From 8710565b2abe5561a0a182a6c72d1b0bdf15e4f4 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Fri, 5 Aug 2022 09:16:43 +0000 Subject: [PATCH 001/118] Use 1 instead of 22 Erlang processes per MQTT connection * Create MQTT connections without proxying via AMQP * Do authn / authz in rabbitmq_mqtt instead of rabbit_direct:connect/5 * Remove rabbit_heartbeat process and per connection supervisors Current status: Creating 10k MQTT connections with clean session succeeds: ./emqtt_bench conn -V 4 -C true -c 10000 -R 500 --- deps/rabbit/src/rabbit_networking.erl | 19 +- deps/rabbit/src/tcp_listener_sup.erl | 12 +- deps/rabbitmq_mqtt/include/mqtt_machine.hrl | 13 +- deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl | 24 +- deps/rabbitmq_mqtt/src/mqtt_node.erl | 5 +- .../src/rabbit_mqtt_connection_sup.erl | 64 --- .../src/rabbit_mqtt_processor.erl | 444 +++++++++--------- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 127 +++-- .../src/rabbit_mqtt_retained_msg_store.erl | 2 + deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl | 55 +-- 10 files changed, 406 insertions(+), 359 deletions(-) delete mode 100644 deps/rabbitmq_mqtt/src/rabbit_mqtt_connection_sup.erl diff --git a/deps/rabbit/src/rabbit_networking.erl b/deps/rabbit/src/rabbit_networking.erl index 9b9ee8213edf..f2095129515b 100644 --- a/deps/rabbit/src/rabbit_networking.erl +++ b/deps/rabbit/src/rabbit_networking.erl @@ -38,7 +38,7 @@ %% Used by TCP-based transports, e.g. STOMP adapter -export([tcp_listener_addresses/1, - tcp_listener_spec/9, tcp_listener_spec/10, + tcp_listener_spec/9, tcp_listener_spec/10, tcp_listener_spec/11, ensure_ssl/0, fix_ssl_options/1, poodle_check/1]). -export([tcp_listener_started/4, tcp_listener_stopped/4]). @@ -206,13 +206,24 @@ tcp_listener_spec(NamePrefix, Address, SocketOpts, Transport, ProtoSup, ProtoOpt any(), protocol(), non_neg_integer(), non_neg_integer(), label()) -> supervisor:child_spec(). -tcp_listener_spec(NamePrefix, {IPAddress, Port, Family}, SocketOpts, +tcp_listener_spec(NamePrefix, Address, SocketOpts, Transport, ProtoSup, ProtoOpts, Protocol, NumAcceptors, ConcurrentConnsSupsCount, Label) -> - Args = [IPAddress, Port, Transport, [Family | SocketOpts], ProtoSup, ProtoOpts, + tcp_listener_spec(NamePrefix, Address, SocketOpts, Transport, ProtoSup, ProtoOpts, + Protocol, NumAcceptors, ConcurrentConnsSupsCount, supervisor, Label). + +-spec tcp_listener_spec + (name_prefix(), address(), [gen_tcp:listen_option()], module(), module(), + any(), protocol(), non_neg_integer(), non_neg_integer(), supervisor:worker(), label()) -> + supervisor:child_spec(). + +tcp_listener_spec(NamePrefix, {IPAddress, Port, Family}, SocketOpts, + Transport, ProtoHandler, ProtoOpts, Protocol, NumAcceptors, + ConcurrentConnsSupsCount, ConnectionType, Label) -> + Args = [IPAddress, Port, Transport, [Family | SocketOpts], ProtoHandler, ProtoOpts, {?MODULE, tcp_listener_started, [Protocol, SocketOpts]}, {?MODULE, tcp_listener_stopped, [Protocol, SocketOpts]}, - NumAcceptors, ConcurrentConnsSupsCount, Label], + NumAcceptors, ConcurrentConnsSupsCount, ConnectionType, Label], {rabbit_misc:tcp_name(NamePrefix, IPAddress, Port), {tcp_listener_sup, start_link, Args}, transient, infinity, supervisor, [tcp_listener_sup]}. diff --git a/deps/rabbit/src/tcp_listener_sup.erl b/deps/rabbit/src/tcp_listener_sup.erl index 57955dcb75fc..7ed78c923236 100644 --- a/deps/rabbit/src/tcp_listener_sup.erl +++ b/deps/rabbit/src/tcp_listener_sup.erl @@ -16,24 +16,24 @@ -behaviour(supervisor). --export([start_link/11]). +-export([start_link/12]). -export([init/1]). -type mfargs() :: {atom(), atom(), [any()]}. -spec start_link (inet:ip_address(), inet:port_number(), module(), [gen_tcp:listen_option()], - module(), any(), mfargs(), mfargs(), integer(), integer(), string()) -> + module(), any(), mfargs(), mfargs(), integer(), integer(), supervisor:worker(), string()) -> rabbit_types:ok_pid_or_error(). start_link(IPAddress, Port, Transport, SocketOpts, ProtoSup, ProtoOpts, OnStartup, OnShutdown, - ConcurrentAcceptorCount, ConcurrentConnsSups, Label) -> + ConcurrentAcceptorCount, ConcurrentConnsSups, ConnectionType, Label) -> supervisor:start_link( ?MODULE, {IPAddress, Port, Transport, SocketOpts, ProtoSup, ProtoOpts, OnStartup, OnShutdown, - ConcurrentAcceptorCount, ConcurrentConnsSups, Label}). + ConcurrentAcceptorCount, ConcurrentConnsSups, ConnectionType, Label}). init({IPAddress, Port, Transport, SocketOpts, ProtoSup, ProtoOpts, OnStartup, OnShutdown, - ConcurrentAcceptorCount, ConcurrentConnsSups, Label}) -> + ConcurrentAcceptorCount, ConcurrentConnsSups, ConnectionType, Label}) -> {ok, AckTimeout} = application:get_env(rabbit, ssl_handshake_timeout), MaxConnections = max_conn(rabbit_misc:get_env(rabbit, connection_max, infinity), ConcurrentConnsSups), @@ -41,7 +41,7 @@ init({IPAddress, Port, Transport, SocketOpts, ProtoSup, ProtoOpts, OnStartup, On num_acceptors => ConcurrentAcceptorCount, max_connections => MaxConnections, handshake_timeout => AckTimeout, - connection_type => supervisor, + connection_type => ConnectionType, socket_opts => [{ip, IPAddress}, {port, Port} | SocketOpts], diff --git a/deps/rabbitmq_mqtt/include/mqtt_machine.hrl b/deps/rabbitmq_mqtt/include/mqtt_machine.hrl index b4c9eb2cdd59..61fc8f759359 100644 --- a/deps/rabbitmq_mqtt/include/mqtt_machine.hrl +++ b/deps/rabbitmq_mqtt/include/mqtt_machine.hrl @@ -5,9 +5,12 @@ %% Copyright (c) 2020-2023 VMware, Inc. or its affiliates. All rights reserved. %% --record(machine_state, {client_ids = #{}, - pids = #{}, - %% add acouple of fields for future extensibility - reserved_1, - reserved_2}). +-record(machine_state, { + %% client ID to connection PID + client_ids = #{}, + %% connection PID to list of client IDs + pids = #{}, + %% add acouple of fields for future extensibility + reserved_1, + reserved_2}). diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl index e75786db15be..a6aaf7c85ea7 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl @@ -14,13 +14,17 @@ deferred_recv, received_connect_frame, connection_state, - keepalive, - keepalive_sup, conserve, parse_state, proc_state, connection, - stats_timer }). + stats_timer, + keepalive }). + +-record(keepalive, {timer :: reference(), + interval_ms :: pos_integer(), + recv_oct :: non_neg_integer(), + received :: boolean()}). %% processor state -record(proc_state, { socket, @@ -36,7 +40,6 @@ channels, connection, exchange, - adapter_info, ssl_login_name, %% Retained messages handler. See rabbit_mqtt_retainer_sup %% and rabbit_mqtt_retainer. @@ -46,11 +49,20 @@ peer_addr, mqtt2amqp_fun, amqp2mqtt_fun, - register_state }). + register_state, + info}). -record(auth_state, {username, user, - vhost}). + vhost, + authz_ctx}). + +-record(info, {prefetch, + host, + port, + peer_host, + peer_port, + protocol}). %% does not include vhost: it is used in %% the table name diff --git a/deps/rabbitmq_mqtt/src/mqtt_node.erl b/deps/rabbitmq_mqtt/src/mqtt_node.erl index 4aaf5862f0f9..ad5f17ea71aa 100644 --- a/deps/rabbitmq_mqtt/src/mqtt_node.erl +++ b/deps/rabbitmq_mqtt/src/mqtt_node.erl @@ -9,9 +9,9 @@ -export([start/0, node_id/0, server_id/0, all_node_ids/0, leave/1, trigger_election/0]). -define(ID_NAME, mqtt_node). --define(START_TIMEOUT, 100000). +-define(START_TIMEOUT, 100_000). -define(RETRY_INTERVAL, 5000). --define(RA_OPERATION_TIMEOUT, 60000). +-define(RA_OPERATION_TIMEOUT, 60_000). -define(RA_SYSTEM, coordination). node_id() -> @@ -42,6 +42,7 @@ start(Delay, AttemptsLeft) -> undefined -> case Nodes of [] -> + %%TODO use global lock instead %% Since cluster members are not known ahead of time and initial boot can be happening in parallel, %% we wait and check a few times (up to a few seconds) to see if we can discover any peers to %% join before forming a cluster. This reduces the probability of N independent clusters being diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_connection_sup.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_connection_sup.erl deleted file mode 100644 index 3d4644ee4efc..000000000000 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_connection_sup.erl +++ /dev/null @@ -1,64 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved. -%% - --module(rabbit_mqtt_connection_sup). - --behaviour(supervisor). --behaviour(ranch_protocol). - --include_lib("rabbit_common/include/rabbit.hrl"). - --export([start_link/3, start_keepalive_link/0]). - --export([init/1]). - -%%---------------------------------------------------------------------------- - -start_link(Ref, _Transport, []) -> - {ok, SupPid} = supervisor:start_link(?MODULE, []), - {ok, KeepaliveSup} = supervisor:start_child( - SupPid, - #{ - id => rabbit_mqtt_keepalive_sup, - start => {rabbit_mqtt_connection_sup, start_keepalive_link, []}, - restart => transient, - significant => true, - shutdown => infinity, - type => supervisor, - modules => [rabbit_keepalive_sup] - } - ), - {ok, ReaderPid} = supervisor:start_child( - SupPid, - #{ - id => rabbit_mqtt_reader, - start => {rabbit_mqtt_reader, start_link, [KeepaliveSup, Ref]}, - restart => transient, - significant => true, - shutdown => ?WORKER_WAIT, - type => worker, - modules => [rabbit_mqtt_reader] - } - ), - {ok, SupPid, ReaderPid}. - -start_keepalive_link() -> - supervisor:start_link(?MODULE, []). - -%%---------------------------------------------------------------------------- - -init([]) -> - {ok, - { - #{ - strategy => one_for_all, - intensity => 0, - period => 1, - auto_shutdown => any_significant - }, - [] - }}. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index a78fb6c3c75d..4d0b986edc59 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -7,14 +7,13 @@ -module(rabbit_mqtt_processor). --export([info/2, initial_state/2, initial_state/5, +-export([info/2, initial_state/2, initial_state/4, process_frame/2, amqp_pub/2, amqp_callback/2, send_will/1, close_connection/1, handle_pre_hibernate/0, handle_ra_event/2]). %% for testing purposes --export([get_vhost_username/1, get_vhost/3, get_vhost_from_user_mapping/2, - add_client_id_to_adapter_info/2, maybe_quorum/3]). +-export([get_vhost_username/1, get_vhost/3, get_vhost_from_user_mapping/2, maybe_quorum/3]). -include_lib("amqp_client/include/amqp_client.hrl"). -include("rabbit_mqtt_frame.hrl"). @@ -28,23 +27,13 @@ initial_state(Socket, SSLLoginName) -> RealSocket = rabbit_net:unwrap_socket(Socket), {ok, {PeerAddr, _PeerPort}} = rabbit_net:peername(RealSocket), - initial_state(RealSocket, SSLLoginName, - adapter_info(Socket, 'MQTT'), - fun serialise_and_send_to_client/2, PeerAddr). + initial_state(RealSocket, SSLLoginName, fun serialise_and_send_to_client/2, PeerAddr). -initial_state(Socket, SSLLoginName, - AdapterInfo0 = #amqp_adapter_info{additional_info = Extra}, - SendFun, PeerAddr) -> +initial_state(Socket, SSLLoginName, SendFun, PeerAddr) -> {ok, {mqtt2amqp_fun, M2A}, {amqp2mqtt_fun, A2M}} = rabbit_mqtt_util:get_topic_translation_funs(), %% MQTT connections use exactly one channel. The frame max is not %% applicable and there is no way to know what client is used. - AdapterInfo = AdapterInfo0#amqp_adapter_info{additional_info = [ - {channels, 1}, - {channel_max, 1}, - {frame_max, 0}, - {client_properties, - [{<<"product">>, longstr, <<"MQTT client">>}]} | Extra]}, #proc_state{ unacked_pubs = gb_trees:empty(), awaiting_ack = gb_trees:empty(), message_id = 1, @@ -53,7 +42,6 @@ initial_state(Socket, SSLLoginName, channels = {undefined, undefined}, exchange = rabbit_mqtt_util:env(exchange), socket = Socket, - adapter_info = AdapterInfo, ssl_login_name = SSLLoginName, send_fun = SendFun, peer_addr = PeerAddr, @@ -61,7 +49,7 @@ initial_state(Socket, SSLLoginName, amqp2mqtt_fun = A2M}. process_frame(#mqtt_frame{ fixed = #mqtt_frame_fixed{ type = Type }}, - PState = #proc_state{ connection = undefined } ) + PState = #proc_state{ auth_state = undefined } ) when Type =/= ?CONNECT -> {error, connect_expected, PState}; process_frame(Frame = #mqtt_frame{ fixed = #mqtt_frame_fixed{ type = Type }}, @@ -80,108 +68,87 @@ process_frame(Frame = #mqtt_frame{ fixed = #mqtt_frame_fixed{ type = Type }}, {error, access_refused, PState} end. -add_client_id_to_adapter_info(ClientId, #amqp_adapter_info{additional_info = AdditionalInfo0} = AdapterInfo) -> - AdditionalInfo1 = [{variable_map, #{<<"client_id">> => ClientId}} - | AdditionalInfo0], - ClientProperties = proplists:get_value(client_properties, AdditionalInfo1, []) - ++ [{client_id, longstr, ClientId}], - AdditionalInfo2 = case lists:keysearch(client_properties, 1, AdditionalInfo1) of - {value, _} -> - lists:keyreplace(client_properties, - 1, - AdditionalInfo1, - {client_properties, ClientProperties}); - false -> - [{client_properties, ClientProperties} | AdditionalInfo1] - end, - AdapterInfo#amqp_adapter_info{additional_info = AdditionalInfo2}. - -process_connect(#mqtt_frame{ variable = #mqtt_frame_connect{ - username = Username, - password = Password, - proto_ver = ProtoVersion, - clean_sess = CleanSess, - client_id = ClientId0, - keep_alive = Keepalive} = Var}, - PState0 = #proc_state{ ssl_login_name = SSLLoginName, - send_fun = SendFun, - adapter_info = AdapterInfo, - peer_addr = Addr}) -> +process_connect(#mqtt_frame{variable = #mqtt_frame_connect{ + username = Username, + password = Password, + proto_ver = ProtoVersion, + clean_sess = CleanSess, + client_id = ClientId0, + keep_alive = Keepalive} = Var}, + PState0 = #proc_state{ssl_login_name = SSLLoginName, + socket = Socket, + send_fun = SendFun, + peer_addr = Addr}) -> ClientId = case ClientId0 of [] -> rabbit_mqtt_util:gen_client_id(); [_|_] -> ClientId0 end, - rabbit_log_connection:debug("Received a CONNECT, client ID: ~tp (expanded to ~tp), username: ~tp, " - "clean session: ~tp, protocol version: ~tp, keepalive: ~tp", - [ClientId0, ClientId, Username, CleanSess, ProtoVersion, Keepalive]), - AdapterInfo1 = add_client_id_to_adapter_info(rabbit_data_coercion:to_binary(ClientId), AdapterInfo), - PState1 = PState0#proc_state{adapter_info = AdapterInfo1}, + rabbit_log_connection:debug("Received a CONNECT, client ID: ~p (expanded to ~p), username: ~p, " + "clean session: ~p, protocol version: ~p, keepalive: ~p", + [ClientId0, ClientId, Username, CleanSess, ProtoVersion, Keepalive]), + % AdapterInfo1 = add_client_id_to_adapter_info(rabbit_data_coercion:to_binary(ClientId), AdapterInfo), + % PState1 = PState0#proc_state{adapter_info = AdapterInfo1}, Ip = list_to_binary(inet:ntoa(Addr)), - {Return, PState5} = - case {lists:member(ProtoVersion, proplists:get_keys(?PROTOCOL_NAMES)), - ClientId0 =:= [] andalso CleanSess =:= false} of - {false, _} -> - {?CONNACK_PROTO_VER, PState1}; - {_, true} -> - {?CONNACK_INVALID_ID, PState1}; - _ -> - case creds(Username, Password, SSLLoginName) of - nocreds -> - rabbit_core_metrics:auth_attempt_failed(Ip, <<>>, mqtt), - rabbit_log_connection:error("MQTT login failed: no credentials provided"), - {?CONNACK_CREDENTIALS, PState1}; - {invalid_creds, {undefined, Pass}} when is_list(Pass) -> - rabbit_core_metrics:auth_attempt_failed(Ip, <<>>, mqtt), - rabbit_log_connection:error("MQTT login failed: no username is provided"), - {?CONNACK_CREDENTIALS, PState1}; - {invalid_creds, {User, undefined}} when is_list(User) -> - rabbit_core_metrics:auth_attempt_failed(Ip, User, mqtt), - rabbit_log_connection:error("MQTT login failed for user '~tp': no password provided", [User]), - {?CONNACK_CREDENTIALS, PState1}; - {UserBin, PassBin} -> - case process_login(UserBin, PassBin, ProtoVersion, PState1) of - connack_dup_auth -> - maybe_clean_sess(PState1); - {?CONNACK_ACCEPT, Conn, VHost, AState} -> - case rabbit_mqtt_collector:register(ClientId, self()) of - {ok, Corr} -> - RetainerPid = rabbit_mqtt_retainer_sup:child_for_vhost(VHost), - link(Conn), - {ok, Ch} = amqp_connection:open_channel(Conn), - link(Ch), - amqp_channel:enable_delivery_flow_control(Ch), - Prefetch = rabbit_mqtt_util:env(prefetch), - #'basic.qos_ok'{} = amqp_channel:call(Ch, - #'basic.qos'{prefetch_count = Prefetch}), + {Return, PState2} = + case {lists:member(ProtoVersion, proplists:get_keys(?PROTOCOL_NAMES)), + ClientId0 =:= [] andalso CleanSess =:= false} of + {false, _} -> + {?CONNACK_PROTO_VER, PState0}; + {_, true} -> + {?CONNACK_INVALID_ID, PState0}; + _ -> + case creds(Username, Password, SSLLoginName) of + nocreds -> + rabbit_core_metrics:auth_attempt_failed(Ip, <<>>, mqtt), + rabbit_log_connection:error("MQTT login failed: no credentials provided"), + {?CONNACK_CREDENTIALS, PState0}; + {invalid_creds, {undefined, Pass}} when is_list(Pass) -> + rabbit_core_metrics:auth_attempt_failed(Ip, <<>>, mqtt), + rabbit_log_connection:error("MQTT login failed: no username is provided"), + {?CONNACK_CREDENTIALS, PState0}; + {invalid_creds, {User, undefined}} when is_list(User) -> + rabbit_core_metrics:auth_attempt_failed(Ip, User, mqtt), + rabbit_log_connection:error("MQTT login failed for user '~p': no password provided", [User]), + {?CONNACK_CREDENTIALS, PState0}; + {UserBin, PassBin} -> + case process_login(UserBin, PassBin, ClientId, ProtoVersion, PState0) of + connack_dup_auth -> + maybe_clean_sess(PState0); + {?CONNACK_ACCEPT, VHost, ProtoVersion, AState} -> + case rabbit_mqtt_collector:register(ClientId, self()) of + {ok, Corr} -> + RetainerPid = rabbit_mqtt_retainer_sup:child_for_vhost(VHost), + Prefetch = rabbit_mqtt_util:env(prefetch), rabbit_mqtt_reader:start_keepalive(self(), Keepalive), - PState3 = PState1#proc_state{ + {ok, {PeerHost, PeerPort, Host, Port}} = rabbit_net:socket_ends(Socket, inbound), + PState1 = PState0#proc_state{ will_msg = make_will_msg(Var), clean_sess = CleanSess, - channels = {Ch, undefined}, - connection = Conn, client_id = ClientId, retainer_pid = RetainerPid, auth_state = AState, - register_state = {pending, Corr}}, - maybe_clean_sess(PState3); - %% e.g. this node was removed from the MQTT cluster members - {error, _} = Err -> + register_state = {pending, Corr}, + info = #info{prefetch = Prefetch, + peer_host = PeerHost, + peer_port = PeerPort, + host = Host, + port = Port, + protocol = {'MQTT', human_readable_mqtt_version(ProtoVersion)}}}, + maybe_clean_sess(PState1); + %% e.g. this node was removed from the MQTT cluster members + {error, _} = Err -> rabbit_log_connection:error("MQTT cannot accept a connection: " - "client ID tracker is unavailable: ~tp", [Err]), - %% ignore all exceptions, we are shutting down - catch amqp_connection:close(Conn), - {?CONNACK_SERVER, PState1}; - {timeout, _} -> + "client ID tracker is unavailable: ~p", [Err]), + {?CONNACK_SERVER, PState0}; + {timeout, _} -> rabbit_log_connection:error("MQTT cannot accept a connection: " "client ID registration timed out"), - %% ignore all exceptions, we are shutting down - catch amqp_connection:close(Conn), - {?CONNACK_SERVER, PState1} - end; - ConnAck -> {ConnAck, PState1} - end - end - end, + {?CONNACK_SERVER, PState0} + end; + ConnAck -> {ConnAck, PState0} + end + end + end, {ReturnCode, SessionPresent} = case Return of {?CONNACK_ACCEPT, Bool} -> {?CONNACK_ACCEPT, Bool}; Other -> {Other, false} @@ -190,14 +157,14 @@ process_connect(#mqtt_frame{ variable = #mqtt_frame_connect{ variable = #mqtt_frame_connack{ session_present = SessionPresent, return_code = ReturnCode}}, - PState5), + PState2), case ReturnCode of - ?CONNACK_ACCEPT -> {ok, PState5}; - ?CONNACK_CREDENTIALS -> {error, unauthenticated, PState5}; - ?CONNACK_AUTH -> {error, unauthorized, PState5}; - ?CONNACK_SERVER -> {error, unavailable, PState5}; - ?CONNACK_INVALID_ID -> {error, invalid_client_id, PState5}; - ?CONNACK_PROTO_VER -> {error, unsupported_protocol_version, PState5} + ?CONNACK_ACCEPT -> {ok, PState2}; + ?CONNACK_CREDENTIALS -> {error, unauthenticated, PState2}; + ?CONNACK_AUTH -> {error, unauthorized, PState2}; + ?CONNACK_SERVER -> {error, unavailable, PState2}; + ?CONNACK_INVALID_ID -> {error, invalid_client_id, PState2}; + ?CONNACK_PROTO_VER -> {error, unsupported_protocol_version, PState2} end. process_request(?CONNECT, Frame, PState = #proc_state{socket = Socket}) -> @@ -535,28 +502,34 @@ maybe_clean_sess(PState = #proc_state { clean_sess = false, erlang:raise(C, E, S) end end; -maybe_clean_sess(PState = #proc_state { clean_sess = true, - connection = Conn, - auth_state = #auth_state{vhost = VHost}, - client_id = ClientId }) -> - {_, Queue} = rabbit_mqtt_util:subcription_queue_name(ClientId), - {ok, Channel} = amqp_connection:open_channel(Conn), - case session_present(VHost, ClientId) of +maybe_clean_sess(PState = #proc_state {clean_sess = true, + client_id = ClientId, + auth_state = #auth_state{user = User, + username = Username, + vhost = VHost, + authz_ctx = AuthzCtx}}) -> + {_, QueueName} = rabbit_mqtt_util:subcription_queue_name(ClientId), + Queue = rabbit_misc:r(VHost, queue, QueueName), + case rabbit_amqqueue:exists(Queue) of false -> {{?CONNACK_ACCEPT, false}, PState}; true -> - try amqp_channel:call(Channel, #'queue.delete'{ queue = Queue }) of - #'queue.delete_ok'{} -> {{?CONNACK_ACCEPT, false}, PState} - catch - exit:({{shutdown, {server_initiated_close, 403, _}}, _}) -> - %% Connection is not yet propagated to #proc_state{}, let's close it here - catch amqp_connection:close(Conn), - rabbit_log_connection:error("MQTT cannot start a clean session: " - "`configure` permission missing for queue `~tp`", [Queue]), - {?CONNACK_SERVER, PState} - after - catch amqp_channel:close(Channel) - end + ok = rabbit_access_control:check_resource_access(User, Queue, configure, AuthzCtx), + rabbit_amqqueue:with( + Queue, + fun (Q) -> + rabbit_queue_type:delete(Q, false, false, Username) + end, + fun (not_found) -> + ok; + ({absent, Q, crashed}) -> + rabbit_classic_queue:delete_crashed(Q, Username); + ({absent, Q, stopped}) -> + rabbit_classic_queue:delete_crashed(Q, Username); + ({absent, _Q, _Reason}) -> + ok + end), + {{?CONNACK_ACCEPT, false}, PState} end. session_present(VHost, ClientId) -> @@ -576,83 +549,134 @@ make_will_msg(#mqtt_frame_connect{ will_retain = Retain, dup = false, payload = Msg }. -process_login(_UserBin, _PassBin, _ProtoVersion, - #proc_state{channels = {Channel, _}, - peer_addr = Addr, +process_login(_UserBin, _PassBin, _ClientId, _ProtoVersion, + #proc_state{peer_addr = Addr, auth_state = #auth_state{username = Username, - vhost = VHost}}) when is_pid(Channel) -> + user = User, + vhost = VHost + }}) + when Username =/= undefined, User =/= undefined, VHost =/= underfined -> UsernameStr = rabbit_data_coercion:to_list(Username), VHostStr = rabbit_data_coercion:to_list(VHost), rabbit_core_metrics:auth_attempt_failed(list_to_binary(inet:ntoa(Addr)), Username, mqtt), rabbit_log_connection:warning("MQTT detected duplicate connect/login attempt for user ~tp, vhost ~tp", [UsernameStr, VHostStr]), connack_dup_auth; -process_login(UserBin, PassBin, ProtoVersion, - #proc_state{channels = {undefined, undefined}, - socket = Sock, - adapter_info = AdapterInfo, +process_login(UserBin, PassBin, ClientId0, ProtoVersion, + #proc_state{socket = Sock, ssl_login_name = SslLoginName, - peer_addr = Addr}) -> - {ok, {_, _, _, ToPort}} = rabbit_net:socket_ends(Sock, inbound), - {VHostPickedUsing, {VHost, UsernameBin}} = get_vhost(UserBin, SslLoginName, ToPort), + peer_addr = Addr, + auth_state = undefined}) -> + {ok, {PeerHost, PeerPort, Host, Port}} = rabbit_net:socket_ends(Sock, inbound), + {VHostPickedUsing, {VHost, UsernameBin}} = get_vhost(UserBin, SslLoginName, Port), rabbit_log_connection:debug( - "MQTT vhost picked using ~ts", - [human_readable_vhost_lookup_strategy(VHostPickedUsing)]), + "MQTT vhost picked using ~s", + [human_readable_vhost_lookup_strategy(VHostPickedUsing)]), RemoteAddress = list_to_binary(inet:ntoa(Addr)), case rabbit_vhost:exists(VHost) of true -> - case amqp_connection:start(#amqp_params_direct{ - username = UsernameBin, - password = PassBin, - virtual_host = VHost, - adapter_info = set_proto_version(AdapterInfo, ProtoVersion)}) of - {ok, Connection} -> - case rabbit_access_control:check_user_loopback(UsernameBin, Addr) of - ok -> - rabbit_core_metrics:auth_attempt_succeeded(RemoteAddress, UsernameBin, - mqtt), - [{internal_user, InternalUser}] = amqp_connection:info( - Connection, [internal_user]), - {?CONNACK_ACCEPT, Connection, VHost, - #auth_state{user = InternalUser, - username = UsernameBin, - vhost = VHost}}; - not_allowed -> - rabbit_core_metrics:auth_attempt_failed(RemoteAddress, UsernameBin, - mqtt), - amqp_connection:close(Connection), - rabbit_log_connection:warning( - "MQTT login failed for user ~ts: " - "this user's access is restricted to localhost", - [binary_to_list(UsernameBin)]), + case rabbit_vhost_limit:is_over_connection_limit(VHost) of + false -> + case rabbit_vhost_sup_sup:is_vhost_alive(VHost) of + true -> + ClientId = rabbit_data_coercion:to_binary(ClientId0), + case rabbit_access_control:check_user_login( + UsernameBin, + [{password, PassBin}, {vhost, VHost}, {client_id, ClientId}]) of + {ok, User = #user{username = Username}} -> + notify_auth_result(Username, + user_authentication_success, + []), + case rabbit_auth_backend_internal:is_over_connection_limit(Username) of + false -> + AuthzCtx = #{<<"client_id">> => ClientId}, + try rabbit_access_control:check_vhost_access(User, + VHost, + {ip, Addr}, + AuthzCtx) of + ok -> + case rabbit_access_control:check_user_loopback(UsernameBin, Addr) of + ok -> + rabbit_core_metrics:auth_attempt_succeeded(RemoteAddress, UsernameBin, + mqtt), + Infos = [{node, node()}, + {host, Host}, + {port, Port}, + {peer_host, PeerHost}, + {peer_port, PeerPort}, + {user, UsernameBin}, + {vhost, VHost}], + rabbit_core_metrics:connection_created(self(), Infos), + rabbit_event:notify(connection_created, Infos), + {?CONNACK_ACCEPT, VHost, ProtoVersion, + #auth_state{user = User, + username = UsernameBin, + vhost = VHost, + authz_ctx = AuthzCtx}}; + not_allowed -> + rabbit_core_metrics:auth_attempt_failed(RemoteAddress, UsernameBin, + mqtt), + rabbit_log_connection:warning( + "MQTT login failed for user ~s: " + "this user's access is restricted to localhost", + [binary_to_list(UsernameBin)]), + ?CONNACK_AUTH + end + catch exit:#amqp_error{name = not_allowed} -> + rabbit_log_connection:error( + "Error on MQTT connection ~p~n" + "access refused for user '~s'", + [self(), Username]), + ?CONNACK_AUTH + end; + {true, Limit} -> + rabbit_log_connection:error( + "Error on MQTT connection ~p~n" + "access refused for user '~s': " + "user connection limit (~p) is reached", + [self(), Username, Limit]), + ?CONNACK_AUTH + end; + {refused, Username, Msg, Args} -> + rabbit_log_connection:error( + "Error on MQTT connection ~p~n" + "access refused for user '~s' in vhost '~s' " + ++ Msg, + [self(), Username, VHost] ++ Args), + notify_auth_result(Username, + user_authentication_failure, + [{error, rabbit_misc:format(Msg, Args)}]), + ?CONNACK_CREDENTIALS + end; + false -> + rabbit_log_connection:error( + "Error on MQTT connection ~p~n" + "access refused for user '~s': " + "vhost is down", + [self(), UsernameBin, VHost]), ?CONNACK_AUTH end; - {error, {auth_failure, Explanation}} -> - rabbit_core_metrics:auth_attempt_failed(RemoteAddress, UsernameBin, mqtt), - rabbit_log_connection:error("MQTT login failed for user '~ts', authentication failed: ~ts", - [binary_to_list(UserBin), Explanation]), - ?CONNACK_CREDENTIALS; - {error, access_refused} -> - rabbit_core_metrics:auth_attempt_failed(RemoteAddress, UsernameBin, mqtt), - rabbit_log_connection:warning("MQTT login failed for user '~ts': " - "virtual host access not allowed", - [binary_to_list(UserBin)]), - ?CONNACK_AUTH; - {error, not_allowed} -> - rabbit_core_metrics:auth_attempt_failed(RemoteAddress, UsernameBin, mqtt), - %% when vhost allowed for TLS connection - rabbit_log_connection:warning("MQTT login failed for user '~ts': " - "virtual host access not allowed", - [binary_to_list(UserBin)]), + {true, Limit} -> + rabbit_log_connection:error( + "Error on MQTT connection ~p~n" + "access to vhost '~s' refused for user '~s': " + "vhost connection limit (~p) is reached", + [self(), VHost, UsernameBin, Limit]), ?CONNACK_AUTH end; false -> rabbit_core_metrics:auth_attempt_failed(RemoteAddress, UsernameBin, mqtt), - rabbit_log_connection:error("MQTT login failed for user '~ts': virtual host '~ts' does not exist", - [UserBin, VHost]), + rabbit_log_connection:error("MQTT login failed for user '~s': virtual host '~s' does not exist", + [UserBin, VHost]), ?CONNACK_CREDENTIALS end. +notify_auth_result(Username, AuthResult, ExtraProps) -> + EventProps = [{connection_type, mqtt}, + {name, case Username of none -> ''; _ -> Username end}] ++ + ExtraProps, + rabbit_event:notify(AuthResult, [P || {_, V} = P <- EventProps, V =/= '']). + get_vhost(UserBin, none, Port) -> get_vhost_no_ssl(UserBin, Port); get_vhost(UserBin, undefined, Port) -> @@ -951,12 +975,9 @@ amqp_pub(#mqtt_msg{ qos = Qos, PState #proc_state{ unacked_pubs = UnackedPubs1, awaiting_seqno = SeqNo1 }. -adapter_info(Sock, ProtoName) -> - amqp_connection:socket_adapter_info(Sock, {ProtoName, "N/A"}). - -set_proto_version(AdapterInfo = #amqp_adapter_info{protocol = {Proto, _}}, Vsn) -> - AdapterInfo#amqp_adapter_info{protocol = {Proto, - human_readable_mqtt_version(Vsn)}}. +% set_proto_version(AdapterInfo = #amqp_adapter_info{protocol = {Proto, _}}, Vsn) -> +% AdapterInfo#amqp_adapter_info{protocol = {Proto, +% human_readable_mqtt_version(Vsn)}}. human_readable_mqtt_version(3) -> "3.1.0"; @@ -1085,36 +1106,35 @@ info(clean_sess, #proc_state{clean_sess = Val}) -> Val; info(will_msg, #proc_state{will_msg = Val}) -> Val; info(channels, #proc_state{channels = Val}) -> Val; info(exchange, #proc_state{exchange = Val}) -> Val; -info(adapter_info, #proc_state{adapter_info = Val}) -> Val; info(ssl_login_name, #proc_state{ssl_login_name = Val}) -> Val; info(retainer_pid, #proc_state{retainer_pid = Val}) -> Val; info(user, #proc_state{auth_state = #auth_state{username = Val}}) -> Val; info(vhost, #proc_state{auth_state = #auth_state{vhost = Val}}) -> Val; -info(host, #proc_state{adapter_info = #amqp_adapter_info{host = Val}}) -> Val; -info(port, #proc_state{adapter_info = #amqp_adapter_info{port = Val}}) -> Val; -info(peer_host, #proc_state{adapter_info = #amqp_adapter_info{peer_host = Val}}) -> Val; -info(peer_port, #proc_state{adapter_info = #amqp_adapter_info{peer_port = Val}}) -> Val; -info(protocol, #proc_state{adapter_info = #amqp_adapter_info{protocol = Val}}) -> +info(host, #proc_state{info = #info{host = Val}}) -> Val; +info(port, #proc_state{info = #info{port = Val}}) -> Val; +info(peer_host, #proc_state{info = #info{peer_host = Val}}) -> Val; +info(peer_port, #proc_state{info = #info{peer_port = Val}}) -> Val; +info(protocol, #proc_state{info = #info{protocol = Val}}) -> case Val of {Proto, Version} -> {Proto, rabbit_data_coercion:to_binary(Version)}; Other -> Other end; -info(channels, PState) -> additional_info(channels, PState); -info(channel_max, PState) -> additional_info(channel_max, PState); -info(frame_max, PState) -> additional_info(frame_max, PState); -info(client_properties, PState) -> additional_info(client_properties, PState); -info(ssl, PState) -> additional_info(ssl, PState); -info(ssl_protocol, PState) -> additional_info(ssl_protocol, PState); -info(ssl_key_exchange, PState) -> additional_info(ssl_key_exchange, PState); -info(ssl_cipher, PState) -> additional_info(ssl_cipher, PState); -info(ssl_hash, PState) -> additional_info(ssl_hash, PState); +% info(channels, PState) -> additional_info(channels, PState); +% info(channel_max, PState) -> additional_info(channel_max, PState); +% info(frame_max, PState) -> additional_info(frame_max, PState); +% info(client_properties, PState) -> additional_info(client_properties, PState); +% info(ssl, PState) -> additional_info(ssl, PState); +% info(ssl_protocol, PState) -> additional_info(ssl_protocol, PState); +% info(ssl_key_exchange, PState) -> additional_info(ssl_key_exchange, PState); +% info(ssl_cipher, PState) -> additional_info(ssl_cipher, PState); +% info(ssl_hash, PState) -> additional_info(ssl_hash, PState); info(Other, _) -> throw({bad_argument, Other}). -additional_info(Key, - #proc_state{adapter_info = - #amqp_adapter_info{additional_info = AddInfo}}) -> - proplists:get_value(Key, AddInfo). +% additional_info(Key, +% #proc_state{adapter_info = +% #amqp_adapter_info{additional_info = AddInfo}}) -> +% proplists:get_value(Key, AddInfo). notify_received(undefined) -> %% no notification for quorum queues and streams diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index 108294df56b3..2a589a8bce1e 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -8,11 +8,15 @@ -module(rabbit_mqtt_reader). -behaviour(gen_server2). +-behaviour(ranch_protocol). --export([start_link/2]). +-export([start_link/3]). -export([init/1, handle_call/3, handle_cast/2, handle_info/2, code_change/3, terminate/2, handle_pre_hibernate/1]). +%%TODO check where to best 'hibernate' when returning from callback +%%TODO use rabbit_global_counters for MQTT protocol + -export([conserve_resources/3, start_keepalive/2, close_connection/2]). @@ -27,10 +31,8 @@ %%---------------------------------------------------------------------------- -start_link(KeepaliveSup, Ref) -> - Pid = proc_lib:spawn_link(?MODULE, init, - [[KeepaliveSup, Ref]]), - +start_link(Ref, _Transport, []) -> + Pid = proc_lib:spawn_link(?MODULE, init, [Ref]), {ok, Pid}. conserve_resources(Pid, _, {_, Conserve, _}) -> @@ -48,7 +50,7 @@ close_connection(Pid, Reason) -> %%---------------------------------------------------------------------------- -init([KeepaliveSup, Ref]) -> +init(Ref) -> process_flag(trap_exit, true), {ok, Sock} = rabbit_networking:handshake(Ref, application:get_env(rabbitmq_mqtt, proxy_protocol, false)), @@ -69,8 +71,6 @@ init([KeepaliveSup, Ref]) -> await_recv = false, connection_state = running, received_connect_frame = false, - keepalive = {none, none}, - keepalive_sup = KeepaliveSup, conserve = false, parse_state = rabbit_mqtt_frame:initial_state(), proc_state = ProcessorState }), #state.stats_timer), @@ -176,20 +176,64 @@ handle_info({bump_credit, Msg}, State) -> credit_flow:handle_bump_msg(Msg), maybe_process_deferred_recv(control_throttle(State)); -handle_info({start_keepalives, Keepalive}, - State = #state { keepalive_sup = KeepaliveSup, socket = Sock }) -> - %% Only the client has the responsibility for sending keepalives - SendFun = fun() -> ok end, - Parent = self(), - ReceiveFun = fun() -> Parent ! keepalive_timeout end, - Heartbeater = rabbit_heartbeat:start( - KeepaliveSup, Sock, 0, SendFun, Keepalive, ReceiveFun), - {noreply, State #state { keepalive = Heartbeater }}; - -handle_info(keepalive_timeout, State = #state {conn_name = ConnStr, - proc_state = PState}) -> - rabbit_log_connection:error("closing MQTT connection ~tp (keepalive timeout)", [ConnStr]), - send_will_and_terminate(PState, {shutdown, keepalive_timeout}, State); +handle_info({start_keepalive, KeepaliveSec}, + State = #state{socket = Sock, + keepalive = undefined}) + when is_number(KeepaliveSec), KeepaliveSec > 0 -> + case rabbit_net:getstat(Sock, [recv_oct]) of + {ok, [{recv_oct, RecvOct}]} -> + %% "If the Keep Alive value is non-zero and the Server does not receive a Control + %% Packet from the Client within one and a half times the Keep Alive time period, + %% it MUST disconnect the Network Connection to the Client as if the network had + %% failed" [MQTT-3.1.2-24] + %% 0.75 * 2 = 1.5 + IntervalMs = timer:seconds(round(0.75 * KeepaliveSec)), + Ref = start_keepalive_timer(#keepalive{interval_ms = IntervalMs}), + {noreply, State#state{keepalive = #keepalive{timer = Ref, + interval_ms = IntervalMs, + recv_oct = RecvOct, + received = true}}}; + {error, einval} -> + %% the socket is dead, most likely because the connection is being shut down + {stop, {shutdown, cannot_get_socket_stats}, State}; + {error, Reason} -> + {stop, Reason, State} + end; + +handle_info({timeout, Ref, keepalive}, + State = #state {socket = Sock, + conn_name = ConnStr, + proc_state = PState, + keepalive = #keepalive{timer = Ref, + recv_oct = SameRecvOct, + received = ReceivedPreviously} = KeepAlive}) -> + case rabbit_net:getstat(Sock, [recv_oct]) of + {ok, [{recv_oct, SameRecvOct}]} + when ReceivedPreviously -> + %% Did not receive from socket for the 1st time. + Ref1 = start_keepalive_timer(KeepAlive), + {noreply, + State#state{keepalive = KeepAlive#keepalive{timer = Ref1, + received = false}}, + hibernate}; + {ok, [{recv_oct, SameRecvOct}]} -> + %% Did not receive from socket for 2nd time successively. + rabbit_log_connection:error("closing MQTT connection ~tp (keepalive timeout)", [ConnStr]), + send_will_and_terminate(PState, {shutdown, keepalive_timeout}, State); + {ok, [{recv_oct, RecvOct}]} -> + %% Received from socket. + Ref1 = start_keepalive_timer(KeepAlive), + {noreply, + State#state{keepalive = KeepAlive#keepalive{timer = Ref1, + recv_oct = RecvOct, + received = true}}, + hibernate}; + {error, einval} -> + %% the socket is dead, most likely because the connection is being shut down + {stop, {shutdown, cannot_get_socket_stats}, State}; + {error, Reason} -> + {stop, Reason, State} + end; handle_info(login_timeout, State = #state{received_connect_frame = true}) -> {noreply, State}; @@ -215,6 +259,12 @@ handle_info({ra_event, _From, Evt}, handle_info(Msg, State) -> {stop, {mqtt_unexpected_msg, Msg}, State}. +start_keepalive_timer(#keepalive{interval_ms = Time}) -> + erlang:start_timer(Time, self(), keepalive). + +cancel_keepalive_timer(#keepalive{timer = Ref}) -> + erlang:cancel_timer(Ref, [{async, true}, {info, false}]). + terminate(Reason, State) -> maybe_emit_stats(State), do_terminate(Reason, State). @@ -369,7 +419,7 @@ callback_reply(State, {error, Reason, ProcState}) -> {stop, Reason, pstate(State, ProcState)}. start_keepalive(_, 0 ) -> ok; -start_keepalive(Pid, Keepalive) -> Pid ! {start_keepalives, Keepalive}. +start_keepalive(Pid, Keepalive) -> Pid ! {start_keepalive, Keepalive}. pstate(State = #state {}, PState = #proc_state{}) -> State #state{ proc_state = PState }. @@ -421,17 +471,28 @@ run_socket(State = #state{ socket = Sock }) -> rabbit_net:setopts(Sock, [{active, once}]), State#state{ await_recv = true }. -control_throttle(State = #state{ connection_state = Flow, - conserve = Conserve }) -> +control_throttle(State = #state{connection_state = Flow, + conserve = Conserve, + keepalive = KeepAlive}) -> case {Flow, Conserve orelse credit_flow:blocked()} of - {running, true} -> ok = rabbit_heartbeat:pause_monitor( - State#state.keepalive), - State #state{ connection_state = blocked }; - {blocked, false} -> ok = rabbit_heartbeat:resume_monitor( - State#state.keepalive), - run_socket(State #state{ - connection_state = running }); - {_, _} -> run_socket(State) + {running, true} + when KeepAlive =:= undefined -> + State#state{connection_state = blocked}; + {running, true} -> + %%TODO Instead of cancelling / setting the timer every time the connection + %% gets blocked / unblocked, restart the timer when it expires and + %% the connection_state is blocked. + ok = cancel_keepalive_timer(KeepAlive), + State#state{connection_state = blocked}; + {blocked, false} + when KeepAlive =:= undefined -> + run_socket(State #state{connection_state = running}); + {blocked, false} -> + Ref = start_keepalive_timer(KeepAlive), + run_socket(State #state{connection_state = running, + keepalive = KeepAlive#keepalive{timer = Ref}}); + {_, _} -> + run_socket(State) end. maybe_process_deferred_recv(State = #state{ deferred_recv = undefined }) -> diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store.erl index 70b18881f031..e4885d068275 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store.erl @@ -21,3 +21,5 @@ behaviour_info(_Other) -> table_name_for(VHost) -> rabbit_mqtt_util:vhost_name_to_table_name(VHost). + +%%TODO could add a rabbitmq_mqtt_retained_msg_store_khepri to have some replication diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl index cbdff3b8e504..807d2f1dd9ad 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl @@ -73,34 +73,35 @@ listener_specs(Fun, Args, Listeners) -> tcp_listener_spec([Address, SocketOpts, NumAcceptors, ConcurrentConnsSups]) -> rabbit_networking:tcp_listener_spec( - rabbit_mqtt_listener_sup, - Address, - SocketOpts, - transport(?TCP_PROTOCOL), - rabbit_mqtt_connection_sup, - [], - mqtt, - NumAcceptors, - ConcurrentConnsSups, - "MQTT TCP listener" - ). + rabbit_mqtt_listener_sup, + Address, + SocketOpts, + transport(?TCP_PROTOCOL), + rabbit_mqtt_reader, + [], + mqtt, + NumAcceptors, + ConcurrentConnsSups, + worker, + "MQTT TCP listener" + ). ssl_listener_spec([Address, SocketOpts, SslOpts, NumAcceptors, ConcurrentConnsSups]) -> rabbit_networking:tcp_listener_spec( - rabbit_mqtt_listener_sup, - Address, - SocketOpts ++ SslOpts, - transport(?TLS_PROTOCOL), - rabbit_mqtt_connection_sup, - [], - 'mqtt/ssl', - NumAcceptors, - ConcurrentConnsSups, - "MQTT TLS listener" - ). + rabbit_mqtt_listener_sup, + Address, + SocketOpts ++ SslOpts, + transport(?TLS_PROTOCOL), + rabbit_mqtt_reader, + [], + 'mqtt/ssl', + NumAcceptors, + ConcurrentConnsSups, + worker, + "MQTT TLS listener" + ). -transport(Protocol) -> - case Protocol of - ?TCP_PROTOCOL -> ranch_tcp; - ?TLS_PROTOCOL -> ranch_ssl - end. +transport(?TCP_PROTOCOL) -> + ranch_tcp; +transport(?TLS_PROTOCOL) -> + ranch_ssl. From 24b0a6bcb2afdf81ea49d59b750a5b26c8810473 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Fri, 12 Aug 2022 13:15:23 +0000 Subject: [PATCH 002/118] Publish with QoS0 via queue_type interface --- deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl | 3 +- .../src/rabbit_mqtt_processor.erl | 144 ++++++++++-------- 2 files changed, 84 insertions(+), 63 deletions(-) diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl index a6aaf7c85ea7..5c687d3de3e0 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl @@ -37,9 +37,10 @@ client_id, clean_sess, will_msg, + queue_states, channels, connection, - exchange, + exchange :: rabbit_exchange:name(), ssl_login_name, %% Retained messages handler. See rabbit_mqtt_retainer_sup %% and rabbit_mqtt_retainer. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 4d0b986edc59..ec7656ede2cd 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -38,9 +38,9 @@ initial_state(Socket, SSLLoginName, SendFun, PeerAddr) -> awaiting_ack = gb_trees:empty(), message_id = 1, subscriptions = #{}, + queue_states = rabbit_queue_type:init(), consumer_tags = {undefined, undefined}, channels = {undefined, undefined}, - exchange = rabbit_mqtt_util:env(exchange), socket = Socket, ssl_login_name = SSLLoginName, send_fun = SendFun, @@ -121,7 +121,10 @@ process_connect(#mqtt_frame{variable = #mqtt_frame_connect{ Prefetch = rabbit_mqtt_util:env(prefetch), rabbit_mqtt_reader:start_keepalive(self(), Keepalive), {ok, {PeerHost, PeerPort, Host, Port}} = rabbit_net:socket_ends(Socket, inbound), + ExchangeBin = rabbit_mqtt_util:env(exchange), + ExchangeName = rabbit_misc:r(VHost, exchange, ExchangeBin), PState1 = PState0#proc_state{ + exchange = ExchangeName, will_msg = make_will_msg(Var), clean_sess = CleanSess, client_id = ClientId, @@ -916,68 +919,84 @@ send_will(PState = #proc_state{will_msg = WillMsg = #mqtt_msg{retain = Retain, end, PState #proc_state{ channels = {undefined, undefined} }. -%% TODO amqp_pub/2 is publishing messages asynchronously, using -%% amqp_channel:cast_flow/3 -%% -%% It does access check using check_publish/3 before submitting, but -%% this is superfluous, as actual publishing will do the same -%% check. While check results cached, it's still some unnecessary -%% work. -%% -%% And the only reason to keep it that way is that it prevents useless -%% crash messages flooding logs, as there is no code to handle async -%% channel crash gracefully. -%% -%% It'd be better to rework the whole thing, removing performance -%% penalty and some 50 lines of duplicate code. Maybe unlinking from -%% channel, and adding it as a child of connection supervisor instead. -%% But exact details are not yet clear. amqp_pub(undefined, PState) -> PState; - -%% set up a qos1 publishing channel if necessary -%% this channel will only be used for publishing, not consuming -amqp_pub(Msg = #mqtt_msg{ qos = ?QOS_1 }, - PState = #proc_state{ channels = {ChQos0, undefined}, - awaiting_seqno = undefined, - connection = Conn }) -> - {ok, Channel} = amqp_connection:open_channel(Conn), - #'confirm.select_ok'{} = amqp_channel:call(Channel, #'confirm.select'{}), - amqp_channel:register_confirm_handler(Channel, self()), - amqp_pub(Msg, PState #proc_state{ channels = {ChQos0, Channel}, - awaiting_seqno = 1 }); - -amqp_pub(#mqtt_msg{ qos = Qos, - topic = Topic, - dup = Dup, - message_id = MessageId, - payload = Payload }, - PState = #proc_state{ channels = {ChQos0, ChQos1}, - exchange = Exchange, - unacked_pubs = UnackedPubs, - awaiting_seqno = SeqNo, - mqtt2amqp_fun = Mqtt2AmqpFun }) -> +amqp_pub(#mqtt_msg{qos = Qos, + topic = Topic, + dup = Dup, + message_id = _MessageId, %%TODO track in unacked_pubs for QoS > 0 + payload = Payload}, + PState = #proc_state{exchange = ExchangeName, + % unacked_pubs = UnackedPubs, + % awaiting_seqno = SeqNo, + mqtt2amqp_fun = Mqtt2AmqpFun}) -> + %%TODO: Use message containers RoutingKey = Mqtt2AmqpFun(Topic), - Method = #'basic.publish'{ exchange = Exchange, - routing_key = RoutingKey }, + Confirm = Qos > ?QOS_0, Headers = [{<<"x-mqtt-publish-qos">>, byte, Qos}, {<<"x-mqtt-dup">>, bool, Dup}], - Msg = #amqp_msg{ props = #'P_basic'{ headers = Headers, - delivery_mode = delivery_mode(Qos)}, - payload = Payload }, - {UnackedPubs1, Ch, SeqNo1} = - case Qos =:= ?QOS_1 andalso MessageId =/= undefined of - true -> {gb_trees:enter(SeqNo, MessageId, UnackedPubs), ChQos1, - SeqNo + 1}; - false -> {UnackedPubs, ChQos0, SeqNo} - end, - amqp_channel:cast_flow(Ch, Method, Msg), - PState #proc_state{ unacked_pubs = UnackedPubs1, - awaiting_seqno = SeqNo1 }. - -% set_proto_version(AdapterInfo = #amqp_adapter_info{protocol = {Proto, _}}, Vsn) -> -% AdapterInfo#amqp_adapter_info{protocol = {Proto, -% human_readable_mqtt_version(Vsn)}}. + Props = #'P_basic'{ + headers = Headers, + delivery_mode = delivery_mode(Qos)}, + {ClassId, _MethodId} = rabbit_framing_amqp_0_9_1:method_id('basic.publish'), + Content = #content{ + class_id = ClassId, + properties = Props, + properties_bin = none, + protocol = none, + payload_fragments_rev = [Payload] + }, + BasicMessage = #basic_message{ + exchange_name = ExchangeName, + routing_keys = [RoutingKey], + content = Content, + id = <<>>, + is_persistent = Confirm + }, + Delivery = #delivery{ + mandatory = false, + confirm = Confirm, + sender = self(), + message = BasicMessage, + msg_seq_no = undefined, %%TODO assumes QoS 0 + flow = noflow %%TODO enable flow control + }, + + Exchange = rabbit_exchange:lookup_or_die(ExchangeName), + QNames = rabbit_exchange:route(Exchange, Delivery), + deliver_to_queues(Delivery, QNames, PState). + +deliver_to_queues(#delivery{confirm = false}, + _RoutedToQueueNames = [], + PState) -> + % rabbit_global_counters:messages_unroutable_dropped(mqtt, 1), + PState; +deliver_to_queues(Delivery = #delivery{message = _Message = #basic_message{exchange_name = _XName}, + confirm = _Confirm, + msg_seq_no = _MsgSeqNo}, + RoutedToQueueNames, + PState = #proc_state{queue_states = QueueStates0}) -> + Qs0 = rabbit_amqqueue:lookup(RoutedToQueueNames), + Qs = rabbit_amqqueue:prepend_extra_bcc(Qs0), + % QueueNames = lists:map(fun amqqueue:get_name/1, Qs), + + {ok, QueueStates, _Actions} = rabbit_queue_type:deliver(Qs, Delivery, QueueStates0), + % rabbit_global_counters:messages_routed(mqtt, length(Qs)), + + %% NB: the order here is important since basic.returns must be + %% sent before confirms. + %% TODO: AMQP 0.9.1 mandatory flag corresponds to MQTT 5 PUBACK reason code "No matching subscribers" + % ok = process_routing_mandatory(Mandatory, Qs, Message, State0), + %% TODO allows QoS > 0 + % State1 = process_routing_confirm(Confirm, QueueNames, + % MsgSeqNo, XName, State0), + + %% Actions must be processed after registering confirms as actions may + %% contain rejections of publishes + %% TODO handle Actions: For example if the messages is rejected, MQTT 5 allows to send a NACK + %% back to the client (via PUBACK Reason Code). + % State = handle_queue_actions(Actions, State1#ch{queue_states = QueueStates}), + PState#proc_state{queue_states = QueueStates}. human_readable_mqtt_version(3) -> "3.1.0"; @@ -1042,6 +1061,7 @@ handle_ra_event(Evt, PState) -> PState. check_publish(TopicName, Fn, PState) -> + %%TODO check additionally write access to exchange as done in channel? case check_topic_access(TopicName, write, PState) of ok -> Fn(); _ -> {error, unauthorized, PState} @@ -1051,7 +1071,7 @@ check_topic_access(TopicName, Access, #proc_state{ auth_state = #auth_state{user = User = #user{username = Username}, vhost = VHost}, - exchange = Exchange, + exchange = #resource{name = ExchangeBin}, client_id = ClientId, mqtt2amqp_fun = Mqtt2AmqpFun }) -> Cache = @@ -1060,14 +1080,14 @@ check_topic_access(TopicName, Access, Other -> Other end, - Key = {TopicName, Username, ClientId, VHost, Exchange, Access}, + Key = {TopicName, Username, ClientId, VHost, ExchangeBin, Access}, case lists:member(Key, Cache) of true -> ok; false -> Resource = #resource{virtual_host = VHost, kind = topic, - name = Exchange}, + name = ExchangeBin}, RoutingKey = Mqtt2AmqpFun(TopicName), Context = #{routing_key => RoutingKey, @@ -1105,7 +1125,7 @@ info(client_id, #proc_state{client_id = Val}) -> info(clean_sess, #proc_state{clean_sess = Val}) -> Val; info(will_msg, #proc_state{will_msg = Val}) -> Val; info(channels, #proc_state{channels = Val}) -> Val; -info(exchange, #proc_state{exchange = Val}) -> Val; +info(exchange, #proc_state{exchange = #resource{name = Val}}) -> Val; info(ssl_login_name, #proc_state{ssl_login_name = Val}) -> Val; info(retainer_pid, #proc_state{retainer_pid = Val}) -> Val; info(user, #proc_state{auth_state = #auth_state{username = Val}}) -> Val; From eac0622f37d3298c86dbc3133b42087931045c85 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Sat, 13 Aug 2022 15:28:30 +0200 Subject: [PATCH 003/118] Consume with QoS0 via queue_type interface --- deps/rabbitmq_mqtt/BUILD.bazel | 6 +- deps/rabbitmq_mqtt/include/mqtt_machine.hrl | 7 + .../include/rabbit_mqtt_frame.hrl | 9 +- deps/rabbitmq_mqtt/src/mqtt_machine.erl | 9 +- .../src/rabbit_mqtt_processor.erl | 307 ++++++++++++++---- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 15 +- 6 files changed, 268 insertions(+), 85 deletions(-) diff --git a/deps/rabbitmq_mqtt/BUILD.bazel b/deps/rabbitmq_mqtt/BUILD.bazel index a337e7b04bbf..07875b615afe 100644 --- a/deps/rabbitmq_mqtt/BUILD.bazel +++ b/deps/rabbitmq_mqtt/BUILD.bazel @@ -48,14 +48,11 @@ BUILD_DEPS = [ DEPS = [ "//deps/amqp_client:erlang_app", "//deps/rabbit_common:erlang_app", + "//deps/rabbit:erlang_app", "@ra//:erlang_app", "@ranch//:erlang_app", ] -RUNTIME_DEPS = [ - "//deps/rabbit:erlang_app", -] - rabbitmq_app( app_description = APP_DESCRIPTION, app_env = APP_ENV, @@ -63,7 +60,6 @@ rabbitmq_app( app_module = APP_MODULE, app_name = APP_NAME, build_deps = BUILD_DEPS, - runtime_deps = RUNTIME_DEPS, deps = DEPS, ) diff --git a/deps/rabbitmq_mqtt/include/mqtt_machine.hrl b/deps/rabbitmq_mqtt/include/mqtt_machine.hrl index 61fc8f759359..c8d4593d9f0b 100644 --- a/deps/rabbitmq_mqtt/include/mqtt_machine.hrl +++ b/deps/rabbitmq_mqtt/include/mqtt_machine.hrl @@ -5,6 +5,13 @@ %% Copyright (c) 2020-2023 VMware, Inc. or its affiliates. All rights reserved. %% +%%TODO decrease per connection memory overhead +%% since the Raft process memory can grow a few GBs with +%% millions of connections. +%% 1. Use binaries instead of string()s for the ConnectionId +%% 2. Use new Erlang 24 function erlang:monitor/3 with tag being the ConnectionId +%% so that we can get rid of pids fields because we won't to lookup the ConnectionId +%% by PID anymore. -record(machine_state, { %% client ID to connection PID client_ids = #{}, diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt_frame.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt_frame.hrl index 82917560cbeb..eb5663f9da8e 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt_frame.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt_frame.hrl @@ -69,15 +69,16 @@ -record(mqtt_frame_publish, {topic_name, message_id}). +-record(mqtt_topic, {name, + qos}). + -record(mqtt_frame_subscribe,{message_id, - topic_table}). + topic_table :: nonempty_list(#mqtt_topic{}) + }). -record(mqtt_frame_suback, {message_id, qos_table = []}). --record(mqtt_topic, {name, - qos}). - -record(mqtt_frame_other, {other}). -record(mqtt_msg, {retain :: boolean(), diff --git a/deps/rabbitmq_mqtt/src/mqtt_machine.erl b/deps/rabbitmq_mqtt/src/mqtt_machine.erl index 4449d989cecc..13e6ce7b2562 100644 --- a/deps/rabbitmq_mqtt/src/mqtt_machine.erl +++ b/deps/rabbitmq_mqtt/src/mqtt_machine.erl @@ -14,7 +14,8 @@ init/1, apply/3, state_enter/2, - notify_connection/2]). + notify_connection/2, + overview/1]). -type state() :: #machine_state{}. @@ -178,6 +179,12 @@ state_enter(leader, State) -> state_enter(_, _) -> []. +-spec overview(state()) -> map(). +overview(#machine_state{client_ids = ClientIds, + pids = Pids}) -> + #{num_client_ids => maps:size(ClientIds), + num_pids => maps:size(Pids)}. + %% ========================== %% Avoids blocking the Raft leader. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index ec7656ede2cd..96af5b8af012 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -10,19 +10,21 @@ -export([info/2, initial_state/2, initial_state/4, process_frame/2, amqp_pub/2, amqp_callback/2, send_will/1, close_connection/1, handle_pre_hibernate/0, - handle_ra_event/2]). + handle_ra_event/2, handle_down/2, handle_queue_event/2]). %% for testing purposes -export([get_vhost_username/1, get_vhost/3, get_vhost_from_user_mapping/2, maybe_quorum/3]). -include_lib("amqp_client/include/amqp_client.hrl"). +-include_lib("rabbit/include/amqqueue.hrl"). -include("rabbit_mqtt_frame.hrl"). -include("rabbit_mqtt.hrl"). -define(APP, rabbitmq_mqtt). -define(FRAME_TYPE(Frame, Type), Frame = #mqtt_frame{ fixed = #mqtt_frame_fixed{ type = Type }}). --define(MAX_TOPIC_PERMISSION_CACHE_SIZE, 12). +-define(MAX_PERMISSION_CACHE_SIZE, 12). +-define(CONSUMER_TAG, mqtt_consumer). initial_state(Socket, SSLLoginName) -> RealSocket = rabbit_net:unwrap_socket(Socket), @@ -236,35 +238,25 @@ process_request(?SUBSCRIBE, message_id = SubscribeMsgId, topic_table = Topics}, payload = undefined}, - #proc_state{channels = {Channel, _}, - exchange = Exchange, - retainer_pid = RPid, + #proc_state{retainer_pid = RPid, send_fun = SendFun, - message_id = StateMsgId, - mqtt2amqp_fun = Mqtt2AmqpFun} = PState0) -> - rabbit_log_connection:debug("Received a SUBSCRIBE for topic(s) ~tp", [Topics]), + message_id = StateMsgId} = PState0) -> + rabbit_log_connection:debug("Received a SUBSCRIBE for topic(s) ~p", [Topics]), {QosResponse, PState1} = lists:foldl(fun (#mqtt_topic{name = TopicName, - qos = Qos}, {QosList, PState}) -> + qos = Qos}, {QosList, S0}) -> SupportedQos = supported_subs_qos(Qos), - {Queue, #proc_state{subscriptions = Subs} = PState1} = - ensure_queue(SupportedQos, PState), - RoutingKey = Mqtt2AmqpFun(TopicName), - Binding = #'queue.bind'{ - queue = Queue, - exchange = Exchange, - routing_key = RoutingKey}, - #'queue.bind_ok'{} = amqp_channel:call(Channel, Binding), + {QueueName, #proc_state{subscriptions = Subs} = S} = + ensure_queue(SupportedQos, S0), + bind(QueueName, TopicName, S), SupportedQosList = case maps:find(TopicName, Subs) of {ok, L} -> [SupportedQos|L]; error -> [SupportedQos] end, {[SupportedQos | QosList], - PState1 #proc_state{ - subscriptions = - maps:put(TopicName, SupportedQosList, Subs)}} - end, {[], PState0}, Topics), + S#proc_state{subscriptions = maps:put(TopicName, SupportedQosList, Subs)}} + end, {[], PState0}, Topics), SendFun(#mqtt_frame{fixed = #mqtt_frame_fixed{type = ?SUBACK}, variable = #mqtt_frame_suback{ message_id = SubscribeMsgId, @@ -839,55 +831,147 @@ maybe_quorum(Qos1Args, CleanSession, Queue) -> %% with appropriate durability and timeout arguments %% this will lead to duplicate messages for overlapping subscriptions %% with different qos values - todo: prevent duplicates -ensure_queue(Qos, #proc_state{ channels = {Channel, _}, - client_id = ClientId, - clean_sess = CleanSess, - consumer_tags = {TagQ0, TagQ1} = Tags} = PState) -> - {QueueQ0, QueueQ1} = rabbit_mqtt_util:subcription_queue_name(ClientId), - Qos1Args = case {rabbit_mqtt_util:env(subscription_ttl), CleanSess} of - {undefined, _} -> - []; - {Ms, false} when is_integer(Ms) -> - [{<<"x-expires">>, long, Ms}]; - _ -> - [] - end, - QueueSetup = - case {TagQ0, TagQ1, Qos} of - {undefined, _, ?QOS_0} -> - {QueueQ0, - #'queue.declare'{ queue = QueueQ0, - durable = false, - auto_delete = true }, - #'basic.consume'{ queue = QueueQ0, - no_ack = true }}; - {_, undefined, ?QOS_1} -> - {QueueQ1, - #'queue.declare'{ queue = QueueQ1, - durable = true, - %% Clean session means a transient connection, - %% translating into auto-delete. - %% - %% see rabbitmq/rabbitmq-mqtt#37 - auto_delete = CleanSess, - arguments = maybe_quorum(Qos1Args, CleanSess, QueueQ1)}, - #'basic.consume'{ queue = QueueQ1, - no_ack = false }}; - {_, _, ?QOS_0} -> - {exists, QueueQ0}; - {_, _, ?QOS_1} -> - {exists, QueueQ1} - end, - case QueueSetup of - {Queue, Declare, Consume} -> - #'queue.declare_ok'{} = amqp_channel:call(Channel, Declare), - #'basic.consume_ok'{ consumer_tag = Tag } = - amqp_channel:call(Channel, Consume), - {Queue, PState #proc_state{ consumer_tags = setelement(Qos+1, Tags, Tag) }}; - {exists, Q} -> - {Q, PState} +%ensure_queue(Qos, #proc_state{ channels = {Channel, _}, +% client_id = ClientId, +% clean_sess = CleanSess, +% consumer_tags = {TagQ0, TagQ1} = Tags} = PState) -> +% {QueueQ0, QueueQ1} = rabbit_mqtt_util:subcription_queue_name(ClientId), +% Qos1Args = case {rabbit_mqtt_util:env(subscription_ttl), CleanSess} of +% {undefined, _} -> +% []; +% {Ms, false} when is_integer(Ms) -> +% [{<<"x-expires">>, long, Ms}]; +% _ -> +% [] +% end, +% QueueSetup = +% case {TagQ0, TagQ1, Qos} of +% {undefined, _, ?QOS_0} -> +% {QueueQ0, +% #'queue.declare'{ queue = QueueQ0, +% durable = false, +% auto_delete = true }, +% #'basic.consume'{ queue = QueueQ0, +% no_ack = true }}; +% {_, undefined, ?QOS_1} -> +% {QueueQ1, +% #'queue.declare'{ queue = QueueQ1, +% durable = true, +% %% Clean session means a transient connection, +% %% translating into auto-delete. +% %% +% %% see rabbitmq/rabbitmq-mqtt#37 +% auto_delete = CleanSess, +% arguments = maybe_quorum(Qos1Args, CleanSess, QueueQ1)}, +% #'basic.consume'{ queue = QueueQ1, +% no_ack = false }}; +% {_, _, ?QOS_0} -> +% {exists, QueueQ0}; +% {_, _, ?QOS_1} -> +% {exists, QueueQ1} +% end, +% case QueueSetup of +% {Queue, Declare, Consume} -> +% #'queue.declare_ok'{} = amqp_channel:call(Channel, Declare), +% #'basic.consume_ok'{ consumer_tag = Tag } = +% amqp_channel:call(Channel, Consume), +% {Queue, PState #proc_state{ consumer_tags = setelement(Qos+1, Tags, Tag) }}; +% {exists, Q} -> +% {Q, PState} +% end. + +ensure_queue(?QOS_0, %% spike handles only QoS0 + #proc_state{ + client_id = ClientId, + clean_sess = _CleanSess, + queue_states = QueueStates0, + auth_state = #auth_state{ + vhost = VHost, + user = User = #user{username = Username}, + authz_ctx = AuthzCtx}, + info = #info{prefetch = Prefetch} + } = PState0) -> + {QueueBin, _QueueQos1Bin} = rabbit_mqtt_util:subcription_queue_name(ClientId), + QueueName = rabbit_misc:r(VHost, queue, QueueBin), + case rabbit_amqqueue:exists(QueueName) of + true -> + {QueueName, PState0}; + false -> + check_resource_access(User, QueueName, read, AuthzCtx), + check_resource_access(User, QueueName, configure, AuthzCtx), + rabbit_core_metrics:queue_declared(QueueName), + Durable = false, + AutoDelete = true, + case rabbit_amqqueue:with( + QueueName, + fun (Q) -> ok = rabbit_amqqueue:assert_equivalence( + Q, Durable, AutoDelete, [], none), + rabbit_amqqueue:stat(Q) + end) of + {error, not_found} -> + case rabbit_vhost_limit:is_over_queue_limit(VHost) of + false -> + case rabbit_amqqueue:declare(QueueName, Durable, AutoDelete, + [], none, Username) of + {new, Q} when ?is_amqqueue(Q) -> + rabbit_core_metrics:queue_created(QueueName), + Spec = #{no_ack => true, + channel_pid => self(), + limiter_pid => none, + limiter_active => false, + prefetch_count => Prefetch, + consumer_tag => ?CONSUMER_TAG, + exclusive_consume => false, + args => [], + ok_msg => undefined, + acting_user => Username}, + case rabbit_queue_type:consume(Q, Spec, QueueStates0) of + {ok, QueueStates, _Actions = []} -> + % rabbit_global_counters:consumer_created(mqtt), + PState = PState0#proc_state{queue_states = QueueStates}, + {QueueName, PState}; + Other -> + exit( + lists:flatten( + io_lib:format("Failed to consume from ~s: ~p", + [rabbit_misc:rs(QueueName), Other]))) + end; + Other -> + exit(lists:flatten( + io_lib:format("Failed to declare ~s: ~p", + [rabbit_misc:rs(QueueName), Other]))) + end; + {true, Limit} -> + exit( + lists:flatten( + io_lib:format("cannot declare ~s because " + "queue limit ~p in vhost '~s' is reached", + [rabbit_misc:rs(QueueName), Limit, VHost]))) + end; + Other -> + exit( + lists:flatten( + io_lib:format("Expected ~s to not exist, got: ~p", + [rabbit_misc:rs(QueueName), Other]))) + end end. +bind(QueueName, + TopicName, + #proc_state{exchange = ExchangeName, + auth_state = #auth_state{ + user = User = #user{username = Username}, + authz_ctx = AuthzCtx}, + mqtt2amqp_fun = Mqtt2AmqpFun} = PState) -> + ok = rabbit_access_control:check_resource_access(User, QueueName, write, AuthzCtx), + ok = rabbit_access_control:check_resource_access(User, ExchangeName, read, AuthzCtx), + ok = check_topic_access(TopicName, read, PState), + RoutingKey = Mqtt2AmqpFun(TopicName), + Binding = #binding{source = ExchangeName, + destination = QueueName, + key = RoutingKey}, + ok = rabbit_binding:add(Binding, Username). + send_will(PState = #proc_state{will_msg = undefined}) -> PState; @@ -924,7 +1008,7 @@ amqp_pub(undefined, PState) -> amqp_pub(#mqtt_msg{qos = Qos, topic = Topic, dup = Dup, - message_id = _MessageId, %%TODO track in unacked_pubs for QoS > 0 + message_id = _MessageId, %% spike handles only QoS0 payload = Payload}, PState = #proc_state{exchange = ExchangeName, % unacked_pubs = UnackedPubs, @@ -958,7 +1042,7 @@ amqp_pub(#mqtt_msg{qos = Qos, confirm = Confirm, sender = self(), message = BasicMessage, - msg_seq_no = undefined, %%TODO assumes QoS 0 + msg_seq_no = undefined, %% spike handles only QoS0 flow = noflow %%TODO enable flow control }, @@ -987,7 +1071,7 @@ deliver_to_queues(Delivery = #delivery{message = _Message = #basic_message{ex %% sent before confirms. %% TODO: AMQP 0.9.1 mandatory flag corresponds to MQTT 5 PUBACK reason code "No matching subscribers" % ok = process_routing_mandatory(Mandatory, Qs, Message, State0), - %% TODO allows QoS > 0 + %% spike handles only QoS0 % State1 = process_routing_confirm(Confirm, QueueNames, % MsgSeqNo, XName, State0), @@ -1006,6 +1090,8 @@ human_readable_mqtt_version(_) -> "N/A". serialise_and_send_to_client(Frame, #proc_state{ socket = Sock }) -> + %%TODO Test sending large frames at high speed: Will we need garbage collection as done + %% in rabbit_writer:maybe_gc_large_msg()? try rabbit_net:port_command(Sock, rabbit_mqtt_frame:serialise(Frame)) of Res -> Res @@ -1036,6 +1122,7 @@ close_connection(PState = #proc_state{ connection = Connection, handle_pre_hibernate() -> erase(topic_permission_cache), + erase(permission_cache), ok. handle_ra_event({applied, [{Corr, ok}]}, @@ -1060,6 +1147,67 @@ handle_ra_event(Evt, PState) -> rabbit_log:debug("unhandled ra_event: ~w ", [Evt]), PState. +handle_down({'DOWN', _MRef, process, QPid, Reason}, + PState0 = #proc_state{queue_states = QStates0}) -> + %% spike handles only QoS0 + case rabbit_queue_type:handle_down(QPid, Reason, QStates0) of + {ok, QStates1, Actions} -> + PState = PState0#proc_state{queue_states = QStates1}, + handle_queue_actions(Actions, PState); + {eol, QStates1, QRef} -> + QStates = rabbit_queue_type:remove(QRef, QStates1), + PState0#proc_state{queue_states = QStates} + end. + +handle_queue_event({queue_event, QRef, Evt}, + PState0 = #proc_state{queue_states = QueueStates0}) -> + case rabbit_queue_type:handle_event(QRef, Evt, QueueStates0) of + {ok, QueueStates, Actions} -> + PState1 = PState0#proc_state{queue_states = QueueStates}, + PState = handle_queue_actions(Actions, PState1), + {ok, PState}; + eol -> + {error, queue_eol, PState0}; + {protocol_error, _Type, _Reason, _ReasonArgs} = Error -> + {error, Error, PState0} + end. + +handle_queue_actions(Actions, #proc_state{} = PState0) -> + lists:foldl( + fun ({deliver, ?CONSUMER_TAG, _AckRequired = false, Msgs}, S) -> + handle_deliver(Msgs, S) + end, PState0, Actions). + +handle_deliver(Msgs, PState) + when is_list(Msgs) -> + lists:foldl(fun(Msg, S) -> + handle_deliver0(Msg, S) + end, PState, Msgs). + +handle_deliver0({_QName, _QPid, _MsgId, Redelivered, + #basic_message{routing_keys = [RoutingKey | _CcRoutes], + content = #content{ + properties = #'P_basic'{headers = Headers}, + payload_fragments_rev = FragmentsRev}}}, + PState = #proc_state{send_fun = SendFun, + amqp2mqtt_fun = Amqp2MqttFun}) -> + Dup = case rabbit_mqtt_util:table_lookup(Headers, <<"x-mqtt-dup">>) of + undefined -> Redelivered; + {bool, Dup0} -> Redelivered orelse Dup0 + end, + %%TODO support iolists when sending to client + Payload = list_to_binary(lists:reverse(FragmentsRev)), + Frame = #mqtt_frame{fixed = #mqtt_frame_fixed{ + type = ?PUBLISH, + qos = ?QOS_0, %% spike handles only QoS0 + dup = Dup}, + variable = #mqtt_frame_publish{ + message_id = undefined, %% spike handles only QoS0 + topic_name = Amqp2MqttFun(RoutingKey)}, + payload = Payload}, + SendFun(Frame, PState), + PState. + check_publish(TopicName, Fn, PState) -> %%TODO check additionally write access to exchange as done in channel? case check_topic_access(TopicName, write, PState) of @@ -1100,7 +1248,7 @@ check_topic_access(TopicName, Access, try rabbit_access_control:check_topic_access(User, Resource, Access, Context) of ok -> - CacheTail = lists:sublist(Cache, ?MAX_TOPIC_PERMISSION_CACHE_SIZE - 1), + CacheTail = lists:sublist(Cache, ?MAX_PERMISSION_CACHE_SIZE - 1), put(topic_permission_cache, [Key | CacheTail]), ok; R -> @@ -1115,6 +1263,21 @@ check_topic_access(TopicName, Access, end end. +%% TODO copied from channel, remove duplication +check_resource_access(User, Resource, Perm, Context) -> + V = {Resource, Context, Perm}, + Cache = case get(permission_cache) of + undefined -> []; + Other -> Other + end, + case lists:member(V, Cache) of + true -> ok; + false -> ok = rabbit_access_control:check_resource_access( + User, Resource, Perm, Context), + CacheTail = lists:sublist(Cache, ?MAX_PERMISSION_CACHE_SIZE-1), + put(permission_cache, [V | CacheTail]) + end. + info(consumer_tags, #proc_state{consumer_tags = Val}) -> Val; info(unacked_pubs, #proc_state{unacked_pubs = Val}) -> Val; info(awaiting_ack, #proc_state{awaiting_ack = Val}) -> Val; diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index 2a589a8bce1e..901241fff25f 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -118,6 +118,10 @@ handle_cast({close_connection, Reason}, [ConnName, rabbit_mqtt_processor:info(client_id, PState), Reason]), {stop, {shutdown, server_initiated_close}, State}; +handle_cast(QueueEvent = {queue_event, _, _}, + State = #state{proc_state = PState}) -> + callback_reply(State, rabbit_mqtt_processor:handle_queue_event(QueueEvent, PState)); + handle_cast(Msg, State) -> {stop, {mqtt_unexpected_cast, Msg}, State}. @@ -250,11 +254,16 @@ handle_info(emit_stats, State) -> {noreply, emit_stats(State), hibernate}; handle_info({ra_event, _From, Evt}, - #state{proc_state = PState} = State) -> + #state{proc_state = PState0} = State) -> %% handle applied event to ensure registration command actually got applied %% handle not_leader notification in case we send the command to a non-leader - PState1 = rabbit_mqtt_processor:handle_ra_event(Evt, PState), - {noreply, State#state{proc_state = PState1}, hibernate}; + PState = rabbit_mqtt_processor:handle_ra_event(Evt, PState0), + {noreply, pstate(State, PState), hibernate}; + +handle_info({'DOWN', _MRef, process, _Pid, _Reason} = Evt, + #state{proc_state = PState0} = State) -> + PState = rabbit_mqtt_processor:handle_down(Evt, PState0), + {noreply, pstate(State, PState), hibernate}; handle_info(Msg, State) -> {stop, {mqtt_unexpected_msg, Msg}, State}. From f4d1f682129f4f5a85aed540c6f12aae9e65b406 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Thu, 1 Sep 2022 14:02:08 +0000 Subject: [PATCH 004/118] Move authn / authz into rabbitmq_mqtt --- deps/rabbit/scripts/rabbitmq-server | 1 + .../src/rabbit_auth_mechanism.erl | 13 + deps/rabbit_common/src/rabbit_misc.erl | 44 +- .../include/rabbit_mqtt_frame.hrl | 18 +- .../src/rabbit_mqtt_processor.erl | 585 ++++++++++-------- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 14 +- deps/rabbitmq_mqtt/test/auth_SUITE.erl | 1 + erlang.mk | 3 +- 8 files changed, 395 insertions(+), 284 deletions(-) diff --git a/deps/rabbit/scripts/rabbitmq-server b/deps/rabbit/scripts/rabbitmq-server index 7ac71d71bfd8..3bd76a1cf68a 100755 --- a/deps/rabbit/scripts/rabbitmq-server +++ b/deps/rabbit/scripts/rabbitmq-server @@ -77,6 +77,7 @@ start_rabbitmq_server() { -syslog logger '[]' \ -syslog syslog_error_logger false \ -kernel prevent_overlapping_partitions false \ + -enable-feature maybe_expr \ "$@" } diff --git a/deps/rabbit_common/src/rabbit_auth_mechanism.erl b/deps/rabbit_common/src/rabbit_auth_mechanism.erl index 6090c4177146..d950769b5046 100644 --- a/deps/rabbit_common/src/rabbit_auth_mechanism.erl +++ b/deps/rabbit_common/src/rabbit_auth_mechanism.erl @@ -5,6 +5,19 @@ %% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved. %% +%% This module defines the interface for SASL mechanisms. +%% +%% The central idea behind SASL is to decouple application protocols +%% from authentication mechanisms. +%% +%% Protocols AMQP 0.9.1, AMQP 1.0, Stream, and MQTT 5.0 support SASL. +%% +%% Therefore, their respective protocol readers call a rabbit_auth_mechanism_* module. +%% The SASL mechanism knows how to extract credentials (e.g. username and password, or just +%% a username from a client certificate) and asks rabbit_access_control to +%% authenticate the client. rabbit_access_control in turn delegates authentication +%% (and authorization) to a rabbit_auth_backend_* module. + -module(rabbit_auth_mechanism). -behaviour(rabbit_registry_class). diff --git a/deps/rabbit_common/src/rabbit_misc.erl b/deps/rabbit_common/src/rabbit_misc.erl index d19f60a2d19a..a79e273d17fb 100644 --- a/deps/rabbit_common/src/rabbit_misc.erl +++ b/deps/rabbit_common/src/rabbit_misc.erl @@ -84,6 +84,7 @@ -export([raw_read_file/1]). -export([find_child/2]). -export([is_regular_file/1]). +-export([pipeline/3]). %% Horrible macro to use in guards -define(IS_BENIGN_EXIT(R), @@ -1471,7 +1472,6 @@ whereis_name(Name) -> %% End copypasta from gen_server2.erl %% ------------------------------------------------------------------------- - %% This will execute a Powershell command without an intervening cmd.exe %% process. Output lines can't exceed 512 bytes. %% @@ -1552,3 +1552,45 @@ find_powershell() -> PwshExe -> PwshExe end. + +%% ------------------------------------------------------------------------- +%% Begin copy from +%% https://github.com/emqx/emqx/blob/cffdcb42843d48bf99d8bd13695bc73149c98a23/apps/emqx/src/emqx_misc.erl#L141-L157 + +%%-------------------------------------------------------------------- +%% Copyright (c) 2017-2022 EMQ Technologies Co., Ltd. All Rights Reserved. +%% +%% Licensed under the Apache License, Version 2.0 (the "License"); +%% you may not use this file except in compliance with the License. +%% You may obtain a copy of the License at +%% +%% http://www.apache.org/licenses/LICENSE-2.0 +%% +%% Unless required by applicable law or agreed to in writing, software +%% distributed under the License is distributed on an "AS IS" BASIS, +%% WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +%% See the License for the specific language governing permissions and +%% limitations under the License. +%%-------------------------------------------------------------------- + +pipeline([], Input, State) -> + {ok, Input, State}; +pipeline([Fun | More], Input, State) -> + case apply_fun(Fun, Input, State) of + ok -> pipeline(More, Input, State); + {ok, NState} -> pipeline(More, Input, NState); + {ok, Output, NState} -> pipeline(More, Output, NState); + {error, Reason} -> {error, Reason, State}; + {error, Reason, NState} -> {error, Reason, NState} + end. + +-compile({inline, [apply_fun/3]}). +apply_fun(Fun, Input, State) -> + case erlang:fun_info(Fun, arity) of + {arity, 1} -> Fun(Input); + {arity, 2} -> Fun(Input, State) + end. + +%% End copy from +%% https://github.com/emqx/emqx/blob/cffdcb42843d48bf99d8bd13695bc73149c98a23/apps/emqx/src/emqx_misc.erl#L141-L157 +%% ------------------------------------------------------------------------- \ No newline at end of file diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt_frame.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt_frame.hrl index eb5663f9da8e..06d9c2f75c56 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt_frame.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt_frame.hrl @@ -26,12 +26,18 @@ %% connect return codes --define(CONNACK_ACCEPT, 0). --define(CONNACK_PROTO_VER, 1). %% unacceptable protocol version --define(CONNACK_INVALID_ID, 2). %% identifier rejected --define(CONNACK_SERVER, 3). %% server unavailable --define(CONNACK_CREDENTIALS, 4). %% bad user name or password --define(CONNACK_AUTH, 5). %% not authorized +%% Connection accepted. +-define(CONNACK_ACCEPT, 0). +%% The Server does not support the level of the MQTT protocol requested by the Client. +-define(CONNACK_UNACCEPTABLE_PROTO_VER, 1). +%% The Client identifier is correct UTF-8 but not allowed by the Server. +-define(CONNACK_ID_REJECTED, 2). +%% The Network Connection has been made but the MQTT service is unavailable. +-define(CONNACK_SERVER_UNAVAILABLE, 3). +%% The data in the user name or password is malformed. +-define(CONNACK_BAD_CREDENTIALS, 4). +%% The Client is not authorized to connect. +-define(CONNACK_NOT_AUTHORIZED, 5). %% qos levels diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 96af5b8af012..7408b10f8494 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -7,7 +7,7 @@ -module(rabbit_mqtt_processor). --export([info/2, initial_state/2, initial_state/4, +-export([info/2, initial_state/1, initial_state/4, process_frame/2, amqp_pub/2, amqp_callback/2, send_will/1, close_connection/1, handle_pre_hibernate/0, handle_ra_event/2, handle_down/2, handle_queue_event/2]). @@ -26,8 +26,9 @@ -define(MAX_PERMISSION_CACHE_SIZE, 12). -define(CONSUMER_TAG, mqtt_consumer). -initial_state(Socket, SSLLoginName) -> +initial_state(Socket) -> RealSocket = rabbit_net:unwrap_socket(Socket), + SSLLoginName = ssl_login_name(RealSocket), {ok, {PeerAddr, _PeerPort}} = rabbit_net:peername(RealSocket), initial_state(RealSocket, SSLLoginName, fun serialise_and_send_to_client/2, PeerAddr). @@ -70,108 +71,6 @@ process_frame(Frame = #mqtt_frame{ fixed = #mqtt_frame_fixed{ type = Type }}, {error, access_refused, PState} end. -process_connect(#mqtt_frame{variable = #mqtt_frame_connect{ - username = Username, - password = Password, - proto_ver = ProtoVersion, - clean_sess = CleanSess, - client_id = ClientId0, - keep_alive = Keepalive} = Var}, - PState0 = #proc_state{ssl_login_name = SSLLoginName, - socket = Socket, - send_fun = SendFun, - peer_addr = Addr}) -> - ClientId = case ClientId0 of - [] -> rabbit_mqtt_util:gen_client_id(); - [_|_] -> ClientId0 - end, - rabbit_log_connection:debug("Received a CONNECT, client ID: ~p (expanded to ~p), username: ~p, " - "clean session: ~p, protocol version: ~p, keepalive: ~p", - [ClientId0, ClientId, Username, CleanSess, ProtoVersion, Keepalive]), - % AdapterInfo1 = add_client_id_to_adapter_info(rabbit_data_coercion:to_binary(ClientId), AdapterInfo), - % PState1 = PState0#proc_state{adapter_info = AdapterInfo1}, - Ip = list_to_binary(inet:ntoa(Addr)), - {Return, PState2} = - case {lists:member(ProtoVersion, proplists:get_keys(?PROTOCOL_NAMES)), - ClientId0 =:= [] andalso CleanSess =:= false} of - {false, _} -> - {?CONNACK_PROTO_VER, PState0}; - {_, true} -> - {?CONNACK_INVALID_ID, PState0}; - _ -> - case creds(Username, Password, SSLLoginName) of - nocreds -> - rabbit_core_metrics:auth_attempt_failed(Ip, <<>>, mqtt), - rabbit_log_connection:error("MQTT login failed: no credentials provided"), - {?CONNACK_CREDENTIALS, PState0}; - {invalid_creds, {undefined, Pass}} when is_list(Pass) -> - rabbit_core_metrics:auth_attempt_failed(Ip, <<>>, mqtt), - rabbit_log_connection:error("MQTT login failed: no username is provided"), - {?CONNACK_CREDENTIALS, PState0}; - {invalid_creds, {User, undefined}} when is_list(User) -> - rabbit_core_metrics:auth_attempt_failed(Ip, User, mqtt), - rabbit_log_connection:error("MQTT login failed for user '~p': no password provided", [User]), - {?CONNACK_CREDENTIALS, PState0}; - {UserBin, PassBin} -> - case process_login(UserBin, PassBin, ClientId, ProtoVersion, PState0) of - connack_dup_auth -> - maybe_clean_sess(PState0); - {?CONNACK_ACCEPT, VHost, ProtoVersion, AState} -> - case rabbit_mqtt_collector:register(ClientId, self()) of - {ok, Corr} -> - RetainerPid = rabbit_mqtt_retainer_sup:child_for_vhost(VHost), - Prefetch = rabbit_mqtt_util:env(prefetch), - rabbit_mqtt_reader:start_keepalive(self(), Keepalive), - {ok, {PeerHost, PeerPort, Host, Port}} = rabbit_net:socket_ends(Socket, inbound), - ExchangeBin = rabbit_mqtt_util:env(exchange), - ExchangeName = rabbit_misc:r(VHost, exchange, ExchangeBin), - PState1 = PState0#proc_state{ - exchange = ExchangeName, - will_msg = make_will_msg(Var), - clean_sess = CleanSess, - client_id = ClientId, - retainer_pid = RetainerPid, - auth_state = AState, - register_state = {pending, Corr}, - info = #info{prefetch = Prefetch, - peer_host = PeerHost, - peer_port = PeerPort, - host = Host, - port = Port, - protocol = {'MQTT', human_readable_mqtt_version(ProtoVersion)}}}, - maybe_clean_sess(PState1); - %% e.g. this node was removed from the MQTT cluster members - {error, _} = Err -> - rabbit_log_connection:error("MQTT cannot accept a connection: " - "client ID tracker is unavailable: ~p", [Err]), - {?CONNACK_SERVER, PState0}; - {timeout, _} -> - rabbit_log_connection:error("MQTT cannot accept a connection: " - "client ID registration timed out"), - {?CONNACK_SERVER, PState0} - end; - ConnAck -> {ConnAck, PState0} - end - end - end, - {ReturnCode, SessionPresent} = case Return of - {?CONNACK_ACCEPT, Bool} -> {?CONNACK_ACCEPT, Bool}; - Other -> {Other, false} - end, - SendFun(#mqtt_frame{fixed = #mqtt_frame_fixed{type = ?CONNACK}, - variable = #mqtt_frame_connack{ - session_present = SessionPresent, - return_code = ReturnCode}}, - PState2), - case ReturnCode of - ?CONNACK_ACCEPT -> {ok, PState2}; - ?CONNACK_CREDENTIALS -> {error, unauthenticated, PState2}; - ?CONNACK_AUTH -> {error, unauthorized, PState2}; - ?CONNACK_SERVER -> {error, unavailable, PState2}; - ?CONNACK_INVALID_ID -> {error, invalid_client_id, PState2}; - ?CONNACK_PROTO_VER -> {error, unsupported_protocol_version, PState2} - end. - process_request(?CONNECT, Frame, PState = #proc_state{socket = Socket}) -> %% Check whether peer closed the connection. %% For example, this can happen when connection was blocked because of resource @@ -320,6 +219,208 @@ process_request(?DISCONNECT, #mqtt_frame{}, PState) -> rabbit_log_connection:debug("Received a DISCONNECT"), {stop, PState}. +process_connect(#mqtt_frame{ + variable = #mqtt_frame_connect{ + username = Username, + proto_ver = ProtoVersion, + clean_sess = CleanSess, + client_id = ClientId, + keep_alive = Keepalive} = FrameConnect}, + #proc_state{send_fun = SendFun} = PState0) -> + rabbit_log_connection:debug("Received a CONNECT, client ID: ~p, username: ~p, " + "clean session: ~p, protocol version: ~p, keepalive: ~p", + [ClientId, Username, CleanSess, ProtoVersion, Keepalive]), + {ReturnCode, SessionPresent, PState} = + case rabbit_misc:pipeline([fun check_protocol_version/1, + fun check_client_id/1, + fun check_credentials/2, + fun login/2, + fun register_client/2], + FrameConnect, PState0) of + {ok, SessionPresent0, PState1} -> + {?CONNACK_ACCEPT, SessionPresent0, PState1}; + {error, ReturnCode0, PState1} -> + {ReturnCode0, false, PState1} + end, + ResponseFrame = #mqtt_frame{fixed = #mqtt_frame_fixed{type = ?CONNACK}, + variable = #mqtt_frame_connack{ + session_present = SessionPresent, + return_code = ReturnCode}}, + SendFun(ResponseFrame, PState), + return_connack(ReturnCode, PState). + +client_id([]) -> + rabbit_mqtt_util:gen_client_id(); +client_id(ClientId) + when is_list(ClientId) -> + ClientId. + +check_protocol_version(#mqtt_frame_connect{proto_ver = ProtoVersion}) -> + case lists:member(ProtoVersion, proplists:get_keys(?PROTOCOL_NAMES)) of + true -> + ok; + false -> + {error, ?CONNACK_UNACCEPTABLE_PROTO_VER} + end. + +check_client_id(#mqtt_frame_connect{clean_sess = false, + client_id = []}) -> + {error, ?CONNACK_ID_REJECTED}; +check_client_id(_) -> + ok. + +check_credentials(Frame = #mqtt_frame_connect{username = Username, + password = Password}, + PState = #proc_state{ssl_login_name = SslLoginName, + peer_addr = PeerAddr}) -> + Ip = list_to_binary(inet:ntoa(PeerAddr)), + case creds(Username, Password, SslLoginName) of + nocreds -> + rabbit_core_metrics:auth_attempt_failed(Ip, <<>>, mqtt), + rabbit_log_connection:error("MQTT login failed: no credentials provided"), + {error, ?CONNACK_BAD_CREDENTIALS}; + {invalid_creds, {undefined, Pass}} when is_list(Pass) -> + rabbit_core_metrics:auth_attempt_failed(Ip, <<>>, mqtt), + rabbit_log_connection:error("MQTT login failed: no username is provided"), + {error, ?CONNACK_BAD_CREDENTIALS}; + {invalid_creds, {User, undefined}} when is_list(User) -> + rabbit_core_metrics:auth_attempt_failed(Ip, User, mqtt), + rabbit_log_connection:error("MQTT login failed for user '~p': no password provided", [User]), + {error, ?CONNACK_BAD_CREDENTIALS}; + {UserBin, PassBin} -> + {ok, {UserBin, PassBin, Frame}, PState} + end. + +login({UserBin, PassBin, + Frame = #mqtt_frame_connect{client_id = ClientId0, + clean_sess = CleanSess}}, + PState) -> + ClientId = client_id(ClientId0), + case process_login(UserBin, PassBin, ClientId, PState) of + connack_dup_auth -> + maybe_clean_sess(PState); + {?CONNACK_ACCEPT, AuthState} -> + {ok, Frame, PState#proc_state{client_id = ClientId, + clean_sess = CleanSess, + auth_state = AuthState}}; + {error, _} = Err -> + Err + end. + +register_client(Frame = #mqtt_frame_connect{ + keep_alive = Keepalive, + proto_ver = ProtoVersion}, + PState0 = #proc_state{client_id = ClientId, + socket = Socket, + auth_state = #auth_state{ + vhost = VHost}}) -> + case rabbit_mqtt_collector:register(ClientId, self()) of + {ok, Corr} -> + RetainerPid = rabbit_mqtt_retainer_sup:child_for_vhost(VHost), + Prefetch = rabbit_mqtt_util:env(prefetch), + rabbit_mqtt_reader:start_keepalive(self(), Keepalive), + {ok, {PeerHost, PeerPort, Host, Port}} = rabbit_net:socket_ends(Socket, inbound), + ExchangeBin = rabbit_mqtt_util:env(exchange), + ExchangeName = rabbit_misc:r(VHost, exchange, ExchangeBin), + Protocol = {'MQTT', human_readable_mqtt_version(ProtoVersion)}, + PState = PState0#proc_state{ + exchange = ExchangeName, + will_msg = make_will_msg(Frame), + retainer_pid = RetainerPid, + register_state = {pending, Corr}, + info = #info{prefetch = Prefetch, + peer_host = PeerHost, + peer_port = PeerPort, + host = Host, + port = Port, + protocol = Protocol}}, + maybe_clean_sess(PState); + {error, _} = Err -> + %% e.g. this node was removed from the MQTT cluster members + rabbit_log_connection:error("MQTT cannot accept a connection: " + "client ID tracker is unavailable: ~p", [Err]), + {error, ?CONNACK_SERVER_UNAVAILABLE}; + {timeout, _} -> + rabbit_log_connection:error("MQTT cannot accept a connection: " + "client ID registration timed out"), + {error, ?CONNACK_SERVER_UNAVAILABLE} + end. + +return_connack(?CONNACK_ACCEPT, S) -> + {ok, S}; +return_connack(?CONNACK_BAD_CREDENTIALS, S) -> + {error, unauthenticated, S}; +return_connack(?CONNACK_NOT_AUTHORIZED, S) -> + {error, unauthorized, S}; +return_connack(?CONNACK_SERVER_UNAVAILABLE, S) -> + {error, unavailable, S}; +return_connack(?CONNACK_ID_REJECTED, S) -> + {error, invalid_client_id, S}; +return_connack(?CONNACK_UNACCEPTABLE_PROTO_VER, S) -> + {error, unsupported_protocol_version, S}. + +maybe_clean_sess(PState = #proc_state {clean_sess = false, + connection = Conn, + auth_state = #auth_state{vhost = VHost}, + client_id = ClientId }) -> + SessionPresent = session_present(VHost, ClientId), + case SessionPresent of + false -> + %% ensure_queue/2 not only ensures that queue is created, but also starts consuming from it. + %% Let's avoid creating that queue until explicitly asked by a client. + %% Then publish-only clients, that connect with clean_sess=true due to some misconfiguration, + %% will consume less resources. + {ok, SessionPresent, PState}; + true -> + try ensure_queue(?QOS_1, PState) of + {_Queue, PState1} -> + {ok, SessionPresent, PState1} + catch + exit:({{shutdown, {server_initiated_close, 403, _}}, _}) -> + %% Connection is not yet propagated to #proc_state{}, let's close it here + catch amqp_connection:close(Conn), + rabbit_log_connection:error("MQTT cannot recover a session, user is missing permissions"), + {error, ?CONNACK_SERVER_UNAVAILABLE}; + C:E:S -> + %% Connection is not yet propagated to + %% #proc_state{}, let's close it here. + %% This is an exceptional situation anyway, but + %% doing this will prevent second crash from + %% amqp client being logged. + catch amqp_connection:close(Conn), + erlang:raise(C, E, S) + end + end; +maybe_clean_sess(PState = #proc_state {clean_sess = true, + client_id = ClientId, + auth_state = #auth_state{user = User, + username = Username, + vhost = VHost, + authz_ctx = AuthzCtx}}) -> + {_, QueueName} = rabbit_mqtt_util:subcription_queue_name(ClientId), + Queue = rabbit_misc:r(VHost, queue, QueueName), + case rabbit_amqqueue:exists(Queue) of + false -> + {ok, false, PState}; + true -> + ok = rabbit_access_control:check_resource_access(User, Queue, configure, AuthzCtx), + rabbit_amqqueue:with( + Queue, + fun (Q) -> + rabbit_queue_type:delete(Q, false, false, Username) + end, + fun (not_found) -> + ok; + ({absent, Q, crashed}) -> + rabbit_classic_queue:delete_crashed(Q, Username); + ({absent, Q, stopped}) -> + rabbit_classic_queue:delete_crashed(Q, Username); + ({absent, _Q, _Reason}) -> + ok + end), + {ok, false, PState} + end. + hand_off_to_retainer(RetainerPid, Amqp2MqttFun, Topic0, #mqtt_msg{payload = <<"">>}) -> Topic1 = Amqp2MqttFun(Topic0), rabbit_mqtt_retainer:clear(RetainerPid, Topic1), @@ -466,67 +567,6 @@ delivery_qos(Tag, Headers, #proc_state{ consumer_tags = {_, Tag} }) -> undefined -> {?QOS_1, ?QOS_1} end. -maybe_clean_sess(PState = #proc_state { clean_sess = false, - connection = Conn, - auth_state = #auth_state{vhost = VHost}, - client_id = ClientId }) -> - SessionPresent = session_present(VHost, ClientId), - case SessionPresent of - false -> - %% ensure_queue/2 not only ensures that queue is created, but also starts consuming from it. - %% Let's avoid creating that queue until explicitly asked by a client. - %% Then publish-only clients, that connect with clean_sess=true due to some misconfiguration, - %% will consume less resources. - {{?CONNACK_ACCEPT, SessionPresent}, PState}; - true -> - try ensure_queue(?QOS_1, PState) of - {_Queue, PState1} -> {{?CONNACK_ACCEPT, SessionPresent}, PState1} - catch - exit:({{shutdown, {server_initiated_close, 403, _}}, _}) -> - %% Connection is not yet propagated to #proc_state{}, let's close it here - catch amqp_connection:close(Conn), - rabbit_log_connection:error("MQTT cannot recover a session, user is missing permissions"), - {?CONNACK_SERVER, PState}; - C:E:S -> - %% Connection is not yet propagated to - %% #proc_state{}, let's close it here. - %% This is an exceptional situation anyway, but - %% doing this will prevent second crash from - %% amqp client being logged. - catch amqp_connection:close(Conn), - erlang:raise(C, E, S) - end - end; -maybe_clean_sess(PState = #proc_state {clean_sess = true, - client_id = ClientId, - auth_state = #auth_state{user = User, - username = Username, - vhost = VHost, - authz_ctx = AuthzCtx}}) -> - {_, QueueName} = rabbit_mqtt_util:subcription_queue_name(ClientId), - Queue = rabbit_misc:r(VHost, queue, QueueName), - case rabbit_amqqueue:exists(Queue) of - false -> - {{?CONNACK_ACCEPT, false}, PState}; - true -> - ok = rabbit_access_control:check_resource_access(User, Queue, configure, AuthzCtx), - rabbit_amqqueue:with( - Queue, - fun (Q) -> - rabbit_queue_type:delete(Q, false, false, Username) - end, - fun (not_found) -> - ok; - ({absent, Q, crashed}) -> - rabbit_classic_queue:delete_crashed(Q, Username); - ({absent, Q, stopped}) -> - rabbit_classic_queue:delete_crashed(Q, Username); - ({absent, _Q, _Reason}) -> - ok - end), - {{?CONNACK_ACCEPT, false}, PState} - end. - session_present(VHost, ClientId) -> {_, QueueQ1} = rabbit_mqtt_util:subcription_queue_name(ClientId), QueueName = rabbit_misc:r(VHost, queue, QueueQ1), @@ -544,7 +584,7 @@ make_will_msg(#mqtt_frame_connect{ will_retain = Retain, dup = false, payload = Msg }. -process_login(_UserBin, _PassBin, _ClientId, _ProtoVersion, +process_login(_UserBin, _PassBin, _ClientId, #proc_state{peer_addr = Addr, auth_state = #auth_state{username = Username, user = User, @@ -557,10 +597,10 @@ process_login(_UserBin, _PassBin, _ClientId, _ProtoVersion, rabbit_log_connection:warning("MQTT detected duplicate connect/login attempt for user ~tp, vhost ~tp", [UsernameStr, VHostStr]), connack_dup_auth; -process_login(UserBin, PassBin, ClientId0, ProtoVersion, - #proc_state{socket = Sock, +process_login(UserBin, PassBin, ClientId0, + #proc_state{socket = Sock, ssl_login_name = SslLoginName, - peer_addr = Addr, + peer_addr = Addr, auth_state = undefined}) -> {ok, {PeerHost, PeerPort, Host, Port}} = rabbit_net:socket_ends(Sock, inbound), {VHostPickedUsing, {VHost, UsernameBin}} = get_vhost(UserBin, SslLoginName, Port), @@ -568,102 +608,108 @@ process_login(UserBin, PassBin, ClientId0, ProtoVersion, "MQTT vhost picked using ~s", [human_readable_vhost_lookup_strategy(VHostPickedUsing)]), RemoteAddress = list_to_binary(inet:ntoa(Addr)), - case rabbit_vhost:exists(VHost) of - true -> - case rabbit_vhost_limit:is_over_connection_limit(VHost) of - false -> - case rabbit_vhost_sup_sup:is_vhost_alive(VHost) of - true -> - ClientId = rabbit_data_coercion:to_binary(ClientId0), - case rabbit_access_control:check_user_login( - UsernameBin, - [{password, PassBin}, {vhost, VHost}, {client_id, ClientId}]) of - {ok, User = #user{username = Username}} -> - notify_auth_result(Username, - user_authentication_success, - []), - case rabbit_auth_backend_internal:is_over_connection_limit(Username) of - false -> - AuthzCtx = #{<<"client_id">> => ClientId}, - try rabbit_access_control:check_vhost_access(User, - VHost, - {ip, Addr}, - AuthzCtx) of - ok -> - case rabbit_access_control:check_user_loopback(UsernameBin, Addr) of - ok -> - rabbit_core_metrics:auth_attempt_succeeded(RemoteAddress, UsernameBin, - mqtt), - Infos = [{node, node()}, - {host, Host}, - {port, Port}, - {peer_host, PeerHost}, - {peer_port, PeerPort}, - {user, UsernameBin}, - {vhost, VHost}], - rabbit_core_metrics:connection_created(self(), Infos), - rabbit_event:notify(connection_created, Infos), - {?CONNACK_ACCEPT, VHost, ProtoVersion, - #auth_state{user = User, - username = UsernameBin, - vhost = VHost, - authz_ctx = AuthzCtx}}; - not_allowed -> - rabbit_core_metrics:auth_attempt_failed(RemoteAddress, UsernameBin, - mqtt), - rabbit_log_connection:warning( - "MQTT login failed for user ~s: " - "this user's access is restricted to localhost", - [binary_to_list(UsernameBin)]), - ?CONNACK_AUTH - end - catch exit:#amqp_error{name = not_allowed} -> - rabbit_log_connection:error( - "Error on MQTT connection ~p~n" - "access refused for user '~s'", - [self(), Username]), - ?CONNACK_AUTH - end; - {true, Limit} -> - rabbit_log_connection:error( - "Error on MQTT connection ~p~n" - "access refused for user '~s': " - "user connection limit (~p) is reached", - [self(), Username, Limit]), - ?CONNACK_AUTH - end; - {refused, Username, Msg, Args} -> - rabbit_log_connection:error( - "Error on MQTT connection ~p~n" - "access refused for user '~s' in vhost '~s' " - ++ Msg, - [self(), Username, VHost] ++ Args), - notify_auth_result(Username, - user_authentication_failure, - [{error, rabbit_misc:format(Msg, Args)}]), - ?CONNACK_CREDENTIALS - end; - false -> - rabbit_log_connection:error( - "Error on MQTT connection ~p~n" - "access refused for user '~s': " - "vhost is down", - [self(), UsernameBin, VHost]), - ?CONNACK_AUTH - end; - {true, Limit} -> + maybe + ok ?= case rabbit_vhost:exists(VHost) of + true -> + ok; + false -> + rabbit_core_metrics:auth_attempt_failed(RemoteAddress, UsernameBin, mqtt), + rabbit_log_connection:error("MQTT login failed for user '~s': virtual host '~s' does not exist", + [UserBin, VHost]), + {error, ?CONNACK_BAD_CREDENTIALS} + end, + ok ?= case rabbit_vhost_limit:is_over_connection_limit(VHost) of + false -> + ok; + {true, Limit0} -> + rabbit_log_connection:error( + "Error on MQTT connection ~p~n" + "access to vhost '~s' refused for user '~s': " + "vhost connection limit (~p) is reached", + [self(), VHost, UsernameBin, Limit0]), + {error, ?CONNACK_NOT_AUTHORIZED} + end, + ok ?= case rabbit_vhost_sup_sup:is_vhost_alive(VHost) of + true -> + ok; + false -> + rabbit_log_connection:error( + "Error on MQTT connection ~p~n" + "access refused for user '~s': " + "vhost is down", + [self(), UsernameBin, VHost]), + {error, ?CONNACK_NOT_AUTHORIZED} + end, + ClientId = rabbit_data_coercion:to_binary(ClientId0), + User = #user{username = Username} ?= + case rabbit_access_control:check_user_login( + UsernameBin, + [{password, PassBin}, {vhost, VHost}, {client_id, ClientId}]) of + {ok, User0} -> + User0; + {refused, Username0, Msg, Args} -> rabbit_log_connection:error( "Error on MQTT connection ~p~n" - "access to vhost '~s' refused for user '~s': " - "vhost connection limit (~p) is reached", - [self(), VHost, UsernameBin, Limit]), - ?CONNACK_AUTH - end; - false -> - rabbit_core_metrics:auth_attempt_failed(RemoteAddress, UsernameBin, mqtt), - rabbit_log_connection:error("MQTT login failed for user '~s': virtual host '~s' does not exist", - [UserBin, VHost]), - ?CONNACK_CREDENTIALS + "access refused for user '~s' in vhost '~s' " + ++ Msg, + [self(), Username0, VHost] ++ Args), + notify_auth_result(Username0, + user_authentication_failure, + [{error, rabbit_misc:format(Msg, Args)}]), + {error, ?CONNACK_BAD_CREDENTIALS} + end, + notify_auth_result(Username, user_authentication_success, []), + ok ?= case rabbit_auth_backend_internal:is_over_connection_limit(Username) of + false -> + ok; + {true, Limit} -> + rabbit_log_connection:error( + "Error on MQTT connection ~p~n" + "access refused for user '~s': " + "user connection limit (~p) is reached", + [self(), Username, Limit]), + {error, ?CONNACK_NOT_AUTHORIZED} + end, + AuthzCtx = #{<<"client_id">> => ClientId}, + try rabbit_access_control:check_vhost_access(User, + VHost, + {ip, Addr}, + AuthzCtx) of + ok -> + case rabbit_access_control:check_user_loopback(UsernameBin, Addr) of + ok -> + rabbit_core_metrics:auth_attempt_succeeded(RemoteAddress, UsernameBin, + mqtt), + Infos = [{node, node()}, + {host, Host}, + {port, Port}, + {peer_host, PeerHost}, + {peer_port, PeerPort}, + {user, UsernameBin}, + {vhost, VHost}], + rabbit_core_metrics:connection_created(self(), Infos), + rabbit_event:notify(connection_created, Infos), + {?CONNACK_ACCEPT, + #auth_state{user = User, + username = UsernameBin, + vhost = VHost, + authz_ctx = AuthzCtx}}; + not_allowed -> + rabbit_core_metrics:auth_attempt_failed(RemoteAddress, UsernameBin, + mqtt), + rabbit_log_connection:warning( + "MQTT login failed for user ~s: " + "this user's access is restricted to localhost", + [binary_to_list(UsernameBin)]), + {error, ?CONNACK_NOT_AUTHORIZED} + end + catch exit:#amqp_error{name = not_allowed} -> + rabbit_log_connection:error( + "Error on MQTT connection ~p~n" + "access refused for user '~s'", + [self(), Username]), + {error, ?CONNACK_NOT_AUTHORIZED} + end end. notify_auth_result(Username, AuthResult, ExtraProps) -> @@ -1325,3 +1371,16 @@ notify_received(undefined) -> notify_received(DeliveryCtx) -> %% notification for flow control amqp_channel:notify_received(DeliveryCtx). + +-spec ssl_login_name(rabbit_net:socket()) -> + none | binary(). +ssl_login_name(Sock) -> + case rabbit_net:peercert(Sock) of + {ok, C} -> case rabbit_ssl:peer_cert_auth_name(C) of + unsafe -> none; + not_found -> none; + Name -> Name + end; + {error, no_peercert} -> none; + nossl -> none + end. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index 901241fff25f..bcaff15a8aea 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -20,7 +20,6 @@ -export([conserve_resources/3, start_keepalive/2, close_connection/2]). --export([ssl_login_name/1]). -export([info/2]). -include_lib("amqp_client/include/amqp_client.hrl"). @@ -62,7 +61,7 @@ init(Ref) -> self(), {?MODULE, conserve_resources, []}), LoginTimeout = application:get_env(rabbitmq_mqtt, login_timeout, 10_000), erlang:send_after(LoginTimeout, self(), login_timeout), - ProcessorState = rabbit_mqtt_processor:initial_state(Sock,ssl_login_name(RealSocket)), + ProcessorState = rabbit_mqtt_processor:initial_state(Sock), gen_server2:enter_loop(?MODULE, [], rabbit_event:init_stats_timer( control_throttle( @@ -326,17 +325,6 @@ do_terminate(_Reason, #state{proc_state = ProcState}) -> code_change(_OldVsn, State, _Extra) -> {ok, State}. -ssl_login_name(Sock) -> - case rabbit_net:peercert(Sock) of - {ok, C} -> case rabbit_ssl:peer_cert_auth_name(C) of - unsafe -> none; - not_found -> none; - Name -> Name - end; - {error, no_peercert} -> none; - nossl -> none - end. - %%---------------------------------------------------------------------------- log_tls_alert(handshake_failure, ConnStr) -> diff --git a/deps/rabbitmq_mqtt/test/auth_SUITE.erl b/deps/rabbitmq_mqtt/test/auth_SUITE.erl index b5cf4d3a3635..3ce31569880e 100644 --- a/deps/rabbitmq_mqtt/test/auth_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/auth_SUITE.erl @@ -13,6 +13,7 @@ -define(CONNECT_TIMEOUT, 10000). -define(WAIT_LOG_NO_CRASHES, {["Generic server.*terminating"], fun () -> exit(there_should_be_no_crashes) end}). +%%TODO check for loopback user, i.e. that rabbit_access_control:check_user_loopback/2 is called. all() -> [{group, anonymous_no_ssl_user}, {group, anonymous_ssl_user}, diff --git a/erlang.mk b/erlang.mk index 4044d80555fe..763c6b6ea8f5 100644 --- a/erlang.mk +++ b/erlang.mk @@ -5146,7 +5146,8 @@ ERLANG_MK_QUERY_SHELL_DEPS_FILE = $(ERLANG_MK_TMP)/query-shell-deps.log # Configuration. ERLC_OPTS ?= -Werror +debug_info +warn_export_vars +warn_shadow_vars \ - +warn_obsolete_guard # +bin_opt_info +warn_export_all +warn_missing_spec + +warn_obsolete_guard "+{feature, maybe_expr, enable}" + # +bin_opt_info +warn_export_all +warn_missing_spec COMPILE_FIRST ?= COMPILE_FIRST_PATHS = $(addprefix src/,$(addsuffix .erl,$(COMPILE_FIRST))) ERLC_EXCLUDE ?= From 73ad3bafe751cbf3ff816062da75e88331197232 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Fri, 2 Sep 2022 12:24:40 +0000 Subject: [PATCH 005/118] Revert maybe expression rabbit_misc:pipeline looks better and doesn't require experimental feature --- deps/rabbit/scripts/rabbitmq-server | 1 - .../src/rabbit_mqtt_processor.erl | 312 ++++++++++-------- deps/rabbitmq_mqtt/test/auth_SUITE.erl | 10 +- erlang.mk | 3 +- 4 files changed, 188 insertions(+), 138 deletions(-) diff --git a/deps/rabbit/scripts/rabbitmq-server b/deps/rabbit/scripts/rabbitmq-server index 3bd76a1cf68a..7ac71d71bfd8 100755 --- a/deps/rabbit/scripts/rabbitmq-server +++ b/deps/rabbit/scripts/rabbitmq-server @@ -77,7 +77,6 @@ start_rabbitmq_server() { -syslog logger '[]' \ -syslog syslog_error_logger false \ -kernel prevent_overlapping_partitions false \ - -enable-feature maybe_expr \ "$@" } diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 7408b10f8494..794ee0d2e58f 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -294,16 +294,15 @@ check_credentials(Frame = #mqtt_frame_connect{username = Username, login({UserBin, PassBin, Frame = #mqtt_frame_connect{client_id = ClientId0, clean_sess = CleanSess}}, - PState) -> + PState0) -> ClientId = client_id(ClientId0), - case process_login(UserBin, PassBin, ClientId, PState) of + case process_login(UserBin, PassBin, ClientId, PState0) of connack_dup_auth -> - maybe_clean_sess(PState); - {?CONNACK_ACCEPT, AuthState} -> - {ok, Frame, PState#proc_state{client_id = ClientId, - clean_sess = CleanSess, - auth_state = AuthState}}; - {error, _} = Err -> + maybe_clean_sess(PState0); + {ok, PState} -> + {ok, Frame, PState#proc_state{clean_sess = CleanSess, + client_id = ClientId}}; + {error, _Reason, _PState} = Err -> Err end. @@ -601,115 +600,105 @@ process_login(UserBin, PassBin, ClientId0, #proc_state{socket = Sock, ssl_login_name = SslLoginName, peer_addr = Addr, - auth_state = undefined}) -> - {ok, {PeerHost, PeerPort, Host, Port}} = rabbit_net:socket_ends(Sock, inbound), + auth_state = undefined} = PState0) -> + {ok, {_PeerHost, _PeerPort, _Host, Port}} = rabbit_net:socket_ends(Sock, inbound), {VHostPickedUsing, {VHost, UsernameBin}} = get_vhost(UserBin, SslLoginName, Port), rabbit_log_connection:debug( "MQTT vhost picked using ~s", [human_readable_vhost_lookup_strategy(VHostPickedUsing)]), - RemoteAddress = list_to_binary(inet:ntoa(Addr)), - maybe - ok ?= case rabbit_vhost:exists(VHost) of - true -> - ok; - false -> - rabbit_core_metrics:auth_attempt_failed(RemoteAddress, UsernameBin, mqtt), - rabbit_log_connection:error("MQTT login failed for user '~s': virtual host '~s' does not exist", - [UserBin, VHost]), - {error, ?CONNACK_BAD_CREDENTIALS} - end, - ok ?= case rabbit_vhost_limit:is_over_connection_limit(VHost) of - false -> - ok; - {true, Limit0} -> - rabbit_log_connection:error( - "Error on MQTT connection ~p~n" - "access to vhost '~s' refused for user '~s': " - "vhost connection limit (~p) is reached", - [self(), VHost, UsernameBin, Limit0]), - {error, ?CONNACK_NOT_AUTHORIZED} - end, - ok ?= case rabbit_vhost_sup_sup:is_vhost_alive(VHost) of - true -> - ok; - false -> - rabbit_log_connection:error( - "Error on MQTT connection ~p~n" - "access refused for user '~s': " - "vhost is down", - [self(), UsernameBin, VHost]), - {error, ?CONNACK_NOT_AUTHORIZED} - end, - ClientId = rabbit_data_coercion:to_binary(ClientId0), - User = #user{username = Username} ?= - case rabbit_access_control:check_user_login( - UsernameBin, - [{password, PassBin}, {vhost, VHost}, {client_id, ClientId}]) of - {ok, User0} -> - User0; - {refused, Username0, Msg, Args} -> - rabbit_log_connection:error( - "Error on MQTT connection ~p~n" - "access refused for user '~s' in vhost '~s' " - ++ Msg, - [self(), Username0, VHost] ++ Args), - notify_auth_result(Username0, - user_authentication_failure, - [{error, rabbit_misc:format(Msg, Args)}]), - {error, ?CONNACK_BAD_CREDENTIALS} - end, - notify_auth_result(Username, user_authentication_success, []), - ok ?= case rabbit_auth_backend_internal:is_over_connection_limit(Username) of - false -> - ok; - {true, Limit} -> - rabbit_log_connection:error( - "Error on MQTT connection ~p~n" - "access refused for user '~s': " - "user connection limit (~p) is reached", - [self(), Username, Limit]), - {error, ?CONNACK_NOT_AUTHORIZED} - end, - AuthzCtx = #{<<"client_id">> => ClientId}, - try rabbit_access_control:check_vhost_access(User, - VHost, - {ip, Addr}, - AuthzCtx) of - ok -> - case rabbit_access_control:check_user_loopback(UsernameBin, Addr) of - ok -> - rabbit_core_metrics:auth_attempt_succeeded(RemoteAddress, UsernameBin, - mqtt), - Infos = [{node, node()}, - {host, Host}, - {port, Port}, - {peer_host, PeerHost}, - {peer_port, PeerPort}, - {user, UsernameBin}, - {vhost, VHost}], - rabbit_core_metrics:connection_created(self(), Infos), - rabbit_event:notify(connection_created, Infos), - {?CONNACK_ACCEPT, - #auth_state{user = User, - username = UsernameBin, - vhost = VHost, - authz_ctx = AuthzCtx}}; - not_allowed -> - rabbit_core_metrics:auth_attempt_failed(RemoteAddress, UsernameBin, - mqtt), - rabbit_log_connection:warning( - "MQTT login failed for user ~s: " - "this user's access is restricted to localhost", - [binary_to_list(UsernameBin)]), - {error, ?CONNACK_NOT_AUTHORIZED} - end - catch exit:#amqp_error{name = not_allowed} -> - rabbit_log_connection:error( - "Error on MQTT connection ~p~n" - "access refused for user '~s'", - [self(), Username]), - {error, ?CONNACK_NOT_AUTHORIZED} - end + RemoteIpAddressBin = list_to_binary(inet:ntoa(Addr)), + ClientId = rabbit_data_coercion:to_binary(ClientId0), + Input = #{vhost => VHost, + username_bin => UsernameBin, + pass_bin => PassBin, + client_id => ClientId}, + case rabbit_misc:pipeline( + [fun check_vhost_exists/1, + fun check_vhost_connection_limit/1, + fun check_vhost_alive/1, + fun check_user_login/2, + fun check_user_connection_limit/1, + fun check_vhost_access/2, + fun check_user_loopback/2 + ], + Input, PState0) of + {ok, _Output, PState} -> + rabbit_core_metrics:auth_attempt_succeeded(RemoteIpAddressBin, UsernameBin, mqtt), + {ok, PState}; + {error, _Reason, _PState} = Err -> + rabbit_core_metrics:auth_attempt_failed(RemoteIpAddressBin, UsernameBin, mqtt), + Err + end. + +check_vhost_exists(#{vhost := VHost, + username_bin := UsernameBin}) -> + case rabbit_vhost:exists(VHost) of + true -> + ok; + false -> + rabbit_log_connection:error("MQTT login failed for user '~s': virtual host '~s' does not exist", + [UsernameBin, VHost]), + {error, ?CONNACK_BAD_CREDENTIALS} + end. + +check_vhost_connection_limit(#{vhost := VHost, + username_bin := UsernameBin}) -> + case rabbit_vhost_limit:is_over_connection_limit(VHost) of + false -> + ok; + {true, Limit} -> + rabbit_log_connection:error( + "Error on MQTT connection ~p~n" + "access to vhost '~s' refused for user '~s': " + "vhost connection limit (~p) is reached", + [self(), VHost, UsernameBin, Limit]), + {error, ?CONNACK_NOT_AUTHORIZED} + end. + +check_vhost_alive(#{vhost := VHost, + username_bin := UsernameBin}) -> + case rabbit_vhost_sup_sup:is_vhost_alive(VHost) of + true -> + ok; + false -> + rabbit_log_connection:error( + "Error on MQTT connection ~p~n" + "access refused for user '~s': " + "vhost is down", + [self(), UsernameBin, VHost]), + {error, ?CONNACK_NOT_AUTHORIZED} + end. + +check_user_login(#{vhost := VHost, + username_bin := UsernameBin, + pass_bin := PassBin, + client_id := ClientId + } = In, PState) -> + AuthProps = case PassBin of + none -> + %% SSL user name provided. + %% Authenticating using username only. + []; + _ -> + [{password, PassBin}, + {vhost, VHost}, + {client_id, ClientId}] + end, + case rabbit_access_control:check_user_login( + UsernameBin, AuthProps) of + {ok, User = #user{username = Username}} -> + notify_auth_result(Username, user_authentication_success, []), + {ok, maps:put(user, User, In), PState}; + {refused, Username, Msg, Args} -> + rabbit_log_connection:error( + "Error on MQTT connection ~p~n" + "access refused for user '~s' in vhost '~s' " + ++ Msg, + [self(), Username, VHost] ++ Args), + notify_auth_result(Username, + user_authentication_failure, + [{error, rabbit_misc:format(Msg, Args)}]), + {error, ?CONNACK_BAD_CREDENTIALS} end. notify_auth_result(Username, AuthResult, ExtraProps) -> @@ -718,6 +707,73 @@ notify_auth_result(Username, AuthResult, ExtraProps) -> ExtraProps, rabbit_event:notify(AuthResult, [P || {_, V} = P <- EventProps, V =/= '']). +check_user_connection_limit(#{user := #user{username = Username}}) -> + case rabbit_auth_backend_internal:is_over_connection_limit(Username) of + false -> + ok; + {true, Limit} -> + rabbit_log_connection:error( + "Error on MQTT connection ~p~n" + "access refused for user '~s': " + "user connection limit (~p) is reached", + [self(), Username, Limit]), + {error, ?CONNACK_NOT_AUTHORIZED} + end. + + +check_vhost_access(#{vhost := VHost, + client_id := ClientId, + user := User = #user{username = Username} + } = In, + #proc_state{peer_addr = PeerAddr} = PState) -> + AuthzCtx = #{<<"client_id">> => ClientId}, + try rabbit_access_control:check_vhost_access( + User, + VHost, + {ip, PeerAddr}, + AuthzCtx) of + ok -> + {ok, maps:put(authz_ctx, AuthzCtx, In), PState} + catch exit:#amqp_error{name = not_allowed} -> + rabbit_log_connection:error( + "Error on MQTT connection ~p~n" + "access refused for user '~s'", + [self(), Username]), + {error, ?CONNACK_NOT_AUTHORIZED} + end. + +check_user_loopback(#{vhost := VHost, + username_bin := UsernameBin, + user := User, + authz_ctx := AuthzCtx + }, + #proc_state{socket = Sock, + peer_addr = PeerAddr} = PState) -> + case rabbit_access_control:check_user_loopback(UsernameBin, PeerAddr) of + ok -> + {ok, {PeerHost, PeerPort, Host, Port}} = rabbit_net:socket_ends(Sock, inbound), + Infos = [{node, node()}, + {host, Host}, + {port, Port}, + {peer_host, PeerHost}, + {peer_port, PeerPort}, + {user, UsernameBin}, + {vhost, VHost}], + rabbit_core_metrics:connection_created(self(), Infos), + rabbit_event:notify(connection_created, Infos), + AuthState = #auth_state{user = User, + username = UsernameBin, + vhost = VHost, + authz_ctx = AuthzCtx}, + {ok, PState#proc_state{auth_state = AuthState}}; + not_allowed -> + rabbit_log_connection:warning( + "MQTT login failed for user ~s: " + "this user's access is restricted to localhost", + [binary_to_list(UsernameBin)]), + {error, ?CONNACK_NOT_AUTHORIZED} + end. + get_vhost(UserBin, none, Port) -> get_vhost_no_ssl(UserBin, Port); get_vhost(UserBin, undefined, Port) -> @@ -1060,7 +1116,6 @@ amqp_pub(#mqtt_msg{qos = Qos, % unacked_pubs = UnackedPubs, % awaiting_seqno = SeqNo, mqtt2amqp_fun = Mqtt2AmqpFun}) -> - %%TODO: Use message containers RoutingKey = Mqtt2AmqpFun(Topic), Confirm = Qos > ?QOS_0, Headers = [{<<"x-mqtt-publish-qos">>, byte, Qos}, @@ -1263,17 +1318,16 @@ check_publish(TopicName, Fn, PState) -> check_topic_access(TopicName, Access, #proc_state{ - auth_state = #auth_state{user = User = #user{username = Username}, - vhost = VHost}, - exchange = #resource{name = ExchangeBin}, - client_id = ClientId, - mqtt2amqp_fun = Mqtt2AmqpFun }) -> - Cache = - case get(topic_permission_cache) of - undefined -> []; - Other -> Other - end, - + auth_state = #auth_state{user = User = #user{username = Username}, + vhost = VHost, + authz_ctx = AuthzCtx}, + exchange = #resource{name = ExchangeBin}, + client_id = ClientId, + mqtt2amqp_fun = Mqtt2AmqpFun}) -> + Cache = case get(topic_permission_cache) of + undefined -> []; + Other -> Other + end, Key = {TopicName, Username, ClientId, VHost, ExchangeBin, Access}, case lists:member(Key, Cache) of true -> @@ -1285,13 +1339,7 @@ check_topic_access(TopicName, Access, RoutingKey = Mqtt2AmqpFun(TopicName), Context = #{routing_key => RoutingKey, - variable_map => #{ - <<"username">> => Username, - <<"vhost">> => VHost, - <<"client_id">> => rabbit_data_coercion:to_binary(ClientId) - } - }, - + variable_map => AuthzCtx}, try rabbit_access_control:check_topic_access(User, Resource, Access, Context) of ok -> CacheTail = lists:sublist(Cache, ?MAX_PERMISSION_CACHE_SIZE - 1), diff --git a/deps/rabbitmq_mqtt/test/auth_SUITE.erl b/deps/rabbitmq_mqtt/test/auth_SUITE.erl index 3ce31569880e..35d459077d0b 100644 --- a/deps/rabbitmq_mqtt/test/auth_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/auth_SUITE.erl @@ -64,6 +64,9 @@ groups() -> {client_id_propagation, [], [client_id_propagation] }, + %%TODO check write access to exchange as done in channel + %% or is that covered in topic_write_permission already? + %% "Topic authorisation is an additional layer on top of existing checks for publishers. Publishing a message to a topic-typed exchange will go through both the basic.publish and the routing key checks. The latter is never called if the former refuses access." {authz_handling, [], [no_queue_bind_permission, no_queue_consume_permission, @@ -459,11 +462,12 @@ client_id_propagation(Config) -> end), %% the setup process will notify us receive - ok -> ok - after 3000 -> ok + ok -> ok + after + 3000 -> ct:fail("timeout waiting for rabbit_auth_backend_mqtt_mock:setup/1") end, ClientId = <<"client-id-propagation">>, - {ok, C} = connect_user(<<"client-id-propagation">>, <<"client-id-propagation">>, + {ok, C} = connect_user(<<"fake-user">>, <<"fake-password">>, Config, ClientId), {ok, _} = emqtt:connect(C), {ok, _, _} = emqtt:subscribe(C, <<"TopicA">>), diff --git a/erlang.mk b/erlang.mk index 763c6b6ea8f5..4044d80555fe 100644 --- a/erlang.mk +++ b/erlang.mk @@ -5146,8 +5146,7 @@ ERLANG_MK_QUERY_SHELL_DEPS_FILE = $(ERLANG_MK_TMP)/query-shell-deps.log # Configuration. ERLC_OPTS ?= -Werror +debug_info +warn_export_vars +warn_shadow_vars \ - +warn_obsolete_guard "+{feature, maybe_expr, enable}" - # +bin_opt_info +warn_export_all +warn_missing_spec + +warn_obsolete_guard # +bin_opt_info +warn_export_all +warn_missing_spec COMPILE_FIRST ?= COMPILE_FIRST_PATHS = $(addprefix src/,$(addsuffix .erl,$(COMPILE_FIRST))) ERLC_EXCLUDE ?= From 77da78f478402da4a25c72ddc2e6028cb7c1af5d Mon Sep 17 00:00:00 2001 From: David Ansari Date: Tue, 6 Sep 2022 15:56:01 +0200 Subject: [PATCH 006/118] Get most auth_SUITE tests green Some tests which require clean_start=false or QoS1 are skipped for now. Differentiate between v3 and v4: v4 allows for an error code in SUBACK frame. --- deps/rabbit/src/rabbit_access_control.erl | 10 +- deps/rabbitmq_ct_helpers/BUILD.bazel | 1 + deps/rabbitmq_ct_helpers/Makefile | 2 +- .../src/rabbit_ct_broker_helpers.erl | 8 + deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl | 3 +- .../include/rabbit_mqtt_frame.hrl | 5 +- deps/rabbitmq_mqtt/src/rabbit_mqtt_frame.erl | 28 +- .../src/rabbit_mqtt_processor.erl | 450 ++++++++-------- deps/rabbitmq_mqtt/test/auth_SUITE.erl | 482 ++++++++++-------- 9 files changed, 551 insertions(+), 438 deletions(-) diff --git a/deps/rabbit/src/rabbit_access_control.erl b/deps/rabbit/src/rabbit_access_control.erl index c47a5dd82391..c7aa589eb235 100644 --- a/deps/rabbit/src/rabbit_access_control.erl +++ b/deps/rabbit/src/rabbit_access_control.erl @@ -18,7 +18,7 @@ -export_type([permission_atom/0]). --type permission_atom() :: 'configure' | 'read' | 'write'. +-type permission_atom() :: 'configure' | 'write' | 'read'. %%---------------------------------------------------------------------------- @@ -194,8 +194,8 @@ check_resource_access(User = #user{username = Username, check_access( fun() -> Module:check_resource_access( auth_user(User, Impl), Resource, Permission, Context) end, - Module, "access to ~ts refused for user '~ts'", - [rabbit_misc:rs(Resource), Username]); + Module, "~s access to ~s refused for user '~s'", + [Permission, rabbit_misc:rs(Resource), Username]); (_, Else) -> Else end, ok, Modules). @@ -207,8 +207,8 @@ check_topic_access(User = #user{username = Username, check_access( fun() -> Module:check_topic_access( auth_user(User, Impl), Resource, Permission, Context) end, - Module, "access to topic '~ts' in exchange ~ts refused for user '~ts'", - [maps:get(routing_key, Context), rabbit_misc:rs(Resource), Username]); + Module, "~s access to topic '~s' in exchange ~s refused for user '~s'", + [Permission, maps:get(routing_key, Context), rabbit_misc:rs(Resource), Username]); (_, Else) -> Else end, ok, Modules). diff --git a/deps/rabbitmq_ct_helpers/BUILD.bazel b/deps/rabbitmq_ct_helpers/BUILD.bazel index 5c46b301f0b1..f323291a06a9 100644 --- a/deps/rabbitmq_ct_helpers/BUILD.bazel +++ b/deps/rabbitmq_ct_helpers/BUILD.bazel @@ -10,5 +10,6 @@ erlang_app( deps = [ "//deps/rabbit_common:erlang_app", "@proper//:erlang_app", + "@meck//:erlang_app", ], ) diff --git a/deps/rabbitmq_ct_helpers/Makefile b/deps/rabbitmq_ct_helpers/Makefile index d81e011ae6fb..5dc6e160bd7b 100644 --- a/deps/rabbitmq_ct_helpers/Makefile +++ b/deps/rabbitmq_ct_helpers/Makefile @@ -1,7 +1,7 @@ PROJECT = rabbitmq_ct_helpers PROJECT_DESCRIPTION = Common Test helpers for RabbitMQ -DEPS = rabbit_common proper inet_tcp_proxy +DEPS = rabbit_common proper inet_tcp_proxy meck TEST_DEPS = rabbit dep_rabbit_common = git-subfolder https://github.com/rabbitmq/rabbitmq-server main deps/rabbit_common diff --git a/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl b/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl index da3b2d9ad24c..7c957c80d4a1 100644 --- a/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl +++ b/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl @@ -25,6 +25,8 @@ rewrite_node_config_file/2, cluster_nodes/1, cluster_nodes/2, + setup_meck/1, + get_node_configs/1, get_node_configs/2, get_node_config/2, get_node_config/3, set_node_config/3, nodename_to_index/2, @@ -1946,3 +1948,9 @@ if_cover(F) -> _ -> F() end. + +setup_meck(Config) -> + {Mod, Bin, File} = code:get_object_code(meck), + [true | _] = rpc_all(Config, code, add_path, [filename:dirname(File)]), + [{module, Mod} | _] = rpc_all(Config, code, load_binary, [Mod, File, Bin]), + ok. diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl index 5c687d3de3e0..e6621e33b1a3 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl @@ -51,6 +51,7 @@ mqtt2amqp_fun, amqp2mqtt_fun, register_state, + proto_ver :: 3 | 4, info}). -record(auth_state, {username, @@ -63,7 +64,7 @@ port, peer_host, peer_port, - protocol}). + proto_human}). %% does not include vhost: it is used in %% the table name diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt_frame.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt_frame.hrl index 06d9c2f75c56..57a3a5fe6281 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt_frame.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt_frame.hrl @@ -5,7 +5,9 @@ %% Copyright (c) 2020-2023 VMware, Inc. or its affiliates. All rights reserved. %% --define(PROTOCOL_NAMES, [{3, "MQIsdp"}, {4, "MQTT"}]). +-define(MQTT_PROTO_V3, 3). +-define(MQTT_PROTO_V4, 4). +-define(PROTOCOL_NAMES, [{?MQTT_PROTO_V3, "MQIsdp"}, {?MQTT_PROTO_V4, "MQTT"}]). %% frame types @@ -44,6 +46,7 @@ -define(QOS_0, 0). -define(QOS_1, 1). -define(QOS_2, 2). +-define(SUBACK_FAILURE, 16#80). %% TODO -type message_id() :: any(). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_frame.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_frame.erl index b79d0bc45857..62a96cb11c60 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_frame.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_frame.erl @@ -8,7 +8,7 @@ -module(rabbit_mqtt_frame). -export([parse/2, initial_state/0]). --export([serialise/1]). +-export([serialise/2]). -include("rabbit_mqtt_frame.hrl"). @@ -146,10 +146,10 @@ bool(1) -> true. %% serialisation -serialise(#mqtt_frame{ fixed = Fixed, - variable = Variable, - payload = Payload }) -> - serialise_variable(Fixed, Variable, serialise_payload(Payload)). +serialise(#mqtt_frame{fixed = Fixed, + variable = Variable, + payload = Payload}, Vsn) -> + serialise_variable(Fixed, Variable, serialise_payload(Payload), Vsn). serialise_payload(undefined) -> <<>>; serialise_payload(B) when is_binary(B) -> B. @@ -157,24 +157,30 @@ serialise_payload(B) when is_binary(B) -> B. serialise_variable(#mqtt_frame_fixed { type = ?CONNACK } = Fixed, #mqtt_frame_connack { session_present = SessionPresent, return_code = ReturnCode }, - <<>> = PayloadBin) -> + <<>> = PayloadBin, _Vsn) -> VariableBin = <>, serialise_fixed(Fixed, VariableBin, PayloadBin); serialise_variable(#mqtt_frame_fixed { type = SubAck } = Fixed, #mqtt_frame_suback { message_id = MessageId, qos_table = Qos }, - <<>> = _PayloadBin) + <<>> = _PayloadBin, Vsn) when SubAck =:= ?SUBACK orelse SubAck =:= ?UNSUBACK -> VariableBin = <>, - QosBin = << <> || Q <- Qos >>, + QosBin = case Vsn of + ?MQTT_PROTO_V3 -> + << <> || Q <- Qos >>; + ?MQTT_PROTO_V4 -> + %% Allow error code (0x80) in the MQTT SUBACK message. + << <> || Q <- Qos >> + end, serialise_fixed(Fixed, VariableBin, QosBin); serialise_variable(#mqtt_frame_fixed { type = ?PUBLISH, qos = Qos } = Fixed, #mqtt_frame_publish { topic_name = TopicName, message_id = MessageId }, - PayloadBin) -> + PayloadBin, _Vsn) -> TopicBin = serialise_utf(TopicName), MessageIdBin = case Qos of 0 -> <<>>; @@ -184,13 +190,13 @@ serialise_variable(#mqtt_frame_fixed { type = ?PUBLISH, serialise_variable(#mqtt_frame_fixed { type = ?PUBACK } = Fixed, #mqtt_frame_publish { message_id = MessageId }, - PayloadBin) -> + PayloadBin, _Vsn) -> MessageIdBin = <>, serialise_fixed(Fixed, MessageIdBin, PayloadBin); serialise_variable(#mqtt_frame_fixed {} = Fixed, undefined, - <<>> = _PayloadBin) -> + <<>> = _PayloadBin, _Vsn) -> serialise_fixed(Fixed, <<>>, <<>>). serialise_fixed(#mqtt_frame_fixed{ type = Type, diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 794ee0d2e58f..8f066285d836 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -55,20 +55,13 @@ process_frame(#mqtt_frame{ fixed = #mqtt_frame_fixed{ type = Type }}, PState = #proc_state{ auth_state = undefined } ) when Type =/= ?CONNECT -> {error, connect_expected, PState}; -process_frame(Frame = #mqtt_frame{ fixed = #mqtt_frame_fixed{ type = Type }}, +process_frame(Frame = #mqtt_frame{fixed = #mqtt_frame_fixed{ type = Type }}, PState) -> - try process_request(Type, Frame, PState) of - {ok, PState1} -> {ok, PState1, PState1#proc_state.connection}; - Ret -> Ret - catch - _:{{shutdown, {server_initiated_close, 403, _}}, _} -> - %% NB: MQTT spec says we should ack normally, ie pretend - %% there was no auth error, but here we are closing the - %% connection with an error. This is what happens anyway - %% if there is an authorization failure at the AMQP 0-9-1 - %% client level. And error was already logged by AMQP - %% channel, so no need for custom logging. - {error, access_refused, PState} + case process_request(Type, Frame, PState) of + {ok, PState1} -> + {ok, PState1, PState1#proc_state.connection}; + Ret -> + Ret end. process_request(?CONNECT, Frame, PState = #proc_state{socket = Socket}) -> @@ -116,61 +109,77 @@ process_request(?PUBLISH, payload = Payload }, PState = #proc_state{retainer_pid = RPid, amqp2mqtt_fun = Amqp2MqttFun}) -> - check_publish(Topic, fun() -> - Msg = #mqtt_msg{retain = Retain, - qos = Qos, - topic = Topic, - dup = Dup, - message_id = MessageId, - payload = Payload}, - Result = amqp_pub(Msg, PState), - case Retain of - false -> ok; - true -> hand_off_to_retainer(RPid, Amqp2MqttFun, Topic, Msg) - end, - {ok, Result} - end, PState); + publish( + Topic, + fun() -> + Msg = #mqtt_msg{retain = Retain, + qos = Qos, + topic = Topic, + dup = Dup, + message_id = MessageId, + payload = Payload}, + Result = amqp_pub(Msg, PState), + case Retain of + false -> ok; + true -> hand_off_to_retainer(RPid, Amqp2MqttFun, Topic, Msg) + end, + {ok, Result} + end, PState); process_request(?SUBSCRIBE, #mqtt_frame{ - variable = #mqtt_frame_subscribe{ - message_id = SubscribeMsgId, - topic_table = Topics}, - payload = undefined}, + variable = #mqtt_frame_subscribe{ + message_id = SubscribeMsgId, + topic_table = Topics}, + payload = undefined}, #proc_state{retainer_pid = RPid, send_fun = SendFun, message_id = StateMsgId} = PState0) -> rabbit_log_connection:debug("Received a SUBSCRIBE for topic(s) ~p", [Topics]), - {QosResponse, PState1} = - lists:foldl(fun (#mqtt_topic{name = TopicName, - qos = Qos}, {QosList, S0}) -> - SupportedQos = supported_subs_qos(Qos), - {QueueName, #proc_state{subscriptions = Subs} = S} = - ensure_queue(SupportedQos, S0), - bind(QueueName, TopicName, S), - SupportedQosList = case maps:find(TopicName, Subs) of - {ok, L} -> [SupportedQos|L]; - error -> [SupportedQos] - end, - {[SupportedQos | QosList], - S#proc_state{subscriptions = maps:put(TopicName, SupportedQosList, Subs)}} - end, {[], PState0}, Topics), + {QosResponse, PState} = + lists:foldl(fun(#mqtt_topic{name = TopicName, + qos = Qos}, {QosList, S0}) -> + SupportedQos = supported_subs_qos(Qos), + case ensure_queue(SupportedQos, S0) of + {ok, QueueName, #proc_state{subscriptions = Subs} = S1} -> + case bind(QueueName, TopicName, S1) of + {ok, _Output, S2} -> + SupportedQosList = case maps:find(TopicName, Subs) of + {ok, L} -> [SupportedQos|L]; + error -> [SupportedQos] + end, + {[SupportedQos | QosList], + S2#proc_state{subscriptions = maps:put(TopicName, SupportedQosList, Subs)}}; + {error, Reason, S2} -> + rabbit_log:error("Failed to bind ~s with topic ~s: ~p", + [rabbit_misc:rs(QueueName), TopicName, Reason]), + {[?SUBACK_FAILURE | QosList], S2} + end; + {error, _Reason} -> + {[?SUBACK_FAILURE | QosList], S0} + end + end, {[], PState0}, Topics), SendFun(#mqtt_frame{fixed = #mqtt_frame_fixed{type = ?SUBACK}, variable = #mqtt_frame_suback{ - message_id = SubscribeMsgId, - qos_table = QosResponse}}, PState1), - %% we may need to send up to length(Topics) messages. - %% if QoS is > 0 then we need to generate a message id, - %% and increment the counter. - StartMsgId = safe_max_id(SubscribeMsgId, StateMsgId), - N = lists:foldl(fun (Topic, Acc) -> - case maybe_send_retained_message(RPid, Topic, Acc, PState1) of - {true, X} -> Acc + X; - false -> Acc - end - end, StartMsgId, Topics), - {ok, PState1#proc_state{message_id = N}}; + message_id = SubscribeMsgId, + qos_table = QosResponse}}, PState), + case lists:member(?SUBACK_FAILURE, QosResponse) of + false -> + %% we may need to send up to length(Topics) messages. + %% if QoS is > 0 then we need to generate a message id, + %% and increment the counter. + StartMsgId = safe_max_id(SubscribeMsgId, StateMsgId), + N = lists:foldl(fun (Topic, Acc) -> + case maybe_send_retained_message(RPid, Topic, Acc, PState) of + {true, X} -> Acc + X; + false -> Acc + end + end, StartMsgId, Topics), + {ok, PState#proc_state{message_id = N}}; + true -> + {error, subscribe_error, PState} + end; process_request(?UNSUBSCRIBE, #mqtt_frame{ @@ -321,18 +330,20 @@ register_client(Frame = #mqtt_frame_connect{ {ok, {PeerHost, PeerPort, Host, Port}} = rabbit_net:socket_ends(Socket, inbound), ExchangeBin = rabbit_mqtt_util:env(exchange), ExchangeName = rabbit_misc:r(VHost, exchange, ExchangeBin), - Protocol = {'MQTT', human_readable_mqtt_version(ProtoVersion)}, + ProtoHumanReadable = {'MQTT', human_readable_mqtt_version(ProtoVersion)}, PState = PState0#proc_state{ exchange = ExchangeName, will_msg = make_will_msg(Frame), retainer_pid = RetainerPid, register_state = {pending, Corr}, + proto_ver = ProtoVersion, info = #info{prefetch = Prefetch, peer_host = PeerHost, peer_port = PeerPort, host = Host, port = Port, - protocol = Protocol}}, + proto_human = ProtoHumanReadable + }}, maybe_clean_sess(PState); {error, _} = Err -> %% e.g. this node was removed from the MQTT cluster members @@ -345,6 +356,11 @@ register_client(Frame = #mqtt_frame_connect{ {error, ?CONNACK_SERVER_UNAVAILABLE} end. +human_readable_mqtt_version(3) -> + <<"3.1.0">>; +human_readable_mqtt_version(4) -> + <<"3.1.1">>. + return_connack(?CONNACK_ACCEPT, S) -> {ok, S}; return_connack(?CONNACK_BAD_CREDENTIALS, S) -> @@ -359,7 +375,6 @@ return_connack(?CONNACK_UNACCEPTABLE_PROTO_VER, S) -> {error, unsupported_protocol_version, S}. maybe_clean_sess(PState = #proc_state {clean_sess = false, - connection = Conn, auth_state = #auth_state{vhost = VHost}, client_id = ClientId }) -> SessionPresent = session_present(VHost, ClientId), @@ -371,23 +386,14 @@ maybe_clean_sess(PState = #proc_state {clean_sess = false, %% will consume less resources. {ok, SessionPresent, PState}; true -> - try ensure_queue(?QOS_1, PState) of - {_Queue, PState1} -> - {ok, SessionPresent, PState1} - catch - exit:({{shutdown, {server_initiated_close, 403, _}}, _}) -> - %% Connection is not yet propagated to #proc_state{}, let's close it here - catch amqp_connection:close(Conn), - rabbit_log_connection:error("MQTT cannot recover a session, user is missing permissions"), - {error, ?CONNACK_SERVER_UNAVAILABLE}; - C:E:S -> - %% Connection is not yet propagated to - %% #proc_state{}, let's close it here. - %% This is an exceptional situation anyway, but - %% doing this will prevent second crash from - %% amqp client being logged. - catch amqp_connection:close(Conn), - erlang:raise(C, E, S) + case ensure_queue(?QOS_1, PState) of + {ok, _QueueName, PState1} -> + {ok, SessionPresent, PState1}; + {error, access_refused} -> + {error, ?CONNACK_NOT_AUTHORIZED}; + {error, _Reason} -> + %% Let's use most generic error return code. + {error, ?CONNACK_SERVER_UNAVAILABLE} end end; maybe_clean_sess(PState = #proc_state {clean_sess = true, @@ -402,22 +408,27 @@ maybe_clean_sess(PState = #proc_state {clean_sess = true, false -> {ok, false, PState}; true -> - ok = rabbit_access_control:check_resource_access(User, Queue, configure, AuthzCtx), - rabbit_amqqueue:with( - Queue, - fun (Q) -> - rabbit_queue_type:delete(Q, false, false, Username) - end, - fun (not_found) -> - ok; - ({absent, Q, crashed}) -> - rabbit_classic_queue:delete_crashed(Q, Username); - ({absent, Q, stopped}) -> - rabbit_classic_queue:delete_crashed(Q, Username); - ({absent, _Q, _Reason}) -> - ok - end), - {ok, false, PState} + %% configure access to queue required for queue.delete + case check_resource_access(User, Queue, configure, AuthzCtx) of + ok -> + rabbit_amqqueue:with( + Queue, + fun (Q) -> + rabbit_queue_type:delete(Q, false, false, Username) + end, + fun (not_found) -> + ok; + ({absent, Q, crashed}) -> + rabbit_classic_queue:delete_crashed(Q, Username); + ({absent, Q, stopped}) -> + rabbit_classic_queue:delete_crashed(Q, Username); + ({absent, _Q, _Reason}) -> + ok + end), + {ok, false, PState}; + {error, access_refused} -> + {error, ?CONNACK_NOT_AUTHORIZED} + end end. hand_off_to_retainer(RetainerPid, Amqp2MqttFun, Topic0, #mqtt_msg{payload = <<"">>}) -> @@ -768,8 +779,7 @@ check_user_loopback(#{vhost := VHost, {ok, PState#proc_state{auth_state = AuthState}}; not_allowed -> rabbit_log_connection:warning( - "MQTT login failed for user ~s: " - "this user's access is restricted to localhost", + "MQTT login failed: user '~s' can only connect via localhost", [binary_to_list(UsernameBin)]), {error, ?CONNACK_NOT_AUTHORIZED} end. @@ -997,82 +1007,113 @@ ensure_queue(?QOS_0, %% spike handles only QoS0 QueueName = rabbit_misc:r(VHost, queue, QueueBin), case rabbit_amqqueue:exists(QueueName) of true -> - {QueueName, PState0}; + {ok, QueueName, PState0}; false -> - check_resource_access(User, QueueName, read, AuthzCtx), - check_resource_access(User, QueueName, configure, AuthzCtx), - rabbit_core_metrics:queue_declared(QueueName), - Durable = false, - AutoDelete = true, - case rabbit_amqqueue:with( - QueueName, - fun (Q) -> ok = rabbit_amqqueue:assert_equivalence( - Q, Durable, AutoDelete, [], none), - rabbit_amqqueue:stat(Q) - end) of - {error, not_found} -> - case rabbit_vhost_limit:is_over_queue_limit(VHost) of - false -> - case rabbit_amqqueue:declare(QueueName, Durable, AutoDelete, - [], none, Username) of - {new, Q} when ?is_amqqueue(Q) -> - rabbit_core_metrics:queue_created(QueueName), - Spec = #{no_ack => true, - channel_pid => self(), - limiter_pid => none, - limiter_active => false, - prefetch_count => Prefetch, - consumer_tag => ?CONSUMER_TAG, - exclusive_consume => false, - args => [], - ok_msg => undefined, - acting_user => Username}, - case rabbit_queue_type:consume(Q, Spec, QueueStates0) of - {ok, QueueStates, _Actions = []} -> - % rabbit_global_counters:consumer_created(mqtt), - PState = PState0#proc_state{queue_states = QueueStates}, - {QueueName, PState}; - Other -> - exit( - lists:flatten( - io_lib:format("Failed to consume from ~s: ~p", - [rabbit_misc:rs(QueueName), Other]))) + %% read access to queue required for basic.consume + case check_resource_access(User, QueueName, read, AuthzCtx) of + ok -> + %% configure access to queue required for queue.declare + case check_resource_access(User, QueueName, configure, AuthzCtx) of + ok -> + rabbit_core_metrics:queue_declared(QueueName), + Durable = false, + AutoDelete = true, + case rabbit_amqqueue:with( + QueueName, + fun (Q) -> ok = rabbit_amqqueue:assert_equivalence( + Q, Durable, AutoDelete, [], none), + rabbit_amqqueue:stat(Q) + end) of + {error, not_found} -> + case rabbit_vhost_limit:is_over_queue_limit(VHost) of + false -> + case rabbit_amqqueue:declare(QueueName, Durable, AutoDelete, + [], none, Username) of + {new, Q} when ?is_amqqueue(Q) -> + rabbit_core_metrics:queue_created(QueueName), + Spec = #{no_ack => true, + channel_pid => self(), + limiter_pid => none, + limiter_active => false, + prefetch_count => Prefetch, + consumer_tag => ?CONSUMER_TAG, + exclusive_consume => false, + args => [], + ok_msg => undefined, + acting_user => Username}, + case rabbit_queue_type:consume(Q, Spec, QueueStates0) of + {ok, QueueStates, _Actions = []} -> + % rabbit_global_counters:consumer_created(mqtt), + PState = PState0#proc_state{queue_states = QueueStates}, + {ok, QueueName, PState}; + Other -> + rabbit_log:error("Failed to consume from ~s: ~p", + [rabbit_misc:rs(QueueName), Other]), + {error, queue_consume} + end; + Other -> + rabbit_log:error("Failed to declare ~s: ~p", + [rabbit_misc:rs(QueueName), Other]), + {error, queue_declare} + end; + {true, Limit} -> + rabbit_log:error("cannot declare ~s because " + "queue limit ~p in vhost '~s' is reached", + [rabbit_misc:rs(QueueName), Limit, VHost]), + {error, access_refused} end; Other -> - exit(lists:flatten( - io_lib:format("Failed to declare ~s: ~p", - [rabbit_misc:rs(QueueName), Other]))) + rabbit_log:error("Expected ~s to not exist, got: ~p", + [rabbit_misc:rs(QueueName), Other]), + {error, queue_access} end; - {true, Limit} -> - exit( - lists:flatten( - io_lib:format("cannot declare ~s because " - "queue limit ~p in vhost '~s' is reached", - [rabbit_misc:rs(QueueName), Limit, VHost]))) + {error, access_refused} = E -> + E end; - Other -> - exit( - lists:flatten( - io_lib:format("Expected ~s to not exist, got: ~p", - [rabbit_misc:rs(QueueName), Other]))) + {error, access_refused} = E -> + E end end. -bind(QueueName, - TopicName, - #proc_state{exchange = ExchangeName, - auth_state = #auth_state{ - user = User = #user{username = Username}, - authz_ctx = AuthzCtx}, - mqtt2amqp_fun = Mqtt2AmqpFun} = PState) -> - ok = rabbit_access_control:check_resource_access(User, QueueName, write, AuthzCtx), - ok = rabbit_access_control:check_resource_access(User, ExchangeName, read, AuthzCtx), - ok = check_topic_access(TopicName, read, PState), +bind(QueueName, TopicName, PState) -> + rabbit_misc:pipeline( + [fun bind_check_queue_write_access/2, + fun bind_check_exchange_read_access/2, + fun bind_check_topic_access/2, + fun bind_add/2 + ], {QueueName, TopicName}, PState). + +bind_check_queue_write_access( + {QueueName, _TopicName}, + #proc_state{auth_state = #auth_state{ + user = User, + authz_ctx = AuthzCtx}}) -> + %% write access to queue required for queue.bind + check_resource_access(User, QueueName, write, AuthzCtx). + +bind_check_exchange_read_access( + {_QueueName, _TopicName}, + #proc_state{exchange = ExchangeName, + auth_state = #auth_state{ + user = User, + authz_ctx = AuthzCtx}}) -> + %% read access to exchange required for queue.bind + check_resource_access(User, ExchangeName, read, AuthzCtx). + +bind_check_topic_access( {_QueueName, TopicName}, PState) -> + check_topic_access(TopicName, read, PState). + +bind_add( + {QueueName, TopicName}, + #proc_state{exchange = ExchangeName, + auth_state = #auth_state{ + user = #user{username = Username}}, + mqtt2amqp_fun = Mqtt2AmqpFun}) -> RoutingKey = Mqtt2AmqpFun(TopicName), Binding = #binding{source = ExchangeName, destination = QueueName, key = RoutingKey}, - ok = rabbit_binding:add(Binding, Username). + rabbit_binding:add(Binding, Username). send_will(PState = #proc_state{will_msg = undefined}) -> PState; @@ -1183,17 +1224,10 @@ deliver_to_queues(Delivery = #delivery{message = _Message = #basic_message{ex % State = handle_queue_actions(Actions, State1#ch{queue_states = QueueStates}), PState#proc_state{queue_states = QueueStates}. -human_readable_mqtt_version(3) -> - "3.1.0"; -human_readable_mqtt_version(4) -> - "3.1.1"; -human_readable_mqtt_version(_) -> - "N/A". - -serialise_and_send_to_client(Frame, #proc_state{ socket = Sock }) -> +serialise_and_send_to_client(Frame, #proc_state{proto_ver = ProtoVer, socket = Sock }) -> %%TODO Test sending large frames at high speed: Will we need garbage collection as done %% in rabbit_writer:maybe_gc_large_msg()? - try rabbit_net:port_command(Sock, rabbit_mqtt_frame:serialise(Frame)) of + try rabbit_net:port_command(Sock, rabbit_mqtt_frame:serialise(Frame, ProtoVer)) of Res -> Res catch _:Error -> @@ -1222,8 +1256,8 @@ close_connection(PState = #proc_state{ connection = Connection, connection = undefined }. handle_pre_hibernate() -> - erase(topic_permission_cache), erase(permission_cache), + erase(topic_permission_cache), ok. handle_ra_event({applied, [{Corr, ok}]}, @@ -1309,12 +1343,42 @@ handle_deliver0({_QName, _QPid, _MsgId, Redelivered, SendFun(Frame, PState), PState. -check_publish(TopicName, Fn, PState) -> - %%TODO check additionally write access to exchange as done in channel? - case check_topic_access(TopicName, write, PState) of - ok -> Fn(); - _ -> {error, unauthorized, PState} - end. +publish(TopicName, PublishFun, + PState = #proc_state{exchange = Exchange, + auth_state = #auth_state{user = User, + authz_ctx = AuthzCtx}}) -> + case check_resource_access(User, Exchange, write, AuthzCtx) of + ok -> + case check_topic_access(TopicName, write, PState) of + ok -> + PublishFun(); + {error, access_refused} -> + {error, unauthorized, PState} + end; + {error, access_refused} -> + {error, unauthorized, PState} + end. + +check_resource_access(User, Resource, Perm, Context) -> + V = {Resource, Context, Perm}, + Cache = case get(permission_cache) of + undefined -> []; + Other -> Other + end, + case lists:member(V, Cache) of + true -> + ok; + false -> + try rabbit_access_control:check_resource_access(User, Resource, Perm, Context) of + ok -> + CacheTail = lists:sublist(Cache, ?MAX_PERMISSION_CACHE_SIZE-1), + put(permission_cache, [V | CacheTail]), + ok + catch exit:{amqp_error, access_refused, Msg, _AmqpMethod} -> + rabbit_log:error("MQTT resource access refused: ~s", [Msg]), + {error, access_refused} + end + end. check_topic_access(TopicName, Access, #proc_state{ @@ -1336,7 +1400,6 @@ check_topic_access(TopicName, Access, Resource = #resource{virtual_host = VHost, kind = topic, name = ExchangeBin}, - RoutingKey = Mqtt2AmqpFun(TopicName), Context = #{routing_key => RoutingKey, variable_map => AuthzCtx}, @@ -1344,34 +1407,14 @@ check_topic_access(TopicName, Access, ok -> CacheTail = lists:sublist(Cache, ?MAX_PERMISSION_CACHE_SIZE - 1), put(topic_permission_cache, [Key | CacheTail]), - ok; - R -> - R + ok catch - _:{amqp_error, access_refused, Msg, _} -> - rabbit_log:error("operation resulted in an error (access_refused): ~tp", [Msg]), - {error, access_refused}; - _:Error -> - rabbit_log:error("~tp", [Error]), + exit:{amqp_error, access_refused, Msg, _AmqpMethod} -> + rabbit_log:error("MQTT topic access refused: ~s", [Msg]), {error, access_refused} end end. -%% TODO copied from channel, remove duplication -check_resource_access(User, Resource, Perm, Context) -> - V = {Resource, Context, Perm}, - Cache = case get(permission_cache) of - undefined -> []; - Other -> Other - end, - case lists:member(V, Cache) of - true -> ok; - false -> ok = rabbit_access_control:check_resource_access( - User, Resource, Perm, Context), - CacheTail = lists:sublist(Cache, ?MAX_PERMISSION_CACHE_SIZE-1), - put(permission_cache, [V | CacheTail]) - end. - info(consumer_tags, #proc_state{consumer_tags = Val}) -> Val; info(unacked_pubs, #proc_state{unacked_pubs = Val}) -> Val; info(awaiting_ack, #proc_state{awaiting_ack = Val}) -> Val; @@ -1391,11 +1434,7 @@ info(host, #proc_state{info = #info{host = Val}}) -> Val; info(port, #proc_state{info = #info{port = Val}}) -> Val; info(peer_host, #proc_state{info = #info{peer_host = Val}}) -> Val; info(peer_port, #proc_state{info = #info{peer_port = Val}}) -> Val; -info(protocol, #proc_state{info = #info{protocol = Val}}) -> - case Val of - {Proto, Version} -> {Proto, rabbit_data_coercion:to_binary(Version)}; - Other -> Other - end; +info(protocol, #proc_state{info = #info{proto_human = Val}}) -> Val; % info(channels, PState) -> additional_info(channels, PState); % info(channel_max, PState) -> additional_info(channel_max, PState); % info(frame_max, PState) -> additional_info(frame_max, PState); @@ -1407,7 +1446,6 @@ info(protocol, #proc_state{info = #info{protocol = Val}}) -> % info(ssl_hash, PState) -> additional_info(ssl_hash, PState); info(Other, _) -> throw({bad_argument, Other}). - % additional_info(Key, % #proc_state{adapter_info = % #amqp_adapter_info{additional_info = AddInfo}}) -> diff --git a/deps/rabbitmq_mqtt/test/auth_SUITE.erl b/deps/rabbitmq_mqtt/test/auth_SUITE.erl index 35d459077d0b..7c2be0a75bf3 100644 --- a/deps/rabbitmq_mqtt/test/auth_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/auth_SUITE.erl @@ -11,16 +11,18 @@ -include_lib("common_test/include/ct.hrl"). -include_lib("eunit/include/eunit.hrl"). -define(CONNECT_TIMEOUT, 10000). --define(WAIT_LOG_NO_CRASHES, {["Generic server.*terminating"], fun () -> exit(there_should_be_no_crashes) end}). +-define(SUBACK_FAILURE, 16#80). +-define(FAIL_IF_CRASH_LOG, {["Generic server.*terminating"], + fun () -> ct:fail(crash_detected) end}). +-import(rabbit_ct_broker_helpers, [rpc/5]). -%%TODO check for loopback user, i.e. that rabbit_access_control:check_user_loopback/2 is called. all() -> [{group, anonymous_no_ssl_user}, {group, anonymous_ssl_user}, {group, no_ssl_user}, {group, ssl_user}, {group, client_id_propagation}, - {group, authz_handling}]. + {group, authz}]. groups() -> [{anonymous_ssl_user, [], @@ -64,17 +66,17 @@ groups() -> {client_id_propagation, [], [client_id_propagation] }, - %%TODO check write access to exchange as done in channel - %% or is that covered in topic_write_permission already? - %% "Topic authorisation is an additional layer on top of existing checks for publishers. Publishing a message to a topic-typed exchange will go through both the basic.publish and the routing key checks. The latter is never called if the former refuses access." - {authz_handling, [], + {authz, [], [no_queue_bind_permission, no_queue_consume_permission, no_queue_consume_permission_on_connect, no_queue_delete_permission, no_queue_declare_permission, + no_publish_permission, no_topic_read_permission, - no_topic_write_permission] + no_topic_write_permission, + loopback_user_connects_from_remote_host + ] } ]. @@ -85,7 +87,7 @@ init_per_suite(Config) -> end_per_suite(Config) -> Config. -init_per_group(authz_handling, Config0) -> +init_per_group(authz, Config0) -> User = <<"mqtt-user">>, Password = <<"mqtt-password">>, VHost = <<"mqtt-vhost">>, @@ -100,7 +102,7 @@ init_per_group(authz_handling, Config0) -> rabbit_ct_client_helpers:setup_steps()), rabbit_ct_broker_helpers:add_user(Config1, User, Password), rabbit_ct_broker_helpers:add_vhost(Config1, VHost), - [Log|_] = rabbit_ct_broker_helpers:rpc(Config1, 0, rabbit, log_locations, []), + [Log|_] = rpc(Config1, 0, rabbit, log_locations, []), [{mqtt_user, User}, {mqtt_vhost, VHost}, {mqtt_password, Password}, {log_location, Log}|Config1]; init_per_group(Group, Config) -> Suffix = rabbit_ct_helpers:testcase_absname(Config, "", "-"), @@ -237,10 +239,7 @@ set_cert_user_on_default_vhost(Config) -> CertFile = filename:join([CertsDir, "client", "cert.pem"]), {ok, CertBin} = file:read_file(CertFile), [{'Certificate', Cert, not_encrypted}] = public_key:pem_decode(CertBin), - UserBin = rabbit_ct_broker_helpers:rpc(Config, 0, - rabbit_ssl, - peer_cert_auth_name, - [Cert]), + UserBin = rpc(Config, 0, rabbit_ssl, peer_cert_auth_name, [Cert]), User = binary_to_list(UserBin), ok = rabbit_ct_broker_helpers:add_user(Config, 0, User, ""), ok = rabbit_ct_broker_helpers:set_full_permissions(Config, User, <<"/">>), @@ -322,13 +321,14 @@ end_per_testcase(Testcase, Config) when Testcase == no_queue_bind_permission; Testcase == no_queue_consume_permission_on_connect; Testcase == no_queue_delete_permission; Testcase == no_queue_declare_permission; + Testcase == no_publish_permission; Testcase == no_topic_read_permission; - Testcase == no_topic_write_permission -> + Testcase == no_topic_write_permission; + Testcase == loopback_user_connects_from_remote_host -> %% So let's wait before logs are surely flushed Marker = "MQTT_AUTH_SUITE_MARKER", - rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_log, error, [Marker]), - wait_log(Config, erlang:system_time(microsecond) + 1_000_000, - [{[Marker], fun () -> stop end}]), + rpc(Config, 0, rabbit_log, error, [Marker]), + wait_log(Config, [{[Marker], fun () -> stop end}]), %% Preserve file contents in case some investigation is needed, before truncating. file:copy(?config(log_location, Config), iolist_to_binary([?config(log_location, Config), ".", atom_to_binary(Testcase)])), @@ -455,11 +455,10 @@ client_id_propagation(Config) -> %% setup creates the ETS table required for the mqtt auth mock %% it blocks indefinitely so we need to spawn Self = self(), - _ = spawn(fun () -> - rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_auth_backend_mqtt_mock, - setup, - [Self]) - end), + _ = spawn( + fun () -> + rpc(Config, 0, rabbit_auth_backend_mqtt_mock, setup, [Self]) + end), %% the setup process will notify us receive ok -> ok @@ -471,29 +470,29 @@ client_id_propagation(Config) -> Config, ClientId), {ok, _} = emqtt:connect(C), {ok, _, _} = emqtt:subscribe(C, <<"TopicA">>), - [{authentication, AuthProps}] = rabbit_ct_broker_helpers:rpc(Config, 0, - rabbit_auth_backend_mqtt_mock, - get, - [authentication]), + [{authentication, AuthProps}] = rpc(Config, 0, + rabbit_auth_backend_mqtt_mock, + get, + [authentication]), ?assertEqual(ClientId, proplists:get_value(client_id, AuthProps)), - [{vhost_access, AuthzData}] = rabbit_ct_broker_helpers:rpc(Config, 0, - rabbit_auth_backend_mqtt_mock, - get, - [vhost_access]), + [{vhost_access, AuthzData}] = rpc(Config, 0, + rabbit_auth_backend_mqtt_mock, + get, + [vhost_access]), ?assertEqual(ClientId, maps:get(<<"client_id">>, AuthzData)), - [{resource_access, AuthzContext}] = rabbit_ct_broker_helpers:rpc(Config, 0, - rabbit_auth_backend_mqtt_mock, - get, - [resource_access]), + [{resource_access, AuthzContext}] = rpc(Config, 0, + rabbit_auth_backend_mqtt_mock, + get, + [resource_access]), ?assertEqual(true, maps:size(AuthzContext) > 0), ?assertEqual(ClientId, maps:get(<<"client_id">>, AuthzContext)), - [{topic_access, TopicContext}] = rabbit_ct_broker_helpers:rpc(Config, 0, - rabbit_auth_backend_mqtt_mock, - get, - [topic_access]), + [{topic_access, TopicContext}] = rpc(Config, 0, + rabbit_auth_backend_mqtt_mock, + get, + [topic_access]), VariableMap = maps:get(variable_map, TopicContext), ?assertEqual(ClientId, maps:get(<<"client_id">>, VariableMap)), @@ -503,108 +502,129 @@ client_id_propagation(Config) -> %% table in https://www.rabbitmq.com/access-control.html#authorisation %% and which MQTT plugin tries to perform. %% -%% Silly MQTT doesn't allow us to see any error codes in the protocol, -%% so the only non-intrusive way to check for `access_refused` +%% MQTT v4 has a single SUBACK error code but does not allow to differentiate +%% between different kind of errors why a subscription request failed. +%% The only non-intrusive way to check for `access_refused` %% codepath is by checking logs. Every testcase from this group %% truncates log file beforehand, so it'd be easier to analyze. There -%% is additional wait in the corresponding end_per_testcase that -%% ensures that logs were for the current testcase were completely +%% is an additional wait in the corresponding end_per_testcase that +%% ensures that logs for the current testcase were completely %% flushed, and won't contaminate following tests from this group. -%% -%% Then each test-case asserts that logs contain following things: -%% 1) Handling of access_refused error handler in MQTT reader: -%% https://github.com/rabbitmq/rabbitmq-server/blob/69dc53fb8938c7f135bf0002b0904cf28c25c571/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl#L332 -%% 2) Mention of which AMQP operation caused that error (that one is -%% kinda superflous, it just makes sure that every AMQP operation -%% in MQTT plugin was tested) no_queue_bind_permission(Config) -> - test_subscribe_permissions_combination(<<".*">>, <<"">>, <<".*">>, Config, - ["operation queue.bind caused a channel exception access_refused"]). + ExpectedLogs = + ["MQTT resource access refused: write access to queue " + "'mqtt-subscription-mqtt-userqos0' in vhost 'mqtt-vhost' " + "refused for user 'mqtt-user'", + "Failed to bind queue 'mqtt-subscription-mqtt-userqos0' " + "in vhost 'mqtt-vhost' with topic test/topic: access_refused" + ], + test_subscribe_permissions_combination(<<".*">>, <<"">>, <<".*">>, Config, ExpectedLogs). no_queue_consume_permission(Config) -> - test_subscribe_permissions_combination(<<".*">>, <<".*">>, <<"^amq\\.topic">>, Config, - ["operation basic.consume caused a channel exception access_refused"]). + ExpectedLogs = + ["MQTT resource access refused: read access to queue " + "'mqtt-subscription-mqtt-userqos0' in vhost 'mqtt-vhost' " + "refused for user 'mqtt-user'"], + test_subscribe_permissions_combination(<<".*">>, <<".*">>, <<"^amq\\.topic">>, Config, ExpectedLogs). no_queue_delete_permission(Config) -> - set_permissions(".*", ".*", ".*", Config), - C1 = open_mqtt_connection(Config, [{clientid, <<"no_queue_delete_permission">>}, {clean_start, false}]), - {ok, _, _} = emqtt:subscribe(C1, {<<"test/topic">>, qos1}), - ok = emqtt:disconnect(C1), - set_permissions(<<>>, ".*", ".*", Config), - - %% And now we have a durable queue that user doesn't have permission to delete. - %% Attempt to establish clean session should fail. - {ok, C2} = connect_user( - ?config(mqtt_user, Config), - ?config(mqtt_password, Config), - Config, - ?config(mqtt_user, Config), - [{clientid, <<"no_queue_delete_permission">>}, - {clean_start, true}]), - unlink(C2), - ?assertMatch({error, _}, - emqtt:connect(C2)), - - wait_log(Config, erlang:system_time(microsecond) + 1_000_000, - [{["Generic server.*terminating"], fun () -> exit(there_should_be_no_crashes) end} - ,{["operation queue.delete caused a channel exception access_refused", - "MQTT cannot start a clean session: `configure` permission missing for queue"], - fun () -> stop end} - ]), - ok. - -no_queue_consume_permission_on_connect(Config) -> - set_permissions(".*", ".*", ".*", Config), - C1 = open_mqtt_connection(Config, [{clientid, <<"no_queue_consume_permission_on_connect">>}, {clean_start, false}]), - {ok, _, _} = emqtt:subscribe(C1, {<<"test/topic">>, qos1}), - ok = emqtt:disconnect(C1), - set_permissions(".*", ".*", "^amq\\.topic", Config), - - {ok, C2} = connect_user( - ?config(mqtt_user, Config), - ?config(mqtt_password, Config), - Config, - ?config(mqtt_user, Config), - [{clientid, <<"no_queue_consume_permission_on_connect">>}, - {clean_start, false}]), - unlink(C2), - ?assertMatch({error, _}, - emqtt:connect(C2)), - - wait_log(Config, erlang:system_time(microsecond) + 1000000, - [{["Generic server.*terminating"], fun () -> exit(there_should_be_no_crashes) end} - ,{["operation basic.consume caused a channel exception access_refused", - "MQTT cannot recover a session, user is missing permissions"], - fun () -> stop end} - ]), - ok. - -no_queue_declare_permission(Config) -> - rabbit_ct_broker_helpers:set_permissions(Config, ?config(mqtt_user, Config), ?config(mqtt_vhost, Config), <<"">>, <<".*">>, <<".*">>), - P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), - {ok, C} = emqtt:start_link([{host, "localhost"}, - {port, P}, - {clientid, <<"no_queue_declare_permission">>}, - {proto_ver, v4}, - {username, ?config(mqtt_user, Config)}, - {password, ?config(mqtt_password, Config)}, - {clean_start, false} - ]), - {ok, _} = emqtt:connect(C), - + {skip, "TODO support clean_start=false"}. + % set_permissions(".*", ".*", ".*", Config), + % C1 = open_mqtt_connection(Config, [{clientid, <<"no_queue_delete_permission">>}, {clean_start, false}]), + % {ok, _, _} = emqtt:subscribe(C1, {<<"test/topic">>, qos1}), + % ok = emqtt:disconnect(C1), + % set_permissions(<<>>, ".*", ".*", Config), + + % %% And now we have a durable queue that user doesn't have permission to delete. + % %% Attempt to establish clean session should fail. + % {ok, C2} = connect_user( + % ?config(mqtt_user, Config), + % ?config(mqtt_password, Config), + % Config, + % ?config(mqtt_user, Config), + % [{clientid, <<"no_queue_delete_permission">>}, + % {clean_start, true}]), + % unlink(C2), + % ?assertMatch({error, _}, + % emqtt:connect(C2)), + + % wait_log(Config, + % [?FAIL_IF_CRASH_LOG + % ,{["operation queue.delete caused a channel exception access_refused", + % "MQTT cannot start a clean session: `configure` permission missing for queue"], + % fun () -> stop end} + % ]), + % ok. + +no_queue_consume_permission_on_connect(_Config) -> + {skip, "TODO support clean_start=false"}. + % set_permissions(".*", ".*", ".*", Config), + % C1 = open_mqtt_connection(Config, [{clientid, <<"no_queue_consume_permission_on_connect">>}, {clean_start, false}]), + % {ok, _, _} = emqtt:subscribe(C1, {<<"test/topic">>, qos1}), + % ok = emqtt:disconnect(C1), + + % set_permissions(".*", ".*", "^amq\\.topic", Config), + % {ok, C2} = connect_user( + % ?config(mqtt_user, Config), + % ?config(mqtt_password, Config), + % Config, + % ?config(mqtt_user, Config), + % [{clientid, <<"no_queue_consume_permission_on_connect">>}, + % {clean_start, false}]), + % unlink(C2), + % ?assertMatch({error, _}, + % emqtt:connect(C2)), + + % wait_log(Config, + % [{["Generic server.*terminating"], fun () -> exit(there_should_be_no_crashes) end} + % ,{["operation basic.consume caused a channel exception access_refused", + % "MQTT cannot recover a session, user is missing permissions"], + % fun () -> stop end} + % ]), + % ok. + +no_queue_declare_permission(_Config) -> + {skip, "TODO support clean_start=false"}. + % rabbit_ct_broker_helpers:set_permissions(Config, ?config(mqtt_user, Config), ?config(mqtt_vhost, Config), <<"">>, <<".*">>, <<".*">>), + % P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), + % {ok, C} = emqtt:start_link([{host, "localhost"}, + % {port, P}, + % {clientid, <<"no_queue_declare_permission">>}, + % {proto_ver, v4}, + % {username, ?config(mqtt_user, Config)}, + % {password, ?config(mqtt_password, Config)}, + % {clean_start, false} + % ]), + % {ok, _} = emqtt:connect(C), + + % process_flag(trap_exit, true), + % try emqtt:subscribe(C, <<"test/topic">>) of + % _ -> exit(this_should_not_succeed) + % catch + % exit:{{shutdown, tcp_closed} , _} -> ok + % end, + % process_flag(trap_exit, false), + + % wait_log(Config, + % [{["Generic server.*terminating"], fun () -> exit(there_should_be_no_crashes) end} + % ,{["MQTT protocol error on connection.*access_refused", + % "operation queue.declare caused a channel exception access_refused"], + % fun () -> stop end} + % ]), + % ok. + +no_publish_permission(Config) -> + set_permissions(".*", "", ".*", Config), + C = open_mqtt_connection(Config), process_flag(trap_exit, true), - try emqtt:subscribe(C, <<"test/topic">>) of - _ -> exit(this_should_not_succeed) - catch - exit:{{shutdown, tcp_closed} , _} -> ok - end, - process_flag(trap_exit, false), - - wait_log(Config, erlang:system_time(microsecond) + 1_000_000, - [{["Generic server.*terminating"], fun () -> exit(there_should_be_no_crashes) end} - ,{["MQTT protocol error on connection.*access_refused", - "operation queue.declare caused a channel exception access_refused"], - fun () -> stop end} + ok = emqtt:publish(C, <<"some/topic">>, <<"payload">>), + assert_connection_closed(C), + wait_log(Config, + [?FAIL_IF_CRASH_LOG + ,{["MQTT resource access refused: write access to exchange " + "'amq.topic' in vhost 'mqtt-vhost' refused for user 'mqtt-user'", + "MQTT connection .* is closing due to an authorization failure"], + fun () -> stop end} ]), ok. @@ -613,57 +633,72 @@ no_topic_read_permission(Config) -> set_topic_permissions("^allow-write\\..*", "^allow-read\\..*", Config), C = open_mqtt_connection(Config), + %% Check topic permission setup is working. + {ok, _, [0]} = emqtt:subscribe(C, <<"allow-read/some/topic">>), - %% Just to be sure that our permission setup is indeed working - {ok, _, _} = emqtt:subscribe(C, <<"allow-read/some/topic">>), - - expect_sync_error(fun () -> - emqtt:subscribe(C, <<"test/topic">>) - end), - wait_log(Config, erlang:system_time(microsecond) + 1_000_000, - [?WAIT_LOG_NO_CRASHES - ,{["MQTT protocol error on connection.*access_refused", - "operation queue.bind caused a channel exception access_refused: access to topic 'test.topic' in exchange 'amq.topic' in vhost 'mqtt-vhost' refused for user 'mqtt-user'"], + process_flag(trap_exit, true), + {ok, _, [?SUBACK_FAILURE]} = emqtt:subscribe(C, <<"test/topic">>), + ok = assert_connection_closed(C), + wait_log(Config, + [?FAIL_IF_CRASH_LOG, + {["MQTT topic access refused: read access to topic 'test.topic' in exchange " + "'amq.topic' in vhost 'mqtt-vhost' refused for user 'mqtt-user'", + "Failed to bind queue 'mqtt-subscription-mqtt-userqos0' " + "in vhost 'mqtt-vhost' with topic test/topic: access_refused" + ], fun () -> stop end} ]), ok. -no_topic_write_permission(Config) -> +no_topic_write_permission(_Config) -> + {skip, "TODO implement QoS1"}. + % set_permissions(".*", ".*", ".*", Config), + % set_topic_permissions("^allow-write\\..*", "^allow-read\\..*", Config), + % C = open_mqtt_connection(Config), + % %% Check topic permission setup is working. + % {ok, _} = emqtt:publish(C, <<"allow-write/some/topic">>, <<"payload">>, qos1), + + % process_flag(trap_exit, true), + % ?assertMatch({error, _}, + % emqtt:publish(C, <<"some/other/topic">>, <<"payload">>, qos1)), + % wait_log(Config, + % [?FAIL_IF_CRASH_LOG + % ,{["MQTT topic access refused: write access to topic 'some.other.topic' in " + % "exchange 'amq.topic' in vhost 'mqtt-vhost' refused for user 'mqtt-user'", + % "MQTT connection .* is closing due to an authorization failure"], + % fun () -> stop end} + % ]), + % ok. + +loopback_user_connects_from_remote_host(Config) -> set_permissions(".*", ".*", ".*", Config), - set_topic_permissions("^allow-write\\..*", "^allow-read\\..*", Config), - C = open_mqtt_connection(Config), + {ok, C} = connect_anonymous(Config), - %% Just to be sure that our permission setup is indeed working - {ok, _} = emqtt:publish(C, <<"allow-write/some/topic">>, <<"payload">>, qos1), + %% CT node runs on the same host as the RabbitMQ node. + %% Instead of changing the IP address of CT node to a non-loopback IP address, + %% we mock rabbit_access_control:check_user_loopback/2. + rabbit_ct_broker_helpers:setup_meck(Config), + Mod = rabbit_access_control, + ok = rpc(Config, 0, meck, new, [Mod, [no_link, passthrough]]), + ok = rpc(Config, 0, meck, expect, [Mod, check_user_loopback, 2, not_allowed]), - unlink(C), - ?assertMatch({error, _}, - emqtt:publish(C, <<"some/other/topic">>, <<"payload">>, qos1)), - - wait_log(Config, erlang:system_time(microsecond) + 1_000_000, - [?WAIT_LOG_NO_CRASHES - ,{["access to topic 'some.other.topic' in exchange 'amq.topic' in vhost 'mqtt-vhost' refused for user 'mqtt-user'", - "MQTT connection.*is closing due to an authorization failure"], + process_flag(trap_exit, true), + ?assertMatch({error, _}, emqtt:connect(C)), + wait_log(Config, + [?FAIL_IF_CRASH_LOG, + {["MQTT login failed: user 'mqtt-user' can only connect via localhost", + "MQTT connection .* is closing due to an authorization failure"], fun () -> stop end} ]), - ok. -expect_sync_error(Fun) -> - process_flag(trap_exit, true), - try Fun() of - _ -> exit(this_should_not_succeed) - catch - exit:{{shutdown, tcp_closed} , _} -> ok - after - process_flag(trap_exit, false) - end. + true = rpc(Config, 0, meck, validate, [Mod]), + ok = rpc(Config, 0, meck, unload, [Mod]). set_topic_permissions(WritePat, ReadPat, Config) -> - rabbit_ct_broker_helpers:rpc(Config, 0, - rabbit_auth_backend_internal, set_topic_permissions, - [?config(mqtt_user, Config), ?config(mqtt_vhost, Config), - <<"amq.topic">>, WritePat, ReadPat, <<"acting-user">>]). - + rpc(Config, 0, + rabbit_auth_backend_internal, set_topic_permissions, + [?config(mqtt_user, Config), ?config(mqtt_vhost, Config), + <<"amq.topic">>, WritePat, ReadPat, <<"acting-user">>]). set_permissions(PermConf, PermWrite, PermRead, Config) -> rabbit_ct_broker_helpers:set_permissions(Config, ?config(mqtt_user, Config), ?config(mqtt_vhost, Config), @@ -679,26 +714,36 @@ open_mqtt_connection(Config, Opts) -> C. test_subscribe_permissions_combination(PermConf, PermWrite, PermRead, Config, ExtraLogChecks) -> - rabbit_ct_broker_helpers:set_permissions(Config, ?config(mqtt_user, Config), ?config(mqtt_vhost, Config), PermConf, PermWrite, PermRead), - - {ok, C} = connect_user(?config(mqtt_user, Config), ?config(mqtt_password, Config), Config), - {ok, _} = emqtt:connect(C), - + rabbit_ct_broker_helpers:set_permissions(Config, + ?config(mqtt_user, Config), + ?config(mqtt_vhost, Config), + PermConf, PermWrite, PermRead), + P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), + User = ?config(mqtt_user, Config), + Opts = [{host, "localhost"}, + {port, P}, + {clientid, User}, + {username, User}, + {password, ?config(mqtt_password, Config)}], + {ok, C1} = emqtt:start_link([{proto_ver, v4} | Opts]), + {ok, _} = emqtt:connect(C1), process_flag(trap_exit, true), - try emqtt:subscribe(C, <<"test/topic">>) of - _ -> exit(this_should_not_succeed) - catch - exit:{{shutdown, tcp_closed} , _} -> ok - end, + %% In v4, we expect to receive a failure return code for our subscription in the SUBACK packet. + ?assertMatch({ok, _Properties, [?SUBACK_FAILURE]}, + emqtt:subscribe(C1, <<"test/topic">>)), + ok = assert_connection_closed(C1), + wait_log(Config, + [?FAIL_IF_CRASH_LOG + ,{["MQTT protocol error on connection.*: subscribe_error"|ExtraLogChecks], fun () -> stop end} + ]), - process_flag(trap_exit, false), + {ok, C2} = emqtt:start_link([{proto_ver, v3} | Opts]), + {ok, _} = emqtt:connect(C2), - wait_log(Config, erlang:system_time(microsecond) + 1_000_000, - [{["Generic server.*terminating"], fun () -> exit(there_should_be_no_crashes) end} - ,{["MQTT protocol error on connection.*access_refused"|ExtraLogChecks], - fun () -> stop end} - ]), - ok. + %% In v3, there is no failure return code in the SUBACK packet. + ?assertMatch({ok, _Properties, [0]}, + emqtt:subscribe(C2, <<"test/topic">>)), + ok = assert_connection_closed(C2). connect_user(User, Pass, Config) -> connect_user(User, Pass, Config, User, []). @@ -721,12 +766,11 @@ connect_user(User, Pass, Config, ClientID0, Opts) -> [{host, "localhost"}, {port, P}, {proto_ver, v4}]). expect_successful_connection(ConnectFun, Config) -> - rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_core_metrics, reset_auth_attempt_metrics, []), + rpc(Config, 0, rabbit_core_metrics, reset_auth_attempt_metrics, []), {ok, C} = ConnectFun(Config), {ok, _} = emqtt:connect(C), ok = emqtt:disconnect(C), - [Attempt] = - rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_core_metrics, get_auth_attempts, []), + [Attempt] = rpc(Config, 0, rabbit_core_metrics, get_auth_attempts, []), ?assertEqual(false, proplists:is_defined(remote_address, Attempt)), ?assertEqual(false, proplists:is_defined(username, Attempt)), ?assertEqual(proplists:get_value(protocol, Attempt), <<"mqtt">>), @@ -735,12 +779,11 @@ expect_successful_connection(ConnectFun, Config) -> ?assertEqual(proplists:get_value(auth_attempts_succeeded, Attempt), 1). expect_authentication_failure(ConnectFun, Config) -> - rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_core_metrics, reset_auth_attempt_metrics, []), + rpc(Config, 0, rabbit_core_metrics, reset_auth_attempt_metrics, []), {ok, C} = ConnectFun(Config), unlink(C), ?assertMatch({error, _}, emqtt:connect(C)), - [Attempt] = - rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_core_metrics, get_auth_attempts, []), + [Attempt] = rpc(Config, 0, rabbit_core_metrics, get_auth_attempts, []), ?assertEqual(false, proplists:is_defined(remote_address, Attempt), <<>>), ?assertEqual(false, proplists:is_defined(username, Attempt)), ?assertEqual(proplists:get_value(protocol, Attempt), <<"mqtt">>), @@ -749,23 +792,27 @@ expect_authentication_failure(ConnectFun, Config) -> ?assertEqual(proplists:get_value(auth_attempts_succeeded, Attempt), 0), ok. -wait_log(Config, Deadline, Clauses) -> +wait_log(Config, Clauses) -> + wait_log(Config, Clauses, erlang:monotonic_time(millisecond) + 1000). + +wait_log(Config, Clauses, Deadline) -> {ok, Content} = file:read_file(?config(log_location, Config)), - case erlang:system_time(microsecond) of - T when T > Deadline -> - lists:foreach(fun - ({REs, _}) -> - Matches = [ io_lib:format("~tp - ~ts~n", [RE, re:run(Content, RE, [{capture, none}])]) || RE <- REs ], - ct:pal("Wait log clause status: ~ts", [Matches]) - end, Clauses), - exit(no_log_lines_detected); - _ -> ok - end, - case wait_log_check_clauses(Content, Clauses) of - stop -> ok; - continue -> - timer:sleep(50), - wait_log(Config, Deadline, Clauses) + case erlang:monotonic_time(millisecond) of + T when T =< Deadline -> + case wait_log_check_clauses(Content, Clauses) of + stop -> ok; + continue -> + timer:sleep(50), + wait_log(Config, Clauses, Deadline) + end; + _ -> + lists:foreach( + fun + ({REs, _}) -> + Matches = [ io_lib:format("~p - ~s~n", [RE, re:run(Content, RE, [{capture, none}])]) || RE <- REs ], + ct:pal("Wait log clause status: ~s", [Matches]) + end, Clauses), + ct:fail(expected_logs_not_found) end, ok. @@ -779,7 +826,16 @@ wait_log_check_clauses(Content, [{REs, Fun}|Rest]) -> end. multiple_re_match(Content, REs) -> - lists:all(fun (RE) -> - match == re:run(Content, RE, [{capture, none}]) - end, - REs). + lists:all( + fun (RE) -> + match == re:run(Content, RE, [{capture, none}]) + end, REs). + +assert_connection_closed(ClientPid) -> + receive + {'EXIT', ClientPid, {shutdown, tcp_closed}} -> + ok + after + 2000 -> + ct:fail("timed out waiting for exit message") + end. From 218ee196c4d8175274e583007c693446f772bb04 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Sun, 18 Sep 2022 19:07:11 +0200 Subject: [PATCH 007/118] Make proxy_protocol tests green --- deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl | 77 ++++++++--------- .../src/rabbit_mqtt_processor.erl | 85 +++++++++++-------- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 9 +- .../test/proxy_protocol_SUITE.erl | 3 +- 4 files changed, 98 insertions(+), 76 deletions(-) diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl index e6621e33b1a3..4799cdb50874 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl @@ -8,18 +8,18 @@ -define(CLIENT_ID_MAXLEN, 23). %% reader state --record(state, { socket, - conn_name, - await_recv, - deferred_recv, - received_connect_frame, - connection_state, - conserve, - parse_state, - proc_state, - connection, - stats_timer, - keepalive }). +-record(state, {socket, + conn_name, + await_recv, + deferred_recv, + received_connect_frame, + connection_state, + conserve, + parse_state, + proc_state, + connection, + stats_timer, + keepalive}). -record(keepalive, {timer :: reference(), interval_ms :: pos_integer(), @@ -27,32 +27,33 @@ received :: boolean()}). %% processor state --record(proc_state, { socket, - subscriptions, - consumer_tags, - unacked_pubs, - awaiting_ack, - awaiting_seqno, - message_id, - client_id, - clean_sess, - will_msg, - queue_states, - channels, - connection, - exchange :: rabbit_exchange:name(), - ssl_login_name, - %% Retained messages handler. See rabbit_mqtt_retainer_sup - %% and rabbit_mqtt_retainer. - retainer_pid, - auth_state, - send_fun, - peer_addr, - mqtt2amqp_fun, - amqp2mqtt_fun, - register_state, - proto_ver :: 3 | 4, - info}). +-record(proc_state, {socket, + conn_name, + subscriptions, + consumer_tags, + unacked_pubs, + awaiting_ack, + awaiting_seqno, + message_id, + client_id, + clean_sess, + will_msg, + queue_states, + channels, + connection, + exchange :: rabbit_exchange:name(), + ssl_login_name, + %% Retained messages handler. See rabbit_mqtt_retainer_sup + %% and rabbit_mqtt_retainer. + retainer_pid, + auth_state, + send_fun, + peer_addr, + mqtt2amqp_fun, + amqp2mqtt_fun, + register_state, + proto_ver :: 3 | 4, + info}). -record(auth_state, {username, user, diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 8f066285d836..b1b7b848b47b 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -7,7 +7,7 @@ -module(rabbit_mqtt_processor). --export([info/2, initial_state/1, initial_state/4, +-export([info/2, initial_state/2, process_frame/2, amqp_pub/2, amqp_callback/2, send_will/1, close_connection/1, handle_pre_hibernate/0, handle_ra_event/2, handle_down/2, handle_queue_event/2]). @@ -26,30 +26,27 @@ -define(MAX_PERMISSION_CACHE_SIZE, 12). -define(CONSUMER_TAG, mqtt_consumer). -initial_state(Socket) -> - RealSocket = rabbit_net:unwrap_socket(Socket), - SSLLoginName = ssl_login_name(RealSocket), - {ok, {PeerAddr, _PeerPort}} = rabbit_net:peername(RealSocket), - initial_state(RealSocket, SSLLoginName, fun serialise_and_send_to_client/2, PeerAddr). - -initial_state(Socket, SSLLoginName, SendFun, PeerAddr) -> +initial_state(Socket, ConnectionName) -> + SSLLoginName = ssl_login_name(Socket), + {ok, {PeerAddr, _PeerPort}} = rabbit_net:peername(Socket), {ok, {mqtt2amqp_fun, M2A}, {amqp2mqtt_fun, A2M}} = - rabbit_mqtt_util:get_topic_translation_funs(), + rabbit_mqtt_util:get_topic_translation_funs(), %% MQTT connections use exactly one channel. The frame max is not %% applicable and there is no way to know what client is used. - #proc_state{ unacked_pubs = gb_trees:empty(), - awaiting_ack = gb_trees:empty(), - message_id = 1, - subscriptions = #{}, - queue_states = rabbit_queue_type:init(), - consumer_tags = {undefined, undefined}, - channels = {undefined, undefined}, - socket = Socket, - ssl_login_name = SSLLoginName, - send_fun = SendFun, - peer_addr = PeerAddr, - mqtt2amqp_fun = M2A, - amqp2mqtt_fun = A2M}. + #proc_state{socket = Socket, + conn_name = ConnectionName, + unacked_pubs = gb_trees:empty(), + awaiting_ack = gb_trees:empty(), + message_id = 1, + subscriptions = #{}, + queue_states = rabbit_queue_type:init(), + consumer_tags = {undefined, undefined}, + channels = {undefined, undefined}, + ssl_login_name = SSLLoginName, + send_fun = fun serialise_and_send_to_client/2, + peer_addr = PeerAddr, + mqtt2amqp_fun = M2A, + amqp2mqtt_fun = A2M}. process_frame(#mqtt_frame{ fixed = #mqtt_frame_fixed{ type = Type }}, PState = #proc_state{ auth_state = undefined } ) @@ -244,7 +241,8 @@ process_connect(#mqtt_frame{ fun check_client_id/1, fun check_credentials/2, fun login/2, - fun register_client/2], + fun register_client/2, + fun notify_connection_created/2], FrameConnect, PState0) of {ok, SessionPresent0, PState1} -> {?CONNACK_ACCEPT, SessionPresent0, PState1}; @@ -356,6 +354,34 @@ register_client(Frame = #mqtt_frame_connect{ {error, ?CONNACK_SERVER_UNAVAILABLE} end. +notify_connection_created( + _Frame, + #proc_state{socket = Sock, + conn_name = ConnName, + info = #info{proto_human = {ProtoName, ProtoVsn}}, + auth_state = #auth_state{vhost = VHost, + username = Username}} = PState) -> + {ok, {PeerHost, PeerPort, Host, Port}} = rabbit_net:socket_ends(Sock, inbound), + ConnectedAt = os:system_time(milli_seconds), + Infos = [{host, Host}, + {port, Port}, + {peer_host, PeerHost}, + {peer_port, PeerPort}, + {vhost, VHost}, + {node, node()}, + {user, Username}, + {name, ConnName}, + {connected_at, ConnectedAt}, + {pid, self()}, + {protocol, {ProtoName, binary_to_list(ProtoVsn)}}, + {type, network} + ], + rabbit_core_metrics:connection_created(self(), Infos), + rabbit_event:notify(connection_created, Infos), + {ok, PState#proc_state{ + %% We won't need conn_name anymore. Use less memmory by setting to undefined. + conn_name = undefined}}. + human_readable_mqtt_version(3) -> <<"3.1.0">>; human_readable_mqtt_version(4) -> @@ -758,20 +784,9 @@ check_user_loopback(#{vhost := VHost, user := User, authz_ctx := AuthzCtx }, - #proc_state{socket = Sock, - peer_addr = PeerAddr} = PState) -> + #proc_state{peer_addr = PeerAddr} = PState) -> case rabbit_access_control:check_user_loopback(UsernameBin, PeerAddr) of ok -> - {ok, {PeerHost, PeerPort, Host, Port}} = rabbit_net:socket_ends(Sock, inbound), - Infos = [{node, node()}, - {host, Host}, - {port, Port}, - {peer_host, PeerHost}, - {peer_port, PeerPort}, - {user, UsernameBin}, - {vhost, VHost}], - rabbit_core_metrics:connection_created(self(), Infos), - rabbit_event:notify(connection_created, Infos), AuthState = #auth_state{user = User, username = UsernameBin, vhost = VHost, diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index bcaff15a8aea..f220f38e1325 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -61,7 +61,10 @@ init(Ref) -> self(), {?MODULE, conserve_resources, []}), LoginTimeout = application:get_env(rabbitmq_mqtt, login_timeout, 10_000), erlang:send_after(LoginTimeout, self(), login_timeout), - ProcessorState = rabbit_mqtt_processor:initial_state(Sock), + ProcessorState = rabbit_mqtt_processor:initial_state(RealSocket, ConnStr), + %%TODO call rabbit_networking:register_non_amqp_connection/1 so that we are notified + %% when need to force load the 'connection_created' event for the management plugin. + %% Same is done for streams. gen_server2:enter_loop(?MODULE, [], rabbit_event:init_stats_timer( control_throttle( @@ -510,6 +513,10 @@ emit_stats(State=#state{connection = C}) when C == none; C == undefined -> State1 = rabbit_event:reset_stats_timer(State, #state.stats_timer), ensure_stats_timer(State1); emit_stats(State) -> + %%TODO only emit stats if rabbit_event:if_enabled() + %% Should be disabled if rabbitmq management agent is disabled, see + %% https://github.com/rabbitmq/rabbitmq-server/blob/7eb4084cf879fe6b1d26dd3c837bd180cb3a8546/deps/rabbitmq_management_agent/src/rabbit_mgmt_db_handler.erl#L57-L72 + %% Otherwise emitting stats for many MQTT connections becomes expensive [{_, Pid}, {_, Recv_oct}, {_, Send_oct}, diff --git a/deps/rabbitmq_mqtt/test/proxy_protocol_SUITE.erl b/deps/rabbitmq_mqtt/test/proxy_protocol_SUITE.erl index 06ffb5930e9c..f50c53c57f0e 100644 --- a/deps/rabbitmq_mqtt/test/proxy_protocol_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/proxy_protocol_SUITE.erl @@ -66,8 +66,7 @@ proxy_protocol(Config) -> ok = inet:send(Socket, "PROXY TCP4 192.168.1.1 192.168.1.2 80 81\r\n"), ok = inet:send(Socket, mqtt_3_1_1_connect_frame()), {ok, _Packet} = gen_tcp:recv(Socket, 0, ?TIMEOUT), - ConnectionName = rabbit_ct_broker_helpers:rpc(Config, 0, - ?MODULE, connection_name, []), + ConnectionName = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, connection_name, []), match = re:run(ConnectionName, <<"^192.168.1.1:80 -> 192.168.1.2:81$">>, [{capture, none}]), gen_tcp:close(Socket), ok. From 99337b84d39d0a95a70b6e534dbaa49f3f73ac5a Mon Sep 17 00:00:00 2001 From: David Ansari Date: Mon, 19 Sep 2022 11:22:42 +0000 Subject: [PATCH 008/118] Emit stats 'connection' field is not needed anymore because it was previously the internal AMQP connection PID --- deps/rabbit/src/rabbit.erl | 6 +- deps/rabbit/src/rabbit_channel.erl | 2 +- deps/rabbitmq_mqtt/include/mqtt_machine.hrl | 2 +- deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl | 2 - .../src/rabbit_mqtt_processor.erl | 39 ++++-------- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 60 ++++++++----------- deps/rabbitmq_mqtt/test/reader_SUITE.erl | 14 ++--- 7 files changed, 46 insertions(+), 79 deletions(-) diff --git a/deps/rabbit/src/rabbit.erl b/deps/rabbit/src/rabbit.erl index 9d5bb05bda6b..ab80ed0e695e 100644 --- a/deps/rabbit/src/rabbit.erl +++ b/deps/rabbit/src/rabbit.erl @@ -1163,12 +1163,12 @@ config_locations() -> % This event is necessary for the stats timer to be initialized with % the correct values once the management agent has started force_event_refresh(Ref) -> - % direct connections, e.g. MQTT, STOMP + % direct connections, e.g. STOMP ok = rabbit_direct:force_event_refresh(Ref), % AMQP connections ok = rabbit_networking:force_connection_event_refresh(Ref), - % "external" connections, which are not handled by the "AMQP core", - % e.g. connections to the stream plugin + % non-AMQP connections, which are not handled by the "AMQP core", + % e.g. connections to the stream and MQTT plugins ok = rabbit_networking:force_non_amqp_connection_event_refresh(Ref), ok = rabbit_channel:force_event_refresh(Ref), ok = rabbit_amqqueue:force_event_refresh(Ref). diff --git a/deps/rabbit/src/rabbit_channel.erl b/deps/rabbit/src/rabbit_channel.erl index 2366437102f9..5c7aaa0d0352 100644 --- a/deps/rabbit/src/rabbit_channel.erl +++ b/deps/rabbit/src/rabbit_channel.erl @@ -1091,7 +1091,7 @@ build_topic_variable_map(AuthzContext, VHost, Username) -> %% Use tuple representation of amqp_params to avoid a dependency on amqp_client. %% Extracts variable map only from amqp_params_direct, not amqp_params_network. -%% amqp_params_direct records are usually used by plugins (e.g. MQTT, STOMP) +%% amqp_params_direct records are usually used by plugins (e.g. STOMP) extract_variable_map_from_amqp_params({amqp_params, {amqp_params_direct, _, _, _, _, {amqp_adapter_info, _,_,_,_,_,_,AdditionalInfo}, _}}) -> proplists:get_value(variable_map, AdditionalInfo, #{}); diff --git a/deps/rabbitmq_mqtt/include/mqtt_machine.hrl b/deps/rabbitmq_mqtt/include/mqtt_machine.hrl index c8d4593d9f0b..bddc2e53c7f6 100644 --- a/deps/rabbitmq_mqtt/include/mqtt_machine.hrl +++ b/deps/rabbitmq_mqtt/include/mqtt_machine.hrl @@ -10,7 +10,7 @@ %% millions of connections. %% 1. Use binaries instead of string()s for the ConnectionId %% 2. Use new Erlang 24 function erlang:monitor/3 with tag being the ConnectionId -%% so that we can get rid of pids fields because we won't to lookup the ConnectionId +%% so that we can get rid of pids fields because we won't lookup the ConnectionId %% by PID anymore. -record(machine_state, { %% client ID to connection PID diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl index 4799cdb50874..892d5b3b684b 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl @@ -17,7 +17,6 @@ conserve, parse_state, proc_state, - connection, stats_timer, keepalive}). @@ -40,7 +39,6 @@ will_msg, queue_states, channels, - connection, exchange :: rabbit_exchange:name(), ssl_login_name, %% Retained messages handler. See rabbit_mqtt_retainer_sup diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index b1b7b848b47b..bc49317bb47b 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -9,7 +9,7 @@ -export([info/2, initial_state/2, process_frame/2, amqp_pub/2, amqp_callback/2, send_will/1, - close_connection/1, handle_pre_hibernate/0, + terminate/1, handle_pre_hibernate/0, handle_ra_event/2, handle_down/2, handle_queue_event/2]). %% for testing purposes @@ -48,18 +48,12 @@ initial_state(Socket, ConnectionName) -> mqtt2amqp_fun = M2A, amqp2mqtt_fun = A2M}. -process_frame(#mqtt_frame{ fixed = #mqtt_frame_fixed{ type = Type }}, - PState = #proc_state{ auth_state = undefined } ) +process_frame(#mqtt_frame{fixed = #mqtt_frame_fixed{type = Type}}, + PState = #proc_state{auth_state = undefined}) when Type =/= ?CONNECT -> {error, connect_expected, PState}; -process_frame(Frame = #mqtt_frame{fixed = #mqtt_frame_fixed{ type = Type }}, - PState) -> - case process_request(Type, Frame, PState) of - {ok, PState1} -> - {ok, PState1, PState1#proc_state.connection}; - Ret -> - Ret - end. +process_frame(Frame = #mqtt_frame{fixed = #mqtt_frame_fixed{type = Type}}, PState) -> + process_request(Type, Frame, PState). process_request(?CONNECT, Frame, PState = #proc_state{socket = Socket}) -> %% Check whether peer closed the connection. @@ -1251,24 +1245,11 @@ serialise_and_send_to_client(Frame, #proc_state{proto_ver = ProtoVer, socket = S [Sock, Error, Frame]) end. -close_connection(PState = #proc_state{ connection = undefined }) -> - PState; -close_connection(PState = #proc_state{ connection = Connection, - client_id = ClientId }) -> - % todo: maybe clean session - case ClientId of - undefined -> ok; - _ -> - case rabbit_mqtt_collector:unregister(ClientId, self()) of - ok -> ok; - %% ignore as we are shutting down - {timeout, _} -> ok - end - end, - %% ignore noproc or other exceptions, we are shutting down - catch amqp_connection:close(Connection), - PState #proc_state{ channels = {undefined, undefined}, - connection = undefined }. +terminate(#proc_state{client_id = undefined}) -> + ok; +terminate(#proc_state{client_id = ClientId}) -> + %% ignore any errors as we are shutting down + rabbit_mqtt_collector:unregister(ClientId, self()). handle_pre_hibernate() -> erase(permission_cache), diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index f220f38e1325..fdc55e67d117 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -63,8 +63,11 @@ init(Ref) -> erlang:send_after(LoginTimeout, self(), login_timeout), ProcessorState = rabbit_mqtt_processor:initial_state(RealSocket, ConnStr), %%TODO call rabbit_networking:register_non_amqp_connection/1 so that we are notified - %% when need to force load the 'connection_created' event for the management plugin. - %% Same is done for streams. + %% when need to force load the 'connection_created' event for the management plugin, see + %% https://github.com/rabbitmq/rabbitmq-management-agent/issues/58 + %% https://github.com/rabbitmq/rabbitmq-server/blob/90cc0e2abf944141feedaf3190d7b6d8b4741b11/deps/rabbitmq_stream/src/rabbit_stream_reader.erl#L536 + %% https://github.com/rabbitmq/rabbitmq-server/blob/90cc0e2abf944141feedaf3190d7b6d8b4741b11/deps/rabbitmq_stream/src/rabbit_stream_reader.erl#L189 + %% https://github.com/rabbitmq/rabbitmq-server/blob/7eb4084cf879fe6b1d26dd3c837bd180cb3a8546/deps/rabbitmq_management_agent/src/rabbit_mgmt_db_handler.erl#L72 gen_server2:enter_loop(?MODULE, [], rabbit_event:init_stats_timer( control_throttle( @@ -317,12 +320,12 @@ do_terminate({network_error, Reason}, _State) -> do_terminate(normal, #state{proc_state = ProcState, conn_name = ConnName}) -> - rabbit_mqtt_processor:close_connection(ProcState), - rabbit_log_connection:info("closing MQTT connection ~tp (~ts)", [self(), ConnName]), + rabbit_mqtt_processor:terminate(ProcState), + rabbit_log_connection:info("closing MQTT connection ~p (~s)", [self(), ConnName]), ok; do_terminate(_Reason, #state{proc_state = ProcState}) -> - rabbit_mqtt_processor:close_connection(ProcState), + rabbit_mqtt_processor:terminate(ProcState), ok. code_change(_OldVsn, State, _Extra) -> @@ -340,18 +343,12 @@ log_tls_alert(Alert, ConnStr) -> rabbit_log_connection:error("MQTT detected TLS upgrade error on ~ts: alert ~ts", [ConnStr, Alert]). -log_new_connection(#state{conn_name = ConnStr, proc_state = PState}) -> - rabbit_log_connection:info("accepting MQTT connection ~tp (~ts, client id: ~ts)", - [self(), ConnStr, rabbit_mqtt_processor:info(client_id, PState)]). - -process_received_bytes(<<>>, State = #state{proc_state = ProcState, - received_connect_frame = false}) -> - MqttConn = ProcState#proc_state.connection, - case MqttConn of - undefined -> ok; - _ -> log_new_connection(State) - end, - {noreply, ensure_stats_timer(State#state{ received_connect_frame = true }), hibernate}; +process_received_bytes(<<>>, State = #state{received_connect_frame = false, + proc_state = PState, + conn_name = ConnStr}) -> + rabbit_log_connection:info("Accepted MQTT connection ~p (~s, client id: ~s)", + [self(), ConnStr, rabbit_mqtt_processor:info(client_id, PState)]), + {noreply, ensure_stats_timer(State#state{received_connect_frame = true}), hibernate}; process_received_bytes(<<>>, State) -> {noreply, ensure_stats_timer(State), hibernate}; process_received_bytes(Bytes, @@ -365,13 +362,12 @@ process_received_bytes(Bytes, hibernate}; {ok, Frame, Rest} -> case rabbit_mqtt_processor:process_frame(Frame, ProcState) of - {ok, ProcState1, ConnPid} -> + {ok, ProcState1} -> PS = rabbit_mqtt_frame:initial_state(), process_received_bytes( Rest, - State #state{ parse_state = PS, - proc_state = ProcState1, - connection = ConnPid }); + State #state{parse_state = PS, + proc_state = ProcState1}); %% PUBLISH and more {error, unauthorized = Reason, ProcState1} -> rabbit_log_connection:error("MQTT connection ~ts is closing due to an authorization failure", [ConnStr]), @@ -444,13 +440,13 @@ send_will_and_terminate(PState, Reason, State = #state{conn_name = ConnStr}) -> network_error(closed, State = #state{conn_name = ConnStr, - proc_state = PState}) -> - MqttConn = PState#proc_state.connection, - Fmt = "MQTT connection ~tp will terminate because peer closed TCP connection", + proc_state = PState, + received_connect_frame = Connected}) -> + Fmt = "MQTT connection ~p will terminate because peer closed TCP connection", Args = [ConnStr], - case MqttConn of - undefined -> rabbit_log_connection:debug(Fmt, Args); - _ -> rabbit_log_connection:info(Fmt, Args) + case Connected of + true -> rabbit_log_connection:info(Fmt, Args); + false -> rabbit_log_connection:debug(Fmt, Args) end, send_will_and_terminate(PState, State); @@ -507,16 +503,12 @@ maybe_emit_stats(State) -> rabbit_event:if_enabled(State, #state.stats_timer, fun() -> emit_stats(State) end). -emit_stats(State=#state{connection = C}) when C == none; C == undefined -> +emit_stats(State=#state{received_connect_frame = false}) -> %% Avoid emitting stats on terminate when the connection has not yet been %% established, as this causes orphan entries on the stats database State1 = rabbit_event:reset_stats_timer(State, #state.stats_timer), ensure_stats_timer(State1); emit_stats(State) -> - %%TODO only emit stats if rabbit_event:if_enabled() - %% Should be disabled if rabbitmq management agent is disabled, see - %% https://github.com/rabbitmq/rabbitmq-server/blob/7eb4084cf879fe6b1d26dd3c837bd180cb3a8546/deps/rabbitmq_management_agent/src/rabbit_mgmt_db_handler.erl#L57-L72 - %% Otherwise emitting stats for many MQTT connections becomes expensive [{_, Pid}, {_, Recv_oct}, {_, Send_oct}, @@ -554,7 +546,7 @@ info_internal(connection_state, #state{received_connect_frame = false}) -> starting; info_internal(connection_state, #state{connection_state = Val}) -> Val; -info_internal(connection, #state{connection = Val}) -> - Val; +info_internal(connection, _State) -> + self(); info_internal(Key, #state{proc_state = ProcState}) -> rabbit_mqtt_processor:info(Key, ProcState). diff --git a/deps/rabbitmq_mqtt/test/reader_SUITE.erl b/deps/rabbitmq_mqtt/test/reader_SUITE.erl index a409b7974d5f..bb3bd58f2fe4 100644 --- a/deps/rabbitmq_mqtt/test/reader_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/reader_SUITE.erl @@ -163,8 +163,10 @@ handle_invalid_frames(Config) -> Bin = <<"GET / HTTP/1.1\r\nHost: www.rabbitmq.com\r\nUser-Agent: curl/7.43.0\r\nAccept: */*">>, gen_tcp:send(C, Bin), gen_tcp:close(C), + %% Wait for stats being emitted (every 100ms) + timer:sleep(300), %% No new stats entries should be inserted as connection never got to initialize - N = rpc(Config, ets, info, [connection_metrics, size]). + ?assertEqual(N, rpc(Config, ets, info, [connection_metrics, size])). login_timeout(Config) -> rpc(Config, application, set_env, [rabbitmq_mqtt, login_timeout, 400]), @@ -179,19 +181,13 @@ login_timeout(Config) -> stats(Config) -> P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), - %% CMN = rpc(Config, ets, info, [connection_metrics, size]), - %% CCMN = rpc(Config, ets, info, [connection_coarse_metrics, size]), {ok, C} = emqtt:start_link([{host, "localhost"}, {port, P}, {clientid, <<"simpleClient">>}, {proto_ver, v4}]), {ok, _Properties} = emqtt:connect(C), - %% Ensure that there are some stats - {ok, _, _} = emqtt:subscribe(C, <<"TopicA">>, qos0), - ok = emqtt:publish(C, <<"TopicA">>, <<"Payload">>), - expect_publishes(<<"TopicA">>, [<<"Payload">>]), - {ok, _, _} = emqtt:unsubscribe(C, <<"TopicA">>), - timer:sleep(1000), %% Wait for stats to be emitted, which it does every 100ms + %% Wait for stats being emitted (every 100ms) + timer:sleep(300), %% Retrieve the connection Pid [{_, Reader}] = rpc(Config, rabbit_mqtt_collector, list, []), [{_, Pid}] = rpc(Config, rabbit_mqtt_reader, info, [Reader, [connection]]), From cdd253ee8723ccfd8fdc78d1167f9aa0507499fb Mon Sep 17 00:00:00 2001 From: David Ansari Date: Tue, 20 Sep 2022 08:39:16 +0000 Subject: [PATCH 009/118] Receive many messages from classic queue Before this commit, a consumer from a classic queue was receiving max 200 messages: https://github.com/rabbitmq/rabbitmq-server/blob/bb5d6263c9b516a0a42fe4105bf110ed5a0ed3a6/deps/rabbit/src/rabbit_queue_consumers.erl#L24 MQTT consumer process must give credit to classic queue process due to internal flow control. --- .../src/rabbit_mqtt_processor.erl | 28 ++++++++++++------- .../java/com/rabbitmq/mqtt/test/MqttTest.java | 2 +- 2 files changed, 19 insertions(+), 11 deletions(-) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index bc49317bb47b..9459e63264a5 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -1234,15 +1234,13 @@ deliver_to_queues(Delivery = #delivery{message = _Message = #basic_message{ex PState#proc_state{queue_states = QueueStates}. serialise_and_send_to_client(Frame, #proc_state{proto_ver = ProtoVer, socket = Sock }) -> - %%TODO Test sending large frames at high speed: Will we need garbage collection as done - %% in rabbit_writer:maybe_gc_large_msg()? - try rabbit_net:port_command(Sock, rabbit_mqtt_frame:serialise(Frame, ProtoVer)) of - Res -> - Res + %%TODO Test sending large frames at high speed: + %% Will we need garbage collection as done in rabbit_writer:maybe_gc_large_msg/1? + try rabbit_net:port_command(Sock, rabbit_mqtt_frame:serialise(Frame, ProtoVer)) catch _:Error -> - rabbit_log_connection:error("MQTT: a socket write failed, the socket might already be closed"), - rabbit_log_connection:debug("Failed to write to socket ~tp, error: ~tp, frame: ~tp", - [Sock, Error, Frame]) + rabbit_log_connection:error("MQTT: a socket write failed, the socket might already be closed"), + rabbit_log_connection:debug("Failed to write to socket ~p, error: ~p, frame: ~p", + [Sock, Error, Frame]) end. terminate(#proc_state{client_id = undefined}) -> @@ -1315,13 +1313,14 @@ handle_deliver(Msgs, PState) handle_deliver0(Msg, S) end, PState, Msgs). -handle_deliver0({_QName, _QPid, _MsgId, Redelivered, +handle_deliver0({QName, QPid, _MsgId, Redelivered, #basic_message{routing_keys = [RoutingKey | _CcRoutes], content = #content{ properties = #'P_basic'{headers = Headers}, payload_fragments_rev = FragmentsRev}}}, PState = #proc_state{send_fun = SendFun, - amqp2mqtt_fun = Amqp2MqttFun}) -> + amqp2mqtt_fun = Amqp2MqttFun, + queue_states = QStates}) -> Dup = case rabbit_mqtt_util:table_lookup(Headers, <<"x-mqtt-dup">>) of undefined -> Redelivered; {bool, Dup0} -> Redelivered orelse Dup0 @@ -1337,6 +1336,15 @@ handle_deliver0({_QName, _QPid, _MsgId, Redelivered, topic_name = Amqp2MqttFun(RoutingKey)}, payload = Payload}, SendFun(Frame, PState), + + {ok, QueueType} = rabbit_queue_type:module(QName, QStates), + case QueueType of + rabbit_classic_queue -> + rabbit_amqqueue:notify_sent(QPid, self()); + _ -> + ok + end, + PState. publish(TopicName, PublishFun, diff --git a/deps/rabbitmq_mqtt/test/java_SUITE_data/src/test/java/com/rabbitmq/mqtt/test/MqttTest.java b/deps/rabbitmq_mqtt/test/java_SUITE_data/src/test/java/com/rabbitmq/mqtt/test/MqttTest.java index b5d9848e8b95..f31497d1c705 100644 --- a/deps/rabbitmq_mqtt/test/java_SUITE_data/src/test/java/com/rabbitmq/mqtt/test/MqttTest.java +++ b/deps/rabbitmq_mqtt/test/java_SUITE_data/src/test/java/com/rabbitmq/mqtt/test/MqttTest.java @@ -740,7 +740,7 @@ public Socket createSocket() { } @Test public void publishMultiple() throws MqttException, InterruptedException { - int pubCount = 50; + int pubCount = 1000; for (int subQos=0; subQos <= 2; subQos++){ for (int pubQos=0; pubQos <= 2; pubQos++){ // avoid reusing the client in this test as a shared From 23dac495adb8dbcf4c2ac7a05b39d14058147c97 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Thu, 22 Sep 2022 19:31:37 +0200 Subject: [PATCH 010/118] Support QoS 1 for sending and receiving --- deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl | 69 +- .../include/rabbit_mqtt_frame.hrl | 5 +- .../include/rabbit_mqtt_types.hrl | 9 + deps/rabbitmq_mqtt/src/rabbit_mqtt.erl | 1 + .../src/rabbit_mqtt_confirms.erl | 89 ++ .../src/rabbit_mqtt_processor.erl | 1160 ++++++++--------- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 23 +- deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl | 4 +- deps/rabbitmq_mqtt/test/auth_SUITE.erl | 221 ++-- 9 files changed, 822 insertions(+), 759 deletions(-) create mode 100644 deps/rabbitmq_mqtt/include/rabbit_mqtt_types.hrl create mode 100644 deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl index 892d5b3b684b..3840b06eec91 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl @@ -7,6 +7,8 @@ -define(CLIENT_ID_MAXLEN, 23). +-include("rabbit_mqtt_types.hrl"). + %% reader state -record(state, {socket, conn_name, @@ -26,32 +28,37 @@ received :: boolean()}). %% processor state --record(proc_state, {socket, - conn_name, - subscriptions, - consumer_tags, - unacked_pubs, - awaiting_ack, - awaiting_seqno, - message_id, - client_id, - clean_sess, - will_msg, - queue_states, - channels, - exchange :: rabbit_exchange:name(), - ssl_login_name, - %% Retained messages handler. See rabbit_mqtt_retainer_sup - %% and rabbit_mqtt_retainer. - retainer_pid, - auth_state, - send_fun, - peer_addr, - mqtt2amqp_fun, - amqp2mqtt_fun, - register_state, - proto_ver :: 3 | 4, - info}). +-record(proc_state, + {socket, + proto_ver :: 3 | 4, + queue_states = rabbit_queue_type:init() :: rabbit_queue_type:state(), + subscriptions = #{} :: #{Topic :: binary() => QoS :: 0..2}, + %% Packet IDs published to queues but not yet confirmed. + unacked_client_pubs = rabbit_mqtt_confirms:init() :: rabbit_mqtt_confirms:state(), + %% Packet IDs published to MQTT subscribers but not yet acknowledged. + unacked_server_pubs = #{} :: #{packet_id() => QueueMsgId :: non_neg_integer()}, + %% Packet ID of next PUBLISH packet (with QoS > 0) sent from server to client. + %% (Not to be confused with packet IDs sent from client to server which can be the + %% same IDs because client and server assign IDs independently of each other.) + packet_id = 1 :: packet_id(), + client_id, + clean_sess, + will_msg, + exchange :: rabbit_exchange:name(), + ssl_login_name, + %% Retained messages handler. See rabbit_mqtt_retainer_sup + %% and rabbit_mqtt_retainer. + retainer_pid, + auth_state, + peer_addr, + %%TODO remove funs from state + mqtt2amqp_fun, + amqp2mqtt_fun, + register_state, + conn_name, + info}). + +-type proc_state() :: #proc_state{}. -record(auth_state, {username, user, @@ -76,8 +83,6 @@ peer_host, peer_port, protocol, - channels, - channel_max, frame_max, client_properties, ssl, @@ -88,11 +93,9 @@ conn_name, connection_state, connection, - consumer_tags, - unacked_pubs, - awaiting_ack, - awaiting_seqno, - message_id, + unacked_client_pubs, + unacked_server_pubs, + packet_id, client_id, clean_sess, will_msg, diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt_frame.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt_frame.hrl index 57a3a5fe6281..168d1c8d15bf 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt_frame.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt_frame.hrl @@ -48,7 +48,8 @@ -define(QOS_2, 2). -define(SUBACK_FAILURE, 16#80). -%% TODO +-type qos() :: ?QOS_0 | ?QOS_1 | ?QOS_2. +%%TODO remove message_id() -type message_id() :: any(). -record(mqtt_frame, {fixed, @@ -91,7 +92,7 @@ -record(mqtt_frame_other, {other}). -record(mqtt_msg, {retain :: boolean(), - qos :: ?QOS_0 | ?QOS_1 | ?QOS_2, + qos :: qos(), topic :: string(), dup :: boolean(), message_id :: message_id(), diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt_types.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt_types.hrl new file mode 100644 index 000000000000..3233d7f69553 --- /dev/null +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt_types.hrl @@ -0,0 +1,9 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2020-2022 VMware, Inc. or its affiliates. All rights reserved. +%% + +%% Packet identifier is a non zero two byte integer. +-type packet_id() :: 1..16#ffff. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl index dee18859f43e..a8c5e3ab4dfc 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl @@ -13,6 +13,7 @@ close_local_client_connections/1]). start(normal, []) -> + rabbit_global_counters:init([{protocol, mqtt}]), {ok, Listeners} = application:get_env(tcp_listeners), {ok, SslListeners} = application:get_env(ssl_listeners), ok = mqtt_node:start(), diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl new file mode 100644 index 000000000000..0fefdadd6d30 --- /dev/null +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl @@ -0,0 +1,89 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2020-2022 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(rabbit_mqtt_confirms). + +-include("rabbit_mqtt_types.hrl"). +-compile({no_auto_import, [size/1]}). + +-export([init/0, + insert/3, + confirm/3, + reject/2, + remove_queue/2, + size/1]). + +-type queue_name() :: rabbit_amqqueue:name(). +-opaque state() :: #{packet_id() => #{queue_name() => ok}}. +-export_type([state/0]). + +-spec init() -> state(). +init() -> + maps:new(). + +-spec size(state()) -> non_neg_integer(). +size(State) -> + maps:size(State). + +-spec insert(packet_id(), [queue_name()], state()) -> + {ok, state()} | {error, duplicate_packet_id}. +insert(PktId, _, State) + when is_map_key(PktId, State) -> + {error, duplicate_packet_id}; +insert(PktId, QNames, State) + when is_integer(PktId) andalso PktId > 0 -> + QMap = maps:from_keys(QNames, ok), + maps:put(PktId, QMap, State). + +-spec confirm([packet_id()], queue_name(), state()) -> + {[packet_id()], state()}. +confirm(PktIds, QName, State0) -> + lists:foldr(fun(PktId, Acc) -> + confirm_one(PktId, QName, Acc) + end, {[], State0}, PktIds). + +-spec reject(packet_id(), state()) -> + {ok, state()} | {error, not_found}. +reject(PktId, State0) + when is_integer(PktId) -> + case maps:take(PktId, State0) of + {_QMap, State} -> + {ok, State}; + error -> + {error, not_found} + end. + +%% idempotent +-spec remove_queue(queue_name(), state()) -> + {[packet_id()], state()}. +remove_queue(QName, State) -> + PktIds = maps:fold( + fun(PktId, QMap, PktIds) + when is_map_key(QName, QMap) -> + [PktId | PktIds]; + (_, _, PktIds) -> + PktIds + end, [], State), + confirm(lists:sort(PktIds), QName, State). + +%% INTERNAL + +confirm_one(PktId, QName, {PktIds, State0}) + when is_integer(PktId) -> + case maps:take(PktId, State0) of + {QMap0, State1} + when is_map_key(QName, QMap0) + andalso map_size(QMap0) =:= 1 -> + %% last queue confirm + {[PktId| PktIds], State1}; + {QMap0, State1} -> + QMap = maps:remove(QName, QMap0), + State = maps:put(PktId, QMap, State1), + {PktIds, State}; + error -> + {PktIds, State0} + end. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 9459e63264a5..f763002ad592 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -7,43 +7,41 @@ -module(rabbit_mqtt_processor). + -export([info/2, initial_state/2, - process_frame/2, amqp_pub/2, amqp_callback/2, send_will/1, + process_frame/2, send_will/1, terminate/1, handle_pre_hibernate/0, handle_ra_event/2, handle_down/2, handle_queue_event/2]). +%%TODO Use single queue per MQTT subscriber connection? +%% * when publishing we store in x-mqtt-publish-qos header the publishing QoS +%% * routing key is present in the delivered message +%% * therefore, we can map routing key -> topic -> subscription -> subscription max QoS +%% Advantages: +%% * better scaling when single client creates subscriptions with different QoS levels +%% * no duplicates when single client creates overlapping subscriptions with different QoS levels + %% for testing purposes --export([get_vhost_username/1, get_vhost/3, get_vhost_from_user_mapping/2, maybe_quorum/3]). +-export([get_vhost_username/1, get_vhost/3, get_vhost_from_user_mapping/2]). --include_lib("amqp_client/include/amqp_client.hrl"). +-include_lib("rabbit_common/include/rabbit.hrl"). +-include_lib("rabbit_common/include/rabbit_framing.hrl"). -include_lib("rabbit/include/amqqueue.hrl"). --include("rabbit_mqtt_frame.hrl"). -include("rabbit_mqtt.hrl"). +-include("rabbit_mqtt_frame.hrl"). -define(APP, rabbitmq_mqtt). --define(FRAME_TYPE(Frame, Type), - Frame = #mqtt_frame{ fixed = #mqtt_frame_fixed{ type = Type }}). -define(MAX_PERMISSION_CACHE_SIZE, 12). --define(CONSUMER_TAG, mqtt_consumer). +-define(CONSUMER_TAG, mqtt). initial_state(Socket, ConnectionName) -> SSLLoginName = ssl_login_name(Socket), {ok, {PeerAddr, _PeerPort}} = rabbit_net:peername(Socket), {ok, {mqtt2amqp_fun, M2A}, {amqp2mqtt_fun, A2M}} = rabbit_mqtt_util:get_topic_translation_funs(), - %% MQTT connections use exactly one channel. The frame max is not - %% applicable and there is no way to know what client is used. #proc_state{socket = Socket, conn_name = ConnectionName, - unacked_pubs = gb_trees:empty(), - awaiting_ack = gb_trees:empty(), - message_id = 1, - subscriptions = #{}, - queue_states = rabbit_queue_type:init(), - consumer_tags = {undefined, undefined}, - channels = {undefined, undefined}, ssl_login_name = SSLLoginName, - send_fun = fun serialise_and_send_to_client/2, peer_addr = PeerAddr, mqtt2amqp_fun = M2A, amqp2mqtt_fun = A2M}. @@ -68,39 +66,53 @@ process_request(?CONNECT, Frame, PState = #proc_state{socket = Socket}) -> process_request(?PUBACK, #mqtt_frame{ - variable = #mqtt_frame_publish{ message_id = MessageId }}, - #proc_state{ channels = {Channel, _}, - awaiting_ack = Awaiting } = PState) -> - %% tag can be missing because of bogus clients and QoS downgrades - case gb_trees:is_defined(MessageId, Awaiting) of - false -> - {ok, PState}; - true -> - Tag = gb_trees:get(MessageId, Awaiting), - amqp_channel:cast(Channel, #'basic.ack'{ delivery_tag = Tag }), - {ok, PState#proc_state{ awaiting_ack = gb_trees:delete(MessageId, Awaiting) }} + variable = #mqtt_frame_publish{message_id = PacketId}}, + #proc_state{unacked_server_pubs = U0, + queue_states = QStates0} = PState) -> + case maps:take(PacketId, U0) of + {QMsgId, U} -> + %% TODO creating binary is expensive + QName = queue_name(?QOS_1, PState), + case rabbit_queue_type:settle(QName, complete, ?CONSUMER_TAG, [QMsgId], QStates0) of + {ok, QStates, [] = _Actions} -> + % incr_queue_stats(QRef, MsgIds, State), + %%TODO handle actions + {ok, PState#proc_state{unacked_server_pubs = U, + queue_states = QStates }}; + {protocol_error, _ErrorType, _Reason, _ReasonArgs} = Err -> + {error, Err, PState} + end; + error -> + {ok, PState} end; process_request(?PUBLISH, Frame = #mqtt_frame{ - fixed = Fixed = #mqtt_frame_fixed{ qos = ?QOS_2 }}, + fixed = Fixed = #mqtt_frame_fixed{qos = ?QOS_2}}, PState) -> % Downgrade QOS_2 to QOS_1 process_request(?PUBLISH, Frame#mqtt_frame{ - fixed = Fixed#mqtt_frame_fixed{ qos = ?QOS_1 }}, + fixed = Fixed#mqtt_frame_fixed{qos = ?QOS_1}}, PState); process_request(?PUBLISH, #mqtt_frame{ - fixed = #mqtt_frame_fixed{ qos = Qos, + fixed = #mqtt_frame_fixed{qos = Qos, retain = Retain, - dup = Dup }, - variable = #mqtt_frame_publish{ topic_name = Topic, + dup = Dup }, + variable = #mqtt_frame_publish{topic_name = Topic, message_id = MessageId }, - payload = Payload }, - PState = #proc_state{retainer_pid = RPid, - amqp2mqtt_fun = Amqp2MqttFun}) -> - publish( + payload = Payload}, + PState = #proc_state{retainer_pid = RPid, + amqp2mqtt_fun = Amqp2MqttFun}) -> + rabbit_global_counters:messages_received(mqtt, 1), + case Qos of + N when N > ?QOS_0 -> + rabbit_global_counters:messages_received_confirm(mqtt, 1); + _ -> + ok + end, + publish_to_queues_with_checks( Topic, fun() -> Msg = #mqtt_msg{retain = Retain, @@ -109,12 +121,16 @@ process_request(?PUBLISH, dup = Dup, message_id = MessageId, payload = Payload}, - Result = amqp_pub(Msg, PState), - case Retain of - false -> ok; - true -> hand_off_to_retainer(RPid, Amqp2MqttFun, Topic, Msg) - end, - {ok, Result} + case publish_to_queues(Msg, PState) of + {ok, _} = Ok -> + case Retain of + false -> ok; + true -> hand_off_to_retainer(RPid, Amqp2MqttFun, Topic, Msg) + end, + Ok; + Error -> + Error + end end, PState); process_request(?SUBSCRIBE, @@ -123,95 +139,96 @@ process_request(?SUBSCRIBE, message_id = SubscribeMsgId, topic_table = Topics}, payload = undefined}, - #proc_state{retainer_pid = RPid, - send_fun = SendFun, - message_id = StateMsgId} = PState0) -> + #proc_state{retainer_pid = RPid} = PState0) -> rabbit_log_connection:debug("Received a SUBSCRIBE for topic(s) ~p", [Topics]), - - {QosResponse, PState} = - lists:foldl(fun(#mqtt_topic{name = TopicName, - qos = Qos}, {QosList, S0}) -> - SupportedQos = supported_subs_qos(Qos), - case ensure_queue(SupportedQos, S0) of - {ok, QueueName, #proc_state{subscriptions = Subs} = S1} -> - case bind(QueueName, TopicName, S1) of - {ok, _Output, S2} -> - SupportedQosList = case maps:find(TopicName, Subs) of - {ok, L} -> [SupportedQos|L]; - error -> [SupportedQos] - end, - {[SupportedQos | QosList], - S2#proc_state{subscriptions = maps:put(TopicName, SupportedQosList, Subs)}}; - {error, Reason, S2} -> + {QosResponse, PState1} = + lists:foldl(fun(_Topic, {[?SUBACK_FAILURE | _] = L, S}) -> + %% Once a subscription failed, mark all following subscriptions + %% as failed instead of creating bindings because we are going + %% to close the client connection anyways. + {[?SUBACK_FAILURE | L], S}; + (#mqtt_topic{name = TopicName, + qos = Qos0}, + {L, S0}) -> + QoS = supported_sub_qos(Qos0), + %%TODO check whether new subscription replaces an existing one + %% (i.e. same topic name and different QoS) + case ensure_queue(QoS, S0) of + {ok, Q} -> + QName = amqqueue:get_name(Q), + case bind(QName, TopicName, S0) of + {ok, _Output, S1 = #proc_state{subscriptions = Subs0}} -> + Subs = maps:put(TopicName, QoS, Subs0), + S2 = S1#proc_state{subscriptions = Subs}, + %%TODO check what happens if we basic.consume multiple times + %% for the same queue + case consume(Q, QoS, S2) of + {ok, S} -> + {[QoS | L], S}; + {error, _Reason} -> + {[?SUBACK_FAILURE | L], S2} + end; + {error, Reason, S} -> rabbit_log:error("Failed to bind ~s with topic ~s: ~p", - [rabbit_misc:rs(QueueName), TopicName, Reason]), - {[?SUBACK_FAILURE | QosList], S2} + [rabbit_misc:rs(QName), TopicName, Reason]), + {[?SUBACK_FAILURE | L], S} end; - {error, _Reason} -> - {[?SUBACK_FAILURE | QosList], S0} + {error, _} -> + {[?SUBACK_FAILURE | L], S0} end end, {[], PState0}, Topics), - SendFun(#mqtt_frame{fixed = #mqtt_frame_fixed{type = ?SUBACK}, - variable = #mqtt_frame_suback{ - message_id = SubscribeMsgId, - qos_table = QosResponse}}, PState), - case lists:member(?SUBACK_FAILURE, QosResponse) of - false -> - %% we may need to send up to length(Topics) messages. - %% if QoS is > 0 then we need to generate a message id, - %% and increment the counter. - StartMsgId = safe_max_id(SubscribeMsgId, StateMsgId), - N = lists:foldl(fun (Topic, Acc) -> - case maybe_send_retained_message(RPid, Topic, Acc, PState) of - {true, X} -> Acc + X; - false -> Acc - end - end, StartMsgId, Topics), - {ok, PState#proc_state{message_id = N}}; - true -> - {error, subscribe_error, PState} + serialise_and_send_to_client( + #mqtt_frame{fixed = #mqtt_frame_fixed{type = ?SUBACK}, + variable = #mqtt_frame_suback{ + message_id = SubscribeMsgId, + %%TODO check correct order of QosResponse + qos_table = QosResponse}}, + PState1), + case QosResponse of + [?SUBACK_FAILURE | _] -> + {error, subscribe_error, PState1}; + _ -> + PState = lists:foldl(fun(Topic, S) -> + maybe_send_retained_message(RPid, Topic, S) + end, PState1, Topics), + {ok, PState} end; process_request(?UNSUBSCRIBE, - #mqtt_frame{ - variable = #mqtt_frame_subscribe{ message_id = MessageId, - topic_table = Topics }, - payload = undefined }, #proc_state{ channels = {Channel, _}, - exchange = Exchange, - client_id = ClientId, - subscriptions = Subs0, - send_fun = SendFun, - mqtt2amqp_fun = Mqtt2AmqpFun } = PState) -> - rabbit_log_connection:debug("Received an UNSUBSCRIBE for topic(s) ~tp", [Topics]), - Queues = rabbit_mqtt_util:subcription_queue_name(ClientId), - Subs1 = - lists:foldl( - fun (#mqtt_topic{ name = TopicName }, Subs) -> - QosSubs = case maps:find(TopicName, Subs) of - {ok, Val} when is_list(Val) -> lists:usort(Val); - error -> [] - end, - RoutingKey = Mqtt2AmqpFun(TopicName), - lists:foreach( - fun (QosSub) -> - Queue = element(QosSub + 1, Queues), - Binding = #'queue.unbind'{ - queue = Queue, - exchange = Exchange, - routing_key = RoutingKey}, - #'queue.unbind_ok'{} = amqp_channel:call(Channel, Binding) - end, QosSubs), - maps:remove(TopicName, Subs) - end, Subs0, Topics), - SendFun(#mqtt_frame{ fixed = #mqtt_frame_fixed { type = ?UNSUBACK }, - variable = #mqtt_frame_suback{ message_id = MessageId }}, - PState), - {ok, PState #proc_state{ subscriptions = Subs1 }}; - -process_request(?PINGREQ, #mqtt_frame{}, #proc_state{ send_fun = SendFun } = PState) -> + #mqtt_frame{variable = #mqtt_frame_subscribe{message_id = MessageId, + topic_table = Topics}, + payload = undefined}, + PState0) -> + rabbit_log_connection:debug("Received an UNSUBSCRIBE for topic(s) ~p", [Topics]), + PState = lists:foldl( + fun(#mqtt_topic{name = TopicName}, + #proc_state{subscriptions = Subs0} = S0) -> + case maps:take(TopicName, Subs0) of + {QoS, Subs} -> + QName = queue_name(QoS, S0), + case unbind(QName, TopicName, S0) of + {ok, _Output, S} -> + S#proc_state{subscriptions = Subs}; + {error, Reason, S} -> + rabbit_log:error("Failed to unbind ~s with topic ~s: ~p", + [rabbit_misc:rs(QName), TopicName, Reason]), + S + end; + error -> + S0 + end + end, PState0, Topics), + serialise_and_send_to_client( + #mqtt_frame{fixed = #mqtt_frame_fixed {type = ?UNSUBACK}, + variable = #mqtt_frame_suback{message_id = MessageId}}, + PState), + {ok, PState}; + +process_request(?PINGREQ, #mqtt_frame{}, PState) -> rabbit_log_connection:debug("Received a PINGREQ"), - SendFun(#mqtt_frame{ fixed = #mqtt_frame_fixed{ type = ?PINGRESP }}, - PState), + serialise_and_send_to_client( + #mqtt_frame{fixed = #mqtt_frame_fixed{type = ?PINGRESP}}, + PState), rabbit_log_connection:debug("Sent a PINGRESP"), {ok, PState}; @@ -226,9 +243,9 @@ process_connect(#mqtt_frame{ clean_sess = CleanSess, client_id = ClientId, keep_alive = Keepalive} = FrameConnect}, - #proc_state{send_fun = SendFun} = PState0) -> - rabbit_log_connection:debug("Received a CONNECT, client ID: ~p, username: ~p, " - "clean session: ~p, protocol version: ~p, keepalive: ~p", + PState0) -> + rabbit_log_connection:debug("Received a CONNECT, client ID: ~s, username: ~s, " + "clean session: ~s, protocol version: ~p, keepalive: ~p", [ClientId, Username, CleanSess, ProtoVersion, Keepalive]), {ReturnCode, SessionPresent, PState} = case rabbit_misc:pipeline([fun check_protocol_version/1, @@ -236,7 +253,8 @@ process_connect(#mqtt_frame{ fun check_credentials/2, fun login/2, fun register_client/2, - fun notify_connection_created/2], + fun notify_connection_created/2, + fun handle_clean_session/2], FrameConnect, PState0) of {ok, SessionPresent0, PState1} -> {?CONNACK_ACCEPT, SessionPresent0, PState1}; @@ -247,7 +265,7 @@ process_connect(#mqtt_frame{ variable = #mqtt_frame_connack{ session_present = SessionPresent, return_code = ReturnCode}}, - SendFun(ResponseFrame, PState), + serialise_and_send_to_client(ResponseFrame, PState), return_connack(ReturnCode, PState). client_id([]) -> @@ -298,8 +316,8 @@ login({UserBin, PassBin, PState0) -> ClientId = client_id(ClientId0), case process_login(UserBin, PassBin, ClientId, PState0) of - connack_dup_auth -> - maybe_clean_sess(PState0); + already_connected -> + {ok, already_connected}; {ok, PState} -> {ok, Frame, PState#proc_state{clean_sess = CleanSess, client_id = ClientId}}; @@ -307,13 +325,15 @@ login({UserBin, PassBin, Err end. +register_client(already_connected, _PState) -> + ok; register_client(Frame = #mqtt_frame_connect{ keep_alive = Keepalive, proto_ver = ProtoVersion}, - PState0 = #proc_state{client_id = ClientId, - socket = Socket, - auth_state = #auth_state{ - vhost = VHost}}) -> + PState = #proc_state{client_id = ClientId, + socket = Socket, + auth_state = #auth_state{ + vhost = VHost}}) -> case rabbit_mqtt_collector:register(ClientId, self()) of {ok, Corr} -> RetainerPid = rabbit_mqtt_retainer_sup:child_for_vhost(VHost), @@ -323,20 +343,19 @@ register_client(Frame = #mqtt_frame_connect{ ExchangeBin = rabbit_mqtt_util:env(exchange), ExchangeName = rabbit_misc:r(VHost, exchange, ExchangeBin), ProtoHumanReadable = {'MQTT', human_readable_mqtt_version(ProtoVersion)}, - PState = PState0#proc_state{ - exchange = ExchangeName, - will_msg = make_will_msg(Frame), - retainer_pid = RetainerPid, - register_state = {pending, Corr}, - proto_ver = ProtoVersion, - info = #info{prefetch = Prefetch, - peer_host = PeerHost, - peer_port = PeerPort, - host = Host, - port = Port, - proto_human = ProtoHumanReadable - }}, - maybe_clean_sess(PState); + {ok, PState#proc_state{ + exchange = ExchangeName, + will_msg = make_will_msg(Frame), + retainer_pid = RetainerPid, + register_state = {pending, Corr}, + proto_ver = ProtoVersion, + info = #info{prefetch = Prefetch, + peer_host = PeerHost, + peer_port = PeerPort, + host = Host, + port = Port, + proto_human = ProtoHumanReadable + }}}; {error, _} = Err -> %% e.g. this node was removed from the MQTT cluster members rabbit_log_connection:error("MQTT cannot accept a connection: " @@ -348,6 +367,8 @@ register_client(Frame = #mqtt_frame_connect{ {error, ?CONNACK_SERVER_UNAVAILABLE} end. +notify_connection_created(already_connected, _PState) -> + ok; notify_connection_created( _Frame, #proc_state{socket = Sock, @@ -394,21 +415,15 @@ return_connack(?CONNACK_ID_REJECTED, S) -> return_connack(?CONNACK_UNACCEPTABLE_PROTO_VER, S) -> {error, unsupported_protocol_version, S}. -maybe_clean_sess(PState = #proc_state {clean_sess = false, - auth_state = #auth_state{vhost = VHost}, - client_id = ClientId }) -> - SessionPresent = session_present(VHost, ClientId), - case SessionPresent of - false -> - %% ensure_queue/2 not only ensures that queue is created, but also starts consuming from it. - %% Let's avoid creating that queue until explicitly asked by a client. - %% Then publish-only clients, that connect with clean_sess=true due to some misconfiguration, - %% will consume less resources. - {ok, SessionPresent, PState}; - true -> - case ensure_queue(?QOS_1, PState) of - {ok, _QueueName, PState1} -> - {ok, SessionPresent, PState1}; +handle_clean_session(_, PState0 = #proc_state{clean_sess = false}) -> + case get_queue(?QOS_1, PState0) of + {error, not_found} -> + %% Queue will be created later when client subscribes. + {ok, _SessionPresent = false, PState0}; + {ok, Q} -> + case consume(Q, ?QOS_1, PState0) of + {ok, PState} -> + {ok, _SessionPresent = true, PState}; {error, access_refused} -> {error, ?CONNACK_NOT_AUTHORIZED}; {error, _Reason} -> @@ -416,23 +431,20 @@ maybe_clean_sess(PState = #proc_state {clean_sess = false, {error, ?CONNACK_SERVER_UNAVAILABLE} end end; -maybe_clean_sess(PState = #proc_state {clean_sess = true, - client_id = ClientId, - auth_state = #auth_state{user = User, - username = Username, - vhost = VHost, - authz_ctx = AuthzCtx}}) -> - {_, QueueName} = rabbit_mqtt_util:subcription_queue_name(ClientId), - Queue = rabbit_misc:r(VHost, queue, QueueName), - case rabbit_amqqueue:exists(Queue) of - false -> - {ok, false, PState}; - true -> +handle_clean_session(_, PState = #proc_state{clean_sess = true, + auth_state = #auth_state{user = User, + username = Username, + authz_ctx = AuthzCtx}}) -> + case get_queue(?QOS_1, PState) of + {error, not_found} -> + {ok, _SessionPresent = false, PState}; + {ok, Q0} -> + QName = amqqueue:get_name(Q0), %% configure access to queue required for queue.delete - case check_resource_access(User, Queue, configure, AuthzCtx) of + case check_resource_access(User, QName, configure, AuthzCtx) of ok -> rabbit_amqqueue:with( - Queue, + QName, fun (Q) -> rabbit_queue_type:delete(Q, false, false, Username) end, @@ -445,12 +457,27 @@ maybe_clean_sess(PState = #proc_state {clean_sess = true, ({absent, _Q, _Reason}) -> ok end), - {ok, false, PState}; + {ok, _SessionPresent = false, PState}; {error, access_refused} -> {error, ?CONNACK_NOT_AUTHORIZED} end end. +-spec get_queue(qos(), proc_state()) -> + {ok, amqqueue:amqqueue()} | {error, not_found}. +get_queue(QoS, PState) -> + QName = queue_name(QoS, PState), + rabbit_amqqueue:lookup(QName). + +queue_name(QoS, #proc_state{auth_state = #auth_state{vhost = VHost}, + client_id = ClientId}) -> + QNameBin = queue_name_bin(QoS, ClientId), + rabbit_misc:r(VHost, queue, QNameBin). + +queue_name_bin(QoS, ClientId) -> + Names = rabbit_mqtt_util:queue_names(ClientId), + element(QoS + 1, Names). + hand_off_to_retainer(RetainerPid, Amqp2MqttFun, Topic0, #mqtt_msg{payload = <<"">>}) -> Topic1 = Amqp2MqttFun(Topic0), rabbit_mqtt_retainer:clear(RetainerPid, Topic1), @@ -460,148 +487,36 @@ hand_off_to_retainer(RetainerPid, Amqp2MqttFun, Topic0, Msg) -> rabbit_mqtt_retainer:retain(RetainerPid, Topic1, Msg), ok. -maybe_send_retained_message(RPid, #mqtt_topic{name = Topic0, qos = SubscribeQos}, MsgId, - #proc_state{ send_fun = SendFun, - amqp2mqtt_fun = Amqp2MqttFun } = PState) -> +maybe_send_retained_message(RPid, #mqtt_topic{name = Topic0, qos = SubscribeQos}, + #proc_state{amqp2mqtt_fun = Amqp2MqttFun, + packet_id = PacketId} = PState0) -> Topic1 = Amqp2MqttFun(Topic0), case rabbit_mqtt_retainer:fetch(RPid, Topic1) of - undefined -> false; + undefined -> PState0; Msg -> %% calculate effective QoS as the lower value of SUBSCRIBE frame QoS %% and retained message QoS. The spec isn't super clear on this, we %% do what Mosquitto does, per user feedback. Qos = erlang:min(SubscribeQos, Msg#mqtt_msg.qos), - Id = case Qos of - ?QOS_0 -> undefined; - ?QOS_1 -> MsgId - end, - SendFun(#mqtt_frame{fixed = #mqtt_frame_fixed{ - type = ?PUBLISH, - qos = Qos, - dup = false, - retain = Msg#mqtt_msg.retain - }, variable = #mqtt_frame_publish{ - message_id = Id, - topic_name = Topic1 - }, - payload = Msg#mqtt_msg.payload}, PState), - case Qos of - ?QOS_0 -> false; - ?QOS_1 -> {true, 1} - end - end. - --spec amqp_callback(#'basic.ack'{} | {#'basic.deliver'{}, #amqp_msg{}, {pid(), pid(), pid()}}, #proc_state{}) -> {'ok', #proc_state{}} | {'error', term(), term()}. -amqp_callback({#'basic.deliver'{ consumer_tag = ConsumerTag, - delivery_tag = DeliveryTag, - routing_key = RoutingKey }, - #amqp_msg{ props = #'P_basic'{ headers = Headers }, - payload = Payload }, - DeliveryCtx} = Delivery, - #proc_state{ channels = {Channel, _}, - awaiting_ack = Awaiting, - message_id = MsgId, - send_fun = SendFun, - amqp2mqtt_fun = Amqp2MqttFun } = PState) -> - notify_received(DeliveryCtx), - case {delivery_dup(Delivery), delivery_qos(ConsumerTag, Headers, PState)} of - {true, {?QOS_0, ?QOS_1}} -> - amqp_channel:cast( - Channel, #'basic.ack'{ delivery_tag = DeliveryTag }), - {ok, PState}; - {true, {?QOS_0, ?QOS_0}} -> - {ok, PState}; - {Dup, {DeliveryQos, _SubQos} = Qos} -> - TopicName = Amqp2MqttFun(RoutingKey), - SendFun( - #mqtt_frame{ fixed = #mqtt_frame_fixed{ + {Id, PState} = case Qos of + ?QOS_0 -> {undefined, PState0}; + ?QOS_1 -> {PacketId, PState0#proc_state{packet_id = increment_packet_id(PacketId)}} + end, + serialise_and_send_to_client( + #mqtt_frame{fixed = #mqtt_frame_fixed{ type = ?PUBLISH, - qos = DeliveryQos, - dup = Dup }, - variable = #mqtt_frame_publish{ - message_id = - case DeliveryQos of - ?QOS_0 -> undefined; - ?QOS_1 -> MsgId - end, - topic_name = TopicName }, - payload = Payload}, PState), - case Qos of - {?QOS_0, ?QOS_0} -> - {ok, PState}; - {?QOS_1, ?QOS_1} -> - Awaiting1 = gb_trees:insert(MsgId, DeliveryTag, Awaiting), - PState1 = PState#proc_state{ awaiting_ack = Awaiting1 }, - PState2 = next_msg_id(PState1), - {ok, PState2}; - {?QOS_0, ?QOS_1} -> - amqp_channel:cast( - Channel, #'basic.ack'{ delivery_tag = DeliveryTag }), - {ok, PState} - end - end; - -amqp_callback(#'basic.ack'{ multiple = true, delivery_tag = Tag } = Ack, - PState = #proc_state{ unacked_pubs = UnackedPubs, - send_fun = SendFun }) -> - case gb_trees:size(UnackedPubs) > 0 andalso - gb_trees:take_smallest(UnackedPubs) of - {TagSmall, MsgId, UnackedPubs1} when TagSmall =< Tag -> - SendFun( - #mqtt_frame{ fixed = #mqtt_frame_fixed{ type = ?PUBACK }, - variable = #mqtt_frame_publish{ message_id = MsgId }}, + qos = Qos, + dup = false, + retain = Msg#mqtt_msg.retain + }, variable = #mqtt_frame_publish{ + message_id = Id, + topic_name = Topic1 + }, + payload = Msg#mqtt_msg.payload}, PState), - amqp_callback(Ack, PState #proc_state{ unacked_pubs = UnackedPubs1 }); - _ -> - {ok, PState} - end; - -amqp_callback(#'basic.ack'{ multiple = false, delivery_tag = Tag }, - PState = #proc_state{ unacked_pubs = UnackedPubs, - send_fun = SendFun }) -> - SendFun( - #mqtt_frame{ fixed = #mqtt_frame_fixed{ type = ?PUBACK }, - variable = #mqtt_frame_publish{ - message_id = gb_trees:get( - Tag, UnackedPubs) }}, PState), - {ok, PState #proc_state{ unacked_pubs = gb_trees:delete(Tag, UnackedPubs) }}. - -delivery_dup({#'basic.deliver'{ redelivered = Redelivered }, - #amqp_msg{ props = #'P_basic'{ headers = Headers }}, - _DeliveryCtx}) -> - case rabbit_mqtt_util:table_lookup(Headers, <<"x-mqtt-dup">>) of - undefined -> Redelivered; - {bool, Dup} -> Redelivered orelse Dup + PState end. -ensure_valid_mqtt_message_id(Id) when Id >= 16#ffff -> - 1; -ensure_valid_mqtt_message_id(Id) -> - Id. - -safe_max_id(Id0, Id1) -> - ensure_valid_mqtt_message_id(erlang:max(Id0, Id1)). - -next_msg_id(PState = #proc_state{ message_id = MsgId0 }) -> - MsgId1 = ensure_valid_mqtt_message_id(MsgId0 + 1), - PState#proc_state{ message_id = MsgId1 }. - -%% decide at which qos level to deliver based on subscription -%% and the message publish qos level. non-MQTT publishes are -%% assumed to be qos 1, regardless of delivery_mode. -delivery_qos(Tag, _Headers, #proc_state{ consumer_tags = {Tag, _} }) -> - {?QOS_0, ?QOS_0}; -delivery_qos(Tag, Headers, #proc_state{ consumer_tags = {_, Tag} }) -> - case rabbit_mqtt_util:table_lookup(Headers, <<"x-mqtt-publish-qos">>) of - {byte, Qos} -> {lists:min([Qos, ?QOS_1]), ?QOS_1}; - undefined -> {?QOS_1, ?QOS_1} - end. - -session_present(VHost, ClientId) -> - {_, QueueQ1} = rabbit_mqtt_util:subcription_queue_name(ClientId), - QueueName = rabbit_misc:r(VHost, queue, QueueQ1), - rabbit_amqqueue:exists(QueueName). - make_will_msg(#mqtt_frame_connect{ will_flag = false }) -> undefined; make_will_msg(#mqtt_frame_connect{ will_retain = Retain, @@ -626,7 +541,7 @@ process_login(_UserBin, _PassBin, _ClientId, rabbit_core_metrics:auth_attempt_failed(list_to_binary(inet:ntoa(Addr)), Username, mqtt), rabbit_log_connection:warning("MQTT detected duplicate connect/login attempt for user ~tp, vhost ~tp", [UsernameStr, VHostStr]), - connack_dup_auth; + already_connected; process_login(UserBin, PassBin, ClientId0, #proc_state{socket = Sock, ssl_login_name = SslLoginName, @@ -924,196 +839,156 @@ creds(User, Pass, SSLLoginName) -> _ -> nocreds end. -supported_subs_qos(?QOS_0) -> ?QOS_0; -supported_subs_qos(?QOS_1) -> ?QOS_1; -supported_subs_qos(?QOS_2) -> ?QOS_1. +supported_sub_qos(?QOS_0) -> ?QOS_0; +supported_sub_qos(?QOS_1) -> ?QOS_1; +supported_sub_qos(?QOS_2) -> ?QOS_1. delivery_mode(?QOS_0) -> 1; delivery_mode(?QOS_1) -> 2; delivery_mode(?QOS_2) -> 2. -maybe_quorum(Qos1Args, CleanSession, Queue) -> - case {rabbit_mqtt_util:env(durable_queue_type), CleanSession} of - %% it is possible to Quorum queues only if Clean Session == False - %% else always use Classic queues - %% Clean Session == True sets auto-delete to True and quorum queues - %% does not support auto-delete flag - {quorum, false} -> lists:append(Qos1Args, - [{<<"x-queue-type">>, longstr, <<"quorum">>}]); - - {quorum, true} -> - rabbit_log:debug("Can't use quorum queue for ~ts. " ++ - "The clean session is true. Classic queue will be used", [Queue]), - Qos1Args; - _ -> Qos1Args - end. -%% different qos subscriptions are received in different queues -%% with appropriate durability and timeout arguments -%% this will lead to duplicate messages for overlapping subscriptions -%% with different qos values - todo: prevent duplicates -%ensure_queue(Qos, #proc_state{ channels = {Channel, _}, -% client_id = ClientId, -% clean_sess = CleanSess, -% consumer_tags = {TagQ0, TagQ1} = Tags} = PState) -> -% {QueueQ0, QueueQ1} = rabbit_mqtt_util:subcription_queue_name(ClientId), -% Qos1Args = case {rabbit_mqtt_util:env(subscription_ttl), CleanSess} of -% {undefined, _} -> -% []; -% {Ms, false} when is_integer(Ms) -> -% [{<<"x-expires">>, long, Ms}]; -% _ -> -% [] -% end, -% QueueSetup = -% case {TagQ0, TagQ1, Qos} of -% {undefined, _, ?QOS_0} -> -% {QueueQ0, -% #'queue.declare'{ queue = QueueQ0, -% durable = false, -% auto_delete = true }, -% #'basic.consume'{ queue = QueueQ0, -% no_ack = true }}; -% {_, undefined, ?QOS_1} -> -% {QueueQ1, -% #'queue.declare'{ queue = QueueQ1, -% durable = true, -% %% Clean session means a transient connection, -% %% translating into auto-delete. -% %% -% %% see rabbitmq/rabbitmq-mqtt#37 -% auto_delete = CleanSess, -% arguments = maybe_quorum(Qos1Args, CleanSess, QueueQ1)}, -% #'basic.consume'{ queue = QueueQ1, -% no_ack = false }}; -% {_, _, ?QOS_0} -> -% {exists, QueueQ0}; -% {_, _, ?QOS_1} -> -% {exists, QueueQ1} -% end, -% case QueueSetup of -% {Queue, Declare, Consume} -> -% #'queue.declare_ok'{} = amqp_channel:call(Channel, Declare), -% #'basic.consume_ok'{ consumer_tag = Tag } = -% amqp_channel:call(Channel, Consume), -% {Queue, PState #proc_state{ consumer_tags = setelement(Qos+1, Tags, Tag) }}; -% {exists, Q} -> -% {Q, PState} -% end. - -ensure_queue(?QOS_0, %% spike handles only QoS0 - #proc_state{ - client_id = ClientId, - clean_sess = _CleanSess, - queue_states = QueueStates0, - auth_state = #auth_state{ - vhost = VHost, - user = User = #user{username = Username}, - authz_ctx = AuthzCtx}, - info = #info{prefetch = Prefetch} - } = PState0) -> - {QueueBin, _QueueQos1Bin} = rabbit_mqtt_util:subcription_queue_name(ClientId), - QueueName = rabbit_misc:r(VHost, queue, QueueBin), - case rabbit_amqqueue:exists(QueueName) of - true -> - {ok, QueueName, PState0}; - false -> - %% read access to queue required for basic.consume - case check_resource_access(User, QueueName, read, AuthzCtx) of +ensure_queue(QoS, #proc_state{ + client_id = ClientId, + clean_sess = CleanSess, + auth_state = #auth_state{ + vhost = VHost, + user = User = #user{username = Username}, + authz_ctx = AuthzCtx} + } = PState) -> + case get_queue(QoS, PState) of + {ok, Q} -> + {ok, Q}; + {error, not_found} -> + QNameBin = queue_name_bin(QoS, ClientId), + QName = rabbit_misc:r(VHost, queue, QNameBin), + %% configure access to queue required for queue.declare + case check_resource_access(User, QName, configure, AuthzCtx) of ok -> - %% configure access to queue required for queue.declare - case check_resource_access(User, QueueName, configure, AuthzCtx) of - ok -> - rabbit_core_metrics:queue_declared(QueueName), - Durable = false, - AutoDelete = true, - case rabbit_amqqueue:with( - QueueName, - fun (Q) -> ok = rabbit_amqqueue:assert_equivalence( - Q, Durable, AutoDelete, [], none), - rabbit_amqqueue:stat(Q) - end) of - {error, not_found} -> - case rabbit_vhost_limit:is_over_queue_limit(VHost) of - false -> - case rabbit_amqqueue:declare(QueueName, Durable, AutoDelete, - [], none, Username) of - {new, Q} when ?is_amqqueue(Q) -> - rabbit_core_metrics:queue_created(QueueName), - Spec = #{no_ack => true, - channel_pid => self(), - limiter_pid => none, - limiter_active => false, - prefetch_count => Prefetch, - consumer_tag => ?CONSUMER_TAG, - exclusive_consume => false, - args => [], - ok_msg => undefined, - acting_user => Username}, - case rabbit_queue_type:consume(Q, Spec, QueueStates0) of - {ok, QueueStates, _Actions = []} -> - % rabbit_global_counters:consumer_created(mqtt), - PState = PState0#proc_state{queue_states = QueueStates}, - {ok, QueueName, PState}; - Other -> - rabbit_log:error("Failed to consume from ~s: ~p", - [rabbit_misc:rs(QueueName), Other]), - {error, queue_consume} - end; - Other -> - rabbit_log:error("Failed to declare ~s: ~p", - [rabbit_misc:rs(QueueName), Other]), - {error, queue_declare} - end; - {true, Limit} -> - rabbit_log:error("cannot declare ~s because " - "queue limit ~p in vhost '~s' is reached", - [rabbit_misc:rs(QueueName), Limit, VHost]), - {error, access_refused} - end; + rabbit_core_metrics:queue_declared(QName), + case rabbit_vhost_limit:is_over_queue_limit(VHost) of + false -> + case rabbit_amqqueue:declare(QName, + _Durable = true, + _AutoDelete = false, + queue_args(QoS, CleanSess), + queue_owner(QoS, CleanSess), + Username) of + {new, Q} when ?is_amqqueue(Q) -> + rabbit_core_metrics:queue_created(QName), + {ok, Q}; Other -> - rabbit_log:error("Expected ~s to not exist, got: ~p", - [rabbit_misc:rs(QueueName), Other]), - {error, queue_access} + rabbit_log:error("Failed to declare ~s: ~p", + [rabbit_misc:rs(QName), Other]), + {error, queue_declare} end; - {error, access_refused} = E -> - E + {true, Limit} -> + rabbit_log:error("cannot declare ~s because " + "queue limit ~p in vhost '~s' is reached", + [rabbit_misc:rs(QName), Limit, VHost]), + {error, access_refused} end; {error, access_refused} = E -> E end end. +queue_owner(QoS, CleanSess) + when QoS =:= ?QOS_0 orelse CleanSess -> + %% Exclusive queues are auto-deleted after node restart while auto-delete queues are not. + %% Therefore make durable queue exclusive. + self(); +queue_owner(_, _) -> + none. + +queue_args(QoS, CleanSess) + when QoS =:= ?QOS_0 orelse CleanSess -> + []; +queue_args(_, _) -> + Args = case rabbit_mqtt_util:env(subscription_ttl) of + Ms when is_integer(Ms) -> + [{<<"x-expires">>, long, Ms}]; + _ -> + [] + end, + case rabbit_mqtt_util:env(durable_queue_type) of + quorum -> + [{<<"x-queue-type">>, longstr, <<"quorum">>} | Args]; + _ -> + Args + end. + +consume(Q, QoS, #proc_state{ + queue_states = QStates0, + auth_state = #auth_state{ + user = User = #user{username = Username}, + authz_ctx = AuthzCtx}, + info = #info{prefetch = Prefetch} + } = PState0) -> + QName = amqqueue:get_name(Q), + %% read access to queue required for basic.consume + case check_resource_access(User, QName, read, AuthzCtx) of + ok -> + Spec = #{no_ack => QoS =:= ?QOS_0, + channel_pid => self(), + limiter_pid => none, + limiter_active => false, + prefetch_count => Prefetch, + consumer_tag => ?CONSUMER_TAG, + exclusive_consume => false, + args => [], + ok_msg => undefined, + acting_user => Username}, + case rabbit_queue_type:consume(Q, Spec, QStates0) of + {ok, QStates, _Actions = []} -> + % rabbit_global_counters:consumer_created(mqtt), + PState = PState0#proc_state{queue_states = QStates}, + {ok, PState}; + {error, Reason} = Err -> + rabbit_log:error("Failed to consume from ~s: ~p", + [rabbit_misc:rs(QName), Reason]), + Err + end; + {error, access_refused} = Err -> + Err + end. + bind(QueueName, TopicName, PState) -> + binding_action_with_checks({QueueName, TopicName, fun rabbit_binding:add/2}, PState). +unbind(QueueName, TopicName, PState) -> + binding_action_with_checks({QueueName, TopicName, fun rabbit_binding:remove/2}, PState). + +binding_action_with_checks(Input, PState) -> + %% Same permission checks required for both binding and unbinding + %% queue to / from topic exchange. rabbit_misc:pipeline( - [fun bind_check_queue_write_access/2, - fun bind_check_exchange_read_access/2, - fun bind_check_topic_access/2, - fun bind_add/2 - ], {QueueName, TopicName}, PState). - -bind_check_queue_write_access( - {QueueName, _TopicName}, + [fun check_queue_write_access/2, + fun check_exchange_read_access/2, + fun check_topic_access/2, + fun binding_action/2], + Input, PState). + +check_queue_write_access( + {QueueName, _, _}, #proc_state{auth_state = #auth_state{ user = User, authz_ctx = AuthzCtx}}) -> - %% write access to queue required for queue.bind + %% write access to queue required for queue.(un)bind check_resource_access(User, QueueName, write, AuthzCtx). -bind_check_exchange_read_access( - {_QueueName, _TopicName}, - #proc_state{exchange = ExchangeName, +check_exchange_read_access( + _, #proc_state{exchange = ExchangeName, auth_state = #auth_state{ user = User, authz_ctx = AuthzCtx}}) -> - %% read access to exchange required for queue.bind + %% read access to exchange required for queue.(un)bind check_resource_access(User, ExchangeName, read, AuthzCtx). -bind_check_topic_access( {_QueueName, TopicName}, PState) -> +check_topic_access({_, TopicName, _}, PState) -> check_topic_access(TopicName, read, PState). -bind_add( - {QueueName, TopicName}, +binding_action( + {QueueName, TopicName, BindingFun}, #proc_state{exchange = ExchangeName, auth_state = #auth_state{ user = #user{username = Username}}, @@ -1122,7 +997,7 @@ bind_add( Binding = #binding{source = ExchangeName, destination = QueueName, key = RoutingKey}, - rabbit_binding:add(Binding, Username). + BindingFun(Binding, Username). send_will(PState = #proc_state{will_msg = undefined}) -> PState; @@ -1130,11 +1005,10 @@ send_will(PState = #proc_state{will_msg = undefined}) -> send_will(PState = #proc_state{will_msg = WillMsg = #mqtt_msg{retain = Retain, topic = Topic}, retainer_pid = RPid, - channels = {ChQos0, ChQos1}, amqp2mqtt_fun = Amqp2MqttFun}) -> case check_topic_access(Topic, write, PState) of ok -> - amqp_pub(WillMsg, PState), + publish_to_queues(WillMsg, PState), case Retain of false -> ok; true -> @@ -1145,27 +1019,27 @@ send_will(PState = #proc_state{will_msg = WillMsg = #mqtt_msg{retain = Retain, "Could not send last will: ~tp", [Error]) end, - case ChQos1 of - undefined -> ok; - _ -> amqp_channel:close(ChQos1) - end, - case ChQos0 of - undefined -> ok; - _ -> amqp_channel:close(ChQos0) - end, - PState #proc_state{ channels = {undefined, undefined} }. + %%TODO cancel queue client? + % case ChQos1 of + % undefined -> ok; + % _ -> amqp_channel:close(ChQos1) + % end, + % case ChQos0 of + % undefined -> ok; + % _ -> amqp_channel:close(ChQos0) + % end, + PState. -amqp_pub(undefined, PState) -> - PState; -amqp_pub(#mqtt_msg{qos = Qos, - topic = Topic, - dup = Dup, - message_id = _MessageId, %% spike handles only QoS0 - payload = Payload}, - PState = #proc_state{exchange = ExchangeName, - % unacked_pubs = UnackedPubs, - % awaiting_seqno = SeqNo, - mqtt2amqp_fun = Mqtt2AmqpFun}) -> +publish_to_queues(undefined, PState) -> + {ok, PState}; +publish_to_queues( + #mqtt_msg{qos = Qos, + topic = Topic, + dup = Dup, + message_id = MessageId, + payload = Payload}, + #proc_state{exchange = ExchangeName, + mqtt2amqp_fun = Mqtt2AmqpFun} = PState) -> RoutingKey = Mqtt2AmqpFun(Topic), Confirm = Qos > ?QOS_0, Headers = [{<<"x-mqtt-publish-qos">>, byte, Qos}, @@ -1185,7 +1059,7 @@ amqp_pub(#mqtt_msg{qos = Qos, exchange_name = ExchangeName, routing_keys = [RoutingKey], content = Content, - id = <<>>, + id = <<>>, %% GUID set in rabbit_classic_queue is_persistent = Confirm }, Delivery = #delivery{ @@ -1193,50 +1067,77 @@ amqp_pub(#mqtt_msg{qos = Qos, confirm = Confirm, sender = self(), message = BasicMessage, - msg_seq_no = undefined, %% spike handles only QoS0 + msg_seq_no = MessageId, flow = noflow %%TODO enable flow control }, + case rabbit_exchange:lookup(ExchangeName) of + {ok, Exchange} -> + QNames = rabbit_exchange:route(Exchange, Delivery), + deliver_to_queues(Delivery, QNames, PState); + {error, not_found} -> + rabbit_log:error("~s not found", [rabbit_misc:rs(ExchangeName)]), + {error, exchange_not_found, PState} + end. - Exchange = rabbit_exchange:lookup_or_die(ExchangeName), - QNames = rabbit_exchange:route(Exchange, Delivery), - deliver_to_queues(Delivery, QNames, PState). - -deliver_to_queues(#delivery{confirm = false}, - _RoutedToQueueNames = [], - PState) -> - % rabbit_global_counters:messages_unroutable_dropped(mqtt, 1), - PState; -deliver_to_queues(Delivery = #delivery{message = _Message = #basic_message{exchange_name = _XName}, - confirm = _Confirm, - msg_seq_no = _MsgSeqNo}, - RoutedToQueueNames, - PState = #proc_state{queue_states = QueueStates0}) -> - Qs0 = rabbit_amqqueue:lookup(RoutedToQueueNames), +deliver_to_queues(Delivery, + RoutedToQNames, + PState0 = #proc_state{queue_states = QStates0}) -> + Qs0 = rabbit_amqqueue:lookup(RoutedToQNames), Qs = rabbit_amqqueue:prepend_extra_bcc(Qs0), - % QueueNames = lists:map(fun amqqueue:get_name/1, Qs), - - {ok, QueueStates, _Actions} = rabbit_queue_type:deliver(Qs, Delivery, QueueStates0), - % rabbit_global_counters:messages_routed(mqtt, length(Qs)), - - %% NB: the order here is important since basic.returns must be - %% sent before confirms. - %% TODO: AMQP 0.9.1 mandatory flag corresponds to MQTT 5 PUBACK reason code "No matching subscribers" - % ok = process_routing_mandatory(Mandatory, Qs, Message, State0), - %% spike handles only QoS0 - % State1 = process_routing_confirm(Confirm, QueueNames, - % MsgSeqNo, XName, State0), - - %% Actions must be processed after registering confirms as actions may - %% contain rejections of publishes - %% TODO handle Actions: For example if the messages is rejected, MQTT 5 allows to send a NACK - %% back to the client (via PUBACK Reason Code). - % State = handle_queue_actions(Actions, State1#ch{queue_states = QueueStates}), - PState#proc_state{queue_states = QueueStates}. + case rabbit_queue_type:deliver(Qs, Delivery, QStates0) of + {ok, QStates, _Actions = []} -> + rabbit_global_counters:messages_routed(mqtt, length(Qs)), + PState = process_routing_confirm(Delivery, Qs, PState0), + %% Actions must be processed after registering confirms as actions may + %% contain rejections of publishes + %% TODO handle Actions: For example if the messages is rejected, MQTT 5 allows to send a NACK + %% back to the client (via PUBACK Reason Code). + % State = handle_queue_actions(Actions, State1#ch{queue_states = QueueStates}), + {ok, PState#proc_state{queue_states = QStates}}; + {error, Reason} -> + rabbit_log:error("Failed to deliver message to queues " + "packet_id=~p, queues=~p, Reason=~p", + [Delivery#delivery.msg_seq_no, queue_names(Qs), Reason]), + {error, Reason, PState0} + end. -serialise_and_send_to_client(Frame, #proc_state{proto_ver = ProtoVer, socket = Sock }) -> +process_routing_confirm(#delivery{confirm = false}, [], PState) -> + rabbit_global_counters:messages_unroutable_dropped(mqtt, 1), + PState; +process_routing_confirm(#delivery{confirm = true, + msg_seq_no = MsgId}, [], PState) -> + rabbit_global_counters:messages_unroutable_returned(mqtt, 1), + %% MQTT 5 spec: + %% If the Server knows that there are no matching subscribers, it MAY use + %% Reason Code 0x10 (No matching subscribers) instead of 0x00 (Success). + send_puback(MsgId, PState), + PState; +process_routing_confirm(#delivery{confirm = false}, _, PState) -> + PState; +process_routing_confirm(#delivery{confirm = true, + msg_seq_no = MsgId}, + Qs, PState = #proc_state{unacked_client_pubs = U0}) -> + QNames = queue_names(Qs), + U = rabbit_mqtt_confirms:insert(MsgId, QNames, U0), + PState#proc_state{unacked_client_pubs = U}. + +send_puback(MsgIds, PState) + when is_list(MsgIds) -> + lists:foreach(fun(Id) -> + send_puback(Id, PState) + end, MsgIds); +send_puback(MsgId, PState) -> + rabbit_global_counters:messages_confirmed(mqtt, 1), + serialise_and_send_to_client( + #mqtt_frame{fixed = #mqtt_frame_fixed{type = ?PUBACK}, + variable = #mqtt_frame_publish{message_id = MsgId}}, + PState). + +serialise_and_send_to_client(Frame, #proc_state{proto_ver = ProtoVer, socket = Sock}) -> %%TODO Test sending large frames at high speed: %% Will we need garbage collection as done in rabbit_writer:maybe_gc_large_msg/1? - try rabbit_net:port_command(Sock, rabbit_mqtt_frame:serialise(Frame, ProtoVer)) + Data = rabbit_mqtt_frame:serialise(Frame, ProtoVer), + try rabbit_net:port_command(Sock, Data) catch _:Error -> rabbit_log_connection:error("MQTT: a socket write failed, the socket might already be closed"), rabbit_log_connection:debug("Failed to write to socket ~p, error: ~p, frame: ~p", @@ -1288,69 +1189,154 @@ handle_down({'DOWN', _MRef, process, QPid, Reason}, PState0#proc_state{queue_states = QStates} end. -handle_queue_event({queue_event, QRef, Evt}, - PState0 = #proc_state{queue_states = QueueStates0}) -> - case rabbit_queue_type:handle_event(QRef, Evt, QueueStates0) of - {ok, QueueStates, Actions} -> - PState1 = PState0#proc_state{queue_states = QueueStates}, +handle_queue_event({queue_event, QName, Evt}, + PState0 = #proc_state{queue_states = QStates0, + unacked_client_pubs = U0}) -> + case rabbit_queue_type:handle_event(QName, Evt, QStates0) of + {ok, QStates, Actions} -> + PState1 = PState0#proc_state{queue_states = QStates}, PState = handle_queue_actions(Actions, PState1), {ok, PState}; eol -> - {error, queue_eol, PState0}; + %%TODO handle consuming queue down + % State1 = handle_consuming_queue_down_or_eol(QRef, State0), + {ConfirmMsgIds, U} = rabbit_mqtt_confirms:remove_queue(QName, U0), + PState = PState0#proc_state{unacked_client_pubs = U}, + send_puback(ConfirmMsgIds, PState), + {ok, PState}; {protocol_error, _Type, _Reason, _ReasonArgs} = Error -> {error, Error, PState0} end. handle_queue_actions(Actions, #proc_state{} = PState0) -> lists:foldl( - fun ({deliver, ?CONSUMER_TAG, _AckRequired = false, Msgs}, S) -> - handle_deliver(Msgs, S) + fun ({deliver, ?CONSUMER_TAG, Ack, Msgs}, S) -> + deliver_to_client(Msgs, Ack, S); + ({settled, QName, MsgIds}, S = #proc_state{unacked_client_pubs = U0}) -> + {ConfirmMsgIds, U} = rabbit_mqtt_confirms:confirm(MsgIds, QName, U0), + send_puback(ConfirmMsgIds, S), + S#proc_state{unacked_client_pubs = U}; + ({rejected, _QName, MsgIds}, S = #proc_state{unacked_client_pubs = U0}) -> + %% Negative acks are supported in MQTT 5 only. + %% Therefore, in MQTT 3 we ignore rejected messages. + U = lists:foldl( + fun(MsgId, Acc0) -> + case rabbit_mqtt_confirms:reject(MsgId, Acc0) of + {ok, Acc} -> Acc; + {error, not_found} -> Acc0 + end + end, U0, MsgIds), + S#proc_state{unacked_client_pubs = U} end, PState0, Actions). -handle_deliver(Msgs, PState) - when is_list(Msgs) -> +deliver_to_client(Msgs, Ack, PState) -> lists:foldl(fun(Msg, S) -> - handle_deliver0(Msg, S) + deliver_one_to_client(Msg, Ack, S) end, PState, Msgs). -handle_deliver0({QName, QPid, _MsgId, Redelivered, - #basic_message{routing_keys = [RoutingKey | _CcRoutes], - content = #content{ - properties = #'P_basic'{headers = Headers}, - payload_fragments_rev = FragmentsRev}}}, - PState = #proc_state{send_fun = SendFun, - amqp2mqtt_fun = Amqp2MqttFun, - queue_states = QStates}) -> - Dup = case rabbit_mqtt_util:table_lookup(Headers, <<"x-mqtt-dup">>) of - undefined -> Redelivered; - {bool, Dup0} -> Redelivered orelse Dup0 - end, +deliver_one_to_client(Msg = {QName, QPid, QMsgId, _Redelivered, + #basic_message{content = #content{properties = #'P_basic'{headers = Headers}}}}, + AckRequired, PState0) -> + PublisherQoS = case rabbit_mqtt_util:table_lookup(Headers, <<"x-mqtt-publish-qos">>) of + {byte, QoS0} -> + QoS0; + undefined -> + %% non-MQTT publishes are assumed to be QoS 1 regardless of delivery_mode + ?QOS_1 + end, + SubscriberQoS = case AckRequired of + true -> + ?QOS_1; + false -> + ?QOS_0 + end, + %% "The QoS of Application Messages sent in response to a Subscription MUST be the minimum + %% of the QoS of the originally published message and the Maximum QoS granted by the Server + %% [MQTT-3.8.4-8]." + QoS = min(PublisherQoS, SubscriberQoS), + PState1 = maybe_publish_to_client(Msg, QoS, PState0), + PState = maybe_ack(AckRequired, QoS, QName, QMsgId, PState1), + %%TODO GC + % case GCThreshold of + % undefined -> ok; + % _ -> rabbit_basic:maybe_gc_large_msg(Content, GCThreshold) + % end, + ok = maybe_notify_sent(QName, QPid, PState), + PState. + +maybe_publish_to_client({_, _, _, _Redelivered = true, _}, ?QOS_0, PState) -> + %% Do no redeliver to MQTT subscriber who gets message at most once. + PState; +maybe_publish_to_client( + {_QName, _QPid, QMsgId, Redelivered, + #basic_message{ + routing_keys = [RoutingKey | _CcRoutes], + content = #content{payload_fragments_rev = FragmentsRev}}}, + QoS, PState0 = #proc_state{amqp2mqtt_fun = Amqp2MqttFun}) -> + {PacketId, PState} = queue_message_id_to_packet_id(QMsgId, QoS, PState0), %%TODO support iolists when sending to client Payload = list_to_binary(lists:reverse(FragmentsRev)), - Frame = #mqtt_frame{fixed = #mqtt_frame_fixed{ - type = ?PUBLISH, - qos = ?QOS_0, %% spike handles only QoS0 - dup = Dup}, - variable = #mqtt_frame_publish{ - message_id = undefined, %% spike handles only QoS0 - topic_name = Amqp2MqttFun(RoutingKey)}, - payload = Payload}, - SendFun(Frame, PState), - - {ok, QueueType} = rabbit_queue_type:module(QName, QStates), - case QueueType of - rabbit_classic_queue -> + Frame = + #mqtt_frame{ + fixed = #mqtt_frame_fixed{ + type = ?PUBLISH, + qos = QoS, + %% "The value of the DUP flag from an incoming PUBLISH packet is not + %% propagated when the PUBLISH Packet is sent to subscribers by the Server. + %% The DUP flag in the outgoing PUBLISH packet is set independently to the + %% incoming PUBLISH packet, its value MUST be determined solely by whether + %% the outgoing PUBLISH packet is a retransmission [MQTT-3.3.1-3]." + %% Therefore, we do not consider header value <<"x-mqtt-dup">> here. + dup = Redelivered}, + variable = #mqtt_frame_publish{ + message_id = PacketId, + topic_name = Amqp2MqttFun(RoutingKey)}, + payload = Payload}, + serialise_and_send_to_client(Frame, PState), + PState. + +queue_message_id_to_packet_id(_, ?QOS_0, PState) -> + %% "A PUBLISH packet MUST NOT contain a Packet Identifier if its QoS value is set to 0 [MQTT-2.2.1-2]." + {undefined, PState}; +queue_message_id_to_packet_id(QMsgId, ?QOS_1, #proc_state{packet_id = PktId, + unacked_server_pubs = U} = PState) -> + {PktId, PState#proc_state{packet_id = increment_packet_id(PktId), + unacked_server_pubs = maps:put(PktId, QMsgId, U)}}. + +-spec increment_packet_id(packet_id()) -> packet_id(). +increment_packet_id(Id) + when Id >= 16#ffff -> + 1; +increment_packet_id(Id) -> + Id + 1. + +maybe_ack(_AckRequired = true, ?QOS_0, QName, QMsgId, + PState = #proc_state{queue_states = QStates0}) -> + case rabbit_queue_type:settle(QName, complete, ?CONSUMER_TAG, [QMsgId], QStates0) of + {ok, QStates, [] = _Actions} -> + % incr_queue_stats(QRef, MsgIds, State), + %%TODO handle actions + PState#proc_state{queue_states = QStates}; + {protocol_error, _ErrorType, _Reason, _ReasonArgs} = Err -> + %%TODO handle error + throw(Err) + end; +maybe_ack(_, _, _, _, PState) -> + PState. + +maybe_notify_sent(QName, QPid, #proc_state{queue_states = QStates}) -> + case rabbit_queue_type:module(QName, QStates) of + {ok, rabbit_classic_queue} -> rabbit_amqqueue:notify_sent(QPid, self()); _ -> ok - end, - - PState. + end. -publish(TopicName, PublishFun, - PState = #proc_state{exchange = Exchange, - auth_state = #auth_state{user = User, - authz_ctx = AuthzCtx}}) -> +publish_to_queues_with_checks( + TopicName, PublishFun, + #proc_state{exchange = Exchange, + auth_state = #auth_state{user = User, + authz_ctx = AuthzCtx}} = PState) -> case check_resource_access(User, Exchange, write, AuthzCtx) of ok -> case check_topic_access(TopicName, write, PState) of @@ -1419,16 +1405,11 @@ check_topic_access(TopicName, Access, end end. -info(consumer_tags, #proc_state{consumer_tags = Val}) -> Val; -info(unacked_pubs, #proc_state{unacked_pubs = Val}) -> Val; -info(awaiting_ack, #proc_state{awaiting_ack = Val}) -> Val; -info(awaiting_seqno, #proc_state{awaiting_seqno = Val}) -> Val; -info(message_id, #proc_state{message_id = Val}) -> Val; +info(unacked_client_pubs, #proc_state{unacked_client_pubs = Val}) -> Val; info(client_id, #proc_state{client_id = Val}) -> rabbit_data_coercion:to_binary(Val); info(clean_sess, #proc_state{clean_sess = Val}) -> Val; info(will_msg, #proc_state{will_msg = Val}) -> Val; -info(channels, #proc_state{channels = Val}) -> Val; info(exchange, #proc_state{exchange = #resource{name = Val}}) -> Val; info(ssl_login_name, #proc_state{ssl_login_name = Val}) -> Val; info(retainer_pid, #proc_state{retainer_pid = Val}) -> Val; @@ -1439,8 +1420,6 @@ info(port, #proc_state{info = #info{port = Val}}) -> Val; info(peer_host, #proc_state{info = #info{peer_host = Val}}) -> Val; info(peer_port, #proc_state{info = #info{peer_port = Val}}) -> Val; info(protocol, #proc_state{info = #info{proto_human = Val}}) -> Val; -% info(channels, PState) -> additional_info(channels, PState); -% info(channel_max, PState) -> additional_info(channel_max, PState); % info(frame_max, PState) -> additional_info(frame_max, PState); % info(client_properties, PState) -> additional_info(client_properties, PState); % info(ssl, PState) -> additional_info(ssl, PState); @@ -1450,18 +1429,6 @@ info(protocol, #proc_state{info = #info{proto_human = Val}}) -> Val; % info(ssl_hash, PState) -> additional_info(ssl_hash, PState); info(Other, _) -> throw({bad_argument, Other}). -% additional_info(Key, -% #proc_state{adapter_info = -% #amqp_adapter_info{additional_info = AddInfo}}) -> -% proplists:get_value(Key, AddInfo). - -notify_received(undefined) -> - %% no notification for quorum queues and streams - ok; -notify_received(DeliveryCtx) -> - %% notification for flow control - amqp_channel:notify_received(DeliveryCtx). - -spec ssl_login_name(rabbit_net:socket()) -> none | binary(). ssl_login_name(Sock) -> @@ -1474,3 +1441,6 @@ ssl_login_name(Sock) -> {error, no_peercert} -> none; nossl -> none end. + +queue_names(Queues) -> + lists:map(fun amqqueue:get_name/1, Queues). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index fdc55e67d117..327b329f5af1 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -22,7 +22,6 @@ -export([info/2]). --include_lib("amqp_client/include/amqp_client.hrl"). -include("rabbit_mqtt.hrl"). -define(SIMPLE_METRICS, [pid, recv_oct, send_oct, reductions]). @@ -130,25 +129,6 @@ handle_cast(QueueEvent = {queue_event, _, _}, handle_cast(Msg, State) -> {stop, {mqtt_unexpected_cast, Msg}, State}. -handle_info({#'basic.deliver'{}, #amqp_msg{}} = Delivery, - State) -> - %% receiving a message from a quorum queue - %% no delivery context - handle_info(erlang:insert_element(3, Delivery, undefined), State); -handle_info({#'basic.deliver'{}, #amqp_msg{}, _DeliveryCtx} = Delivery, - State = #state{ proc_state = ProcState }) -> - callback_reply(State, rabbit_mqtt_processor:amqp_callback(Delivery, - ProcState)); - -handle_info(#'basic.ack'{} = Ack, State = #state{ proc_state = ProcState }) -> - callback_reply(State, rabbit_mqtt_processor:amqp_callback(Ack, ProcState)); - -handle_info(#'basic.consume_ok'{}, State) -> - {noreply, State, hibernate}; - -handle_info(#'basic.cancel'{}, State) -> - {stop, {shutdown, subscription_cancelled}, State}; - handle_info({'EXIT', _Conn, Reason}, State) -> {stop, {connection_died, Reason}, State}; @@ -434,8 +414,7 @@ send_will_and_terminate(PState, State) -> send_will_and_terminate(PState, Reason, State = #state{conn_name = ConnStr}) -> rabbit_mqtt_processor:send_will(PState), - rabbit_log_connection:debug("MQTT: about to send will message (if any) on connection ~tp", [ConnStr]), - % todo: flush channel after publish + rabbit_log_connection:debug("MQTT: about to send will message (if any) on connection ~p", [ConnStr]), {stop, Reason, State}. network_error(closed, diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl index c711250b030b..6ad9ed36b356 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl @@ -9,7 +9,7 @@ -include("rabbit_mqtt.hrl"). --export([subcription_queue_name/1, +-export([queue_names/1, gen_client_id/0, env/1, table_lookup/2, @@ -21,7 +21,7 @@ -define(MAX_TOPIC_TRANSLATION_CACHE_SIZE, 12). -subcription_queue_name(ClientId) -> +queue_names(ClientId) -> Base = "mqtt-subscription-" ++ ClientId ++ "qos", {list_to_binary(Base ++ "0"), list_to_binary(Base ++ "1")}. diff --git a/deps/rabbitmq_mqtt/test/auth_SUITE.erl b/deps/rabbitmq_mqtt/test/auth_SUITE.erl index 7c2be0a75bf3..672409e03875 100644 --- a/deps/rabbitmq_mqtt/test/auth_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/auth_SUITE.erl @@ -10,8 +10,10 @@ -include_lib("common_test/include/ct.hrl"). -include_lib("eunit/include/eunit.hrl"). --define(CONNECT_TIMEOUT, 10000). + +%% defined in MQTT v4 and v5 (not in v3) -define(SUBACK_FAILURE, 16#80). + -define(FAIL_IF_CRASH_LOG, {["Generic server.*terminating"], fun () -> ct:fail(crash_detected) end}). -import(rabbit_ct_broker_helpers, [rpc/5]). @@ -528,90 +530,99 @@ no_queue_consume_permission(Config) -> test_subscribe_permissions_combination(<<".*">>, <<".*">>, <<"^amq\\.topic">>, Config, ExpectedLogs). no_queue_delete_permission(Config) -> - {skip, "TODO support clean_start=false"}. - % set_permissions(".*", ".*", ".*", Config), - % C1 = open_mqtt_connection(Config, [{clientid, <<"no_queue_delete_permission">>}, {clean_start, false}]), - % {ok, _, _} = emqtt:subscribe(C1, {<<"test/topic">>, qos1}), - % ok = emqtt:disconnect(C1), - % set_permissions(<<>>, ".*", ".*", Config), - - % %% And now we have a durable queue that user doesn't have permission to delete. - % %% Attempt to establish clean session should fail. - % {ok, C2} = connect_user( - % ?config(mqtt_user, Config), - % ?config(mqtt_password, Config), - % Config, - % ?config(mqtt_user, Config), - % [{clientid, <<"no_queue_delete_permission">>}, - % {clean_start, true}]), - % unlink(C2), - % ?assertMatch({error, _}, - % emqtt:connect(C2)), - - % wait_log(Config, - % [?FAIL_IF_CRASH_LOG - % ,{["operation queue.delete caused a channel exception access_refused", - % "MQTT cannot start a clean session: `configure` permission missing for queue"], - % fun () -> stop end} - % ]), - % ok. - -no_queue_consume_permission_on_connect(_Config) -> - {skip, "TODO support clean_start=false"}. - % set_permissions(".*", ".*", ".*", Config), - % C1 = open_mqtt_connection(Config, [{clientid, <<"no_queue_consume_permission_on_connect">>}, {clean_start, false}]), - % {ok, _, _} = emqtt:subscribe(C1, {<<"test/topic">>, qos1}), - % ok = emqtt:disconnect(C1), - - % set_permissions(".*", ".*", "^amq\\.topic", Config), - % {ok, C2} = connect_user( - % ?config(mqtt_user, Config), - % ?config(mqtt_password, Config), - % Config, - % ?config(mqtt_user, Config), - % [{clientid, <<"no_queue_consume_permission_on_connect">>}, - % {clean_start, false}]), - % unlink(C2), - % ?assertMatch({error, _}, - % emqtt:connect(C2)), - - % wait_log(Config, - % [{["Generic server.*terminating"], fun () -> exit(there_should_be_no_crashes) end} - % ,{["operation basic.consume caused a channel exception access_refused", - % "MQTT cannot recover a session, user is missing permissions"], - % fun () -> stop end} - % ]), - % ok. - -no_queue_declare_permission(_Config) -> - {skip, "TODO support clean_start=false"}. - % rabbit_ct_broker_helpers:set_permissions(Config, ?config(mqtt_user, Config), ?config(mqtt_vhost, Config), <<"">>, <<".*">>, <<".*">>), - % P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), - % {ok, C} = emqtt:start_link([{host, "localhost"}, - % {port, P}, - % {clientid, <<"no_queue_declare_permission">>}, - % {proto_ver, v4}, - % {username, ?config(mqtt_user, Config)}, - % {password, ?config(mqtt_password, Config)}, - % {clean_start, false} - % ]), - % {ok, _} = emqtt:connect(C), - - % process_flag(trap_exit, true), - % try emqtt:subscribe(C, <<"test/topic">>) of - % _ -> exit(this_should_not_succeed) - % catch - % exit:{{shutdown, tcp_closed} , _} -> ok - % end, - % process_flag(trap_exit, false), - - % wait_log(Config, - % [{["Generic server.*terminating"], fun () -> exit(there_should_be_no_crashes) end} - % ,{["MQTT protocol error on connection.*access_refused", - % "operation queue.declare caused a channel exception access_refused"], - % fun () -> stop end} - % ]), - % ok. + set_permissions(".*", ".*", ".*", Config), + ClientId = <<"no_queue_delete_permission">>, + {ok, C1} = connect_user( + ?config(mqtt_user, Config), + ?config(mqtt_password, Config), + Config, + ClientId, + [{clean_start, false}]), + {ok, _} = emqtt:connect(C1), + {ok, _, _} = emqtt:subscribe(C1, {<<"test/topic">>, qos1}), + ok = emqtt:disconnect(C1), + + set_permissions(<<>>, ".*", ".*", Config), + %% Now we have a durable queue that user doesn't have permission to delete. + %% Attempt to establish clean session should fail. + {ok, C2} = connect_user( + ?config(mqtt_user, Config), + ?config(mqtt_password, Config), + Config, + ClientId, + [{clean_start, true}]), + unlink(C2), + ?assertMatch({error, _}, + emqtt:connect(C2)), + wait_log( + Config, + [?FAIL_IF_CRASH_LOG + ,{[io_lib:format("MQTT resource access refused: configure access to queue " + "'mqtt-subscription-~sqos1' in vhost 'mqtt-vhost' refused for user 'mqtt-user'", + [ClientId]), + "MQTT connection .* is closing due to an authorization failure"], + fun() -> stop end} + ]), + ok. + +no_queue_consume_permission_on_connect(Config) -> + set_permissions(".*", ".*", ".*", Config), + ClientId = <<"no_queue_consume_permission_on_connect">>, + {ok, C1} = connect_user( + ?config(mqtt_user, Config), + ?config(mqtt_password, Config), + Config, + ClientId, + [{clean_start, false}]), + {ok, _} = emqtt:connect(C1), + {ok, _, _} = emqtt:subscribe(C1, {<<"test/topic">>, qos1}), + ok = emqtt:disconnect(C1), + + set_permissions(".*", ".*", "^amq\\.topic", Config), + {ok, C2} = connect_user( + ?config(mqtt_user, Config), + ?config(mqtt_password, Config), + Config, + ClientId, + [{clean_start, false}]), + unlink(C2), + ?assertMatch({error, _}, + emqtt:connect(C2)), + wait_log( + Config, + [?FAIL_IF_CRASH_LOG + ,{[io_lib:format("MQTT resource access refused: read access to queue " + "'mqtt-subscription-~sqos1' in vhost 'mqtt-vhost' refused for user 'mqtt-user'", + [ClientId]), + "MQTT connection .* is closing due to an authorization failure"], + fun () -> stop end} + ]), + ok. + +no_queue_declare_permission(Config) -> + set_permissions("", ".*", ".*", Config), + ClientId = <<"no_queue_declare_permission">>, + {ok, C} = connect_user( + ?config(mqtt_user, Config), + ?config(mqtt_password, Config), + Config, + ClientId, + [{clean_start, true}]), + {ok, _} = emqtt:connect(C), + + process_flag(trap_exit, true), + {ok, _, [?SUBACK_FAILURE]} = emqtt:subscribe(C, <<"test/topic">>, qos0), + ok = assert_connection_closed(C), + wait_log( + Config, + [?FAIL_IF_CRASH_LOG + ,{[io_lib:format("MQTT resource access refused: configure access to queue " + "'mqtt-subscription-~sqos0' in vhost 'mqtt-vhost' refused for user 'mqtt-user'", + [ClientId]), + "MQTT protocol error on connection .*: subscribe_error"], + fun () -> stop end} + ]), + ok. no_publish_permission(Config) -> set_permissions(".*", "", ".*", Config), @@ -631,8 +642,8 @@ no_publish_permission(Config) -> no_topic_read_permission(Config) -> set_permissions(".*", ".*", ".*", Config), set_topic_permissions("^allow-write\\..*", "^allow-read\\..*", Config), - C = open_mqtt_connection(Config), + %% Check topic permission setup is working. {ok, _, [0]} = emqtt:subscribe(C, <<"allow-read/some/topic">>), @@ -650,25 +661,25 @@ no_topic_read_permission(Config) -> ]), ok. -no_topic_write_permission(_Config) -> - {skip, "TODO implement QoS1"}. - % set_permissions(".*", ".*", ".*", Config), - % set_topic_permissions("^allow-write\\..*", "^allow-read\\..*", Config), - % C = open_mqtt_connection(Config), - % %% Check topic permission setup is working. - % {ok, _} = emqtt:publish(C, <<"allow-write/some/topic">>, <<"payload">>, qos1), - - % process_flag(trap_exit, true), - % ?assertMatch({error, _}, - % emqtt:publish(C, <<"some/other/topic">>, <<"payload">>, qos1)), - % wait_log(Config, - % [?FAIL_IF_CRASH_LOG - % ,{["MQTT topic access refused: write access to topic 'some.other.topic' in " - % "exchange 'amq.topic' in vhost 'mqtt-vhost' refused for user 'mqtt-user'", - % "MQTT connection .* is closing due to an authorization failure"], - % fun () -> stop end} - % ]), - % ok. +no_topic_write_permission(Config) -> + set_permissions(".*", ".*", ".*", Config), + set_topic_permissions("^allow-write\\..*", "^allow-read\\..*", Config), + C = open_mqtt_connection(Config), + + %% Check topic permission setup is working. + {ok, _} = emqtt:publish(C, <<"allow-write/some/topic">>, <<"payload">>, qos1), + + process_flag(trap_exit, true), + ?assertMatch({error, _}, + emqtt:publish(C, <<"some/other/topic">>, <<"payload">>, qos1)), + wait_log(Config, + [?FAIL_IF_CRASH_LOG + ,{["MQTT topic access refused: write access to topic 'some.other.topic' in " + "exchange 'amq.topic' in vhost 'mqtt-vhost' refused for user 'mqtt-user'", + "MQTT connection .* is closing due to an authorization failure"], + fun () -> stop end} + ]), + ok. loopback_user_connects_from_remote_host(Config) -> set_permissions(".*", ".*", ".*", Config), From a02cbb73a15158ff892f2f754380bd712b808f1a Mon Sep 17 00:00:00 2001 From: David Ansari Date: Mon, 26 Sep 2022 10:48:44 +0200 Subject: [PATCH 011/118] Get all existing rabbitmq_mqtt tests green --- deps/rabbit/src/rabbit_core_ff.erl | 3 +- deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl | 63 +++++---- .../src/rabbit_mqtt_confirms.erl | 2 +- .../src/rabbit_mqtt_connection_info.erl | 25 ---- .../src/rabbit_mqtt_processor.erl | 128 +++++++++--------- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 26 +++- .../src/rabbit_mqtt_retained_msg_store.erl | 11 +- .../rabbit_mqtt_retained_msg_store_dets.erl | 11 +- .../rabbit_mqtt_retained_msg_store_ets.erl | 3 +- .../rabbit_mqtt_retained_msg_store_noop.erl | 1 - .../src/rabbit_mqtt_retainer.erl | 3 +- deps/rabbitmq_mqtt/test/command_SUITE.erl | 17 +-- deps/rabbitmq_mqtt/test/java_SUITE.erl | 3 +- .../test/java_SUITE_data/pom.xml | 4 +- .../java/com/rabbitmq/mqtt/test/MqttTest.java | 2 + .../rabbitmq_mqtt/test/mqtt_machine_SUITE.erl | 5 +- deps/rabbitmq_mqtt/test/processor_SUITE.erl | 36 +---- .../test/proxy_protocol_SUITE.erl | 2 +- deps/rabbitmq_mqtt/test/reader_SUITE.erl | 49 +++---- deps/rabbitmq_mqtt/test/retainer_SUITE.erl | 2 +- deps/rabbitmq_mqtt/test/util_SUITE.erl | 2 +- 21 files changed, 184 insertions(+), 214 deletions(-) delete mode 100644 deps/rabbitmq_mqtt/src/rabbit_mqtt_connection_info.erl diff --git a/deps/rabbit/src/rabbit_core_ff.erl b/deps/rabbit/src/rabbit_core_ff.erl index 1ec4d5f06531..651da0925089 100644 --- a/deps/rabbit/src/rabbit_core_ff.erl +++ b/deps/rabbit/src/rabbit_core_ff.erl @@ -34,7 +34,8 @@ {stream_queue, #{desc => "Support queues of type `stream`", doc_url => "https://www.rabbitmq.com/stream.html", - stability => stable, + %%TODO remove compatibility code + stability => required, depends_on => [quorum_queue] }}). diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl index 3840b06eec91..04d5baadb8d1 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl @@ -11,6 +11,7 @@ %% reader state -record(state, {socket, + proxy_socket, conn_name, await_recv, deferred_recv, @@ -42,7 +43,7 @@ %% same IDs because client and server assign IDs independently of each other.) packet_id = 1 :: packet_id(), client_id, - clean_sess, + clean_sess :: boolean(), will_msg, exchange :: rabbit_exchange:name(), ssl_login_name, @@ -72,37 +73,41 @@ peer_port, proto_human}). -%% does not include vhost: it is used in -%% the table name +%% does not include vhost because vhost is used in the (D)ETS table name -record(retained_message, {topic, mqtt_msg}). -define(INFO_ITEMS, - [host, - port, - peer_host, - peer_port, - protocol, - frame_max, - client_properties, - ssl, - ssl_protocol, - ssl_key_exchange, - ssl_cipher, - ssl_hash, - conn_name, - connection_state, - connection, - unacked_client_pubs, - unacked_server_pubs, - packet_id, - client_id, - clean_sess, - will_msg, - exchange, - ssl_login_name, - retainer_pid, - user, - vhost]). + [protocol, + host, + port, + peer_host, + peer_port, + connection, + conn_name, + connection_state, + ssl, + ssl_protocol, + ssl_key_exchange, + ssl_cipher, + ssl_hash, + ssl_login_name, + client_id, + vhost, + user, + recv_cnt, + recv_oct, + send_cnt, + send_oct, + send_pend, + clean_sess, + will_msg, + retainer_pid, + exchange, + subscriptions, + prefetch, + messages_unconfirmed, + messages_unacknowledged + ]). -define(MQTT_GUIDE_URL, <<"https://rabbitmq.com/mqtt.html">>). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl index 0fefdadd6d30..79ad1f22c591 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl @@ -37,7 +37,7 @@ insert(PktId, _, State) insert(PktId, QNames, State) when is_integer(PktId) andalso PktId > 0 -> QMap = maps:from_keys(QNames, ok), - maps:put(PktId, QMap, State). + {ok, maps:put(PktId, QMap, State)}. -spec confirm([packet_id()], queue_name(), state()) -> {[packet_id()], state()}. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_connection_info.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_connection_info.erl deleted file mode 100644 index 2beb99b43916..000000000000 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_connection_info.erl +++ /dev/null @@ -1,25 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2017-2023 VMware, Inc. or its affiliates. All rights reserved. -%% --module(rabbit_mqtt_connection_info). - -%% Module to add the MQTT client ID to authentication properties - -%% API --export([additional_authn_params/4]). - -additional_authn_params(_Creds, _VHost, _Pid, Infos) -> - case proplists:get_value(variable_map, Infos, undefined) of - VariableMap when is_map(VariableMap) -> - case maps:get(<<"client_id">>, VariableMap, []) of - ClientId when is_binary(ClientId)-> - [{client_id, ClientId}]; - [] -> - [] - end; - _ -> - [] - end. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index f763002ad592..3c011b1133c6 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -489,19 +489,19 @@ hand_off_to_retainer(RetainerPid, Amqp2MqttFun, Topic0, Msg) -> maybe_send_retained_message(RPid, #mqtt_topic{name = Topic0, qos = SubscribeQos}, #proc_state{amqp2mqtt_fun = Amqp2MqttFun, - packet_id = PacketId} = PState0) -> + packet_id = PacketId0} = PState0) -> Topic1 = Amqp2MqttFun(Topic0), case rabbit_mqtt_retainer:fetch(RPid, Topic1) of - undefined -> PState0; - Msg -> - %% calculate effective QoS as the lower value of SUBSCRIBE frame QoS - %% and retained message QoS. The spec isn't super clear on this, we - %% do what Mosquitto does, per user feedback. - Qos = erlang:min(SubscribeQos, Msg#mqtt_msg.qos), - {Id, PState} = case Qos of - ?QOS_0 -> {undefined, PState0}; - ?QOS_1 -> {PacketId, PState0#proc_state{packet_id = increment_packet_id(PacketId)}} - end, + undefined -> + PState0; + Msg -> + Qos = effective_qos(Msg#mqtt_msg.qos, SubscribeQos), + {PacketId, PState} = case Qos of + ?QOS_0 -> + {undefined, PState0}; + ?QOS_1 -> + {PacketId0, PState0#proc_state{packet_id = increment_packet_id(PacketId0)}} + end, serialise_and_send_to_client( #mqtt_frame{fixed = #mqtt_frame_fixed{ type = ?PUBLISH, @@ -509,7 +509,7 @@ maybe_send_retained_message(RPid, #mqtt_topic{name = Topic0, qos = SubscribeQos} dup = false, retain = Msg#mqtt_msg.retain }, variable = #mqtt_frame_publish{ - message_id = Id, + message_id = PacketId, topic_name = Topic1 }, payload = Msg#mqtt_msg.payload}, @@ -517,17 +517,17 @@ maybe_send_retained_message(RPid, #mqtt_topic{name = Topic0, qos = SubscribeQos} PState end. -make_will_msg(#mqtt_frame_connect{ will_flag = false }) -> +make_will_msg(#mqtt_frame_connect{will_flag = false}) -> undefined; -make_will_msg(#mqtt_frame_connect{ will_retain = Retain, - will_qos = Qos, - will_topic = Topic, - will_msg = Msg }) -> - #mqtt_msg{ retain = Retain, - qos = Qos, - topic = Topic, - dup = false, - payload = Msg }. +make_will_msg(#mqtt_frame_connect{will_retain = Retain, + will_qos = Qos, + will_topic = Topic, + will_msg = Msg}) -> + #mqtt_msg{retain = Retain, + qos = Qos, + topic = Topic, + dup = false, + payload = Msg}. process_login(_UserBin, _PassBin, _ClientId, #proc_state{peer_addr = Addr, @@ -999,9 +999,8 @@ binding_action( key = RoutingKey}, BindingFun(Binding, Username). -send_will(PState = #proc_state{will_msg = undefined}) -> - PState; - +send_will(#proc_state{will_msg = undefined}) -> + ok; send_will(PState = #proc_state{will_msg = WillMsg = #mqtt_msg{retain = Retain, topic = Topic}, retainer_pid = RPid, @@ -1010,25 +1009,14 @@ send_will(PState = #proc_state{will_msg = WillMsg = #mqtt_msg{retain = Retain, ok -> publish_to_queues(WillMsg, PState), case Retain of - false -> ok; - true -> + false -> + ok; + true -> hand_off_to_retainer(RPid, Amqp2MqttFun, Topic, WillMsg) end; - Error -> - rabbit_log:warning( - "Could not send last will: ~tp", - [Error]) - end, - %%TODO cancel queue client? - % case ChQos1 of - % undefined -> ok; - % _ -> amqp_channel:close(ChQos1) - % end, - % case ChQos0 of - % undefined -> ok; - % _ -> amqp_channel:close(ChQos0) - % end, - PState. + {error, access_refused = Reason} -> + rabbit_log:error("failed to send will message: ~p", [Reason]) + end. publish_to_queues(undefined, PState) -> {ok, PState}; @@ -1104,6 +1092,11 @@ deliver_to_queues(Delivery, process_routing_confirm(#delivery{confirm = false}, [], PState) -> rabbit_global_counters:messages_unroutable_dropped(mqtt, 1), PState; +process_routing_confirm(#delivery{confirm = true, + msg_seq_no = undefined}, [], PState) -> + %% unroutable will message with QoS > 0 + rabbit_global_counters:messages_unroutable_dropped(mqtt, 1), + PState; process_routing_confirm(#delivery{confirm = true, msg_seq_no = MsgId}, [], PState) -> rabbit_global_counters:messages_unroutable_returned(mqtt, 1), @@ -1114,11 +1107,15 @@ process_routing_confirm(#delivery{confirm = true, PState; process_routing_confirm(#delivery{confirm = false}, _, PState) -> PState; +process_routing_confirm(#delivery{confirm = true, + msg_seq_no = undefined}, [_|_], PState) -> + %% routable will message with QoS > 0 + PState; process_routing_confirm(#delivery{confirm = true, msg_seq_no = MsgId}, Qs, PState = #proc_state{unacked_client_pubs = U0}) -> QNames = queue_names(Qs), - U = rabbit_mqtt_confirms:insert(MsgId, QNames, U0), + {ok, U} = rabbit_mqtt_confirms:insert(MsgId, QNames, U0), PState#proc_state{unacked_client_pubs = U}. send_puback(MsgIds, PState) @@ -1250,10 +1247,7 @@ deliver_one_to_client(Msg = {QName, QPid, QMsgId, _Redelivered, false -> ?QOS_0 end, - %% "The QoS of Application Messages sent in response to a Subscription MUST be the minimum - %% of the QoS of the originally published message and the Maximum QoS granted by the Server - %% [MQTT-3.8.4-8]." - QoS = min(PublisherQoS, SubscriberQoS), + QoS = effective_qos(PublisherQoS, SubscriberQoS), PState1 = maybe_publish_to_client(Msg, QoS, PState0), PState = maybe_ack(AckRequired, QoS, QName, QMsgId, PState1), %%TODO GC @@ -1264,6 +1258,13 @@ deliver_one_to_client(Msg = {QName, QPid, QMsgId, _Redelivered, ok = maybe_notify_sent(QName, QPid, PState), PState. +-spec effective_qos(qos(), qos()) -> qos(). +effective_qos(PublisherQoS, SubscriberQoS) -> + %% "The QoS of Application Messages sent in response to a Subscription MUST be the minimum + %% of the QoS of the originally published message and the Maximum QoS granted by the Server + %% [MQTT-3.8.4-8]." + erlang:min(PublisherQoS, SubscriberQoS). + maybe_publish_to_client({_, _, _, _Redelivered = true, _}, ?QOS_0, PState) -> %% Do no redeliver to MQTT subscriber who gets message at most once. PState; @@ -1405,28 +1406,27 @@ check_topic_access(TopicName, Access, end end. -info(unacked_client_pubs, #proc_state{unacked_client_pubs = Val}) -> Val; +info(protocol, #proc_state{info = #info{proto_human = Val}}) -> Val; +info(host, #proc_state{info = #info{host = Val}}) -> Val; +info(port, #proc_state{info = #info{port = Val}}) -> Val; +info(peer_host, #proc_state{info = #info{peer_host = Val}}) -> Val; +info(peer_port, #proc_state{info = #info{peer_port = Val}}) -> Val; +info(ssl_login_name, #proc_state{ssl_login_name = Val}) -> Val; info(client_id, #proc_state{client_id = Val}) -> rabbit_data_coercion:to_binary(Val); +info(vhost, #proc_state{auth_state = #auth_state{vhost = Val}}) -> Val; +info(user, #proc_state{auth_state = #auth_state{username = Val}}) -> Val; info(clean_sess, #proc_state{clean_sess = Val}) -> Val; info(will_msg, #proc_state{will_msg = Val}) -> Val; -info(exchange, #proc_state{exchange = #resource{name = Val}}) -> Val; -info(ssl_login_name, #proc_state{ssl_login_name = Val}) -> Val; info(retainer_pid, #proc_state{retainer_pid = Val}) -> Val; -info(user, #proc_state{auth_state = #auth_state{username = Val}}) -> Val; -info(vhost, #proc_state{auth_state = #auth_state{vhost = Val}}) -> Val; -info(host, #proc_state{info = #info{host = Val}}) -> Val; -info(port, #proc_state{info = #info{port = Val}}) -> Val; -info(peer_host, #proc_state{info = #info{peer_host = Val}}) -> Val; -info(peer_port, #proc_state{info = #info{peer_port = Val}}) -> Val; -info(protocol, #proc_state{info = #info{proto_human = Val}}) -> Val; -% info(frame_max, PState) -> additional_info(frame_max, PState); -% info(client_properties, PState) -> additional_info(client_properties, PState); -% info(ssl, PState) -> additional_info(ssl, PState); -% info(ssl_protocol, PState) -> additional_info(ssl_protocol, PState); -% info(ssl_key_exchange, PState) -> additional_info(ssl_key_exchange, PState); -% info(ssl_cipher, PState) -> additional_info(ssl_cipher, PState); -% info(ssl_hash, PState) -> additional_info(ssl_hash, PState); +info(exchange, #proc_state{exchange = #resource{name = Val}}) -> Val; +info(subscriptions, #proc_state{subscriptions = Val}) -> + maps:keys(Val); +info(prefetch, #proc_state{info = #info{prefetch = Val}}) -> Val; +info(messages_unconfirmed, #proc_state{unacked_client_pubs = Val}) -> + rabbit_mqtt_confirms:size(Val); +info(messages_unacknowledged, #proc_state{unacked_server_pubs = Val}) -> + maps:size(Val); info(Other, _) -> throw({bad_argument, Other}). -spec ssl_login_name(rabbit_net:socket()) -> diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index 327b329f5af1..2e09afa1e99f 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -71,6 +71,7 @@ init(Ref) -> rabbit_event:init_stats_timer( control_throttle( #state{socket = RealSocket, + proxy_socket = rabbit_net:maybe_get_proxy_socket(Sock), conn_name = ConnStr, await_recv = false, connection_state = running, @@ -409,12 +410,17 @@ parse(Bytes, ParseState) -> {error, {cannot_parse, Reason, Stacktrace}} end. +%% TODO Send will message in all abnormal shutdowns? +%% => in terminate/2 depending on Reason +%% "The Will Message MUST be published when the Network Connection is subsequently +%% closed unless the Will Message has been deleted by the Server on receipt of a +%% DISCONNECT Packet [MQTT-3.1.2-8]." send_will_and_terminate(PState, State) -> send_will_and_terminate(PState, {shutdown, conn_closed}, State). send_will_and_terminate(PState, Reason, State = #state{conn_name = ConnStr}) -> - rabbit_mqtt_processor:send_will(PState), rabbit_log_connection:debug("MQTT: about to send will message (if any) on connection ~p", [ConnStr]), + rabbit_mqtt_processor:send_will(PState), {stop, Reason, State}. network_error(closed, @@ -527,5 +533,23 @@ info_internal(connection_state, #state{connection_state = Val}) -> Val; info_internal(connection, _State) -> self(); +info_internal(ssl, #state{socket = Sock, proxy_socket = ProxySock}) -> + rabbit_net:proxy_ssl_info(Sock, ProxySock) /= nossl; +info_internal(ssl_protocol, S) -> ssl_info(fun ({P, _}) -> P end, S); +info_internal(ssl_key_exchange, S) -> ssl_info(fun ({_, {K, _, _}}) -> K end, S); +info_internal(ssl_cipher, S) -> ssl_info(fun ({_, {_, C, _}}) -> C end, S); +info_internal(ssl_hash, S) -> ssl_info(fun ({_, {_, _, H}}) -> H end, S); info_internal(Key, #state{proc_state = ProcState}) -> rabbit_mqtt_processor:info(Key, ProcState). + +ssl_info(F, #state{socket = Sock, proxy_socket = ProxySock}) -> + case rabbit_net:proxy_ssl_info(Sock, ProxySock) of + nossl -> ''; + {error, _} -> ''; + {ok, Items} -> + P = proplists:get_value(protocol, Items), + #{cipher := C, + key_exchange := K, + mac := H} = proplists:get_value(selected_cipher_suite, Items), + F({P, {K, C, H}}) + end. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store.erl index e4885d068275..cbe5752884b5 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store.erl @@ -22,4 +22,13 @@ behaviour_info(_Other) -> table_name_for(VHost) -> rabbit_mqtt_util:vhost_name_to_table_name(VHost). -%%TODO could add a rabbitmq_mqtt_retained_msg_store_khepri to have some replication +%% TODO +%% Support retained messages in RabbitMQ cluster: +%% * SUBSCRIBE on a different node than PUBLISH with retain +%% * replicate retained message for data safety +%% +%% Possible solutions: +%% * rabbitmq_mqtt_retained_msg_store_mnesia +%% * rabbitmq_mqtt_retained_msg_store_khepri +%% * rabbitmq_mqtt_retained_msg_store_ra (implementing our own ra machine) or +%% * use existing mqtt_machine diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_dets.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_dets.erl index 4cfc11279b05..f449b1e8e45e 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_dets.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_dets.erl @@ -48,7 +48,10 @@ open_table(Dir, VHost) -> table_options(rabbit_mqtt_util:path_for(Dir, VHost, ".dets"))). table_options(Path) -> - [{type, set}, {keypos, #retained_message.topic}, - {file, Path}, {ram_file, true}, {repair, true}, - {auto_save, rabbit_misc:get_env(rabbit_mqtt, - retained_message_store_dets_sync_interval, 2000)}]. + [{type, set}, + {keypos, #retained_message.topic}, + {file, Path}, + {ram_file, true}, + {repair, true}, + {auto_save, rabbit_misc:get_env(rabbit_mqtt, retained_message_store_dets_sync_interval, 2000)} + ]. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_ets.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_ets.erl index c25c9f20984c..b427750cc178 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_ets.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_ets.erl @@ -50,5 +50,4 @@ delete(Topic, #store_state{table = T}) -> ok. terminate(#store_state{table = T, filename = Path}) -> - ok = ets:tab2file(T, Path, - [{extended_info, [object_count]}]). + ok = ets:tab2file(T, Path, [{extended_info, [object_count]}]). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_noop.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_noop.erl index 79cb74d7d9c9..1e62ed5abee7 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_noop.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_noop.erl @@ -8,7 +8,6 @@ -module(rabbit_mqtt_retained_msg_store_noop). -behaviour(rabbit_mqtt_retained_msg_store). --include("rabbit_mqtt.hrl"). -export([new/2, recover/2, insert/3, lookup/2, delete/2, terminate/1]). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer.erl index 32de0bc6bc59..baae37ea6b80 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer.erl @@ -16,8 +16,7 @@ -export([retain/3, fetch/2, clear/2, store_module/0]). --define(SERVER, ?MODULE). --define(TIMEOUT, 30000). +-define(TIMEOUT, 30_000). -record(retainer_state, {store_mod, store}). diff --git a/deps/rabbitmq_mqtt/test/command_SUITE.erl b/deps/rabbitmq_mqtt/test/command_SUITE.erl index 0b3301a95243..f87b96f509e2 100644 --- a/deps/rabbitmq_mqtt/test/command_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/command_SUITE.erl @@ -6,14 +6,12 @@ -module(command_SUITE). --compile([export_all]). +-compile([export_all, nowarn_export_all]). --include_lib("common_test/include/ct.hrl"). -include_lib("eunit/include/eunit.hrl"). -include_lib("amqp_client/include/amqp_client.hrl"). -include("rabbit_mqtt.hrl"). - -define(COMMAND, 'Elixir.RabbitMQ.CLI.Ctl.Commands.ListMqttConnectionsCommand'). all() -> @@ -71,9 +69,8 @@ merge_defaults(_Config) -> run(Config) -> - Node = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), - Opts = #{node => Node, timeout => 10000, verbose => false}, + Opts = #{node => Node, timeout => 10_000, verbose => false}, %% No connections [] = 'Elixir.Enum':to_list(?COMMAND:run([], Opts)), @@ -85,7 +82,7 @@ run(Config) -> {proto_ver, v4}, {ack_timeout, 1}]), {ok, _} = emqtt:connect(C1), - ct:sleep(100), + timer:sleep(100), [[{client_id, <<"simpleClient">>}]] = 'Elixir.Enum':to_list(?COMMAND:run([<<"client_id">>], Opts)), @@ -99,7 +96,7 @@ run(Config) -> {password, <<"guest">>}, {ack_timeout, 1}]), {ok, _} = emqtt:connect(C2), - ct:sleep(200), + timer:sleep(200), [[{client_id, <<"simpleClient">>}, {user, <<"guest">>}], [{client_id, <<"simpleClient1">>}, {user, <<"guest">>}]] = @@ -110,15 +107,15 @@ run(Config) -> Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_amqp), start_amqp_connection(network, Node, Port), - %% There are still just two connections + %% There are still just two MQTT connections [[{client_id, <<"simpleClient">>}], [{client_id, <<"simpleClient1">>}]] = lists:sort('Elixir.Enum':to_list(?COMMAND:run([<<"client_id">>], Opts))), start_amqp_connection(direct, Node, Port), - ct:sleep(200), + timer:sleep(200), - %% Still two MQTT connections, one direct AMQP 0-9-1 connection + %% Still two MQTT connections [[{client_id, <<"simpleClient">>}], [{client_id, <<"simpleClient1">>}]] = lists:sort('Elixir.Enum':to_list(?COMMAND:run([<<"client_id">>], Opts))), diff --git a/deps/rabbitmq_mqtt/test/java_SUITE.erl b/deps/rabbitmq_mqtt/test/java_SUITE.erl index e0038a796c22..5155f1ba778d 100644 --- a/deps/rabbitmq_mqtt/test/java_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/java_SUITE.erl @@ -6,12 +6,11 @@ %% -module(java_SUITE). --compile([export_all]). +-compile([export_all, nowarn_export_all]). -include_lib("common_test/include/ct.hrl"). -include_lib("eunit/include/eunit.hrl"). --define(BASE_CONF_RABBIT, {rabbit, [{ssl_options, [{fail_if_no_peer_cert, false}]}]}). -define(BASE_CONF_MQTT, {rabbitmq_mqtt, [ {ssl_cert_login, true}, diff --git a/deps/rabbitmq_mqtt/test/java_SUITE_data/pom.xml b/deps/rabbitmq_mqtt/test/java_SUITE_data/pom.xml index b27b58c172e4..ce790d213095 100644 --- a/deps/rabbitmq_mqtt/test/java_SUITE_data/pom.xml +++ b/deps/rabbitmq_mqtt/test/java_SUITE_data/pom.xml @@ -22,13 +22,13 @@ com.rabbitmq amqp-client - 5.7.3 + 5.16.0 test org.junit.jupiter junit-jupiter - 5.5.2 + 5.9.1 test diff --git a/deps/rabbitmq_mqtt/test/java_SUITE_data/src/test/java/com/rabbitmq/mqtt/test/MqttTest.java b/deps/rabbitmq_mqtt/test/java_SUITE_data/src/test/java/com/rabbitmq/mqtt/test/MqttTest.java index f31497d1c705..c7ac646cfae6 100644 --- a/deps/rabbitmq_mqtt/test/java_SUITE_data/src/test/java/com/rabbitmq/mqtt/test/MqttTest.java +++ b/deps/rabbitmq_mqtt/test/java_SUITE_data/src/test/java/com/rabbitmq/mqtt/test/MqttTest.java @@ -34,6 +34,8 @@ import static org.junit.jupiter.api.Assertions.*; +//TODO Add quorum queue test + /*** * MQTT v3.1 tests * diff --git a/deps/rabbitmq_mqtt/test/mqtt_machine_SUITE.erl b/deps/rabbitmq_mqtt/test/mqtt_machine_SUITE.erl index 7ce08cbc2cdf..274877cdc841 100644 --- a/deps/rabbitmq_mqtt/test/mqtt_machine_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/mqtt_machine_SUITE.erl @@ -1,9 +1,6 @@ -module(mqtt_machine_SUITE). --compile(export_all). - --export([ - ]). +-compile([export_all, nowarn_export_all]). -include_lib("common_test/include/ct.hrl"). -include_lib("eunit/include/eunit.hrl"). diff --git a/deps/rabbitmq_mqtt/test/processor_SUITE.erl b/deps/rabbitmq_mqtt/test/processor_SUITE.erl index 310c2a8cdb66..15ae0dd5374c 100644 --- a/deps/rabbitmq_mqtt/test/processor_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/processor_SUITE.erl @@ -6,7 +6,7 @@ -module(processor_SUITE). --compile([export_all]). +-compile([export_all, nowarn_export_all]). -include_lib("common_test/include/ct.hrl"). -include_lib("eunit/include/eunit.hrl"). @@ -23,9 +23,7 @@ groups() -> ignores_colons_in_username_if_option_set, interprets_colons_in_username_if_option_not_set, get_vhosts_from_global_runtime_parameter, - get_vhost, - add_client_id_to_adapter_info, - quorum_configuration + get_vhost ]} ]. @@ -186,17 +184,6 @@ get_vhost(_Config) -> clear_vhost_global_parameters(), ok. -add_client_id_to_adapter_info(_Config) -> - TestFun = fun(AdapterInfo) -> - Info0 = rabbit_mqtt_processor:add_client_id_to_adapter_info(<<"my-client-id">>, AdapterInfo), - AdditionalInfo0 = Info0#amqp_adapter_info.additional_info, - ?assertEqual(#{<<"client_id">> => <<"my-client-id">>}, proplists:get_value(variable_map, AdditionalInfo0)), - ClientProperties = proplists:get_value(client_properties, AdditionalInfo0), - ?assertEqual([{client_id,longstr,<<"my-client-id">>}], ClientProperties) - end, - lists:foreach(TestFun, [#amqp_adapter_info{}, #amqp_adapter_info{additional_info = [{client_properties, []}]}]), - ok. - set_global_parameter(Key, Term) -> InsertParameterFun = fun () -> mnesia:write(rabbit_runtime_parameters, #runtime_parameters{key = Key, value = Term}, write) @@ -210,22 +197,3 @@ clear_vhost_global_parameters() -> ok = mnesia:delete(rabbit_runtime_parameters, mqtt_port_to_vhost_mapping, write) end, {atomic, ok} = mnesia:transaction(DeleteParameterFun). - -quorum_configuration(_Config) -> - MyArgs = [], -%% default setting with CleanSession = true of false - QMustBeClassic = rabbit_mqtt_processor:maybe_quorum(MyArgs, true, <<"">>), - ?assertEqual(QMustBeClassic, []), -%% default setting with CleanSession = true of false - QMustBeClassicEvenFalse = rabbit_mqtt_processor:maybe_quorum(MyArgs, false, <<"">>), - ?assertEqual(QMustBeClassicEvenFalse, []), - application:set_env(rabbitmq_mqtt, durable_queue_type, quorum), -%% quorum setting with CleanSession == false must me quorum - QMustBeQuorum = rabbit_mqtt_processor:maybe_quorum(MyArgs, false, <<"">>), - ?assertEqual(QMustBeQuorum, [{<<"x-queue-type">>, longstr, <<"quorum">>}]), - - %% quorum setting with CleanSession == true must me classic since - %% quorum does not support auto-delete - QEvenQuorumMustBeClassic = rabbit_mqtt_processor:maybe_quorum(MyArgs, true, <<"">>), - ?assertEqual(QEvenQuorumMustBeClassic, []), - ok. diff --git a/deps/rabbitmq_mqtt/test/proxy_protocol_SUITE.erl b/deps/rabbitmq_mqtt/test/proxy_protocol_SUITE.erl index f50c53c57f0e..87b6faead203 100644 --- a/deps/rabbitmq_mqtt/test/proxy_protocol_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/proxy_protocol_SUITE.erl @@ -5,7 +5,7 @@ %% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved. %% -module(proxy_protocol_SUITE). --compile([export_all]). +-compile([export_all, nowarn_export_all]). -include_lib("common_test/include/ct.hrl"). -include_lib("eunit/include/eunit.hrl"). diff --git a/deps/rabbitmq_mqtt/test/reader_SUITE.erl b/deps/rabbitmq_mqtt/test/reader_SUITE.erl index bb3bd58f2fe4..97d9b255128f 100644 --- a/deps/rabbitmq_mqtt/test/reader_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/reader_SUITE.erl @@ -24,10 +24,10 @@ groups() -> handle_invalid_frames, login_timeout, stats, - quorum_session_false, - quorum_session_true, - classic_session_true, - classic_session_false + quorum_clean_session_false, + quorum_clean_session_true, + classic_clean_session_true, + classic_clean_session_false ]} ]. @@ -199,10 +199,10 @@ stats(Config) -> [connection_coarse_metrics, Pid]), ok = emqtt:disconnect(C). -get_durable_queue_type(Server, Q0) -> - QNameRes = rabbit_misc:r(<<"/">>, queue, Q0), - {ok, Q1} = rpc:call(Server, rabbit_amqqueue, lookup, [QNameRes]), - amqqueue:get_type(Q1). +get_durable_queue_type(Server, QNameBin) -> + QName = rabbit_misc:r(<<"/">>, queue, QNameBin), + {ok, Q} = rpc:call(Server, rabbit_amqqueue, lookup, [QName]), + amqqueue:get_type(Q). set_env(QueueType) -> application:set_env(rabbitmq_mqtt, durable_queue_type, QueueType). @@ -210,8 +210,7 @@ set_env(QueueType) -> get_env() -> rabbit_mqtt_util:env(durable_queue_type). - -validate_durable_queue_type(Config, ClientName, CleanSession, Expected) -> +validate_durable_queue_type(Config, ClientName, CleanSession, ExpectedQueueType) -> P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), {ok, C} = emqtt:start_link([{host, "localhost"}, @@ -226,35 +225,29 @@ validate_durable_queue_type(Config, ClientName, CleanSession, Expected) -> {ok, _, _} = emqtt:unsubscribe(C, <<"TopicB">>), Prefix = <<"mqtt-subscription-">>, Suffix = <<"qos1">>, - Q= <>, - ?assertEqual(Expected,get_durable_queue_type(Server,Q)), - timer:sleep(500), + QNameBin = <>, + ?assertEqual(ExpectedQueueType, get_durable_queue_type(Server, QNameBin)), ok = emqtt:disconnect(C). -%% quorum queue test when enable -quorum_session_false(Config) -> - %% test if the quorum queue is enable after the setting +quorum_clean_session_false(Config) -> Default = rpc(Config, reader_SUITE, get_env, []), rpc(Config, reader_SUITE, set_env, [quorum]), - validate_durable_queue_type(Config, <<"qCleanSessionFalse">>, false, rabbit_quorum_queue), + validate_durable_queue_type(Config, <<"quorumCleanSessionFalse">>, false, rabbit_quorum_queue), rpc(Config, reader_SUITE, set_env, [Default]). -quorum_session_true(Config) -> - %% in case clean session == true must be classic since quorum - %% doesn't support auto-delete +quorum_clean_session_true(Config) -> Default = rpc(Config, reader_SUITE, get_env, []), rpc(Config, reader_SUITE, set_env, [quorum]), - validate_durable_queue_type(Config, <<"qCleanSessionTrue">>, true, rabbit_classic_queue), + %% Since we use a clean session and quorum queues cannot be auto-delete or exclusive, + %% we expect a classic queue. + validate_durable_queue_type(Config, <<"quorumCleanSessionTrue">>, true, rabbit_classic_queue), rpc(Config, reader_SUITE, set_env, [Default]). -classic_session_true(Config) -> - %% with default configuration the queue is classic - validate_durable_queue_type(Config, <<"cCleanSessionTrue">>, true, rabbit_classic_queue). - -classic_session_false(Config) -> - %% with default configuration the queue is classic - validate_durable_queue_type(Config, <<"cCleanSessionFalse">>, false, rabbit_classic_queue). +classic_clean_session_true(Config) -> + validate_durable_queue_type(Config, <<"classicCleanSessionTrue">>, true, rabbit_classic_queue). +classic_clean_session_false(Config) -> + validate_durable_queue_type(Config, <<"classicCleanSessionFalse">>, false, rabbit_classic_queue). expect_publishes(_Topic, []) -> ok; expect_publishes(Topic, [Payload|Rest]) -> diff --git a/deps/rabbitmq_mqtt/test/retainer_SUITE.erl b/deps/rabbitmq_mqtt/test/retainer_SUITE.erl index 6001f8cfe269..1bee53413ad1 100644 --- a/deps/rabbitmq_mqtt/test/retainer_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/retainer_SUITE.erl @@ -5,7 +5,7 @@ %% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved. %% -module(retainer_SUITE). --compile([export_all]). +-compile([export_all, nowarn_export_all]). -include_lib("common_test/include/ct.hrl"). diff --git a/deps/rabbitmq_mqtt/test/util_SUITE.erl b/deps/rabbitmq_mqtt/test/util_SUITE.erl index c15eb1dee407..5f5c4327b151 100644 --- a/deps/rabbitmq_mqtt/test/util_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/util_SUITE.erl @@ -5,7 +5,7 @@ %% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved. -module(util_SUITE). --compile([export_all]). +-compile([export_all, nowarn_export_all]). -include_lib("common_test/include/ct.hrl"). -include_lib("eunit/include/eunit.hrl"). From 6f00ccb3adb228a48d954e6e510efc5662f3f5ae Mon Sep 17 00:00:00 2001 From: David Ansari Date: Wed, 28 Sep 2022 11:37:25 +0000 Subject: [PATCH 012/118] Get all existing rabbitmq_web_mqtt tests green --- deps/rabbit/src/rabbit_core_ff.erl | 3 +- deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl | 3 +- deps/rabbitmq_mqtt/src/rabbit_mqtt.erl | 3 + .../src/rabbit_mqtt_processor.erl | 51 +++++---- .../src/rabbit_web_mqtt_connection_info.erl | 26 ----- .../src/rabbit_web_mqtt_handler.erl | 104 +++++++++++------- deps/rabbitmq_web_mqtt/test/system_SUITE.erl | 4 +- 7 files changed, 104 insertions(+), 90 deletions(-) delete mode 100644 deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_connection_info.erl diff --git a/deps/rabbit/src/rabbit_core_ff.erl b/deps/rabbit/src/rabbit_core_ff.erl index 651da0925089..1ec4d5f06531 100644 --- a/deps/rabbit/src/rabbit_core_ff.erl +++ b/deps/rabbit/src/rabbit_core_ff.erl @@ -34,8 +34,7 @@ {stream_queue, #{desc => "Support queues of type `stream`", doc_url => "https://www.rabbitmq.com/stream.html", - %%TODO remove compatibility code - stability => required, + stability => stable, depends_on => [quorum_queue] }}). diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl index 04d5baadb8d1..5db1b8a16a8f 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl @@ -52,7 +52,8 @@ retainer_pid, auth_state, peer_addr, - %%TODO remove funs from state + send_fun :: fun((Frame :: tuple(), proc_state()) -> term()), + %%TODO remove funs from state? mqtt2amqp_fun, amqp2mqtt_fun, register_state, diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl index a8c5e3ab4dfc..c7f5877b7ec2 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl @@ -13,6 +13,9 @@ close_local_client_connections/1]). start(normal, []) -> + %%TODO make feature flag stream_queue 'required' for 3.12 + %% because we rely on rabbit_queue_type interface. + ok = rabbit_feature_flags:enable(stream_queue), rabbit_global_counters:init([{protocol, mqtt}]), {ok, Listeners} = application:get_env(tcp_listeners), {ok, SslListeners} = application:get_env(ssl_listeners), diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 3c011b1133c6..7de05834a77d 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -8,8 +8,8 @@ -module(rabbit_mqtt_processor). --export([info/2, initial_state/2, - process_frame/2, send_will/1, +-export([info/2, initial_state/2, initial_state/4, + process_frame/2, serialise/2, send_will/1, terminate/1, handle_pre_hibernate/0, handle_ra_event/2, handle_down/2, handle_queue_event/2]). @@ -35,14 +35,19 @@ -define(CONSUMER_TAG, mqtt). initial_state(Socket, ConnectionName) -> - SSLLoginName = ssl_login_name(Socket), {ok, {PeerAddr, _PeerPort}} = rabbit_net:peername(Socket), - {ok, {mqtt2amqp_fun, M2A}, {amqp2mqtt_fun, A2M}} = - rabbit_mqtt_util:get_topic_translation_funs(), + initial_state(Socket, + ConnectionName, + fun serialise_and_send_to_client/2, + PeerAddr). + +initial_state(Socket, ConnectionName, SendFun, PeerAddr) -> + {ok, {mqtt2amqp_fun, M2A}, {amqp2mqtt_fun, A2M}} = rabbit_mqtt_util:get_topic_translation_funs(), #proc_state{socket = Socket, conn_name = ConnectionName, - ssl_login_name = SSLLoginName, + ssl_login_name = ssl_login_name(Socket), peer_addr = PeerAddr, + send_fun = SendFun, mqtt2amqp_fun = M2A, amqp2mqtt_fun = A2M}. @@ -139,7 +144,8 @@ process_request(?SUBSCRIBE, message_id = SubscribeMsgId, topic_table = Topics}, payload = undefined}, - #proc_state{retainer_pid = RPid} = PState0) -> + #proc_state{send_fun = SendFun, + retainer_pid = RPid} = PState0) -> rabbit_log_connection:debug("Received a SUBSCRIBE for topic(s) ~p", [Topics]), {QosResponse, PState1} = lists:foldl(fun(_Topic, {[?SUBACK_FAILURE | _] = L, S}) -> @@ -177,7 +183,7 @@ process_request(?SUBSCRIBE, {[?SUBACK_FAILURE | L], S0} end end, {[], PState0}, Topics), - serialise_and_send_to_client( + SendFun( #mqtt_frame{fixed = #mqtt_frame_fixed{type = ?SUBACK}, variable = #mqtt_frame_suback{ message_id = SubscribeMsgId, @@ -198,7 +204,7 @@ process_request(?UNSUBSCRIBE, #mqtt_frame{variable = #mqtt_frame_subscribe{message_id = MessageId, topic_table = Topics}, payload = undefined}, - PState0) -> + PState0 = #proc_state{send_fun = SendFun}) -> rabbit_log_connection:debug("Received an UNSUBSCRIBE for topic(s) ~p", [Topics]), PState = lists:foldl( fun(#mqtt_topic{name = TopicName}, @@ -218,15 +224,15 @@ process_request(?UNSUBSCRIBE, S0 end end, PState0, Topics), - serialise_and_send_to_client( + SendFun( #mqtt_frame{fixed = #mqtt_frame_fixed {type = ?UNSUBACK}, variable = #mqtt_frame_suback{message_id = MessageId}}, PState), {ok, PState}; -process_request(?PINGREQ, #mqtt_frame{}, PState) -> +process_request(?PINGREQ, #mqtt_frame{}, PState = #proc_state{send_fun = SendFun}) -> rabbit_log_connection:debug("Received a PINGREQ"), - serialise_and_send_to_client( + SendFun( #mqtt_frame{fixed = #mqtt_frame_fixed{type = ?PINGRESP}}, PState), rabbit_log_connection:debug("Sent a PINGRESP"), @@ -243,7 +249,7 @@ process_connect(#mqtt_frame{ clean_sess = CleanSess, client_id = ClientId, keep_alive = Keepalive} = FrameConnect}, - PState0) -> + PState0 = #proc_state{send_fun = SendFun}) -> rabbit_log_connection:debug("Received a CONNECT, client ID: ~s, username: ~s, " "clean session: ~s, protocol version: ~p, keepalive: ~p", [ClientId, Username, CleanSess, ProtoVersion, Keepalive]), @@ -265,7 +271,7 @@ process_connect(#mqtt_frame{ variable = #mqtt_frame_connack{ session_present = SessionPresent, return_code = ReturnCode}}, - serialise_and_send_to_client(ResponseFrame, PState), + SendFun(ResponseFrame, PState), return_connack(ReturnCode, PState). client_id([]) -> @@ -489,7 +495,8 @@ hand_off_to_retainer(RetainerPid, Amqp2MqttFun, Topic0, Msg) -> maybe_send_retained_message(RPid, #mqtt_topic{name = Topic0, qos = SubscribeQos}, #proc_state{amqp2mqtt_fun = Amqp2MqttFun, - packet_id = PacketId0} = PState0) -> + packet_id = PacketId0, + send_fun = SendFun} = PState0) -> Topic1 = Amqp2MqttFun(Topic0), case rabbit_mqtt_retainer:fetch(RPid, Topic1) of undefined -> @@ -502,7 +509,7 @@ maybe_send_retained_message(RPid, #mqtt_topic{name = Topic0, qos = SubscribeQos} ?QOS_1 -> {PacketId0, PState0#proc_state{packet_id = increment_packet_id(PacketId0)}} end, - serialise_and_send_to_client( + SendFun( #mqtt_frame{fixed = #mqtt_frame_fixed{ type = ?PUBLISH, qos = Qos, @@ -1123,9 +1130,9 @@ send_puback(MsgIds, PState) lists:foreach(fun(Id) -> send_puback(Id, PState) end, MsgIds); -send_puback(MsgId, PState) -> +send_puback(MsgId, PState = #proc_state{send_fun = SendFun}) -> rabbit_global_counters:messages_confirmed(mqtt, 1), - serialise_and_send_to_client( + SendFun( #mqtt_frame{fixed = #mqtt_frame_fixed{type = ?PUBACK}, variable = #mqtt_frame_publish{message_id = MsgId}}, PState). @@ -1141,6 +1148,9 @@ serialise_and_send_to_client(Frame, #proc_state{proto_ver = ProtoVer, socket = S [Sock, Error, Frame]) end. +serialise(Frame, #proc_state{proto_ver = ProtoVer}) -> + rabbit_mqtt_frame:serialise(Frame, ProtoVer). + terminate(#proc_state{client_id = undefined}) -> ok; terminate(#proc_state{client_id = ClientId}) -> @@ -1273,7 +1283,8 @@ maybe_publish_to_client( #basic_message{ routing_keys = [RoutingKey | _CcRoutes], content = #content{payload_fragments_rev = FragmentsRev}}}, - QoS, PState0 = #proc_state{amqp2mqtt_fun = Amqp2MqttFun}) -> + QoS, PState0 = #proc_state{amqp2mqtt_fun = Amqp2MqttFun, + send_fun = SendFun}) -> {PacketId, PState} = queue_message_id_to_packet_id(QMsgId, QoS, PState0), %%TODO support iolists when sending to client Payload = list_to_binary(lists:reverse(FragmentsRev)), @@ -1293,7 +1304,7 @@ maybe_publish_to_client( message_id = PacketId, topic_name = Amqp2MqttFun(RoutingKey)}, payload = Payload}, - serialise_and_send_to_client(Frame, PState), + SendFun(Frame, PState), PState. queue_message_id_to_packet_id(_, ?QOS_0, PState) -> diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_connection_info.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_connection_info.erl deleted file mode 100644 index 9e0c17d1713b..000000000000 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_connection_info.erl +++ /dev/null @@ -1,26 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved. -%% - --module(rabbit_web_mqtt_connection_info). - -%% Module to add the MQTT client ID to authentication properties - -%% API --export([additional_authn_params/4]). - -additional_authn_params(_Creds, _VHost, _Pid, Infos) -> - case proplists:get_value(variable_map, Infos, undefined) of - VariableMap when is_map(VariableMap) -> - case maps:get(<<"client_id">>, VariableMap, []) of - ClientId when is_binary(ClientId)-> - [{client_id, ClientId}]; - [] -> - [] - end; - _ -> - [] - end. diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index bc1b9abfbbbf..d84bfef25c6b 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -23,8 +23,6 @@ upgrade/5, takeover/7]). --include_lib("amqp_client/include/amqp_client.hrl"). - -record(state, { conn_name, keepalive, @@ -36,9 +34,16 @@ socket, peername, stats_timer, - connection + received_connect_frame }). +%%TODO Use 1 Erlang process per connection +%% => remove rabbit_heartbeat processes +%% => partly revert https://github.com/rabbitmq/rabbitmq-server/commit/9c153b2d405 ? + +%%TODO move from deprecated callback results to new callback results +%% see cowboy_websocket.erl + %% cowboy_sub_protcol upgrade(Req, Env, Handler, HandlerState) -> upgrade(Req, Env, Handler, HandlerState, #{}). @@ -62,9 +67,18 @@ init(Req, Opts) -> {_, KeepaliveSup} = lists:keyfind(keepalive_sup, 1, Opts), SockInfo = maps:get(proxy_header, Req, undefined), WsOpts0 = proplists:get_value(ws_opts, Opts, #{}), + %%TODO return idle_timeout? + %% Do we need both MQTT Keepalives and WebSocket pings or is the latter just enough to determine + %% when we need to close the connection? + %% Check how other MQTT over WebSocket brokers handle it. + %% + %%TODO is compress needed? WsOpts = maps:merge(#{compress => true}, WsOpts0), Req2 = case cowboy_req:header(<<"sec-websocket-protocol">>, Req) of undefined -> Req; + %%TODO check whether client offers mqtt: + %% MQTT spec: + %% "The Client MUST include “mqtt” in the list of WebSocket Sub Protocols it offers" SecWsProtocol -> cowboy_req:set_resp_header(<<"sec-websocket-protocol">>, SecWsProtocol, Req) end, @@ -75,7 +89,8 @@ init(Req, Opts) -> state = running, conserve_resources = false, socket = SockInfo, - peername = PeerAddr + peername = PeerAddr, + received_connect_frame = false }, WsOpts}. websocket_init(State0 = #state{socket = Sock, peername = PeerAddr}) -> @@ -86,12 +101,10 @@ websocket_init(State0 = #state{socket = Sock, peername = PeerAddr}) -> conn_name = ConnStr, socket = Sock }, - rabbit_log_connection:info("accepting Web MQTT connection ~tp (~ts)", [self(), ConnStr]), - AdapterInfo = amqp_connection:socket_adapter_info(Sock, {'Web MQTT', "N/A"}), + rabbit_log_connection:info("accepting Web MQTT connection ~p (~s)", [self(), ConnStr]), RealSocket = rabbit_net:unwrap_socket(Sock), - ProcessorState = rabbit_mqtt_processor:initial_state(Sock, - rabbit_mqtt_reader:ssl_login_name(RealSocket), - AdapterInfo, + ProcessorState = rabbit_mqtt_processor:initial_state(RealSocket, + ConnStr, fun send_reply/2, PeerAddr), process_flag(trap_exit, true), @@ -113,15 +126,20 @@ close_connection(Pid, Reason) -> websocket_handle({binary, Data}, State) -> handle_data(Data, State); -%% Silently ignore ping and pong frames. -websocket_handle({Ping, _}, State) when Ping =:= ping; Ping =:= pong -> +%% Silently ignore ping and pong frames as Cowboy will automatically reply to ping frames. +websocket_handle({Ping, _}, State) + when Ping =:= ping orelse Ping =:= pong -> {ok, State, hibernate}; -websocket_handle(Ping, State) when Ping =:= ping; Ping =:= pong -> +websocket_handle(Ping, State) + when Ping =:= ping orelse Ping =:= pong -> {ok, State, hibernate}; %% Log any other unexpected frames. websocket_handle(Frame, State) -> rabbit_log_connection:info("Web MQTT: unexpected WebSocket frame ~tp", [Frame]), + %%TODO close connection instead? + %%"MQTT Control Packets MUST be sent in WebSocket binary data frames. + %% If any other type of data frame is received the recipient MUST close the Network Connection" {ok, State, hibernate}. %% `rabbit_mqtt_processor:amqp_callback/2` doesn't actually return @@ -141,19 +159,21 @@ websocket_info({conserve_resources, Conserve}, State) -> websocket_info({bump_credit, Msg}, State) -> credit_flow:handle_bump_msg(Msg), handle_credits(control_throttle(State)); -websocket_info({#'basic.deliver'{}, #amqp_msg{}, _DeliveryCtx} = Delivery, - State = #state{ proc_state = ProcState0 }) -> - callback_reply(State, rabbit_mqtt_processor:amqp_callback(Delivery, ProcState0)); -websocket_info(#'basic.ack'{} = Ack, State = #state{ proc_state = ProcState0 }) -> - callback_reply(State, rabbit_mqtt_processor:amqp_callback(Ack, ProcState0)); -websocket_info(#'basic.consume_ok'{}, State) -> - {ok, State, hibernate}; -websocket_info(#'basic.cancel'{}, State) -> - stop(State); + %%TODO return hibernate? websocket_info({reply, Data}, State) -> {reply, {binary, Data}, State, hibernate}; websocket_info({'EXIT', _, _}, State) -> stop(State); +websocket_info({'$gen_cast', QueueEvent = {queue_event, _, _}}, + State = #state{proc_state = PState0}) -> + case rabbit_mqtt_processor:handle_queue_event(QueueEvent, PState0) of + {ok, PState} -> + {ok, State#state{proc_state = PState}, hibernate}; + {error, Reason, PState} -> + rabbit_log_connection:error("Web MQTT connection ~p failed to handle queue event: ~p", + [State#state.conn_name, Reason]), + stop(State#state{proc_state = PState}) + end; websocket_info({'$gen_cast', duplicate_id}, State = #state{ proc_state = ProcState, conn_name = ConnName }) -> rabbit_log_connection:warning("Web MQTT disconnecting a client with duplicate ID '~ts' (~tp)", @@ -164,7 +184,7 @@ websocket_info({'$gen_cast', {close_connection, Reason}}, State = #state{ proc_s rabbit_log_connection:warning("Web MQTT disconnecting client with ID '~ts' (~tp), reason: ~ts", [rabbit_mqtt_processor:info(client_id, ProcState), ConnName, Reason]), stop(State); -websocket_info({start_keepalives, Keepalive}, +websocket_info({start_keepalive, Keepalive}, State = #state{ socket = Sock, keepalive_sup = KeepaliveSup }) -> %% Only the client has the responsibility for sending keepalives SendFun = fun() -> ok end, @@ -178,11 +198,12 @@ websocket_info(keepalive_timeout, State = #state{conn_name = ConnStr}) -> stop(State); websocket_info(emit_stats, State) -> {ok, emit_stats(State), hibernate}; -websocket_info({ra_event, _, _}, State) -> - {ok, State, hibernate}; +websocket_info({ra_event, _From, Evt}, + #state{proc_state = PState0} = State) -> + PState = rabbit_mqtt_processor:handle_ra_event(Evt, PState0), + {ok, State#state{proc_state = PState}, hibernate}; websocket_info(Msg, State) -> - rabbit_log_connection:info("Web MQTT: unexpected message ~tp", - [Msg]), + rabbit_log_connection:warning("Web MQTT: unexpected message ~p", [Msg]), {ok, State, hibernate}. terminate(_, _, #state{ proc_state = undefined }) -> @@ -203,6 +224,13 @@ handle_data(Data, State0 = #state{conn_name = ConnStr}) -> Other end. +handle_data1(<<>>, State0 = #state{received_connect_frame = false, + proc_state = PState, + conn_name = ConnStr}) -> + rabbit_log_connection:info("Accepted web MQTT connection ~p (~s, client id: ~s)", + [self(), ConnStr, rabbit_mqtt_processor:info(client_id, PState)]), + State = State0#state{received_connect_frame = true}, + {ok, ensure_stats_timer(control_throttle(State)), hibernate}; handle_data1(<<>>, State) -> {ok, ensure_stats_timer(control_throttle(State)), hibernate}; handle_data1(Data, State = #state{ parse_state = ParseState, @@ -214,13 +242,12 @@ handle_data1(Data, State = #state{ parse_state = ParseState, State #state{ parse_state = ParseState1 })), hibernate}; {ok, Frame, Rest} -> case rabbit_mqtt_processor:process_frame(Frame, ProcState) of - {ok, ProcState1, ConnPid} -> + {ok, ProcState1} -> PS = rabbit_mqtt_frame:initial_state(), handle_data1( Rest, - State #state{ parse_state = PS, - proc_state = ProcState1, - connection = ConnPid }); + State#state{parse_state = PS, + proc_state = ProcState1}); {error, Reason, _} -> rabbit_log_connection:info("MQTT protocol error ~tp for connection ~tp", [Reason, ConnStr]), @@ -252,8 +279,8 @@ stop_rabbit_mqtt_processor(State = #state{state = running, conn_name = ConnName}) -> maybe_emit_stats(State), rabbit_log_connection:info("closing Web MQTT connection ~tp (~ts)", [self(), ConnName]), - _ = rabbit_mqtt_processor:send_will(ProcState), - rabbit_mqtt_processor:close_connection(ProcState). + rabbit_mqtt_processor:send_will(ProcState), + rabbit_mqtt_processor:terminate(ProcState). handle_credits(State0) -> case control_throttle(State0) of @@ -275,8 +302,8 @@ control_throttle(State = #state{ state = CS, {_, _} -> State end. -send_reply(Frame, _) -> - self() ! {reply, rabbit_mqtt_frame:serialise(Frame)}. +send_reply(Frame, PState) -> + self() ! {reply, rabbit_mqtt_processor:serialise(Frame, PState)}. ensure_stats_timer(State) -> rabbit_event:ensure_stats_timer(State, #state.stats_timer, emit_stats). @@ -285,19 +312,18 @@ maybe_emit_stats(State) -> rabbit_event:if_enabled(State, #state.stats_timer, fun() -> emit_stats(State) end). -emit_stats(State=#state{connection = C}) when C == none; C == undefined -> +emit_stats(State=#state{received_connect_frame = false}) -> %% Avoid emitting stats on terminate when the connection has not yet been %% established, as this causes orphan entries on the stats database State1 = rabbit_event:reset_stats_timer(State, #state.stats_timer), State1; -emit_stats(State=#state{socket=Sock, state=RunningState, connection=Conn}) -> +emit_stats(State=#state{socket=Sock, state=RunningState}) -> SockInfos = case rabbit_net:getstat(Sock, [recv_oct, recv_cnt, send_oct, send_cnt, send_pend]) of {ok, SI} -> SI; {error, _} -> [] end, - Infos = [{pid, Conn}, {state, RunningState}|SockInfos], - rabbit_core_metrics:connection_stats(Conn, Infos), - rabbit_event:notify(connection_stats, Infos), + Infos = [{pid, self()}, {state, RunningState}|SockInfos], + rabbit_core_metrics:connection_stats(self(), Infos), State1 = rabbit_event:reset_stats_timer(State, #state.stats_timer), State1. diff --git a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl index e4216d96cca1..f63ac60b674b 100644 --- a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl +++ b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl @@ -10,7 +10,7 @@ -include_lib("eunit/include/eunit.hrl"). -include("src/emqttc_packet.hrl"). --compile(export_all). +-compile([export_all, nowarn_export_all]). all() -> [ @@ -55,7 +55,7 @@ init_per_testcase(Testcase, Config) -> end_per_testcase(Testcase, Config) -> rabbit_ct_helpers:testcase_finished(Config, Testcase). --define(DEFAULT_TIMEOUT, 15000). +-define(DEFAULT_TIMEOUT, 15_000). connection(Config) -> From fc33719d773e40e62614456515fd2015da8665c7 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Fri, 30 Sep 2022 14:55:10 +0200 Subject: [PATCH 013/118] Use 1 instead of 4 processes per WebMQTT connection Reducing number of Erlang processes allows better scaling with millions of clients connecting via MQTT over WebSockets. In this commit, we partially revert https://github.com/rabbitmq/rabbitmq-server/commit/9c153b2d4054a2a60231f30b5951561f7573eb82 where support for heartbeats were introduced. Prior to this commit there were 4 processes per WebMQTT connection supervised by ranch_conns_sup in the ranch application: 1. rabbit_web_mqtt_connection_sup 2. rabbit_web_mqtt_connection_sup (id=rabbit_web_mqtt_keepalive_sup) 3. rabbit_hearbeat (receiver) 4. rabbit_web_mqtt_handler connection process After this commit, there is only the 4th process supervised directly by ranch_conns_sup. --- .../src/rabbit_web_mqtt_app.erl | 104 +++++++++--------- .../src/rabbit_web_mqtt_connection_sup.erl | 75 ------------- .../src/rabbit_web_mqtt_handler.erl | 46 ++++---- .../src/rabbit_web_mqtt_middleware.erl | 20 ---- 4 files changed, 71 insertions(+), 174 deletions(-) delete mode 100644 deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_connection_sup.erl delete mode 100644 deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_middleware.erl diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_app.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_app.erl index a30295756318..0fde0e4e20d5 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_app.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_app.erl @@ -80,14 +80,14 @@ cowboy_ws_connection_pid(RanchConnPid) -> mqtt_init() -> CowboyOpts0 = maps:from_list(get_env(cowboy_opts, [])), CowboyWsOpts = maps:from_list(get_env(cowboy_ws_opts, [])), - Routes = cowboy_router:compile([{'_', [ {get_env(ws_path, "/ws"), rabbit_web_mqtt_handler, [{ws_opts, CowboyWsOpts}]} ]}]), - CowboyOpts = CowboyOpts0#{env => #{dispatch => Routes}, - middlewares => [cowboy_router, rabbit_web_mqtt_middleware, cowboy_handler], - proxy_header => get_env(proxy_protocol, false), - stream_handlers => [rabbit_web_mqtt_stream_handler, cowboy_stream_h]}, + CowboyOpts = CowboyOpts0#{ + env => #{dispatch => Routes}, + proxy_header => get_env(proxy_protocol, false), + stream_handlers => [rabbit_web_mqtt_stream_handler, cowboy_stream_h] + }, case get_env(tcp_config, []) of [] -> ok; TCPConf0 -> start_tcp_listener(TCPConf0, CowboyOpts) @@ -99,59 +99,55 @@ mqtt_init() -> ok. start_tcp_listener(TCPConf0, CowboyOpts) -> - {TCPConf, IpStr, Port} = get_tcp_conf(TCPConf0), - RanchTransportOpts = #{ - socket_opts => TCPConf, - connection_type => supervisor, - max_connections => get_max_connections(), - num_acceptors => get_env(num_tcp_acceptors, 10), - num_conns_sups => get_env(num_conns_sup, 1) - }, - case ranch:start_listener(rabbit_networking:ranch_ref(TCPConf), - ranch_tcp, - RanchTransportOpts, - rabbit_web_mqtt_connection_sup, - CowboyOpts) of - {ok, _} -> ok; - {error, {already_started, _}} -> ok; - {error, ErrTCP} -> - rabbit_log_connection:error( - "Failed to start a WebSocket (HTTP) listener. Error: ~tp," - " listener settings: ~tp", + {TCPConf, IpStr, Port} = get_tcp_conf(TCPConf0), + RanchRef = rabbit_networking:ranch_ref(TCPConf), + RanchTransportOpts = + #{ + socket_opts => TCPConf, + max_connections => get_max_connections(), + num_acceptors => get_env(num_tcp_acceptors, 10), + num_conns_sups => get_env(num_conns_sup, 1) + }, + case cowboy:start_clear(RanchRef, RanchTransportOpts, CowboyOpts) of + {ok, _} -> + ok; + {error, {already_started, _}} -> + ok; + {error, ErrTCP} -> + rabbit_log_connection:error( + "Failed to start a WebSocket (HTTP) listener. Error: ~p, listener settings: ~p", [ErrTCP, TCPConf]), - throw(ErrTCP) - end, - listener_started(?TCP_PROTOCOL, TCPConf), - rabbit_log:info("rabbit_web_mqtt: listening for HTTP connections on ~ts:~w", - [IpStr, Port]). + throw(ErrTCP) + end, + listener_started(?TCP_PROTOCOL, TCPConf), + rabbit_log:info("rabbit_web_mqtt: listening for HTTP connections on ~s:~w", + [IpStr, Port]). start_tls_listener(TLSConf0, CowboyOpts) -> - _ = rabbit_networking:ensure_ssl(), - {TLSConf, TLSIpStr, TLSPort} = get_tls_conf(TLSConf0), - RanchTransportOpts = #{ - socket_opts => TLSConf, - connection_type => supervisor, - max_connections => get_max_connections(), - num_acceptors => get_env(num_ssl_acceptors, 10), - num_conns_sups => get_env(num_conns_sup, 1) - }, - case ranch:start_listener(rabbit_networking:ranch_ref(TLSConf), - ranch_ssl, - RanchTransportOpts, - rabbit_web_mqtt_connection_sup, - CowboyOpts) of - {ok, _} -> ok; - {error, {already_started, _}} -> ok; - {error, ErrTLS} -> - rabbit_log_connection:error( - "Failed to start a TLS WebSocket (HTTPS) listener. Error: ~tp," - " listener settings: ~tp", + rabbit_networking:ensure_ssl(), + {TLSConf, TLSIpStr, TLSPort} = get_tls_conf(TLSConf0), + RanchRef = rabbit_networking:ranch_ref(TLSConf), + RanchTransportOpts = + #{ + socket_opts => TLSConf, + max_connections => get_max_connections(), + num_acceptors => get_env(num_ssl_acceptors, 10), + num_conns_sups => get_env(num_conns_sup, 1) + }, + case cowboy:start_tls(RanchRef, RanchTransportOpts, CowboyOpts) of + {ok, _} -> + ok; + {error, {already_started, _}} -> + ok; + {error, ErrTLS} -> + rabbit_log_connection:error( + "Failed to start a TLS WebSocket (HTTPS) listener. Error: ~p, listener settings: ~p", [ErrTLS, TLSConf]), - throw(ErrTLS) - end, - listener_started(?TLS_PROTOCOL, TLSConf), - rabbit_log:info("rabbit_web_mqtt: listening for HTTPS connections on ~ts:~w", - [TLSIpStr, TLSPort]). + throw(ErrTLS) + end, + listener_started(?TLS_PROTOCOL, TLSConf), + rabbit_log:info("rabbit_web_mqtt: listening for HTTPS connections on ~s:~w", + [TLSIpStr, TLSPort]). listener_started(Protocol, Listener) -> Port = rabbit_misc:pget(port, Listener), diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_connection_sup.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_connection_sup.erl deleted file mode 100644 index c37488ecf4f5..000000000000 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_connection_sup.erl +++ /dev/null @@ -1,75 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved. -%% - --module(rabbit_web_mqtt_connection_sup). - --behaviour(supervisor). --behaviour(ranch_protocol). - --include_lib("rabbit_common/include/rabbit.hrl"). - --export([start_link/3, start_keepalive_link/0]). - --export([init/1]). - -%%---------------------------------------------------------------------------- - -start_link(Ref, Transport, CowboyOpts0) -> - {ok, SupPid} = supervisor:start_link(?MODULE, []), - {ok, KeepaliveSup} = supervisor:start_child( - SupPid, - #{ - id => rabbit_web_mqtt_keepalive_sup, - start => {?MODULE, start_keepalive_link, []}, - restart => transient, - significant => true, - shutdown => infinity, - type => supervisor, - modules => [rabbit_keepalive_sup] - } - ), - - %% In order for the Websocket handler to receive the KeepaliveSup - %% variable, we need to pass it first through the environment and - %% then have the middleware rabbit_web_mqtt_middleware place it - %% in the initial handler state. - Env = maps:get(env, CowboyOpts0), - CowboyOpts = CowboyOpts0#{env => Env#{keepalive_sup => KeepaliveSup}}, - Protocol = case Transport of - ranch_tcp -> cowboy_clear; - ranch_ssl -> cowboy_tls - end, - {ok, ReaderPid} = supervisor:start_child( - SupPid, - #{ - id => Protocol, - start => {Protocol, start_link, [Ref, Transport, CowboyOpts]}, - restart => transient, - significant => true, - shutdown => ?WORKER_WAIT, - type => worker, - modules => [Protocol] - } - ), - {ok, SupPid, ReaderPid}. - -start_keepalive_link() -> - supervisor:start_link(?MODULE, []). - -%%---------------------------------------------------------------------------- - -init([]) -> - {ok, - { - #{ - strategy => one_for_all, - intensity => 0, - period => 1, - auto_shutdown => any_significant - }, - [] - }}. diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index d84bfef25c6b..c9dcf99531ac 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -25,8 +25,6 @@ -record(state, { conn_name, - keepalive, - keepalive_sup, parse_state, proc_state, state, @@ -44,6 +42,13 @@ %%TODO move from deprecated callback results to new callback results %% see cowboy_websocket.erl +%%TODO call rabbit_networking:register_non_amqp_connection/1 so that we are notified +%% when need to force load the 'connection_created' event for the management plugin, see +%% https://github.com/rabbitmq/rabbitmq-management-agent/issues/58 +%% https://github.com/rabbitmq/rabbitmq-server/blob/90cc0e2abf944141feedaf3190d7b6d8b4741b11/deps/rabbitmq_stream/src/rabbit_stream_reader.erl#L536 +%% https://github.com/rabbitmq/rabbitmq-server/blob/90cc0e2abf944141feedaf3190d7b6d8b4741b11/deps/rabbitmq_stream/src/rabbit_stream_reader.erl#L189 +%% https://github.com/rabbitmq/rabbitmq-server/blob/7eb4084cf879fe6b1d26dd3c837bd180cb3a8546/deps/rabbitmq_management_agent/src/rabbit_mgmt_db_handler.erl#L72 + %% cowboy_sub_protcol upgrade(Req, Env, Handler, HandlerState) -> upgrade(Req, Env, Handler, HandlerState, #{}). @@ -64,7 +69,6 @@ takeover(Parent, Ref, Socket, Transport, Opts, Buffer, {Handler, HandlerState}) %% cowboy_websocket init(Req, Opts) -> {PeerAddr, _PeerPort} = maps:get(peer, Req), - {_, KeepaliveSup} = lists:keyfind(keepalive_sup, 1, Opts), SockInfo = maps:get(proxy_header, Req, undefined), WsOpts0 = proplists:get_value(ws_opts, Opts, #{}), %%TODO return idle_timeout? @@ -83,8 +87,6 @@ init(Req, Opts) -> cowboy_req:set_resp_header(<<"sec-websocket-protocol">>, SecWsProtocol, Req) end, {?MODULE, Req2, #state{ - keepalive = {none, none}, - keepalive_sup = KeepaliveSup, parse_state = rabbit_mqtt_frame:initial_state(), state = running, conserve_resources = false, @@ -101,7 +103,7 @@ websocket_init(State0 = #state{socket = Sock, peername = PeerAddr}) -> conn_name = ConnStr, socket = Sock }, - rabbit_log_connection:info("accepting Web MQTT connection ~p (~s)", [self(), ConnStr]), + rabbit_log_connection:info("Accepting Web MQTT connection ~p (~s)", [self(), ConnStr]), RealSocket = rabbit_net:unwrap_socket(Sock), ProcessorState = rabbit_mqtt_processor:initial_state(RealSocket, ConnStr, @@ -184,18 +186,12 @@ websocket_info({'$gen_cast', {close_connection, Reason}}, State = #state{ proc_s rabbit_log_connection:warning("Web MQTT disconnecting client with ID '~ts' (~tp), reason: ~ts", [rabbit_mqtt_processor:info(client_id, ProcState), ConnName, Reason]), stop(State); -websocket_info({start_keepalive, Keepalive}, - State = #state{ socket = Sock, keepalive_sup = KeepaliveSup }) -> - %% Only the client has the responsibility for sending keepalives - SendFun = fun() -> ok end, - Parent = self(), - ReceiveFun = fun() -> Parent ! keepalive_timeout end, - Heartbeater = rabbit_heartbeat:start( - KeepaliveSup, Sock, 0, SendFun, Keepalive, ReceiveFun), - {ok, State #state { keepalive = Heartbeater }, hibernate}; -websocket_info(keepalive_timeout, State = #state{conn_name = ConnStr}) -> - rabbit_log_connection:error("closing Web MQTT connection ~tp (keepalive timeout)", [ConnStr]), - stop(State); +websocket_info({start_keepalive, _Keepalive}, State) -> + %%TODO use timer as done in rabbit_mqtt_reader + {ok, State, hibernate}; +% websocket_info(keepalive_timeout, State = #state{conn_name = ConnStr}) -> +% rabbit_log_connection:error("closing Web MQTT connection ~p (keepalive timeout)", [ConnStr]), +% stop(State); websocket_info(emit_stats, State) -> {ok, emit_stats(State), hibernate}; websocket_info({ra_event, _From, Evt}, @@ -293,13 +289,13 @@ handle_credits(State0) -> control_throttle(State = #state{ state = CS, conserve_resources = Mem }) -> case {CS, Mem orelse credit_flow:blocked()} of - {running, true} -> ok = rabbit_heartbeat:pause_monitor( - State#state.keepalive), - State #state{ state = blocked }; - {blocked, false} -> ok = rabbit_heartbeat:resume_monitor( - State#state.keepalive), - State #state{ state = running }; - {_, _} -> State + %%TODO cancel / resume keepalive timer as done in rabbit_mqtt_reader + {running, true} -> + State #state{state = blocked}; + {blocked,false} -> + State #state{state = running}; + {_, _} -> + State end. send_reply(Frame, PState) -> diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_middleware.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_middleware.erl deleted file mode 100644 index 23cfa04f16b1..000000000000 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_middleware.erl +++ /dev/null @@ -1,20 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved. -%% - --module(rabbit_web_mqtt_middleware). --behavior(cowboy_middleware). - --export([execute/2]). - -execute(Req, Env) -> - #{keepalive_sup := KeepaliveSup} = Env, - case maps:get(handler_opts, Env, undefined) of - undefined -> {ok, Req, Env}; - Opts when is_list(Opts) -> - {ok, Req, Env#{handler_opts => [{keepalive_sup, KeepaliveSup} - |Opts]}} - end. From 5710a9474a5e5e2aef68f3e2045dd068eb657f7d Mon Sep 17 00:00:00 2001 From: David Ansari Date: Mon, 3 Oct 2022 18:44:33 +0000 Subject: [PATCH 014/118] Support MQTT Keepalive in WebMQTT Share the same MQTT keepalive code between rabbit_mqtt_reader and rabbit_web_mqtt_handler. Add MQTT keepalive test in both plugins rabbitmq_mqtt and rabbitmq_web_mqtt. --- deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl | 30 ++--- .../src/rabbit_mqtt_keepalive.erl | 92 ++++++++++++++ .../src/rabbit_mqtt_processor.erl | 13 +- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 118 ++++-------------- deps/rabbitmq_mqtt/test/reader_SUITE.erl | 38 ++++++ .../src/rabbit_web_mqtt_handler.erl | 110 ++++++++-------- deps/rabbitmq_web_mqtt/test/system_SUITE.erl | 28 +++++ 7 files changed, 260 insertions(+), 169 deletions(-) create mode 100644 deps/rabbitmq_mqtt/src/rabbit_mqtt_keepalive.erl diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl index 5db1b8a16a8f..4743780f910b 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl @@ -10,23 +10,19 @@ -include("rabbit_mqtt_types.hrl"). %% reader state --record(state, {socket, - proxy_socket, - conn_name, - await_recv, - deferred_recv, - received_connect_frame, - connection_state, - conserve, - parse_state, - proc_state, - stats_timer, - keepalive}). - --record(keepalive, {timer :: reference(), - interval_ms :: pos_integer(), - recv_oct :: non_neg_integer(), - received :: boolean()}). +-record(state, + {socket, + proxy_socket, + conn_name, + await_recv, + deferred_recv, + received_connect_frame, + connection_state, + conserve, + parse_state, + proc_state, + stats_timer, + keepalive :: rabbit_mqtt_keepalive:state()}). %% processor state -record(proc_state, diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_keepalive.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_keepalive.erl new file mode 100644 index 000000000000..46ab4a129277 --- /dev/null +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_keepalive.erl @@ -0,0 +1,92 @@ +-module(rabbit_mqtt_keepalive). + +-export([start/2, + handle/2, + start_timer/1, + cancel_timer/1]). + +-export_type([state/0]). + +-record(state, {timer :: reference(), + interval_ms :: pos_integer(), + socket :: inet:socket(), + recv_oct :: non_neg_integer(), + received :: boolean()}). + +-opaque(state() :: undefined | #state{}). + +-spec start(IntervalSeconds :: non_neg_integer(), inet:socket()) -> ok. +start(0, _Sock) -> + ok; +start(Seconds, Sock) + when is_integer(Seconds) andalso Seconds > 0 -> + self() ! {keepalive, {init, Seconds, Sock}}, + ok. + +-spec handle(Request :: term(), state()) -> + {ok, state()} | {error, Reason :: term()}. +handle({init, IntervalSecs, Sock}, _State) -> + case rabbit_net:getstat(Sock, [recv_oct]) of + {ok, [{recv_oct, RecvOct}]} -> + %% "If the Keep Alive value is non-zero and the Server does not receive a Control + %% Packet from the Client within one and a half times the Keep Alive time period, + %% it MUST disconnect the Network Connection to the Client as if the network had + %% failed" [MQTT-3.1.2-24] + %% + %% We check every (1.5 / 2 = 0.75) * KeepaliveInterval whether we received + %% any data from the client. If there was no activity for two consecutive times, + %% we close the connection. + %% We choose 0.75 (instead of a larger or smaller factor) to have the right balance + %% between not checking too often (since it could become expensive when there are + %% millions of clients) and not checking too rarely (to detect dead clients promptly). + %% + %% See https://github.com/emqx/emqx/issues/460 + %% PING + %% | DOWN + %% | |<-------Delay Time---------> + %% t0---->|----------|----------|----------|---->tn + %% | | | + %% Ok Retry Timeout + IntervalMs = round(0.75 * timer:seconds(IntervalSecs)), + State = #state{socket = Sock, + interval_ms = IntervalMs, + recv_oct = RecvOct, + received = true}, + {ok, start_timer(State)}; + {error, _} = Err -> + Err + end; +handle(check, State = #state{socket = Sock, + recv_oct = SameRecvOct, + received = ReceivedPreviously}) -> + case rabbit_net:getstat(Sock, [recv_oct]) of + {ok, [{recv_oct, SameRecvOct}]} + when ReceivedPreviously -> + %% Did not receive from socket for the 1st time. + {ok, start_timer(State#state{received = false})}; + {ok, [{recv_oct, SameRecvOct}]} -> + %% Did not receive from socket for 2nd time. + {error, timeout}; + {ok, [{recv_oct, NewRecvOct}]} -> + %% Received from socket. + {ok, start_timer(State#state{recv_oct = NewRecvOct, + received = true})}; + {error, _} = Err -> + Err + end. + +-spec start_timer(state()) -> state(). +start_timer(undefined) -> + undefined; +start_timer(#state{interval_ms = IntervalMs} = State) -> + Ref = erlang:send_after(IntervalMs, self(), {keepalive, check}), + State#state{timer = Ref}. + +-spec cancel_timer(state()) -> state(). +cancel_timer(undefined) -> + undefined; +cancel_timer(#state{timer = Ref} = State) + when is_reference(Ref) -> + ok = erlang:cancel_timer(Ref, [{async, true}, + {info, false}]), + State. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 7de05834a77d..05d1d002530a 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -260,6 +260,7 @@ process_connect(#mqtt_frame{ fun login/2, fun register_client/2, fun notify_connection_created/2, + fun start_keepalive/2, fun handle_clean_session/2], FrameConnect, PState0) of {ok, SessionPresent0, PState1} -> @@ -333,18 +334,14 @@ login({UserBin, PassBin, register_client(already_connected, _PState) -> ok; -register_client(Frame = #mqtt_frame_connect{ - keep_alive = Keepalive, - proto_ver = ProtoVersion}, +register_client(Frame = #mqtt_frame_connect{proto_ver = ProtoVersion}, PState = #proc_state{client_id = ClientId, socket = Socket, - auth_state = #auth_state{ - vhost = VHost}}) -> + auth_state = #auth_state{vhost = VHost}}) -> case rabbit_mqtt_collector:register(ClientId, self()) of {ok, Corr} -> RetainerPid = rabbit_mqtt_retainer_sup:child_for_vhost(VHost), Prefetch = rabbit_mqtt_util:env(prefetch), - rabbit_mqtt_reader:start_keepalive(self(), Keepalive), {ok, {PeerHost, PeerPort, Host, Port}} = rabbit_net:socket_ends(Socket, inbound), ExchangeBin = rabbit_mqtt_util:env(exchange), ExchangeName = rabbit_misc:r(VHost, exchange, ExchangeBin), @@ -421,6 +418,10 @@ return_connack(?CONNACK_ID_REJECTED, S) -> return_connack(?CONNACK_UNACCEPTABLE_PROTO_VER, S) -> {error, unsupported_protocol_version, S}. +start_keepalive(#mqtt_frame_connect{keep_alive = Seconds}, + #proc_state{socket = Socket}) -> + ok = rabbit_mqtt_keepalive:start(Seconds, Socket). + handle_clean_session(_, PState0 = #proc_state{clean_sess = false}) -> case get_queue(?QOS_1, PState0) of {error, not_found} -> diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index 2e09afa1e99f..29cf66ecf5ed 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -17,7 +17,7 @@ %%TODO check where to best 'hibernate' when returning from callback %%TODO use rabbit_global_counters for MQTT protocol --export([conserve_resources/3, start_keepalive/2, +-export([conserve_resources/3, close_connection/2]). -export([info/2]). @@ -166,61 +166,14 @@ handle_info({bump_credit, Msg}, State) -> credit_flow:handle_bump_msg(Msg), maybe_process_deferred_recv(control_throttle(State)); -handle_info({start_keepalive, KeepaliveSec}, - State = #state{socket = Sock, - keepalive = undefined}) - when is_number(KeepaliveSec), KeepaliveSec > 0 -> - case rabbit_net:getstat(Sock, [recv_oct]) of - {ok, [{recv_oct, RecvOct}]} -> - %% "If the Keep Alive value is non-zero and the Server does not receive a Control - %% Packet from the Client within one and a half times the Keep Alive time period, - %% it MUST disconnect the Network Connection to the Client as if the network had - %% failed" [MQTT-3.1.2-24] - %% 0.75 * 2 = 1.5 - IntervalMs = timer:seconds(round(0.75 * KeepaliveSec)), - Ref = start_keepalive_timer(#keepalive{interval_ms = IntervalMs}), - {noreply, State#state{keepalive = #keepalive{timer = Ref, - interval_ms = IntervalMs, - recv_oct = RecvOct, - received = true}}}; - {error, einval} -> - %% the socket is dead, most likely because the connection is being shut down - {stop, {shutdown, cannot_get_socket_stats}, State}; - {error, Reason} -> - {stop, Reason, State} - end; - -handle_info({timeout, Ref, keepalive}, - State = #state {socket = Sock, - conn_name = ConnStr, - proc_state = PState, - keepalive = #keepalive{timer = Ref, - recv_oct = SameRecvOct, - received = ReceivedPreviously} = KeepAlive}) -> - case rabbit_net:getstat(Sock, [recv_oct]) of - {ok, [{recv_oct, SameRecvOct}]} - when ReceivedPreviously -> - %% Did not receive from socket for the 1st time. - Ref1 = start_keepalive_timer(KeepAlive), - {noreply, - State#state{keepalive = KeepAlive#keepalive{timer = Ref1, - received = false}}, - hibernate}; - {ok, [{recv_oct, SameRecvOct}]} -> - %% Did not receive from socket for 2nd time successively. - rabbit_log_connection:error("closing MQTT connection ~tp (keepalive timeout)", [ConnStr]), - send_will_and_terminate(PState, {shutdown, keepalive_timeout}, State); - {ok, [{recv_oct, RecvOct}]} -> - %% Received from socket. - Ref1 = start_keepalive_timer(KeepAlive), - {noreply, - State#state{keepalive = KeepAlive#keepalive{timer = Ref1, - recv_oct = RecvOct, - received = true}}, - hibernate}; - {error, einval} -> - %% the socket is dead, most likely because the connection is being shut down - {stop, {shutdown, cannot_get_socket_stats}, State}; +handle_info({keepalive, Req}, State = #state{keepalive = KState0, + conn_name = ConnName}) -> + case rabbit_mqtt_keepalive:handle(Req, KState0) of + {ok, KState} -> + {noreply, State#state{keepalive = KState}, hibernate}; + {error, timeout} -> + rabbit_log_connection:error("closing MQTT connection ~p (keepalive timeout)", [ConnName]), + send_will_and_terminate({shutdown, keepalive_timeout}, State); {error, Reason} -> {stop, Reason, State} end; @@ -254,14 +207,9 @@ handle_info({'DOWN', _MRef, process, _Pid, _Reason} = Evt, handle_info(Msg, State) -> {stop, {mqtt_unexpected_msg, Msg}, State}. -start_keepalive_timer(#keepalive{interval_ms = Time}) -> - erlang:start_timer(Time, self(), keepalive). - -cancel_keepalive_timer(#keepalive{timer = Ref}) -> - erlang:cancel_timer(Ref, [{async, true}, {info, false}]). - -terminate(Reason, State) -> - maybe_emit_stats(State), +terminate(Reason, State = #state{keepalive = KState0}) -> + KState = rabbit_mqtt_keepalive:cancel_timer(KState0), + maybe_emit_stats(State#state{keepalive = KState}), do_terminate(Reason, State). handle_pre_hibernate(State) -> @@ -300,7 +248,7 @@ do_terminate({network_error, Reason}, _State) -> rabbit_log_connection:error("MQTT detected network error: ~tp", [Reason]); do_terminate(normal, #state{proc_state = ProcState, - conn_name = ConnName}) -> + conn_name = ConnName}) -> rabbit_mqtt_processor:terminate(ProcState), rabbit_log_connection:info("closing MQTT connection ~p (~s)", [self(), ConnName]), ok; @@ -395,9 +343,6 @@ callback_reply(State, {ok, ProcState}) -> callback_reply(State, {error, Reason, ProcState}) -> {stop, Reason, pstate(State, ProcState)}. -start_keepalive(_, 0 ) -> ok; -start_keepalive(Pid, Keepalive) -> Pid ! {start_keepalive, Keepalive}. - pstate(State = #state {}, PState = #proc_state{}) -> State #state{ proc_state = PState }. @@ -415,17 +360,17 @@ parse(Bytes, ParseState) -> %% "The Will Message MUST be published when the Network Connection is subsequently %% closed unless the Will Message has been deleted by the Server on receipt of a %% DISCONNECT Packet [MQTT-3.1.2-8]." -send_will_and_terminate(PState, State) -> - send_will_and_terminate(PState, {shutdown, conn_closed}, State). +send_will_and_terminate(State) -> + send_will_and_terminate({shutdown, conn_closed}, State). -send_will_and_terminate(PState, Reason, State = #state{conn_name = ConnStr}) -> +send_will_and_terminate(Reason, State = #state{conn_name = ConnStr, + proc_state = PState}) -> rabbit_log_connection:debug("MQTT: about to send will message (if any) on connection ~p", [ConnStr]), rabbit_mqtt_processor:send_will(PState), {stop, Reason, State}. network_error(closed, State = #state{conn_name = ConnStr, - proc_state = PState, received_connect_frame = Connected}) -> Fmt = "MQTT connection ~p will terminate because peer closed TCP connection", Args = [ConnStr], @@ -433,14 +378,13 @@ network_error(closed, true -> rabbit_log_connection:info(Fmt, Args); false -> rabbit_log_connection:debug(Fmt, Args) end, - send_will_and_terminate(PState, State); + send_will_and_terminate(State); network_error(Reason, - State = #state{conn_name = ConnStr, - proc_state = PState}) -> - rabbit_log_connection:info("MQTT detected network error for ~tp: ~tp", + State = #state{conn_name = ConnStr}) -> + rabbit_log_connection:info("MQTT detected network error for ~p: ~p", [ConnStr, Reason]), - send_will_and_terminate(PState, State). + send_will_and_terminate(State). run_socket(State = #state{ connection_state = blocked }) -> State; @@ -454,24 +398,14 @@ run_socket(State = #state{ socket = Sock }) -> control_throttle(State = #state{connection_state = Flow, conserve = Conserve, - keepalive = KeepAlive}) -> + keepalive = KState}) -> case {Flow, Conserve orelse credit_flow:blocked()} of - {running, true} - when KeepAlive =:= undefined -> - State#state{connection_state = blocked}; {running, true} -> - %%TODO Instead of cancelling / setting the timer every time the connection - %% gets blocked / unblocked, restart the timer when it expires and - %% the connection_state is blocked. - ok = cancel_keepalive_timer(KeepAlive), - State#state{connection_state = blocked}; - {blocked, false} - when KeepAlive =:= undefined -> - run_socket(State #state{connection_state = running}); + State#state{connection_state = blocked, + keepalive = rabbit_mqtt_keepalive:cancel_timer(KState)}; {blocked, false} -> - Ref = start_keepalive_timer(KeepAlive), - run_socket(State #state{connection_state = running, - keepalive = KeepAlive#keepalive{timer = Ref}}); + run_socket(State#state{connection_state = running, + keepalive = rabbit_mqtt_keepalive:start_timer(KState)}); {_, _} -> run_socket(State) end. diff --git a/deps/rabbitmq_mqtt/test/reader_SUITE.erl b/deps/rabbitmq_mqtt/test/reader_SUITE.erl index 97d9b255128f..8284cf2648f0 100644 --- a/deps/rabbitmq_mqtt/test/reader_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/reader_SUITE.erl @@ -11,6 +11,8 @@ -include_lib("common_test/include/ct.hrl"). -include_lib("eunit/include/eunit.hrl"). +-import(rabbit_ct_broker_helpers, [rpc/5]). + all() -> [ {group, non_parallel_tests} @@ -23,6 +25,7 @@ groups() -> block_connack_timeout, handle_invalid_frames, login_timeout, + keepalive, stats, quorum_clean_session_false, quorum_clean_session_true, @@ -179,6 +182,41 @@ login_timeout(Config) -> rpc(Config, application, unset_env, [rabbitmq_mqtt, login_timeout]) end. +keepalive(Config) -> + KeepaliveSecs = 1, + KeepaliveMs = timer:seconds(KeepaliveSecs), + P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), + {ok, C} = emqtt:start_link([{keepalive, KeepaliveSecs}, + {host, "localhost"}, + {port, P}, + {clientid, <<"simpleClient">>}, + {proto_ver, v4} + ]), + {ok, _Properties} = emqtt:connect(C), + + %% Connection should stay up when client sends PING requests. + timer:sleep(KeepaliveMs), + + %% Mock the server socket to not have received any bytes. + rabbit_ct_broker_helpers:setup_meck(Config), + Mod = rabbit_net, + ok = rpc(Config, 0, meck, new, [Mod, [no_link, passthrough]]), + ok = rpc(Config, 0, meck, expect, [Mod, getstat, 2, {ok, [{recv_oct, 999}]} ]), + + process_flag(trap_exit, true), + receive + {'EXIT', C, {shutdown, tcp_closed}} -> + ok + after + ceil(3 * 0.75 * KeepaliveMs) -> + ct:fail("server did not respect keepalive") + end, + %%TODO Introduce Prometheus counter for number of connections closed + %% due to keepalive timeout and assert here that this counter is 1. + + true = rpc(Config, 0, meck, validate, [Mod]), + ok = rpc(Config, 0, meck, unload, [Mod]). + stats(Config) -> P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), {ok, C} = emqtt:start_link([{host, "localhost"}, diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index c9dcf99531ac..5a413fc802f3 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -24,20 +24,17 @@ takeover/7]). -record(state, { - conn_name, - parse_state, - proc_state, - state, - conserve_resources, - socket, - peername, - stats_timer, - received_connect_frame -}). - -%%TODO Use 1 Erlang process per connection -%% => remove rabbit_heartbeat processes -%% => partly revert https://github.com/rabbitmq/rabbitmq-server/commit/9c153b2d405 ? + conn_name, + parse_state, + proc_state, + state, + conserve_resources, + socket, + peername, + stats_timer, + received_connect_frame, + keepalive :: rabbit_mqtt_keepalive:state() + }). %%TODO move from deprecated callback results to new callback results %% see cowboy_websocket.erl @@ -71,11 +68,6 @@ init(Req, Opts) -> {PeerAddr, _PeerPort} = maps:get(peer, Req), SockInfo = maps:get(proxy_header, Req, undefined), WsOpts0 = proplists:get_value(ws_opts, Opts, #{}), - %%TODO return idle_timeout? - %% Do we need both MQTT Keepalives and WebSocket pings or is the latter just enough to determine - %% when we need to close the connection? - %% Check how other MQTT over WebSocket brokers handle it. - %% %%TODO is compress needed? WsOpts = maps:merge(#{compress => true}, WsOpts0), Req2 = case cowboy_req:header(<<"sec-websocket-protocol">>, Req) of @@ -177,21 +169,29 @@ websocket_info({'$gen_cast', QueueEvent = {queue_event, _, _}}, stop(State#state{proc_state = PState}) end; websocket_info({'$gen_cast', duplicate_id}, State = #state{ proc_state = ProcState, - conn_name = ConnName }) -> - rabbit_log_connection:warning("Web MQTT disconnecting a client with duplicate ID '~ts' (~tp)", + conn_name = ConnName }) -> + rabbit_log_connection:warning("Web MQTT disconnecting a client with duplicate ID '~s' (~p)", [rabbit_mqtt_processor:info(client_id, ProcState), ConnName]), stop(State); websocket_info({'$gen_cast', {close_connection, Reason}}, State = #state{ proc_state = ProcState, - conn_name = ConnName }) -> - rabbit_log_connection:warning("Web MQTT disconnecting client with ID '~ts' (~tp), reason: ~ts", + conn_name = ConnName }) -> + rabbit_log_connection:warning("Web MQTT disconnecting client with ID '~s' (~p), reason: ~s", [rabbit_mqtt_processor:info(client_id, ProcState), ConnName, Reason]), stop(State); -websocket_info({start_keepalive, _Keepalive}, State) -> - %%TODO use timer as done in rabbit_mqtt_reader - {ok, State, hibernate}; -% websocket_info(keepalive_timeout, State = #state{conn_name = ConnStr}) -> -% rabbit_log_connection:error("closing Web MQTT connection ~p (keepalive timeout)", [ConnStr]), -% stop(State); +websocket_info({keepalive, Req}, State = #state{keepalive = KState0, + conn_name = ConnName}) -> + case rabbit_mqtt_keepalive:handle(Req, KState0) of + {ok, KState} -> + {ok, State#state{keepalive = KState}, hibernate}; + {error, timeout} -> + rabbit_log_connection:error("keepalive timeout in Web MQTT connection ~p", + [ConnName]), + stop(State, 1000, <<"MQTT keepalive timeout">>); + {error, Reason} -> + rabbit_log_connection:error("keepalive error in Web MQTT connection ~p: ~p", + [ConnName, Reason]), + stop(State) + end; websocket_info(emit_stats, State) -> {ok, emit_stats(State), hibernate}; websocket_info({ra_event, _From, Evt}, @@ -202,11 +202,15 @@ websocket_info(Msg, State) -> rabbit_log_connection:warning("Web MQTT: unexpected message ~p", [Msg]), {ok, State, hibernate}. -terminate(_, _, #state{ proc_state = undefined }) -> - ok; -terminate(_, _, State) -> - _ = stop_rabbit_mqtt_processor(State), - ok. +terminate(_Reason, _Request, + #state{conn_name = ConnName, + proc_state = PState, + keepalive = KState} = State) -> + rabbit_log_connection:info("closing Web MQTT connection ~p (~s)", [self(), ConnName]), + maybe_emit_stats(State), + rabbit_mqtt_keepalive:cancel_timer(KState), + ok = file_handle_cache:release(), + stop_rabbit_mqtt_processor(PState). %% Internal. @@ -255,28 +259,24 @@ handle_data1(Data, State = #state{ parse_state = ParseState, Other end. +stop_with_framing_error(State, Error0, ConnStr) -> + Error1 = rabbit_misc:format("~tp", [Error0]), + rabbit_log_connection:error("MQTT detected framing error '~ts' for connection ~tp", + [Error1, ConnStr]), + stop(State, 1007, Error1). + stop(State) -> stop(State, 1000, "MQTT died"). stop(State, CloseCode, Error0) -> - ok = file_handle_cache:release(), - _ = stop_rabbit_mqtt_processor(State), Error1 = rabbit_data_coercion:to_binary(Error0), {[{close, CloseCode, Error1}], State}. -stop_with_framing_error(State, Error0, ConnStr) -> - Error1 = rabbit_misc:format("~tp", [Error0]), - rabbit_log_connection:error("MQTT detected framing error '~ts' for connection ~tp", - [Error1, ConnStr]), - stop(State, 1007, Error1). - -stop_rabbit_mqtt_processor(State = #state{state = running, - proc_state = ProcState, - conn_name = ConnName}) -> - maybe_emit_stats(State), - rabbit_log_connection:info("closing Web MQTT connection ~tp (~ts)", [self(), ConnName]), - rabbit_mqtt_processor:send_will(ProcState), - rabbit_mqtt_processor:terminate(ProcState). +stop_rabbit_mqtt_processor(undefined) -> + ok; +stop_rabbit_mqtt_processor(PState) -> + rabbit_mqtt_processor:send_will(PState), + rabbit_mqtt_processor:terminate(PState). handle_credits(State0) -> case control_throttle(State0) of @@ -286,14 +286,16 @@ handle_credits(State0) -> {ok, State} end. -control_throttle(State = #state{ state = CS, - conserve_resources = Mem }) -> +control_throttle(State = #state{state = CS, + conserve_resources = Mem, + keepalive = KState}) -> case {CS, Mem orelse credit_flow:blocked()} of - %%TODO cancel / resume keepalive timer as done in rabbit_mqtt_reader {running, true} -> - State #state{state = blocked}; + State#state{state = blocked, + keepalive = rabbit_mqtt_keepalive:cancel_timer(KState)}; {blocked,false} -> - State #state{state = running}; + State#state{state = running, + keepalive = rabbit_mqtt_keepalive:start_timer(KState)}; {_, _} -> State end. diff --git a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl index f63ac60b674b..0bf59e9c3263 100644 --- a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl +++ b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl @@ -26,6 +26,7 @@ groups() -> , last_will_enabled , last_will_disabled , disconnect + , keepalive ]} ]. @@ -227,6 +228,33 @@ disconnect(Config) -> ok. +keepalive(Config) -> + Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt), + PortStr = integer_to_list(Port), + WS = rfc6455_client:new("ws://127.0.0.1:" ++ PortStr ++ "/ws", self()), + {ok, _} = rfc6455_client:open(WS), + + KeepaliveSecs = 1, + KeepaliveMs = timer:seconds(KeepaliveSecs), + ok = raw_send(WS, + ?CONNECT_PACKET( + #mqtt_packet_connect{ + keep_alive = KeepaliveSecs, + clean_sess = true, + client_id = <<"web-mqtt-tests-disconnect">>, + username = <<"guest">>, + password = <<"guest">>})), + {ok, ?CONNACK_PACKET(?CONNACK_ACCEPT), _} = raw_recv(WS), + + %% Sanity check that MQTT ping request and ping response work. + timer:sleep(KeepaliveMs), + ok = raw_send(WS, #mqtt_packet{header = #mqtt_packet_header{type = ?PINGREQ}}), + {ok, #mqtt_packet{header = #mqtt_packet_header{type = ?PINGRESP}}, <<>>} = raw_recv(WS), + + %% Stop sending any data to the server (including ping requests). + %% The server should disconnect us. + ?assertEqual({close, {1000, <<"MQTT keepalive timeout">>}}, + rfc6455_client:recv(WS, ceil(3 * 0.75 * KeepaliveMs))). raw_send(WS, Packet) -> Frame = emqttc_serialiser:serialise(Packet), From 7a325a3a99d1205671eaa385e85116f575fade6a Mon Sep 17 00:00:00 2001 From: Chunyi Lyu Date: Tue, 4 Oct 2022 15:46:50 +0100 Subject: [PATCH 015/118] Remove deprecated cowboy return types from web mqtt - change introduced in cowboy 2.5; see: https://github.com/ninenines/cowboy/commit/8404b1c908ac890925496ce839e5b2b2b407a6f7 --- .../src/rabbit_web_mqtt_handler.erl | 52 ++++++++++--------- 1 file changed, 28 insertions(+), 24 deletions(-) diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index 5a413fc802f3..4ceecbae06c8 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -64,6 +64,10 @@ takeover(Parent, Ref, Socket, Transport, Opts, Buffer, {Handler, HandlerState}) {Handler, HandlerState#state{socket = Sock}}). %% cowboy_websocket +-spec init(Req, any()) -> + {ok | module(), Req, any()} | + {module(), Req, any(), any()} + when Req::cowboy_req:req(). init(Req, Opts) -> {PeerAddr, _PeerPort} = maps:get(peer, Req), SockInfo = maps:get(proxy_header, Req, undefined), @@ -87,6 +91,9 @@ init(Req, Opts) -> received_connect_frame = false }, WsOpts}. +-spec websocket_init(State) -> + {cowboy_websocket:commands(), State} | + {cowboy_websocket:commands(), State, hibernate}. websocket_init(State0 = #state{socket = Sock, peername = PeerAddr}) -> ok = file_handle_cache:obtain(), case rabbit_net:connection_string(Sock, inbound) of @@ -102,13 +109,13 @@ websocket_init(State0 = #state{socket = Sock, peername = PeerAddr}) -> fun send_reply/2, PeerAddr), process_flag(trap_exit, true), - {ok, + {[], rabbit_event:init_stats_timer( State#state{proc_state = ProcessorState}, #state.stats_timer), hibernate}; - _ -> - {stop, State0} + {error, Reason} -> + {[{shutdown_reason, Reason}], State0} end. -spec close_connection(pid(), string()) -> 'ok'. @@ -118,15 +125,18 @@ close_connection(Pid, Reason) -> sys:terminate(Pid, Reason), ok. +-spec websocket_handle(ping | pong | {text | binary | ping | pong, binary()}, State) -> + {cowboy_websocket:commands(), State} | + {cowboy_websocket:commands(), State, hibernate}. websocket_handle({binary, Data}, State) -> handle_data(Data, State); %% Silently ignore ping and pong frames as Cowboy will automatically reply to ping frames. websocket_handle({Ping, _}, State) when Ping =:= ping orelse Ping =:= pong -> - {ok, State, hibernate}; + {[], State, hibernate}; websocket_handle(Ping, State) when Ping =:= ping orelse Ping =:= pong -> - {ok, State, hibernate}; + {[], State, hibernate}; %% Log any other unexpected frames. websocket_handle(Frame, State) -> rabbit_log_connection:info("Web MQTT: unexpected WebSocket frame ~tp", @@ -134,19 +144,11 @@ websocket_handle(Frame, State) -> %%TODO close connection instead? %%"MQTT Control Packets MUST be sent in WebSocket binary data frames. %% If any other type of data frame is received the recipient MUST close the Network Connection" - {ok, State, hibernate}. - -%% `rabbit_mqtt_processor:amqp_callback/2` doesn't actually return -%% {'error', _, _}, so this small function is a place to silence -%% unmatched warning. This allows to keep currently-unused -%% error-handling code. --spec callback_reply(#state{}, {'ok', any()} | {'error', any(), any()}) -> {'ok', #state{}, 'hibernate'}. --dialyzer({no_match, callback_reply/2}). -callback_reply(State, {ok, ProcState}) -> - {ok, State #state { proc_state = ProcState }, hibernate}; -callback_reply(State, {error, _Reason, _ProcState}) -> - stop(State). + {[], State, hibernate}. +-spec websocket_info(any(), State) -> + {cowboy_websocket:commands(), State} | + {cowboy_websocket:commands(), State, hibernate}. websocket_info({conserve_resources, Conserve}, State) -> NewState = State#state{conserve_resources = Conserve}, handle_credits(control_throttle(NewState)); @@ -155,14 +157,14 @@ websocket_info({bump_credit, Msg}, State) -> handle_credits(control_throttle(State)); %%TODO return hibernate? websocket_info({reply, Data}, State) -> - {reply, {binary, Data}, State, hibernate}; + {[{binary, Data}], State, hibernate}; websocket_info({'EXIT', _, _}, State) -> stop(State); websocket_info({'$gen_cast', QueueEvent = {queue_event, _, _}}, State = #state{proc_state = PState0}) -> case rabbit_mqtt_processor:handle_queue_event(QueueEvent, PState0) of {ok, PState} -> - {ok, State#state{proc_state = PState}, hibernate}; + {[], State#state{proc_state = PState}, hibernate}; {error, Reason, PState} -> rabbit_log_connection:error("Web MQTT connection ~p failed to handle queue event: ~p", [State#state.conn_name, Reason]), @@ -182,7 +184,7 @@ websocket_info({keepalive, Req}, State = #state{keepalive = KState0, conn_name = ConnName}) -> case rabbit_mqtt_keepalive:handle(Req, KState0) of {ok, KState} -> - {ok, State#state{keepalive = KState}, hibernate}; + {[], State#state{keepalive = KState}, hibernate}; {error, timeout} -> rabbit_log_connection:error("keepalive timeout in Web MQTT connection ~p", [ConnName]), @@ -193,15 +195,16 @@ websocket_info({keepalive, Req}, State = #state{keepalive = KState0, stop(State) end; websocket_info(emit_stats, State) -> - {ok, emit_stats(State), hibernate}; + {[], emit_stats(State), hibernate}; websocket_info({ra_event, _From, Evt}, #state{proc_state = PState0} = State) -> PState = rabbit_mqtt_processor:handle_ra_event(Evt, PState0), - {ok, State#state{proc_state = PState}, hibernate}; + {[], State#state{proc_state = PState}, hibernate}; websocket_info(Msg, State) -> rabbit_log_connection:warning("Web MQTT: unexpected message ~p", [Msg]), - {ok, State, hibernate}. + {[], State, hibernate}. +-spec terminate(any(), cowboy_req:req(), any()) -> ok. terminate(_Reason, _Request, #state{conn_name = ConnName, proc_state = PState, @@ -283,7 +286,7 @@ handle_credits(State0) -> State = #state{state = running} -> {[{active, true}], State}; State -> - {ok, State} + {[], State} end. control_throttle(State = #state{state = CS, @@ -306,6 +309,7 @@ send_reply(Frame, PState) -> ensure_stats_timer(State) -> rabbit_event:ensure_stats_timer(State, #state.stats_timer, emit_stats). +%% TODO if #state.stats_timer is undefined, rabbit_event:if_enabled crashes maybe_emit_stats(State) -> rabbit_event:if_enabled(State, #state.stats_timer, fun() -> emit_stats(State) end). From 43bd548dccc70141dcb12679edb0e2a352acab2a Mon Sep 17 00:00:00 2001 From: David Ansari Date: Mon, 10 Oct 2022 16:10:02 +0000 Subject: [PATCH 016/118] Handle deprecated classic queue delivery when feature flag classic_queue_type_delivery_support is disabled. --- deps/rabbit/src/rabbit_classic_queue.erl | 2 +- deps/rabbit/src/rabbit_core_ff.erl | 3 ++- deps/rabbitmq_mqtt/src/rabbit_mqtt.erl | 3 --- deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl | 8 +++++++- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 3 +++ deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl | 4 ++++ 6 files changed, 17 insertions(+), 6 deletions(-) diff --git a/deps/rabbit/src/rabbit_classic_queue.erl b/deps/rabbit/src/rabbit_classic_queue.erl index e349b5e57254..b7d3d1f8b0a6 100644 --- a/deps/rabbit/src/rabbit_classic_queue.erl +++ b/deps/rabbit/src/rabbit_classic_queue.erl @@ -545,7 +545,7 @@ send_rejection(Pid, QName, MsgSeqNo) -> end. deliver_to_consumer(Pid, QName, CTag, AckRequired, Message) -> - case has_classic_queue_type_delivery_support() of + case has_classic_queue_type_delivery_support() of true -> Deliver = {deliver, CTag, AckRequired, [Message]}, Evt = {queue_event, QName, Deliver}, diff --git a/deps/rabbit/src/rabbit_core_ff.erl b/deps/rabbit/src/rabbit_core_ff.erl index 1ec4d5f06531..651da0925089 100644 --- a/deps/rabbit/src/rabbit_core_ff.erl +++ b/deps/rabbit/src/rabbit_core_ff.erl @@ -34,7 +34,8 @@ {stream_queue, #{desc => "Support queues of type `stream`", doc_url => "https://www.rabbitmq.com/stream.html", - stability => stable, + %%TODO remove compatibility code + stability => required, depends_on => [quorum_queue] }}). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl index c7f5877b7ec2..a8c5e3ab4dfc 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl @@ -13,9 +13,6 @@ close_local_client_connections/1]). start(normal, []) -> - %%TODO make feature flag stream_queue 'required' for 3.12 - %% because we rely on rabbit_queue_type interface. - ok = rabbit_feature_flags:enable(stream_queue), rabbit_global_counters:init([{protocol, mqtt}]), {ok, Listeners} = application:get_env(tcp_listeners), {ok, SslListeners} = application:get_env(ssl_listeners), diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 05d1d002530a..cfaa8cfa7f9e 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -11,7 +11,8 @@ -export([info/2, initial_state/2, initial_state/4, process_frame/2, serialise/2, send_will/1, terminate/1, handle_pre_hibernate/0, - handle_ra_event/2, handle_down/2, handle_queue_event/2]). + handle_ra_event/2, handle_down/2, handle_queue_event/2, + handle_deprecated_delivery/2]). %%TODO Use single queue per MQTT subscriber connection? %% * when publishing we store in x-mqtt-publish-qos header the publishing QoS @@ -1197,6 +1198,11 @@ handle_down({'DOWN', _MRef, process, QPid, Reason}, PState0#proc_state{queue_states = QStates} end. +%% Handle deprecated delivery from classic queue. This function is to be +%% removed when feature flag classic_queue_type_delivery_support becomes required. +handle_deprecated_delivery({deliver, ?CONSUMER_TAG, AckRequired, Msg}, PState) -> + {ok, deliver_one_to_client(Msg, AckRequired, PState)}. + handle_queue_event({queue_event, QName, Evt}, PState0 = #proc_state{queue_states = QStates0, unacked_client_pubs = U0}) -> diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index 29cf66ecf5ed..549bd3eaf5b1 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -127,6 +127,9 @@ handle_cast(QueueEvent = {queue_event, _, _}, State = #state{proc_state = PState}) -> callback_reply(State, rabbit_mqtt_processor:handle_queue_event(QueueEvent, PState)); +handle_cast(Delivery = {deliver, _, _, _}, State = #state{proc_state = PState}) -> + callback_reply(State, rabbit_mqtt_processor:handle_deprecated_delivery(Delivery, PState)); + handle_cast(Msg, State) -> {stop, {mqtt_unexpected_cast, Msg}, State}. diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index 4ceecbae06c8..ac523fbd8cd1 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -170,6 +170,10 @@ websocket_info({'$gen_cast', QueueEvent = {queue_event, _, _}}, [State#state.conn_name, Reason]), stop(State#state{proc_state = PState}) end; +websocket_info({'$gen_cast', Delivery = {deliver, _, _, _}}, + State = #state{proc_state = PState0}) -> + {ok, PState} = rabbit_mqtt_processor:handle_deprecated_delivery(Delivery, PState0), + {[], State#state{proc_state = PState}, hibernate}; websocket_info({'$gen_cast', duplicate_id}, State = #state{ proc_state = ProcState, conn_name = ConnName }) -> rabbit_log_connection:warning("Web MQTT disconnecting a client with duplicate ID '~s' (~p)", From ab8957ba9c51461c560695885e3ea142436a96af Mon Sep 17 00:00:00 2001 From: David Ansari Date: Mon, 10 Oct 2022 10:40:20 +0200 Subject: [PATCH 017/118] Use best-effort client ID tracking "Each Client connecting to the Server has a unique ClientId" "If the ClientId represents a Client already connected to the Server then the Server MUST disconnect the existing Client [MQTT-3.1.4-2]." Instead of tracking client IDs via Raft, we use local ETS tables in this commit. Previous tracking of client IDs via Raft: (+) consistency (does the right thing) (-) state of Ra process becomes large > 1GB with many (> 1 Million) MQTT clients (-) Ra process becomes a bottleneck when many MQTT clients (e.g. 300k) disconnect at the same time because monitor (DOWN) Ra commands get written resulting in Ra machine timeout. (-) if we need consistency, we ideally want a single source of truth, e.g. only Mnesia, or only Khepri (but not Mnesia + MQTT ra process) While above downsides could be fixed (e.g. avoiding DOWN commands by instead doing periodic cleanups of client ID entries using session interval in MQTT 5 or using subscription_ttl parameter in current RabbitMQ MQTT config), in this case we do not necessarily need the consistency guarantees Raft provides. In this commit, we try to comply with [MQTT-3.1.4-2] on a best-effort basis: If there are no network failures and no messages get lost, existing clients with duplicate client IDs get disconnected. In the presence of network failures / lost messages, two clients with the same client ID can end up publishing or receiving from the same queue. Arguably, that's acceptable and less worse than the scaling issues we experience when we want stronger consistency. Note that it is also the responsibility of the client to not connect twice with the same client ID. This commit also ensures that the client ID is a binary to save memory. A new feature flag is introduced, which when enabled, deletes the Ra cluster named 'mqtt_node'. Independent of that feature flag, client IDs are tracked locally in ETS tables. If that feature flag is disabled, client IDs are additionally tracked in Ra. The feature flag is required such that clients can continue to connect to all nodes except for the node being udpated in a rolling update. This commit also fixes a bug where previously all MQTT connections were cluster-wide closed when one RabbitMQ node was put into maintenance mode. --- .../test/direct_exchange_routing_v2_SUITE.erl | 6 + deps/rabbit_common/include/rabbit.hrl | 6 +- deps/rabbitmq_mqtt/BUILD.bazel | 7 + deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl | 2 +- deps/rabbitmq_mqtt/src/mqtt_node.erl | 35 ++++- deps/rabbitmq_mqtt/src/rabbit_mqtt.erl | 26 +++- .../src/rabbit_mqtt_clientid.erl | 123 ++++++++++++++++++ deps/rabbitmq_mqtt/src/rabbit_mqtt_ff.erl | 20 +++ deps/rabbitmq_mqtt/src/rabbit_mqtt_frame.erl | 5 +- .../src/rabbit_mqtt_processor.erl | 96 +++++++++----- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 59 +++++---- deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl | 53 +++++--- deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl | 9 +- deps/rabbitmq_mqtt/test/cluster_SUITE.erl | 54 ++++---- deps/rabbitmq_mqtt/test/ff_SUITE.erl | 100 ++++++++++++++ deps/rabbitmq_mqtt/test/reader_SUITE.erl | 22 +++- 16 files changed, 493 insertions(+), 130 deletions(-) create mode 100644 deps/rabbitmq_mqtt/src/rabbit_mqtt_clientid.erl create mode 100644 deps/rabbitmq_mqtt/src/rabbit_mqtt_ff.erl create mode 100644 deps/rabbitmq_mqtt/test/ff_SUITE.erl diff --git a/deps/rabbit/test/direct_exchange_routing_v2_SUITE.erl b/deps/rabbit/test/direct_exchange_routing_v2_SUITE.erl index 6593c2d6d9e2..3065c5394468 100644 --- a/deps/rabbit/test/direct_exchange_routing_v2_SUITE.erl +++ b/deps/rabbit/test/direct_exchange_routing_v2_SUITE.erl @@ -1,3 +1,9 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2016-2022 VMware, Inc. or its affiliates. All rights reserved. + -module(direct_exchange_routing_v2_SUITE). %% Test suite for the feature flag direct_exchange_routing_v2 diff --git a/deps/rabbit_common/include/rabbit.hrl b/deps/rabbit_common/include/rabbit.hrl index e4976267a2b2..444e807fbf35 100644 --- a/deps/rabbit_common/include/rabbit.hrl +++ b/deps/rabbit_common/include/rabbit.hrl @@ -226,11 +226,11 @@ -define(SUPERVISOR_WAIT, rabbit_misc:get_env(rabbit, supervisor_shutdown_timeout, infinity)). -define(WORKER_WAIT, - rabbit_misc:get_env(rabbit, worker_shutdown_timeout, 300000)). + rabbit_misc:get_env(rabbit, worker_shutdown_timeout, 300_000)). -define(MSG_STORE_WORKER_WAIT, - rabbit_misc:get_env(rabbit, msg_store_shutdown_timeout, 600000)). + rabbit_misc:get_env(rabbit, msg_store_shutdown_timeout, 600_000)). -define(CLASSIC_QUEUE_WORKER_WAIT, - rabbit_misc:get_env(rabbit, classic_queue_shutdown_timeout, 600000)). + rabbit_misc:get_env(rabbit, classic_queue_shutdown_timeout, 600_000)). -define(HIBERNATE_AFTER_MIN, 1000). -define(DESIRED_HIBERNATE, 10000). diff --git a/deps/rabbitmq_mqtt/BUILD.bazel b/deps/rabbitmq_mqtt/BUILD.bazel index 07875b615afe..66378e0efb1d 100644 --- a/deps/rabbitmq_mqtt/BUILD.bazel +++ b/deps/rabbitmq_mqtt/BUILD.bazel @@ -159,6 +159,13 @@ rabbitmq_integration_suite( ], ) +rabbitmq_integration_suite( + name = "ff_SUITE", + runtime_deps = [ + "@emqtt//:erlang_app", + ], +) + rabbitmq_suite( name = "util_SUITE", size = "small", diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl index 4743780f910b..df76dbec0584 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl @@ -38,7 +38,7 @@ %% (Not to be confused with packet IDs sent from client to server which can be the %% same IDs because client and server assign IDs independently of each other.) packet_id = 1 :: packet_id(), - client_id, + client_id :: binary(), clean_sess :: boolean(), will_msg, exchange :: rabbit_exchange:name(), diff --git a/deps/rabbitmq_mqtt/src/mqtt_node.erl b/deps/rabbitmq_mqtt/src/mqtt_node.erl index ad5f17ea71aa..a9c253e4a923 100644 --- a/deps/rabbitmq_mqtt/src/mqtt_node.erl +++ b/deps/rabbitmq_mqtt/src/mqtt_node.erl @@ -6,7 +6,8 @@ %% -module(mqtt_node). --export([start/0, node_id/0, server_id/0, all_node_ids/0, leave/1, trigger_election/0]). +-export([start/0, node_id/0, server_id/0, all_node_ids/0, leave/1, trigger_election/0, + delete/1]). -define(ID_NAME, mqtt_node). -define(START_TIMEOUT, 100_000). @@ -131,3 +132,35 @@ can_participate_in_clientid_tracking(Node) -> {badrpc, _} -> false; _ -> true end. + +-spec delete(Args) -> Ret when + Args :: rabbit_feature_flags:enable_callback_args(), + Ret :: rabbit_feature_flags:enable_callback_ret(). +delete(_) -> + RaNodes = all_node_ids(), + Nodes = lists:map(fun({_, N}) -> N end, RaNodes), + LockId = {?ID_NAME, node_id()}, + rabbit_log:info("Trying to acquire lock ~p on nodes ~p ...", [LockId, Nodes]), + true = global:set_lock(LockId, Nodes), + rabbit_log:info("Acquired lock ~p", [LockId]), + try whereis(?ID_NAME) of + undefined -> + rabbit_log:info("Local Ra process ~s does not exist", [?ID_NAME]), + ok; + _ -> + rabbit_log:info("Deleting Ra cluster ~s ...", [?ID_NAME]), + try ra:delete_cluster(RaNodes, ?RA_OPERATION_TIMEOUT) of + {ok, _Leader} -> + rabbit_log:info("Successfully deleted Ra cluster ~s", [?ID_NAME]), + ok; + {error, _} = Err -> + rabbit_log:info("Failed to delete Ra cluster ~s: ~p", [?ID_NAME, Err]), + Err + catch exit:{{shutdown, delete}, _Stacktrace} -> + rabbit_log:info("Ra cluster ~s already being deleted", [?ID_NAME]), + ok + end + after + true = global:del_lock(LockId, Nodes), + rabbit_log:info("Released lock ~p", [LockId]) + end. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl index a8c5e3ab4dfc..1baf1a52c5ab 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl @@ -7,6 +7,8 @@ -module(rabbit_mqtt). +-include("rabbit_mqtt.hrl"). + -behaviour(application). -export([start/2, stop/1]). -export([emit_connection_info_all/4, @@ -16,7 +18,12 @@ start(normal, []) -> rabbit_global_counters:init([{protocol, mqtt}]), {ok, Listeners} = application:get_env(tcp_listeners), {ok, SslListeners} = application:get_env(ssl_listeners), - ok = mqtt_node:start(), + case rabbit_mqtt_ff:track_client_id_in_ra() of + true -> + ok = mqtt_node:start(); + false -> + ok + end, Result = rabbit_mqtt_sup:start_link({Listeners, SslListeners}, []), EMPid = case rabbit_event:start_link() of {ok, Pid} -> Pid; @@ -40,7 +47,20 @@ emit_connection_info_all(_Nodes, Items, Ref, AggregatorPid) -> -spec close_local_client_connections(string() | binary()) -> {'ok', non_neg_integer()}. close_local_client_connections(Reason) -> - AllPids = rabbit_mqtt_collector:list_pids(), - LocalPids = lists:filter(fun(Pid) -> node(Pid) =:= node() end, AllPids), + LocalPids = local_connection_pids(), [rabbit_mqtt_reader:close_connection(Pid, Reason) || Pid <- LocalPids], {ok, length(LocalPids)}. + +-spec local_connection_pids() -> [pid()]. +local_connection_pids() -> + case rabbit_mqtt_ff:track_client_id_in_ra() of + true -> + AllPids = rabbit_mqtt_collector:list_pids(), + LocalPids = lists:filter(fun(Pid) -> node(Pid) =:= node() end, AllPids), + LocalPids; + false -> + PgScope = persistent_term:get(?PG_SCOPE), + lists:flatmap(fun(Group) -> + pg:get_local_members(PgScope, Group) + end, pg:which_groups(PgScope)) + end. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_clientid.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_clientid.erl new file mode 100644 index 000000000000..ea75724b214f --- /dev/null +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_clientid.erl @@ -0,0 +1,123 @@ +-module(rabbit_mqtt_clientid). + +-behaviour(gen_server). + +%% client API +-export([ + start_link/1, + register/2, + unregister/2 + ]). + +%% debugging +-export([ + list_local/0, + list_all/0, + lookup_local/2 + ]). + +%% gen_server callbacks +-export([ + init/1, + handle_cast/2, + handle_call/3, + handle_info/2, + terminate/2 + ]). + +%% ------------------------------------------------------------------- +%% client API +%% ------------------------------------------------------------------- + +start_link(ServerName) -> + gen_server:start_link(ServerName, ?MODULE, #{}, []). + +register(Vhost, ClientId) + when is_binary(Vhost), is_binary(ClientId) -> + gen_server:cast(?MODULE, {register, {Vhost, ClientId}, self()}). + +unregister(Vhost, ClientId) + when is_binary(Vhost), is_binary(ClientId) -> + gen_server:cast(?MODULE, {unregister, {Vhost, ClientId}, self()}). + +%% ------------------------------------------------------------------- +%% debugging +%% ------------------------------------------------------------------- + +list_local() -> + gen_server:call(?MODULE, list, 30_000). + +list_all() -> + {Replies, _BadNodes} = gen_server:multi_call([node() | nodes()], ?MODULE, list, 30_000), + lists:flatmap(fun({_Node, Reply}) -> + Reply + end, Replies). + +lookup_local(Vhost, ClientId) + when is_binary(Vhost), is_binary(ClientId) -> + gen_server:call(?MODULE, {lookup, {Vhost, ClientId}}, 30_000). + +%% ------------------------------------------------------------------- +%% gen_server callbacks +%% ------------------------------------------------------------------- + +init(#{}) -> + Tid = ets:new(?MODULE, [private]), + {ok, Tid}. + +handle_cast({register, Key, Pid}, Tid) -> + gen_server:abcast(nodes(), ?MODULE, {maybe_remove_duplicate, Key}), + %% optimize for no duplicate ClientId + case ets:insert_new(Tid, {Key, Pid}) of + true -> + ok; + false -> + remove_duplicate(Tid, Key), + true = ets:insert(Tid, {Key, Pid}) + end, + {noreply, Tid}; + +handle_cast({maybe_remove_duplicate, Key}, Tid) -> + %% optimize for no duplicate ClientId + case ets:member(Tid, Key) of + false -> + ok; + true -> + remove_duplicate(Tid, Key) + end, + {noreply, Tid}; + +handle_cast({unregister, Key, Pid}, Tid) -> + true = ets:delete_object(Tid, {Key, Pid}), + {noreply, Tid}; + +handle_cast(Request, Tid) -> + rabbit_log:warning("~s received unkown request ~p", [?MODULE, Request]), + {noreply, Tid}. + +handle_call(list, _From, Tid) -> + {reply, ets:tab2list(Tid), Tid}; + +handle_call({lookup, Key}, _From, Tid) -> + {reply, ets:lookup(Tid, Key), Tid}; + +handle_call(Request, From, Tid) -> + rabbit_log:warning("~s received unkown request ~p from ~p", + [?MODULE, Request, From]), + {noreply, Tid}. + +handle_info(Info, Tid) -> + rabbit_log:warning("~s received unkown info ~p", [?MODULE, Info]), + {noreply, Tid}. + +terminate(Reason, Tid) -> + rabbit_log:info("~s terminates with ~b entries and reason ~p", + [?MODULE, ets:info(Tid, size), Reason]). + +%% ------------------------------------------------------------------- +%% internal +%% ------------------------------------------------------------------- + +remove_duplicate(Tid, Key) -> + [{Key, OldPid}] = ets:take(Tid, Key), + gen_server:cast(OldPid, duplicate_id). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_ff.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_ff.erl new file mode 100644 index 000000000000..f8a10782fe82 --- /dev/null +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_ff.erl @@ -0,0 +1,20 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2018-2022 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(rabbit_mqtt_ff). + +-export([track_client_id_in_ra/0]). + +-rabbit_feature_flag( + {delete_ra_cluster_mqtt_node, + #{desc => "Delete Ra cluster 'mqtt_node' because from now on MQTT client IDs are tracked locally", + stability => stable, + callbacks => #{enable => {mqtt_node, delete}} + }}). + +track_client_id_in_ra() -> + not rabbit_feature_flags:is_enabled(delete_ra_cluster_mqtt_node). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_frame.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_frame.erl index 62a96cb11c60..fa8724afdb76 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_frame.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_frame.erl @@ -58,7 +58,7 @@ parse_frame(Bin, #mqtt_frame_fixed{ type = Type, _Reserved : 1, KeepAlive : 16/big, Rest3/binary>> = Rest2, - {ClientId, Rest4} = parse_utf(Rest3), + {ClientId, Rest4} = parse_utf_as_binary(Rest3), {WillTopic, Rest5} = parse_utf(Rest4, WillFlag), {WillMsg, Rest6} = parse_msg(Rest5, WillFlag), {UserName, Rest7} = parse_utf(Rest6, UsernameFlag), @@ -136,6 +136,9 @@ parse_utf(Bin, _) -> parse_utf(<>) -> {binary_to_list(Str), Rest}. +parse_utf_as_binary(<>) -> + {Str, Rest}. + parse_msg(Bin, 0) -> {undefined, Bin}; parse_msg(<>, _) -> diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index cfaa8cfa7f9e..1d417d1ee158 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -276,10 +276,10 @@ process_connect(#mqtt_frame{ SendFun(ResponseFrame, PState), return_connack(ReturnCode, PState). -client_id([]) -> +client_id(<<>>) -> rabbit_mqtt_util:gen_client_id(); client_id(ClientId) - when is_list(ClientId) -> + when is_binary(ClientId) -> ClientId. check_protocol_version(#mqtt_frame_connect{proto_ver = ProtoVersion}) -> @@ -339,36 +339,46 @@ register_client(Frame = #mqtt_frame_connect{proto_ver = ProtoVersion}, PState = #proc_state{client_id = ClientId, socket = Socket, auth_state = #auth_state{vhost = VHost}}) -> - case rabbit_mqtt_collector:register(ClientId, self()) of - {ok, Corr} -> + NewProcState = + fun(RegisterState) -> + rabbit_mqtt_clientid:register(VHost, ClientId), RetainerPid = rabbit_mqtt_retainer_sup:child_for_vhost(VHost), Prefetch = rabbit_mqtt_util:env(prefetch), {ok, {PeerHost, PeerPort, Host, Port}} = rabbit_net:socket_ends(Socket, inbound), ExchangeBin = rabbit_mqtt_util:env(exchange), ExchangeName = rabbit_misc:r(VHost, exchange, ExchangeBin), ProtoHumanReadable = {'MQTT', human_readable_mqtt_version(ProtoVersion)}, - {ok, PState#proc_state{ - exchange = ExchangeName, - will_msg = make_will_msg(Frame), - retainer_pid = RetainerPid, - register_state = {pending, Corr}, - proto_ver = ProtoVersion, - info = #info{prefetch = Prefetch, - peer_host = PeerHost, - peer_port = PeerPort, - host = Host, - port = Port, - proto_human = ProtoHumanReadable - }}}; - {error, _} = Err -> - %% e.g. this node was removed from the MQTT cluster members - rabbit_log_connection:error("MQTT cannot accept a connection: " - "client ID tracker is unavailable: ~p", [Err]), - {error, ?CONNACK_SERVER_UNAVAILABLE}; - {timeout, _} -> - rabbit_log_connection:error("MQTT cannot accept a connection: " - "client ID registration timed out"), - {error, ?CONNACK_SERVER_UNAVAILABLE} + PState#proc_state{ + exchange = ExchangeName, + will_msg = make_will_msg(Frame), + retainer_pid = RetainerPid, + register_state = RegisterState, + proto_ver = ProtoVersion, + info = #info{prefetch = Prefetch, + peer_host = PeerHost, + peer_port = PeerPort, + host = Host, + port = Port, + proto_human = ProtoHumanReadable + }} + end, + case rabbit_mqtt_ff:track_client_id_in_ra() of + true -> + case rabbit_mqtt_collector:register(ClientId, self()) of + {ok, Corr} -> + {ok, NewProcState({pending, Corr})}; + {error, _} = Err -> + %% e.g. this node was removed from the MQTT cluster members + rabbit_log_connection:error("MQTT cannot accept a connection: " + "client ID tracker is unavailable: ~p", [Err]), + {error, ?CONNACK_SERVER_UNAVAILABLE}; + {timeout, _} -> + rabbit_log_connection:error("MQTT cannot accept a connection: " + "client ID registration timed out"), + {error, ?CONNACK_SERVER_UNAVAILABLE} + end; + false -> + {ok, NewProcState(undefined)} end. notify_connection_created(already_connected, _PState) -> @@ -551,7 +561,7 @@ process_login(_UserBin, _PassBin, _ClientId, rabbit_log_connection:warning("MQTT detected duplicate connect/login attempt for user ~tp, vhost ~tp", [UsernameStr, VHostStr]), already_connected; -process_login(UserBin, PassBin, ClientId0, +process_login(UserBin, PassBin, ClientId, #proc_state{socket = Sock, ssl_login_name = SslLoginName, peer_addr = Addr, @@ -562,7 +572,6 @@ process_login(UserBin, PassBin, ClientId0, "MQTT vhost picked using ~s", [human_readable_vhost_lookup_strategy(VHostPickedUsing)]), RemoteIpAddressBin = list_to_binary(inet:ntoa(Addr)), - ClientId = rabbit_data_coercion:to_binary(ClientId0), Input = #{vhost => VHost, username_bin => UsernameBin, pass_bin => PassBin, @@ -1155,9 +1164,16 @@ serialise(Frame, #proc_state{proto_ver = ProtoVer}) -> terminate(#proc_state{client_id = undefined}) -> ok; -terminate(#proc_state{client_id = ClientId}) -> +terminate(#proc_state{client_id = ClientId, + auth_state = #auth_state{vhost = VHost}}) -> %% ignore any errors as we are shutting down - rabbit_mqtt_collector:unregister(ClientId, self()). + case rabbit_mqtt_ff:track_client_id_in_ra() of + true -> + rabbit_mqtt_collector:unregister(ClientId, self()); + false -> + ok + end, + rabbit_mqtt_clientid:unregister(VHost, ClientId). handle_pre_hibernate() -> erase(permission_cache), @@ -1171,14 +1187,24 @@ handle_ra_event({applied, [{Corr, ok}]}, handle_ra_event({not_leader, Leader, Corr}, PState = #proc_state{register_state = {pending, Corr}, client_id = ClientId}) -> - %% retry command against actual leader - {ok, NewCorr} = rabbit_mqtt_collector:register(Leader, ClientId, self()), - PState#proc_state{register_state = {pending, NewCorr}}; + case rabbit_mqtt_ff:track_client_id_in_ra() of + true -> + %% retry command against actual leader + {ok, NewCorr} = rabbit_mqtt_collector:register(Leader, ClientId, self()), + PState#proc_state{register_state = {pending, NewCorr}}; + false -> + PState + end; handle_ra_event(register_timeout, PState = #proc_state{register_state = {pending, _Corr}, client_id = ClientId}) -> - {ok, NewCorr} = rabbit_mqtt_collector:register(ClientId, self()), - PState#proc_state{register_state = {pending, NewCorr}}; + case rabbit_mqtt_ff:track_client_id_in_ra() of + true -> + {ok, NewCorr} = rabbit_mqtt_collector:register(ClientId, self()), + PState#proc_state{register_state = {pending, NewCorr}}; + false -> + PState + end; handle_ra_event(register_timeout, PState) -> PState; handle_ra_event(Evt, PState) -> diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index 549bd3eaf5b1..16cdfed50f0b 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -210,54 +210,53 @@ handle_info({'DOWN', _MRef, process, _Pid, _Reason} = Evt, handle_info(Msg, State) -> {stop, {mqtt_unexpected_msg, Msg}, State}. -terminate(Reason, State = #state{keepalive = KState0}) -> +terminate(Reason, State = #state{keepalive = KState0, + proc_state = PState}) -> KState = rabbit_mqtt_keepalive:cancel_timer(KState0), maybe_emit_stats(State#state{keepalive = KState}), - do_terminate(Reason, State). + rabbit_mqtt_processor:terminate(PState), + log_terminate(Reason, State). handle_pre_hibernate(State) -> rabbit_mqtt_processor:handle_pre_hibernate(), {hibernate, State}. -do_terminate({network_error, {ssl_upgrade_error, closed}, ConnStr}, _State) -> - rabbit_log_connection:error("MQTT detected TLS upgrade error on ~ts: connection closed", - [ConnStr]); +log_terminate({network_error, {ssl_upgrade_error, closed}, ConnStr}, _State) -> + rabbit_log_connection:error("MQTT detected TLS upgrade error on ~s: connection closed", + [ConnStr]); -do_terminate({network_error, - {ssl_upgrade_error, - {tls_alert, "handshake failure"}}, ConnStr}, _State) -> +log_terminate({network_error, + {ssl_upgrade_error, + {tls_alert, "handshake failure"}}, ConnStr}, _State) -> log_tls_alert(handshake_failure, ConnStr); -do_terminate({network_error, - {ssl_upgrade_error, - {tls_alert, "unknown ca"}}, ConnStr}, _State) -> +log_terminate({network_error, + {ssl_upgrade_error, + {tls_alert, "unknown ca"}}, ConnStr}, _State) -> log_tls_alert(unknown_ca, ConnStr); -do_terminate({network_error, - {ssl_upgrade_error, - {tls_alert, {Err, _}}}, ConnStr}, _State) -> +log_terminate({network_error, + {ssl_upgrade_error, + {tls_alert, {Err, _}}}, ConnStr}, _State) -> log_tls_alert(Err, ConnStr); -do_terminate({network_error, - {ssl_upgrade_error, - {tls_alert, Alert}}, ConnStr}, _State) -> +log_terminate({network_error, + {ssl_upgrade_error, + {tls_alert, Alert}}, ConnStr}, _State) -> log_tls_alert(Alert, ConnStr); -do_terminate({network_error, {ssl_upgrade_error, Reason}, ConnStr}, _State) -> - rabbit_log_connection:error("MQTT detected TLS upgrade error on ~ts: ~tp", - [ConnStr, Reason]); +log_terminate({network_error, {ssl_upgrade_error, Reason}, ConnStr}, _State) -> + rabbit_log_connection:error("MQTT detected TLS upgrade error on ~s: ~p", + [ConnStr, Reason]); -do_terminate({network_error, Reason, ConnStr}, _State) -> - rabbit_log_connection:error("MQTT detected network error on ~ts: ~tp", - [ConnStr, Reason]); +log_terminate({network_error, Reason, ConnStr}, _State) -> + rabbit_log_connection:error("MQTT detected network error on ~s: ~p", + [ConnStr, Reason]); -do_terminate({network_error, Reason}, _State) -> - rabbit_log_connection:error("MQTT detected network error: ~tp", [Reason]); +log_terminate({network_error, Reason}, _State) -> + rabbit_log_connection:error("MQTT detected network error: ~p", [Reason]); -do_terminate(normal, #state{proc_state = ProcState, - conn_name = ConnName}) -> - rabbit_mqtt_processor:terminate(ProcState), +log_terminate(normal, #state{conn_name = ConnName}) -> rabbit_log_connection:info("closing MQTT connection ~p (~s)", [self(), ConnName]), ok; -do_terminate(_Reason, #state{proc_state = ProcState}) -> - rabbit_mqtt_processor:terminate(ProcState), +log_terminate(_Reason, _State) -> ok. code_change(_OldVsn, State, _Extra) -> diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl index 807d2f1dd9ad..9f682d1a8a0c 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl @@ -33,27 +33,38 @@ init([{Listeners, SslListeners0}]) -> end} end, {ok, - {#{strategy => one_for_all, intensity => 10, period => 10}, [ - #{ - id => rabbit_mqtt_retainer_sup, - start => - {rabbit_mqtt_retainer_sup, start_link, [{local, rabbit_mqtt_retainer_sup}]}, - restart => transient, - shutdown => ?SUPERVISOR_WAIT, - type => supervisor, - modules => [rabbit_mqtt_retainer_sup] - } - | listener_specs( - fun tcp_listener_spec/1, - [SocketOpts, NumTcpAcceptors, ConcurrentConnsSups], - Listeners - ) ++ - listener_specs( - fun ssl_listener_spec/1, - [SocketOpts, SslOpts, NumSslAcceptors, ConcurrentConnsSups], - SslListeners - ) - ]}}. + {#{strategy => one_for_all, + intensity => 10, + period => 10}, + [ + #{id => rabbit_mqtt_clientid, + start => {rabbit_mqtt_clientid, start_link, + [{local, rabbit_mqtt_clientid}]}, + restart => transient, + shutdown => ?WORKER_WAIT, + type => worker, + modules => [rabbit_mqtt_clientid] + }, + #{ + id => rabbit_mqtt_retainer_sup, + start => {rabbit_mqtt_retainer_sup, start_link, + [{local, rabbit_mqtt_retainer_sup}]}, + restart => transient, + shutdown => ?SUPERVISOR_WAIT, + type => supervisor, + modules => [rabbit_mqtt_retainer_sup] + } + | listener_specs( + fun tcp_listener_spec/1, + [SocketOpts, NumTcpAcceptors, ConcurrentConnsSups], + Listeners + ) ++ + listener_specs( + fun ssl_listener_spec/1, + [SocketOpts, SslOpts, NumSslAcceptors, ConcurrentConnsSups], + SslListeners + ) + ]}}. stop_listeners() -> rabbit_networking:stop_ranch_listener_of_protocol(?TCP_PROTOCOL), diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl index 6ad9ed36b356..552d9574446a 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl @@ -21,9 +21,11 @@ -define(MAX_TOPIC_TRANSLATION_CACHE_SIZE, 12). +-spec queue_names(binary()) -> + {binary(), binary()}. queue_names(ClientId) -> - Base = "mqtt-subscription-" ++ ClientId ++ "qos", - {list_to_binary(Base ++ "0"), list_to_binary(Base ++ "1")}. + Base = <<"mqtt-subscription-", ClientId/binary, "qos">>, + {<>, <>}. cached(CacheName, Fun, Arg) -> Cache = @@ -100,8 +102,9 @@ get_topic_translation_funs() -> end, {ok, {mqtt2amqp_fun, M2AFun}, {amqp2mqtt_fun, A2MFun}}. +-spec gen_client_id() -> binary(). gen_client_id() -> - lists:nthtail(1, rabbit_guid:string(rabbit_guid:gen_secure(), [])). + rabbit_misc:base64url(rabbit_guid:gen_secure()). env(Key) -> case application:get_env(rabbitmq_mqtt, Key) of diff --git a/deps/rabbitmq_mqtt/test/cluster_SUITE.erl b/deps/rabbitmq_mqtt/test/cluster_SUITE.erl index 510557285b29..a5505d53549e 100644 --- a/deps/rabbitmq_mqtt/test/cluster_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/cluster_SUITE.erl @@ -142,20 +142,19 @@ connection_id_tracking(Config) -> expect_publishes(<<"TopicA">>, [<<"Payload">>]), %% there's one connection - assert_connection_count(Config, 10, 2, 1), + assert_connection_count(Config, 4, 2, 1), %% connect to the same node (A or 0) {ok, MRef2, _C2} = connect_to_node(Config, 0, ID), - %% C1 is disconnected await_disconnection(MRef1), + assert_connection_count(Config, 4, 2, 1), %% connect to a different node (C or 2) {ok, _, C3} = connect_to_node(Config, 2, ID), - assert_connection_count(Config, 10, 2, 1), - %% C2 is disconnected await_disconnection(MRef2), + assert_connection_count(Config, 4, 2, 1), ok = emqtt:disconnect(C3). connection_id_tracking_on_nodedown(Config) -> @@ -164,43 +163,46 @@ connection_id_tracking_on_nodedown(Config) -> {ok, _, _} = emqtt:subscribe(C, <<"TopicA">>, qos0), ok = emqtt:publish(C, <<"TopicA">>, <<"Payload">>), expect_publishes(<<"TopicA">>, [<<"Payload">>]), - assert_connection_count(Config, 10, 2, 1), + assert_connection_count(Config, 4, 2, 1), ok = stop_node(Config, Server), await_disconnection(MRef), - assert_connection_count(Config, 10, 2, 0), + assert_connection_count(Config, 4, 2, 0), ok. connection_id_tracking_with_decommissioned_node(Config) -> - Server = get_node_config(Config, 0, nodename), - {ok, MRef, C} = connect_to_node(Config, 0, <<"simpleClient">>), - {ok, _, _} = emqtt:subscribe(C, <<"TopicA">>, qos0), - ok = emqtt:publish(C, <<"TopicA">>, <<"Payload">>), - expect_publishes(<<"TopicA">>, [<<"Payload">>]), - - assert_connection_count(Config, 10, 2, 1), - {ok, _} = rabbitmqctl(Config, 0, ["decommission_mqtt_node", Server]), - await_disconnection(MRef), - assert_connection_count(Config, 10, 2, 0), - ok. + case rpc(Config, 0, rabbit_mqtt_ff, track_client_id_in_ra, []) of + false -> + {skip, "This test requires client ID tracking in Ra"}; + true -> + Server = get_node_config(Config, 0, nodename), + {ok, MRef, C} = connect_to_node(Config, 0, <<"simpleClient">>), + {ok, _, _} = emqtt:subscribe(C, <<"TopicA">>, qos0), + ok = emqtt:publish(C, <<"TopicA">>, <<"Payload">>), + expect_publishes(<<"TopicA">>, [<<"Payload">>]), + + assert_connection_count(Config, 4, 2, 1), + {ok, _} = rabbitmqctl(Config, 0, ["decommission_mqtt_node", Server]), + await_disconnection(MRef), + assert_connection_count(Config, 4, 2, 0), + ok + end. %% %% Helpers %% -assert_connection_count(_Config, 0, _, _) -> - ct:fail("failed to complete rabbit_mqtt_collector:list/0"); +assert_connection_count(_Config, 0, _, NumElements) -> + ct:fail("failed to match connection count ~b", [NumElements]); assert_connection_count(Config, Retries, NodeId, NumElements) -> - List = rpc(Config, NodeId, rabbit_mqtt_collector, list, []), - case length(List) == NumElements of - true -> + case util:all_connection_pids(Config) of + Pids + when length(Pids) =:= NumElements -> ok; - false -> - timer:sleep(200), + _ -> + timer:sleep(500), assert_connection_count(Config, Retries-1, NodeId, NumElements) end. - - connect_to_node(Config, Node, ClientID) -> Port = get_node_config(Config, Node, tcp_port_mqtt), {ok, C} = connect(Port, ClientID), diff --git a/deps/rabbitmq_mqtt/test/ff_SUITE.erl b/deps/rabbitmq_mqtt/test/ff_SUITE.erl new file mode 100644 index 000000000000..3ccb17a8bf5a --- /dev/null +++ b/deps/rabbitmq_mqtt/test/ff_SUITE.erl @@ -0,0 +1,100 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2016-2022 VMware, Inc. or its affiliates. All rights reserved. + +-module(ff_SUITE). + +%% Test suite for the feature flag delete_ra_cluster_mqtt_node + +-compile([export_all, nowarn_export_all]). + +-include_lib("common_test/include/ct.hrl"). +-include_lib("eunit/include/eunit.hrl"). + +-define(FEATURE_FLAG, delete_ra_cluster_mqtt_node). + +all() -> + [ + {group, cluster_size_3} + ]. + +groups() -> + [ + {cluster_size_3, [], [enable_feature_flag]} + ]. + +suite() -> + [ + {timetrap, {minutes, 5}} + ]. + +init_per_suite(Config) -> + rabbit_ct_helpers:log_environment(), + rabbit_ct_helpers:run_setup_steps(Config, []). + +end_per_suite(Config) -> + rabbit_ct_helpers:run_teardown_steps(Config). + +init_per_group(Group = cluster_size_3, Config0) -> + Config1 = rabbit_ct_helpers:set_config(Config0, [{rmq_nodes_count, 3}, + {rmq_nodename_suffix, Group}]), + Config2 = rabbit_ct_helpers:merge_app_env( + Config1, {rabbit, [{forced_feature_flags_on_init, []}]}), + Config = rabbit_ct_helpers:run_steps(Config2, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps()), + case rabbit_ct_broker_helpers:is_feature_flag_supported(Config, ?FEATURE_FLAG) of + true -> + Config; + false -> + end_per_group(Group, Config), + {skip, io_lib:format("feature flag ~s is unsupported", [?FEATURE_FLAG])} + end. + +end_per_group(_Group, Config) -> + rabbit_ct_helpers:run_steps(Config, + rabbit_ct_client_helpers:teardown_steps() ++ + rabbit_ct_broker_helpers:teardown_steps()). + +init_per_testcase(_TestCase, Config) -> + Config. + +end_per_testcase(_TestCase, Config) -> + Config. + +enable_feature_flag(Config) -> + C = connect_to_node(Config, 1, <<"my-client">>), + timer:sleep(500), + %% old client ID tracking works + ?assertMatch([{<<"my-client">>, _ConnectionPid}], + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_mqtt_collector, list, [])), + %% Ra processes are alive + ?assert(lists:all(fun erlang:is_pid/1, + rabbit_ct_broker_helpers:rpc_all(Config, erlang, whereis, [mqtt_node]))), + %% new client ID tracking works + ?assertEqual(1, + length(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_mqtt_clientid, list_all, []))), + + ?assertEqual(ok, rabbit_ct_broker_helpers:enable_feature_flag(Config, ?FEATURE_FLAG)), + + %% Ra processes should be gone + ?assert(lists:all(fun(Pid) -> Pid =:= undefined end, + rabbit_ct_broker_helpers:rpc_all(Config, erlang, whereis, [mqtt_node]))), + %% new client ID tracking still works + ?assertEqual(1, + length(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_mqtt_clientid, list_all, []))), + ?assert(erlang:is_process_alive(C)), + ok = emqtt:disconnect(C). + +connect_to_node(Config, Node, ClientID) -> + Port = rabbit_ct_broker_helpers:get_node_config(Config, Node, tcp_port_mqtt), + {ok, C} = emqtt:start_link([{host, "localhost"}, + {port, Port}, + {clientid, ClientID}, + {proto_ver, v4}, + {connect_timeout, 1}, + {ack_timeout, 1}]), + {ok, _Properties} = emqtt:connect(C), + C. diff --git a/deps/rabbitmq_mqtt/test/reader_SUITE.erl b/deps/rabbitmq_mqtt/test/reader_SUITE.erl index 8284cf2648f0..4a88b08272d2 100644 --- a/deps/rabbitmq_mqtt/test/reader_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/reader_SUITE.erl @@ -86,7 +86,7 @@ block(Config) -> P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), {ok, C} = emqtt:start_link([{host, "localhost"}, {port, P}, - {clientid, <<"simpleClient">>}, + {clientid, atom_to_binary(?FUNCTION_NAME)}, {proto_ver, v4}]), {ok, _Properties} = emqtt:connect(C), @@ -125,7 +125,7 @@ block_connack_timeout(Config) -> %% We can still connect via TCP, but CONNECT frame will not be processed on the server. {ok, Client} = emqtt:start_link([{host, "localhost"}, {port, P}, - {clientid, <<"simpleClient">>}, + {clientid, atom_to_binary(?FUNCTION_NAME)}, {proto_ver, v4}, {connect_timeout, 1}]), unlink(Client), @@ -156,7 +156,12 @@ block_connack_timeout(Config) -> ct:fail("missing peername_not_known from server") end, %% Ensure that our client is not registered. - [] = rpc(Config, rabbit_mqtt_collector, list, []), + [] = case rpc(Config, rabbit_mqtt_ff, track_client_id_in_ra, []) of + true -> + rpc(Config, rabbit_mqtt_collector, list, []); + false -> + rpc(Config, rabbit_mqtt_clientid, list_all, []) + end, ok. handle_invalid_frames(Config) -> @@ -189,7 +194,7 @@ keepalive(Config) -> {ok, C} = emqtt:start_link([{keepalive, KeepaliveSecs}, {host, "localhost"}, {port, P}, - {clientid, <<"simpleClient">>}, + {clientid, atom_to_binary(?FUNCTION_NAME)}, {proto_ver, v4} ]), {ok, _Properties} = emqtt:connect(C), @@ -221,13 +226,18 @@ stats(Config) -> P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), {ok, C} = emqtt:start_link([{host, "localhost"}, {port, P}, - {clientid, <<"simpleClient">>}, + {clientid, atom_to_binary(?FUNCTION_NAME)}, {proto_ver, v4}]), {ok, _Properties} = emqtt:connect(C), %% Wait for stats being emitted (every 100ms) timer:sleep(300), %% Retrieve the connection Pid - [{_, Reader}] = rpc(Config, rabbit_mqtt_collector, list, []), + [{_, Reader}] = case rpc(Config, rabbit_mqtt_ff, track_client_id_in_ra, []) of + true -> + rpc(Config, rabbit_mqtt_collector, list, []); + false -> + rpc(Config, rabbit_mqtt_clientid, list_local, []) + end, [{_, Pid}] = rpc(Config, rabbit_mqtt_reader, info, [Reader, [connection]]), %% Verify the content of the metrics, garbage_collection must be present [{Pid, Props}] = rpc(Config, ets, lookup, [connection_metrics, Pid]), From 199238d76eaeb91c723479b1289a39d7c4019e1c Mon Sep 17 00:00:00 2001 From: David Ansari Date: Wed, 12 Oct 2022 14:16:58 +0000 Subject: [PATCH 018/118] Use pg to track MQTT client IDs Instead of tracking {Vhost, ClientId} to ConnectionPid mappings in our custom process registry, i.e. custom local ETS table with a custom gen_server process managing that ETS table, this commit uses the pg module because pg is better tested. To save memory with millions of MQTT client connections, we want to save the mappings only locally on the node where the connection resides and therfore not be replicated across all nodes. According to Maxim Fedorov: "The easy way to provide per-node unique pg scope is to start it like pg:start_link(node()). At least that's what we've been doing to have node-local scopes. It will still try to discover scopes on nodeup from nodes joining the cluster, but since you cannot have nodes with the same name in one cluster, using node() for local-only scopes worked well for us." So that's what we're doing in this commit. --- deps/rabbitmq_mqtt/BUILD.bazel | 19 +++ deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl | 1 + ...tl.Commands.ListMqttConnectionsCommand.erl | 3 +- deps/rabbitmq_mqtt/src/rabbit_mqtt.erl | 1 + .../src/rabbit_mqtt_clientid.erl | 123 ------------------ .../src/rabbit_mqtt_processor.erl | 8 +- deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl | 12 +- deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl | 21 ++- deps/rabbitmq_mqtt/test/ff_SUITE.erl | 11 +- deps/rabbitmq_mqtt/test/reader_SUITE.erl | 14 +- deps/rabbitmq_mqtt/test/util.erl | 12 ++ 11 files changed, 70 insertions(+), 155 deletions(-) delete mode 100644 deps/rabbitmq_mqtt/src/rabbit_mqtt_clientid.erl create mode 100644 deps/rabbitmq_mqtt/test/util.erl diff --git a/deps/rabbitmq_mqtt/BUILD.bazel b/deps/rabbitmq_mqtt/BUILD.bazel index 66378e0efb1d..9a7f53ee4a4c 100644 --- a/deps/rabbitmq_mqtt/BUILD.bazel +++ b/deps/rabbitmq_mqtt/BUILD.bazel @@ -91,6 +91,16 @@ rabbitmq_test_helper( ], ) +rabbitmq_test_helper( + name = "util", + srcs = [ + "test/util.erl", + ], + deps = [ + "//deps/rabbit_common:erlang_app", + ], +) + rabbitmq_integration_suite( name = "auth_SUITE", additional_beam = [ @@ -109,6 +119,9 @@ rabbitmq_integration_suite( runtime_deps = [ "@emqtt//:erlang_app", ], + additional_beam = [ + ":util", + ], ) rabbitmq_integration_suite( @@ -150,6 +163,9 @@ rabbitmq_integration_suite( runtime_deps = [ "@emqtt//:erlang_app", ], + additional_beam = [ + ":util", + ], ) rabbitmq_integration_suite( @@ -164,6 +180,9 @@ rabbitmq_integration_suite( runtime_deps = [ "@emqtt//:erlang_app", ], + additional_beam = [ + ":util", + ], ) rabbitmq_suite( diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl index df76dbec0584..a2b8638c7498 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl @@ -6,6 +6,7 @@ %% -define(CLIENT_ID_MAXLEN, 23). +-define(PG_SCOPE, pg_scope_rabbitmq_mqtt_clientid). -include("rabbit_mqtt_types.hrl"). diff --git a/deps/rabbitmq_mqtt/src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListMqttConnectionsCommand.erl b/deps/rabbitmq_mqtt/src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListMqttConnectionsCommand.erl index 1aca58265f10..1a8b8e78e38f 100644 --- a/deps/rabbitmq_mqtt/src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListMqttConnectionsCommand.erl +++ b/deps/rabbitmq_mqtt/src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListMqttConnectionsCommand.erl @@ -68,8 +68,7 @@ run(Args, #{node := NodeName, false -> 'Elixir.RabbitMQ.CLI.Ctl.InfoKeys':prepare_info_keys(Args) end, - %% a node uses the Raft-based collector to list connections, which knows about all connections in the cluster - %% so no need to reach out to all the nodes + %% List only connections on the target node. Nodes = [NodeName], 'Elixir.RabbitMQ.CLI.Ctl.RpcStream':receive_list_items( diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl index 1baf1a52c5ab..7d4e5adc6fb3 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl @@ -10,6 +10,7 @@ -include("rabbit_mqtt.hrl"). -behaviour(application). + -export([start/2, stop/1]). -export([emit_connection_info_all/4, close_local_client_connections/1]). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_clientid.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_clientid.erl deleted file mode 100644 index ea75724b214f..000000000000 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_clientid.erl +++ /dev/null @@ -1,123 +0,0 @@ --module(rabbit_mqtt_clientid). - --behaviour(gen_server). - -%% client API --export([ - start_link/1, - register/2, - unregister/2 - ]). - -%% debugging --export([ - list_local/0, - list_all/0, - lookup_local/2 - ]). - -%% gen_server callbacks --export([ - init/1, - handle_cast/2, - handle_call/3, - handle_info/2, - terminate/2 - ]). - -%% ------------------------------------------------------------------- -%% client API -%% ------------------------------------------------------------------- - -start_link(ServerName) -> - gen_server:start_link(ServerName, ?MODULE, #{}, []). - -register(Vhost, ClientId) - when is_binary(Vhost), is_binary(ClientId) -> - gen_server:cast(?MODULE, {register, {Vhost, ClientId}, self()}). - -unregister(Vhost, ClientId) - when is_binary(Vhost), is_binary(ClientId) -> - gen_server:cast(?MODULE, {unregister, {Vhost, ClientId}, self()}). - -%% ------------------------------------------------------------------- -%% debugging -%% ------------------------------------------------------------------- - -list_local() -> - gen_server:call(?MODULE, list, 30_000). - -list_all() -> - {Replies, _BadNodes} = gen_server:multi_call([node() | nodes()], ?MODULE, list, 30_000), - lists:flatmap(fun({_Node, Reply}) -> - Reply - end, Replies). - -lookup_local(Vhost, ClientId) - when is_binary(Vhost), is_binary(ClientId) -> - gen_server:call(?MODULE, {lookup, {Vhost, ClientId}}, 30_000). - -%% ------------------------------------------------------------------- -%% gen_server callbacks -%% ------------------------------------------------------------------- - -init(#{}) -> - Tid = ets:new(?MODULE, [private]), - {ok, Tid}. - -handle_cast({register, Key, Pid}, Tid) -> - gen_server:abcast(nodes(), ?MODULE, {maybe_remove_duplicate, Key}), - %% optimize for no duplicate ClientId - case ets:insert_new(Tid, {Key, Pid}) of - true -> - ok; - false -> - remove_duplicate(Tid, Key), - true = ets:insert(Tid, {Key, Pid}) - end, - {noreply, Tid}; - -handle_cast({maybe_remove_duplicate, Key}, Tid) -> - %% optimize for no duplicate ClientId - case ets:member(Tid, Key) of - false -> - ok; - true -> - remove_duplicate(Tid, Key) - end, - {noreply, Tid}; - -handle_cast({unregister, Key, Pid}, Tid) -> - true = ets:delete_object(Tid, {Key, Pid}), - {noreply, Tid}; - -handle_cast(Request, Tid) -> - rabbit_log:warning("~s received unkown request ~p", [?MODULE, Request]), - {noreply, Tid}. - -handle_call(list, _From, Tid) -> - {reply, ets:tab2list(Tid), Tid}; - -handle_call({lookup, Key}, _From, Tid) -> - {reply, ets:lookup(Tid, Key), Tid}; - -handle_call(Request, From, Tid) -> - rabbit_log:warning("~s received unkown request ~p from ~p", - [?MODULE, Request, From]), - {noreply, Tid}. - -handle_info(Info, Tid) -> - rabbit_log:warning("~s received unkown info ~p", [?MODULE, Info]), - {noreply, Tid}. - -terminate(Reason, Tid) -> - rabbit_log:info("~s terminates with ~b entries and reason ~p", - [?MODULE, ets:info(Tid, size), Reason]). - -%% ------------------------------------------------------------------- -%% internal -%% ------------------------------------------------------------------- - -remove_duplicate(Tid, Key) -> - [{Key, OldPid}] = ets:take(Tid, Key), - gen_server:cast(OldPid, duplicate_id). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 1d417d1ee158..3548c65647b6 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -341,7 +341,7 @@ register_client(Frame = #mqtt_frame_connect{proto_ver = ProtoVersion}, auth_state = #auth_state{vhost = VHost}}) -> NewProcState = fun(RegisterState) -> - rabbit_mqtt_clientid:register(VHost, ClientId), + rabbit_mqtt_util:register_clientid(VHost, ClientId), RetainerPid = rabbit_mqtt_retainer_sup:child_for_vhost(VHost), Prefetch = rabbit_mqtt_util:env(prefetch), {ok, {PeerHost, PeerPort, Host, Port}} = rabbit_net:socket_ends(Socket, inbound), @@ -1164,16 +1164,14 @@ serialise(Frame, #proc_state{proto_ver = ProtoVer}) -> terminate(#proc_state{client_id = undefined}) -> ok; -terminate(#proc_state{client_id = ClientId, - auth_state = #auth_state{vhost = VHost}}) -> +terminate(#proc_state{client_id = ClientId}) -> %% ignore any errors as we are shutting down case rabbit_mqtt_ff:track_client_id_in_ra() of true -> rabbit_mqtt_collector:unregister(ClientId, self()); false -> ok - end, - rabbit_mqtt_clientid:unregister(VHost, ClientId). + end. handle_pre_hibernate() -> erase(permission_cache), diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl index 9f682d1a8a0c..900559781fb1 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl @@ -9,6 +9,7 @@ -behaviour(supervisor). -include_lib("rabbit_common/include/rabbit.hrl"). +-include("rabbit_mqtt.hrl"). -export([start_link/2, init/1, stop_listeners/0]). @@ -32,18 +33,21 @@ init([{Listeners, SslListeners0}]) -> danger -> [] end} end, + %% Use separate process group scope per RabbitMQ node. This achieves a local-only + %% process group which requires less memory with millions of connections. + PgScope = list_to_atom(io_lib:format("~s_~s", [?PG_SCOPE, node()])), + persistent_term:put(?PG_SCOPE, PgScope), {ok, {#{strategy => one_for_all, intensity => 10, period => 10}, [ - #{id => rabbit_mqtt_clientid, - start => {rabbit_mqtt_clientid, start_link, - [{local, rabbit_mqtt_clientid}]}, + #{id => PgScope, + start => {pg, start_link, [PgScope]}, restart => transient, shutdown => ?WORKER_WAIT, type => worker, - modules => [rabbit_mqtt_clientid] + modules => [pg] }, #{ id => rabbit_mqtt_retainer_sup, diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl index 552d9574446a..1d805220eb3c 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl @@ -16,7 +16,9 @@ path_for/2, path_for/3, vhost_name_to_table_name/1, - get_topic_translation_funs/0 + get_topic_translation_funs/0, + register_clientid/2, + remove_duplicate_clientid_connections/2 ]). -define(MAX_TOPIC_TRANSLATION_CACHE_SIZE, 12). @@ -140,3 +142,20 @@ path_for(Dir, VHost, Suffix) -> vhost_name_to_table_name(VHost) -> <> = erlang:md5(VHost), list_to_atom("rabbit_mqtt_retained_" ++ rabbit_misc:format("~36.16.0b", [Num])). + +-spec register_clientid(rabbit_types:vhost(), binary()) -> ok. +register_clientid(Vhost, ClientId) + when is_binary(Vhost), is_binary(ClientId) -> + PgGroup = {Vhost, ClientId}, + ok = pg:join(persistent_term:get(?PG_SCOPE), PgGroup, self()), + ok = erpc:multicast([node() | nodes()], + ?MODULE, + remove_duplicate_clientid_connections, + [PgGroup, self()]). + +-spec remove_duplicate_clientid_connections({rabbit_types:vhost(), binary()}, pid()) -> ok. +remove_duplicate_clientid_connections(PgGroup, PidToKeep) -> + Pids = pg:get_local_members(persistent_term:get(?PG_SCOPE), PgGroup), + lists:foreach(fun(Pid) -> + gen_server:cast(Pid, duplicate_id) + end, Pids -- [PidToKeep]). diff --git a/deps/rabbitmq_mqtt/test/ff_SUITE.erl b/deps/rabbitmq_mqtt/test/ff_SUITE.erl index 3ccb17a8bf5a..a101a4f548d8 100644 --- a/deps/rabbitmq_mqtt/test/ff_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/ff_SUITE.erl @@ -68,23 +68,18 @@ enable_feature_flag(Config) -> C = connect_to_node(Config, 1, <<"my-client">>), timer:sleep(500), %% old client ID tracking works - ?assertMatch([{<<"my-client">>, _ConnectionPid}], - rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_mqtt_collector, list, [])), + ?assertEqual(1, length(util:all_connection_pids(Config))), %% Ra processes are alive ?assert(lists:all(fun erlang:is_pid/1, rabbit_ct_broker_helpers:rpc_all(Config, erlang, whereis, [mqtt_node]))), - %% new client ID tracking works - ?assertEqual(1, - length(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_mqtt_clientid, list_all, []))), ?assertEqual(ok, rabbit_ct_broker_helpers:enable_feature_flag(Config, ?FEATURE_FLAG)), %% Ra processes should be gone ?assert(lists:all(fun(Pid) -> Pid =:= undefined end, rabbit_ct_broker_helpers:rpc_all(Config, erlang, whereis, [mqtt_node]))), - %% new client ID tracking still works - ?assertEqual(1, - length(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_mqtt_clientid, list_all, []))), + %% new client ID tracking works + ?assertEqual(1, length(util:all_connection_pids(Config))), ?assert(erlang:is_process_alive(C)), ok = emqtt:disconnect(C). diff --git a/deps/rabbitmq_mqtt/test/reader_SUITE.erl b/deps/rabbitmq_mqtt/test/reader_SUITE.erl index 4a88b08272d2..34a9da26d9c3 100644 --- a/deps/rabbitmq_mqtt/test/reader_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/reader_SUITE.erl @@ -156,12 +156,7 @@ block_connack_timeout(Config) -> ct:fail("missing peername_not_known from server") end, %% Ensure that our client is not registered. - [] = case rpc(Config, rabbit_mqtt_ff, track_client_id_in_ra, []) of - true -> - rpc(Config, rabbit_mqtt_collector, list, []); - false -> - rpc(Config, rabbit_mqtt_clientid, list_all, []) - end, + ?assertEqual([], util:all_connection_pids(Config)), ok. handle_invalid_frames(Config) -> @@ -232,12 +227,7 @@ stats(Config) -> %% Wait for stats being emitted (every 100ms) timer:sleep(300), %% Retrieve the connection Pid - [{_, Reader}] = case rpc(Config, rabbit_mqtt_ff, track_client_id_in_ra, []) of - true -> - rpc(Config, rabbit_mqtt_collector, list, []); - false -> - rpc(Config, rabbit_mqtt_clientid, list_local, []) - end, + [Reader] = util:all_connection_pids(Config), [{_, Pid}] = rpc(Config, rabbit_mqtt_reader, info, [Reader, [connection]]), %% Verify the content of the metrics, garbage_collection must be present [{Pid, Props}] = rpc(Config, ets, lookup, [connection_metrics, Pid]), diff --git a/deps/rabbitmq_mqtt/test/util.erl b/deps/rabbitmq_mqtt/test/util.erl new file mode 100644 index 000000000000..80d33584cb25 --- /dev/null +++ b/deps/rabbitmq_mqtt/test/util.erl @@ -0,0 +1,12 @@ +-module(util). + +-export([all_connection_pids/1]). + +all_connection_pids(Config) -> + Nodes = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Result = erpc:multicall(Nodes, rabbit_mqtt, local_connection_pids, [], 5000), + lists:foldl(fun({ok, Pids}, Acc) -> + Pids ++ Acc; + (_, Acc) -> + Acc + end, [], Result). From 3e28a520668dccd67e9faec0f868bb65f3f67a35 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Wed, 12 Oct 2022 16:54:28 +0000 Subject: [PATCH 019/118] Convert rabbit_mqtt_reader from gen_server2 to gen_server There is no need to use gen_server2. gen_server2 requires lots of memory with millions of MQTT connections because it creates 1 entry per connection into ETS table 'gen_server2_metrics'. Instead of using handle_pre_hibernate, erasing the permission cache is now done by using a timeout. We do not need a hibernate backoff feature, simply hibernate after 1 second. It's okay for MQTT connection processes to hibernate because they usually send data rather rarely (compared to AMQP connections). --- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 45 ++++++++++--------- 1 file changed, 23 insertions(+), 22 deletions(-) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index 16cdfed50f0b..35faa58fc142 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -7,12 +7,12 @@ -module(rabbit_mqtt_reader). --behaviour(gen_server2). +-behaviour(gen_server). -behaviour(ranch_protocol). -export([start_link/3]). -export([init/1, handle_call/3, handle_cast/2, handle_info/2, - code_change/3, terminate/2, handle_pre_hibernate/1]). + code_change/3, terminate/2]). %%TODO check where to best 'hibernate' when returning from callback %%TODO use rabbit_global_counters for MQTT protocol @@ -26,6 +26,7 @@ -define(SIMPLE_METRICS, [pid, recv_oct, send_oct, reductions]). -define(OTHER_METRICS, [recv_cnt, send_cnt, send_pend, garbage_collection, state]). +-define(HIBERNATE_AFTER, 1000). %%---------------------------------------------------------------------------- @@ -39,7 +40,7 @@ conserve_resources(Pid, _, {_, Conserve, _}) -> info(Pid, InfoItems) -> case InfoItems -- ?INFO_ITEMS of - [] -> gen_server2:call(Pid, {info, InfoItems}); + [] -> gen_server:call(Pid, {info, InfoItems}); UnknownItems -> throw({bad_argument, UnknownItems}) end. @@ -67,7 +68,7 @@ init(Ref) -> %% https://github.com/rabbitmq/rabbitmq-server/blob/90cc0e2abf944141feedaf3190d7b6d8b4741b11/deps/rabbitmq_stream/src/rabbit_stream_reader.erl#L536 %% https://github.com/rabbitmq/rabbitmq-server/blob/90cc0e2abf944141feedaf3190d7b6d8b4741b11/deps/rabbitmq_stream/src/rabbit_stream_reader.erl#L189 %% https://github.com/rabbitmq/rabbitmq-server/blob/7eb4084cf879fe6b1d26dd3c837bd180cb3a8546/deps/rabbitmq_management_agent/src/rabbit_mgmt_db_handler.erl#L72 - gen_server2:enter_loop(?MODULE, [], + gen_server:enter_loop(?MODULE, [], rabbit_event:init_stats_timer( control_throttle( #state{socket = RealSocket, @@ -78,8 +79,8 @@ init(Ref) -> received_connect_frame = false, conserve = false, parse_state = rabbit_mqtt_frame:initial_state(), - proc_state = ProcessorState }), #state.stats_timer), - {backoff, 1000, 1000, 10000}); + proc_state = ProcessorState }), #state.stats_timer) + ); {network_error, Reason} -> rabbit_net:fast_close(RealSocket), terminate({shutdown, Reason}, undefined); @@ -133,13 +134,17 @@ handle_cast(Delivery = {deliver, _, _, _}, State = #state{proc_state = PState}) handle_cast(Msg, State) -> {stop, {mqtt_unexpected_cast, Msg}, State}. +handle_info(timeout, State) -> + rabbit_mqtt_processor:handle_pre_hibernate(), + {noreply, State, hibernate}; + handle_info({'EXIT', _Conn, Reason}, State) -> {stop, {connection_died, Reason}, State}; handle_info({Tag, Sock, Data}, State = #state{ socket = Sock, connection_state = blocked }) - when Tag =:= tcp; Tag =:= ssl -> - {noreply, State#state{ deferred_recv = Data }, hibernate}; + when Tag =:= tcp; Tag =:= ssl -> + {noreply, State#state{ deferred_recv = Data }, ?HIBERNATE_AFTER}; handle_info({Tag, Sock, Data}, State = #state{ socket = Sock, connection_state = running }) @@ -156,7 +161,7 @@ handle_info({Tag, Sock, Reason}, State = #state{socket = Sock}) network_error(Reason, State); handle_info({inet_reply, Sock, ok}, State = #state{socket = Sock}) -> - {noreply, State, hibernate}; + {noreply, State, ?HIBERNATE_AFTER}; handle_info({inet_reply, Sock, {error, Reason}}, State = #state{socket = Sock}) -> network_error(Reason, State); @@ -173,7 +178,7 @@ handle_info({keepalive, Req}, State = #state{keepalive = KState0, conn_name = ConnName}) -> case rabbit_mqtt_keepalive:handle(Req, KState0) of {ok, KState} -> - {noreply, State#state{keepalive = KState}, hibernate}; + {noreply, State#state{keepalive = KState}, ?HIBERNATE_AFTER}; {error, timeout} -> rabbit_log_connection:error("closing MQTT connection ~p (keepalive timeout)", [ConnName]), send_will_and_terminate({shutdown, keepalive_timeout}, State); @@ -193,19 +198,19 @@ handle_info(login_timeout, State = #state{conn_name = ConnStr}) -> {stop, {shutdown, login_timeout}, State}; handle_info(emit_stats, State) -> - {noreply, emit_stats(State), hibernate}; + {noreply, emit_stats(State), ?HIBERNATE_AFTER}; handle_info({ra_event, _From, Evt}, #state{proc_state = PState0} = State) -> %% handle applied event to ensure registration command actually got applied %% handle not_leader notification in case we send the command to a non-leader PState = rabbit_mqtt_processor:handle_ra_event(Evt, PState0), - {noreply, pstate(State, PState), hibernate}; + {noreply, pstate(State, PState), ?HIBERNATE_AFTER}; handle_info({'DOWN', _MRef, process, _Pid, _Reason} = Evt, #state{proc_state = PState0} = State) -> PState = rabbit_mqtt_processor:handle_down(Evt, PState0), - {noreply, pstate(State, PState), hibernate}; + {noreply, pstate(State, PState), ?HIBERNATE_AFTER}; handle_info(Msg, State) -> {stop, {mqtt_unexpected_msg, Msg}, State}. @@ -217,10 +222,6 @@ terminate(Reason, State = #state{keepalive = KState0, rabbit_mqtt_processor:terminate(PState), log_terminate(Reason, State). -handle_pre_hibernate(State) -> - rabbit_mqtt_processor:handle_pre_hibernate(), - {hibernate, State}. - log_terminate({network_error, {ssl_upgrade_error, closed}, ConnStr}, _State) -> rabbit_log_connection:error("MQTT detected TLS upgrade error on ~s: connection closed", [ConnStr]); @@ -279,9 +280,9 @@ process_received_bytes(<<>>, State = #state{received_connect_frame = false, conn_name = ConnStr}) -> rabbit_log_connection:info("Accepted MQTT connection ~p (~s, client id: ~s)", [self(), ConnStr, rabbit_mqtt_processor:info(client_id, PState)]), - {noreply, ensure_stats_timer(State#state{received_connect_frame = true}), hibernate}; + {noreply, ensure_stats_timer(State#state{received_connect_frame = true}), ?HIBERNATE_AFTER}; process_received_bytes(<<>>, State) -> - {noreply, ensure_stats_timer(State), hibernate}; + {noreply, ensure_stats_timer(State), ?HIBERNATE_AFTER}; process_received_bytes(Bytes, State = #state{ parse_state = ParseState, proc_state = ProcState, @@ -290,7 +291,7 @@ process_received_bytes(Bytes, {more, ParseState1} -> {noreply, ensure_stats_timer( State #state{ parse_state = ParseState1 }), - hibernate}; + ?HIBERNATE_AFTER}; {ok, Frame, Rest} -> case rabbit_mqtt_processor:process_frame(Frame, ProcState) of {ok, ProcState1} -> @@ -341,7 +342,7 @@ process_received_bytes(Bytes, end. callback_reply(State, {ok, ProcState}) -> - {noreply, pstate(State, ProcState), hibernate}; + {noreply, pstate(State, ProcState), ?HIBERNATE_AFTER}; callback_reply(State, {error, Reason, ProcState}) -> {stop, Reason, pstate(State, ProcState)}. @@ -413,7 +414,7 @@ control_throttle(State = #state{connection_state = Flow, end. maybe_process_deferred_recv(State = #state{ deferred_recv = undefined }) -> - {noreply, State, hibernate}; + {noreply, State, ?HIBERNATE_AFTER}; maybe_process_deferred_recv(State = #state{ deferred_recv = Data, socket = Sock }) -> handle_info({tcp, Sock, Data}, State#state{ deferred_recv = undefined }). From 4b1c2c870bfafb09e5aacaf5b1ac98180c1f5606 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Fri, 14 Oct 2022 14:35:36 +0200 Subject: [PATCH 020/118] Emit cluster-wide MQTT connection infos When listing MQTT connections with the CLI, whether feature flag delete_ra_cluster_mqtt_node is enabled or disabled, in both cases return cluster wide MQTT connections. If connection tracking is done in Ra, the CLI target node returns all connection infos because Ra is aware of all MQTT connections. If connection tracking is done in (local-only) pg, all nodes return their local MQTT connection infos. --- ...tl.Commands.ListMqttConnectionsCommand.erl | 5 +-- deps/rabbitmq_mqtt/src/rabbit_mqtt.erl | 45 ++++++++++++++----- deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl | 14 ++++-- deps/rabbitmq_mqtt/test/command_SUITE.erl | 5 +++ 4 files changed, 50 insertions(+), 19 deletions(-) diff --git a/deps/rabbitmq_mqtt/src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListMqttConnectionsCommand.erl b/deps/rabbitmq_mqtt/src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListMqttConnectionsCommand.erl index 1a8b8e78e38f..f5534f37b428 100644 --- a/deps/rabbitmq_mqtt/src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListMqttConnectionsCommand.erl +++ b/deps/rabbitmq_mqtt/src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListMqttConnectionsCommand.erl @@ -30,7 +30,7 @@ scopes() -> [ctl, diagnostics]. switches() -> [{verbose, boolean}]. aliases() -> [{'V', verbose}]. -description() -> <<"Lists MQTT connections on the target node">>. +description() -> <<"Lists all MQTT connections">>. help_section() -> {plugin, mqtt}. @@ -68,8 +68,7 @@ run(Args, #{node := NodeName, false -> 'Elixir.RabbitMQ.CLI.Ctl.InfoKeys':prepare_info_keys(Args) end, - %% List only connections on the target node. - Nodes = [NodeName], + Nodes = 'Elixir.RabbitMQ.CLI.Core.Helpers':nodes_in_cluster(NodeName), 'Elixir.RabbitMQ.CLI.Ctl.RpcStream':receive_list_items( NodeName, diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl index 7d4e5adc6fb3..ac42da990e2b 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl @@ -13,7 +13,10 @@ -export([start/2, stop/1]). -export([emit_connection_info_all/4, - close_local_client_connections/1]). + emit_connection_info_local/3, + close_local_client_connections/1, + %% exported for tests + local_connection_pids/0]). start(normal, []) -> rabbit_global_counters:init([{protocol, mqtt}]), @@ -36,29 +39,47 @@ start(normal, []) -> stop(_) -> rabbit_mqtt_sup:stop_listeners(). -emit_connection_info_all(_Nodes, Items, Ref, AggregatorPid) -> +emit_connection_info_all(Nodes, Items, Ref, AggregatorPid) -> + case rabbit_mqtt_ff:track_client_id_in_ra() of + true -> + %% Ra tracks connections cluster-wide. + AllPids = rabbit_mqtt_collector:list_pids(), + emit_connection_info(Items, Ref, AggregatorPid, AllPids), + %% Our node already emitted infos for all connections. Therefore, for the + %% remaining nodes, we send back 'finished' so that the CLI does not time out. + [AggregatorPid ! {Ref, finished} || _ <- lists:seq(1, length(Nodes) - 1)]; + false -> + Pids = [spawn_link(Node, rabbit_mqtt, emit_connection_info_local, + [Items, Ref, AggregatorPid]) + || Node <- Nodes], + rabbit_control_misc:await_emitters_termination(Pids) + end. + +emit_connection_info_local(Items, Ref, AggregatorPid) -> + LocalPids = local_connection_pids(), + emit_connection_info(Items, Ref, AggregatorPid, LocalPids). + +emit_connection_info(Items, Ref, AggregatorPid, Pids) -> rabbit_control_misc:emitting_map_with_exit_handler( - AggregatorPid, - Ref, + AggregatorPid, Ref, fun(Pid) -> rabbit_mqtt_reader:info(Pid, Items) - end, - rabbit_mqtt_collector:list_pids() - ). + end, Pids). -spec close_local_client_connections(string() | binary()) -> {'ok', non_neg_integer()}. close_local_client_connections(Reason) -> - LocalPids = local_connection_pids(), - [rabbit_mqtt_reader:close_connection(Pid, Reason) || Pid <- LocalPids], - {ok, length(LocalPids)}. + Pids = local_connection_pids(), + lists:foreach(fun(Pid) -> + rabbit_mqtt_reader:close_connection(Pid, Reason) + end, Pids), + {ok, length(Pids)}. -spec local_connection_pids() -> [pid()]. local_connection_pids() -> case rabbit_mqtt_ff:track_client_id_in_ra() of true -> AllPids = rabbit_mqtt_collector:list_pids(), - LocalPids = lists:filter(fun(Pid) -> node(Pid) =:= node() end, AllPids), - LocalPids; + lists:filter(fun(Pid) -> node(Pid) =:= node() end, AllPids); false -> PgScope = persistent_term:get(?PG_SCOPE), lists:flatmap(fun(Group) -> diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl index 1d805220eb3c..b2872deb138e 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl @@ -155,7 +155,13 @@ register_clientid(Vhost, ClientId) -spec remove_duplicate_clientid_connections({rabbit_types:vhost(), binary()}, pid()) -> ok. remove_duplicate_clientid_connections(PgGroup, PidToKeep) -> - Pids = pg:get_local_members(persistent_term:get(?PG_SCOPE), PgGroup), - lists:foreach(fun(Pid) -> - gen_server:cast(Pid, duplicate_id) - end, Pids -- [PidToKeep]). + try persistent_term:get(?PG_SCOPE) of + PgScope -> + Pids = pg:get_local_members(PgScope, PgGroup), + lists:foreach(fun(Pid) -> + gen_server:cast(Pid, duplicate_id) + end, Pids -- [PidToKeep]) + catch _:badarg -> + %% MQTT supervision tree on this node not fully started + ok + end. diff --git a/deps/rabbitmq_mqtt/test/command_SUITE.erl b/deps/rabbitmq_mqtt/test/command_SUITE.erl index f87b96f509e2..78ef887888ad 100644 --- a/deps/rabbitmq_mqtt/test/command_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/command_SUITE.erl @@ -27,6 +27,11 @@ groups() -> ]} ]. +suite() -> + [ + {timetrap, {minutes, 3}} + ]. + init_per_suite(Config) -> rabbit_ct_helpers:log_environment(), Config1 = rabbit_ct_helpers:set_config(Config, [ From af68fb4484584fb3fa16dd3a15f8c8e98372c969 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Tue, 18 Oct 2022 10:29:15 +0000 Subject: [PATCH 021/118] Decrease memory usage of queue_type state Prior to this commit, 1 MQTT publisher publishing to 1 Million target classic queues requires around 680 MB of process memory. After this commit, it requires around 290 MB of process memory. This commit requires feature flag classic_queue_type_delivery_support and introduces a new one called no_queue_name_in_classic_queue_client. Instead of storing the binary queue name 4 times, this commit now stores it only 1 time. The monitor_registry is removed since only classic queue clients monitor their classic queue server processes. The classic queue client does not store the queue name anymore. Instead the queue name is included in messages handled by the classic queue client. Storing the queue name in the record ctx was unnecessary. More potential future memory optimisations: * When routing to destination queues, looking up the queue record, delivering to queue: Use streaming / batching instead of fetching all at once * Only fetch ETS columns that are necessary instead of whole queue records * Do not hold the same vhost binary in memory many times. Instead, maintain a mapping. * Remove unnecessary tuple fields. --- deps/rabbit/src/rabbit_amqqueue.erl | 2 +- deps/rabbit/src/rabbit_channel.erl | 51 +----- deps/rabbit/src/rabbit_classic_queue.erl | 152 +++++++++--------- deps/rabbit/src/rabbit_core_ff.erl | 12 +- deps/rabbit/src/rabbit_fifo_client.erl | 42 ++--- deps/rabbit/src/rabbit_fifo_dlx_worker.erl | 4 +- deps/rabbit/src/rabbit_queue_type.erl | 113 +++---------- deps/rabbit/src/rabbit_quorum_queue.erl | 22 +-- deps/rabbit/src/rabbit_stream_queue.erl | 3 +- deps/rabbit/test/backing_queue_SUITE.erl | 5 +- deps/rabbit_common/src/rabbit_writer.erl | 2 +- .../src/rabbit_mqtt_processor.erl | 17 +- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 5 +- .../src/rabbit_web_mqtt_handler.erl | 7 - rabbitmq.bzl | 4 +- 15 files changed, 162 insertions(+), 279 deletions(-) diff --git a/deps/rabbit/src/rabbit_amqqueue.erl b/deps/rabbit/src/rabbit_amqqueue.erl index 880992a03845..9c14c2a29c72 100644 --- a/deps/rabbit/src/rabbit_amqqueue.erl +++ b/deps/rabbit/src/rabbit_amqqueue.erl @@ -1763,7 +1763,7 @@ basic_get(Q, NoAck, LimiterPid, CTag, QStates) -> non_neg_integer(), rabbit_types:ctag(), boolean(), rabbit_framing:amqp_table(), any(), rabbit_types:username(), rabbit_queue_type:state()) -> - {ok, rabbit_queue_type:state(), rabbit_queue_type:actions()} | + {ok, rabbit_queue_type:state()} | {error, term()} | {protocol_error, Type :: atom(), Reason :: string(), Args :: term()}. basic_consume(Q, NoAck, ChPid, LimiterPid, diff --git a/deps/rabbit/src/rabbit_channel.erl b/deps/rabbit/src/rabbit_channel.erl index 5c7aaa0d0352..c6d342442bf8 100644 --- a/deps/rabbit/src/rabbit_channel.erl +++ b/deps/rabbit/src/rabbit_channel.erl @@ -743,27 +743,6 @@ handle_cast({mandatory_received, _MsgSeqNo}, State) -> %% NB: don't call noreply/1 since we don't want to send confirms. noreply_coalesce(State); -handle_cast({reject_publish, _MsgSeqNo, QPid} = Evt, State) -> - %% For backwards compatibility - QRef = rabbit_queue_type:find_name_from_pid(QPid, State#ch.queue_states), - case QRef of - undefined -> - %% ignore if no queue could be found for the given pid - noreply(State); - _ -> - handle_cast({queue_event, QRef, Evt}, State) - end; - -handle_cast({confirm, _MsgSeqNo, QPid} = Evt, State) -> - %% For backwards compatibility - QRef = rabbit_queue_type:find_name_from_pid(QPid, State#ch.queue_states), - case QRef of - undefined -> - %% ignore if no queue could be found for the given pid - noreply(State); - _ -> - handle_cast({queue_event, QRef, Evt}, State) - end; handle_cast({queue_event, QRef, Evt}, #ch{queue_states = QueueStates0} = State0) -> case rabbit_queue_type:handle_event(QRef, Evt, QueueStates0) of @@ -786,11 +765,6 @@ handle_cast({queue_event, QRef, Evt}, rabbit_misc:protocol_error(Type, Reason, ReasonArgs) end. -handle_info({ra_event, {Name, _} = From, Evt}, State) -> - %% For backwards compatibility - QRef = find_queue_name_from_quorum_name(Name, State#ch.queue_states), - handle_cast({queue_event, QRef, {From, Evt}}, State); - handle_info({bump_credit, Msg}, State) -> %% A rabbit_amqqueue_process is granting credit to our channel. If %% our channel was being blocked by this process, and no other @@ -811,11 +785,11 @@ handle_info(emit_stats, State) -> %% stats timer. {noreply, send_confirms_and_nacks(State1), hibernate}; -handle_info({'DOWN', _MRef, process, QPid, Reason}, +handle_info({{'DOWN', QName}, _MRef, process, QPid, Reason}, #ch{queue_states = QStates0, queue_monitors = _QMons} = State0) -> credit_flow:peer_down(QPid), - case rabbit_queue_type:handle_down(QPid, Reason, QStates0) of + case rabbit_queue_type:handle_down(QPid, QName, Reason, QStates0) of {ok, QState1, Actions} -> State1 = State0#ch{queue_states = QState1}, State = handle_queue_actions(Actions, State1), @@ -1813,7 +1787,7 @@ basic_consume(QueueName, NoAck, ConsumerPrefetch, ActualConsumerTag, Username, QueueStates0), Q} end) of - {{ok, QueueStates, Actions}, Q} when ?is_amqqueue(Q) -> + {{ok, QueueStates}, Q} when ?is_amqqueue(Q) -> rabbit_global_counters:consumer_created(amqp091), CM1 = maps:put( ActualConsumerTag, @@ -1822,10 +1796,9 @@ basic_consume(QueueName, NoAck, ConsumerPrefetch, ActualConsumerTag, State1 = State#ch{consumer_mapping = CM1, queue_states = QueueStates}, - State2 = handle_queue_actions(Actions, State1), {ok, case NoWait of - true -> consumer_monitor(ActualConsumerTag, State2); - false -> State2 + true -> consumer_monitor(ActualConsumerTag, State1); + false -> State1 end}; {{error, exclusive_consume_unavailable} = E, _Q} -> E; @@ -2891,20 +2864,6 @@ handle_queue_actions(Actions, #ch{} = State0) -> end, State0, Actions). -find_queue_name_from_quorum_name(Name, QStates) -> - Fun = fun(K, _V, undefined) -> - {ok, Q} = rabbit_amqqueue:lookup(K), - case amqqueue:get_pid(Q) of - {Name, _} -> - amqqueue:get_name(Q); - _ -> - undefined - end; - (_, _, Acc) -> - Acc - end, - rabbit_queue_type:fold_state(Fun, undefined, QStates). - maybe_increase_global_publishers(#ch{publishing_mode = true} = State0) -> State0; maybe_increase_global_publishers(State0) -> diff --git a/deps/rabbit/src/rabbit_classic_queue.erl b/deps/rabbit/src/rabbit_classic_queue.erl index b7d3d1f8b0a6..f842b214f201 100644 --- a/deps/rabbit/src/rabbit_classic_queue.erl +++ b/deps/rabbit/src/rabbit_classic_queue.erl @@ -4,14 +4,20 @@ -include("amqqueue.hrl"). -include_lib("rabbit_common/include/rabbit.hrl"). +%% TODO possible to use sets / maps instead of lists? +%% Check performance with QoS 1 and 1 million target queues. -record(msg_status, {pending :: [pid()], confirmed = [] :: [pid()]}). -define(STATE, ?MODULE). --record(?STATE, {pid :: undefined | pid(), %% the current master pid - qref :: term(), %% TODO - unconfirmed = #{} :: - #{non_neg_integer() => #msg_status{}}}). +-record(?STATE, { + %% the current master pid + pid :: undefined | pid(), + %% undefined if feature flag no_queue_name_in_classic_queue_client enabled + qref :: term(), + unconfirmed = #{} :: #{non_neg_integer() => #msg_status{}}, + monitored = #{} :: #{pid() => ok} + }). -opaque state() :: #?STATE{}. @@ -156,9 +162,14 @@ stat(Q) -> -spec init(amqqueue:amqqueue()) -> {ok, state()}. init(Q) when ?amqqueue_is_classic(Q) -> - QName = amqqueue:get_name(Q), + QRef = case rabbit_feature_flags:is_enabled(no_queue_name_in_classic_queue_client) of + true -> + undefined; + false -> + amqqueue:get_name(Q) + end, {ok, #?STATE{pid = amqqueue:get_pid(Q), - qref = QName}}. + qref = QRef}}. -spec close(state()) -> ok. close(_State) -> @@ -174,7 +185,7 @@ update(Q, #?STATE{pid = Pid} = State) when ?amqqueue_is_classic(Q) -> State#?STATE{pid = NewPid} end. -consume(Q, Spec, State) when ?amqqueue_is_classic(Q) -> +consume(Q, Spec, State0) when ?amqqueue_is_classic(Q) -> QPid = amqqueue:get_pid(Q), QRef = amqqueue:get_name(Q), #{no_ack := NoAck, @@ -194,9 +205,9 @@ consume(Q, Spec, State) when ?amqqueue_is_classic(Q) -> ExclusiveConsume, Args, OkMsg, ActingUser}, infinity]}) of ok -> - %% ask the host process to monitor this pid %% TODO: track pids as they change - {ok, State#?STATE{pid = QPid}, [{monitor, QPid, QRef}]}; + State = ensure_monitor(QPid, QRef, State0), + {ok, State#?STATE{pid = QPid}}; Err -> Err end. @@ -233,8 +244,10 @@ credit(CTag, Credit, Drain, State) -> [{credit, ChPid, CTag, Credit, Drain}]}), {State, []}. -handle_event({confirm, MsgSeqNos, Pid}, #?STATE{qref = QRef, - unconfirmed = U0} = State) -> +handle_event({confirm, MsgSeqNos, Pid}, #?STATE{qref = QRef} = State) -> + %% backwards compatibility when feature flag no_queue_name_in_classic_queue_client disabled + handle_event({confirm, MsgSeqNos, Pid, QRef}, State); +handle_event({confirm, MsgSeqNos, Pid, QRef}, #?STATE{unconfirmed = U0} = State) -> %% confirms should never result in rejections {Unconfirmed, ConfirmedSeqNos, []} = settle_seq_nos(MsgSeqNos, Pid, U0, confirm), @@ -247,17 +260,20 @@ handle_event({confirm, MsgSeqNos, Pid}, #?STATE{qref = QRef, {ok, State#?STATE{unconfirmed = Unconfirmed}, Actions}; handle_event({deliver, _, _, _} = Delivery, #?STATE{} = State) -> {ok, State, [Delivery]}; -handle_event({reject_publish, SeqNo, _QPid}, - #?STATE{qref = QRef, - unconfirmed = U0} = State) -> +handle_event({reject_publish, SeqNo, QPid}, #?STATE{qref = QRef} = State) -> + %% backwards compatibility when feature flag no_queue_name_in_classic_queue_client disabled + handle_event({reject_publish, SeqNo, QPid, QRef}, State); +handle_event({reject_publish, SeqNo, _QPid, QRef}, + #?STATE{unconfirmed = U0} = State) -> %% It does not matter which queue rejected the message, %% if any queue did, it should not be confirmed. {U, Rejected} = reject_seq_no(SeqNo, U0), Actions = [{rejected, QRef, Rejected}], {ok, State#?STATE{unconfirmed = U}, Actions}; -handle_event({down, Pid, Info}, #?STATE{qref = QRef, - pid = MasterPid, - unconfirmed = U0} = State0) -> +handle_event({down, Pid, QRef, Info}, #?STATE{monitored = Monitored, + pid = MasterPid, + unconfirmed = U0} = State0) -> + State = State0#?STATE{monitored = maps:remove(Pid, Monitored)}, Actions0 = case Pid =:= MasterPid of true -> [{queue_down, QRef}]; @@ -279,7 +295,7 @@ handle_event({down, Pid, Info}, #?STATE{qref = QRef, Actions = settlement_action( settled, QRef, Settled, settlement_action(rejected, QRef, Rejected, Actions0)), - {ok, State0#?STATE{unconfirmed = Unconfirmed}, Actions}; + {ok, State#?STATE{unconfirmed = Unconfirmed}, Actions}; true -> %% any abnormal exit should be considered a full reject of the %% oustanding message ids - If the message didn't get to all @@ -294,7 +310,7 @@ handle_event({down, Pid, Info}, #?STATE{qref = QRef, end end, [], U0), U = maps:without(MsgIds, U0), - {ok, State0#?STATE{unconfirmed = U}, + {ok, State#?STATE{unconfirmed = U}, [{rejected, QRef, MsgIds} | Actions0]} end; handle_event({send_drained, _} = Action, State) -> @@ -319,7 +335,7 @@ deliver(Qs0, #delivery{flow = Flow, Msg = Msg0#basic_message{id = rabbit_guid:gen()}, Delivery = Delivery0#delivery{message = Msg}, - {MPids, SPids, Qs, Actions} = qpids(Qs0, Confirm, MsgNo), + {MPids, SPids, Qs} = qpids(Qs0, Confirm, MsgNo), case Flow of %% Here we are tracking messages sent by the rabbit_channel %% process. We are accessing the rabbit_channel process @@ -334,7 +350,7 @@ deliver(Qs0, #delivery{flow = Flow, SMsg = {deliver, Delivery, true}, delegate:invoke_no_result(MPids, {gen_server2, cast, [MMsg]}), delegate:invoke_no_result(SPids, {gen_server2, cast, [SMsg]}), - {Qs, Actions}. + {Qs, []}. -spec dequeue(NoAck :: boolean(), LimiterPid :: pid(), @@ -382,14 +398,16 @@ purge(Q) when ?is_amqqueue(Q) -> qpids(Qs, Confirm, MsgNo) -> lists:foldl( - fun ({Q, S0}, {MPidAcc, SPidAcc, Qs0, Actions0}) -> + fun ({Q, S0}, {MPidAcc, SPidAcc, Qs0}) -> QPid = amqqueue:get_pid(Q), SPids = amqqueue:get_slave_pids(Q), QRef = amqqueue:get_name(Q), - Actions = [{monitor, QPid, QRef} - | [{monitor, P, QRef} || P <- SPids]] ++ Actions0, + S1 = ensure_monitor(QPid, QRef, S0), + S2 = lists:foldl(fun(SPid, Acc) -> + ensure_monitor(SPid, QRef, Acc) + end, S1, SPids), %% confirm record only if necessary - S = case S0 of + S = case S2 of #?STATE{unconfirmed = U0} -> Rec = [QPid | SPids], U = case Confirm of @@ -398,14 +416,14 @@ qpids(Qs, Confirm, MsgNo) -> true -> U0#{MsgNo => #msg_status{pending = Rec}} end, - S0#?STATE{pid = QPid, + S2#?STATE{pid = QPid, unconfirmed = U}; stateless -> - S0 + S2 end, {[QPid | MPidAcc], SPidAcc ++ SPids, - [{Q, S} | Qs0], Actions} - end, {[], [], [], []}, Qs). + [{Q, S} | Qs0]} + end, {[], [], []}, Qs). %% internal-ish -spec wait_for_promoted_or_stopped(amqqueue:amqqueue()) -> @@ -522,59 +540,43 @@ update_msg_status(confirm, Pid, #msg_status{pending = P, update_msg_status(down, Pid, #msg_status{pending = P} = S) -> S#msg_status{pending = lists:delete(Pid, P)}. +ensure_monitor(_, _, State = stateless) -> + State; +ensure_monitor(Pid, _, State = #?STATE{monitored = Monitored}) + when is_map_key(Pid, Monitored) -> + State; +ensure_monitor(Pid, QName, State = #?STATE{monitored = Monitored}) -> + _ = erlang:monitor(process, Pid, [{tag, {'DOWN', QName}}]), + State#?STATE{monitored = Monitored#{Pid => ok}}. + %% part of channel <-> queue api confirm_to_sender(Pid, QName, MsgSeqNos) -> - %% the stream queue included the queue type refactoring and thus requires - %% a different message format - case rabbit_queue_type:is_supported() of - true -> - gen_server:cast(Pid, - {queue_event, QName, - {confirm, MsgSeqNos, self()}}); - false -> - gen_server2:cast(Pid, {confirm, MsgSeqNos, self()}) - end. + Msg = case rabbit_feature_flags:is_enabled(no_queue_name_in_classic_queue_client) of + true -> + {confirm, MsgSeqNos, self(), QName}; + false -> + {confirm, MsgSeqNos, self()} + end, + gen_server:cast(Pid, {queue_event, QName, Msg}). send_rejection(Pid, QName, MsgSeqNo) -> - case rabbit_queue_type:is_supported() of - true -> - gen_server:cast(Pid, {queue_event, QName, - {reject_publish, MsgSeqNo, self()}}); - false -> - gen_server2:cast(Pid, {reject_publish, MsgSeqNo, self()}) - end. + Msg = case rabbit_feature_flags:is_enabled(no_queue_name_in_classic_queue_client) of + true -> + {reject_publish, MsgSeqNo, self(), QName}; + false -> + {reject_publish, MsgSeqNo, self()} + end, + gen_server:cast(Pid, {queue_event, QName, Msg}). deliver_to_consumer(Pid, QName, CTag, AckRequired, Message) -> - case has_classic_queue_type_delivery_support() of - true -> - Deliver = {deliver, CTag, AckRequired, [Message]}, - Evt = {queue_event, QName, Deliver}, - gen_server:cast(Pid, Evt); - false -> - Deliver = {deliver, CTag, AckRequired, Message}, - gen_server2:cast(Pid, Deliver) - end. + Deliver = {deliver, CTag, AckRequired, [Message]}, + Evt = {queue_event, QName, Deliver}, + gen_server:cast(Pid, Evt). send_drained(Pid, QName, CTagCredits) -> - case has_classic_queue_type_delivery_support() of - true -> - gen_server:cast(Pid, {queue_event, QName, - {send_drained, CTagCredits}}); - false -> - gen_server2:cast(Pid, {send_drained, CTagCredits}) - end. + gen_server:cast(Pid, {queue_event, QName, + {send_drained, CTagCredits}}). send_credit_reply(Pid, QName, Len) when is_integer(Len) -> - case rabbit_queue_type:is_supported() of - true -> - gen_server:cast(Pid, {queue_event, QName, - {send_credit_reply, Len}}); - false -> - gen_server2:cast(Pid, {send_credit_reply, Len}) - end. - -has_classic_queue_type_delivery_support() -> - %% some queue_events were missed in the initial queue_type implementation - %% this feature flag enables those and completes the initial queue type - %% API for classic queues - rabbit_feature_flags:is_enabled(classic_queue_type_delivery_support). + gen_server:cast(Pid, {queue_event, QName, + {send_credit_reply, Len}}). diff --git a/deps/rabbit/src/rabbit_core_ff.erl b/deps/rabbit/src/rabbit_core_ff.erl index 651da0925089..b3e43e053ac4 100644 --- a/deps/rabbit/src/rabbit_core_ff.erl +++ b/deps/rabbit/src/rabbit_core_ff.erl @@ -112,18 +112,26 @@ {classic_queue_type_delivery_support, #{desc => "Bug fix for classic queue deliveries using mixed versions", doc_url => "https://github.com/rabbitmq/rabbitmq-server/issues/5931", - stability => stable, + %%TODO remove compatibility code + stability => required, depends_on => [stream_queue] }}). -rabbit_feature_flag( {restart_streams, #{desc => "Support for restarting streams with optional preferred next leader argument. " - "Used to implement stream leader rebalancing", + "Used to implement stream leader rebalancing", stability => stable, depends_on => [stream_queue] }}). +-rabbit_feature_flag( + {no_queue_name_in_classic_queue_client, + #{desc => "Remove queue name from classic queue type client to save memory", + stability => stable, + depends_on => [classic_queue_type_delivery_support] + }}). + %% ------------------------------------------------------------------- %% Direct exchange routing v2. %% ------------------------------------------------------------------- diff --git a/deps/rabbit/src/rabbit_fifo_client.erl b/deps/rabbit/src/rabbit_fifo_client.erl index bec5a2bacefe..c00d33b5326f 100644 --- a/deps/rabbit/src/rabbit_fifo_client.erl +++ b/deps/rabbit/src/rabbit_fifo_client.erl @@ -27,7 +27,7 @@ handle_ra_event/3, untracked_enqueue/2, purge/1, - cluster_name/1, + queue_name/1, update_machine_state/2, pending_size/1, stat/1, @@ -47,13 +47,13 @@ {rabbit_fifo:consumer_tag(), non_neg_integer()}}. -type actions() :: [action()]. --type cluster_name() :: rabbit_types:r(queue). +-type queue_name() :: rabbit_types:r(queue). -record(consumer, {last_msg_id :: seq() | -1, ack = false :: boolean(), delivery_count = 0 :: non_neg_integer()}). --record(cfg, {cluster_name :: cluster_name(), +-record(cfg, {queue_name :: queue_name(), servers = [] :: [ra:server_id()], soft_limit = ?SOFT_LIMIT :: non_neg_integer(), block_handler = fun() -> ok end :: fun(() -> term()), @@ -87,33 +87,33 @@ %% @doc Create the initial state for a new rabbit_fifo sessions. A state is needed %% to interact with a rabbit_fifo queue using @module. -%% @param ClusterName the id of the cluster to interact with +%% @param QueueName the id of the cluster to interact with %% @param Servers The known servers of the queue. If the current leader is known %% ensure the leader node is at the head of the list. --spec init(cluster_name(), [ra:server_id()]) -> state(). -init(ClusterName, Servers) -> - init(ClusterName, Servers, ?SOFT_LIMIT). +-spec init(queue_name(), [ra:server_id()]) -> state(). +init(QueueName, Servers) -> + init(QueueName, Servers, ?SOFT_LIMIT). %% @doc Create the initial state for a new rabbit_fifo sessions. A state is needed %% to interact with a rabbit_fifo queue using @module. -%% @param ClusterName the id of the cluster to interact with +%% @param QueueName the id of the cluster to interact with %% @param Servers The known servers of the queue. If the current leader is known %% ensure the leader node is at the head of the list. %% @param MaxPending size defining the max number of pending commands. --spec init(cluster_name(), [ra:server_id()], non_neg_integer()) -> state(). -init(ClusterName = #resource{}, Servers, SoftLimit) -> +-spec init(queue_name(), [ra:server_id()], non_neg_integer()) -> state(). +init(QueueName = #resource{}, Servers, SoftLimit) -> Timeout = application:get_env(kernel, net_ticktime, 60) + 5, - #state{cfg = #cfg{cluster_name = ClusterName, + #state{cfg = #cfg{queue_name = QueueName, servers = Servers, soft_limit = SoftLimit, timeout = Timeout * 1000}}. --spec init(cluster_name(), [ra:server_id()], non_neg_integer(), fun(() -> ok), +-spec init(queue_name(), [ra:server_id()], non_neg_integer(), fun(() -> ok), fun(() -> ok)) -> state(). -init(ClusterName = #resource{}, Servers, SoftLimit, BlockFun, UnblockFun) -> +init(QueueName = #resource{}, Servers, SoftLimit, BlockFun, UnblockFun) -> %% net ticktime is in seconds Timeout = application:get_env(kernel, net_ticktime, 60) + 5, - #state{cfg = #cfg{cluster_name = ClusterName, + #state{cfg = #cfg{queue_name = QueueName, servers = Servers, block_handler = BlockFun, unblock_handler = UnblockFun, @@ -237,7 +237,7 @@ enqueue(Msg, State) -> | {empty, state()} | {error | timeout, term()}. dequeue(ConsumerTag, Settlement, #state{cfg = #cfg{timeout = Timeout, - cluster_name = QName}} = State0) -> + queue_name = QName}} = State0) -> Node = pick_server(State0), ConsumerId = consumer_id(ConsumerTag), case ra:process_command(Node, @@ -502,9 +502,9 @@ stat(Leader, Timeout) -> end. %% @doc returns the cluster name --spec cluster_name(state()) -> cluster_name(). -cluster_name(#state{cfg = #cfg{cluster_name = ClusterName}}) -> - ClusterName. +-spec queue_name(state()) -> queue_name(). +queue_name(#state{cfg = #cfg{queue_name = QueueName}}) -> + QueueName. update_machine_state(Server, Conf) -> case ra:process_command(Server, rabbit_fifo:make_update_config(Conf), ?COMMAND_TIMEOUT) of @@ -561,7 +561,7 @@ update_machine_state(Server, Conf) -> {internal, Correlators :: [term()], actions(), state()} | {rabbit_fifo:client_msg(), state()} | eol. handle_ra_event(From, {applied, Seqs}, - #state{cfg = #cfg{cluster_name = QRef, + #state{cfg = #cfg{queue_name = QRef, soft_limit = SftLmt, unblock_handler = UnblockFun}} = State0) -> @@ -738,7 +738,7 @@ maybe_auto_ack(false, {deliver, Tag, _Ack, Msgs} = Deliver, State0) -> {ok, State, [Deliver] ++ Actions}. handle_delivery(Leader, {delivery, Tag, [{FstId, _} | _] = IdMsgs}, - #state{cfg = #cfg{cluster_name = QName}, + #state{cfg = #cfg{queue_name = QName}, consumer_deliveries = CDels0} = State0) when is_map_key(Tag, CDels0) -> QRef = qref(Leader), @@ -905,7 +905,7 @@ add_command(Cid, discard, MsgIds, Acc) -> set_timer(#state{leader = Leader0, cfg = #cfg{servers = [Server | _], - cluster_name = QName}} = State) -> + queue_name = QName}} = State) -> Leader = case Leader0 of undefined -> Server; _ -> diff --git a/deps/rabbit/src/rabbit_fifo_dlx_worker.erl b/deps/rabbit/src/rabbit_fifo_dlx_worker.erl index 8322de87bb80..9e46c0996e40 100644 --- a/deps/rabbit/src/rabbit_fifo_dlx_worker.erl +++ b/deps/rabbit/src/rabbit_fifo_dlx_worker.erl @@ -177,10 +177,10 @@ handle_info({'DOWN', Ref, process, _, _}, rabbit_log:debug("~ts terminating itself because leader of ~ts is down...", [?MODULE, rabbit_misc:rs(QRef)]), supervisor:terminate_child(rabbit_fifo_dlx_sup, self()); -handle_info({'DOWN', _MRef, process, QPid, Reason}, +handle_info({{'DOWN', QName}, _MRef, process, QPid, Reason}, #state{queue_type_state = QTypeState0} = State0) -> %% received from target classic queue - case rabbit_queue_type:handle_down(QPid, Reason, QTypeState0) of + case rabbit_queue_type:handle_down(QPid, QName, Reason, QTypeState0) of {ok, QTypeState, Actions} -> State = State0#state{queue_type_state = QTypeState}, {noreply, handle_queue_actions(Actions, State)}; diff --git a/deps/rabbit/src/rabbit_queue_type.erl b/deps/rabbit/src/rabbit_queue_type.erl index a29f8cec6855..4feb912b31bd 100644 --- a/deps/rabbit/src/rabbit_queue_type.erl +++ b/deps/rabbit/src/rabbit_queue_type.erl @@ -15,7 +15,6 @@ discover/1, feature_flag_name/1, default/0, - is_supported/0, is_enabled/1, is_compatible/4, declare/2, @@ -34,7 +33,7 @@ new/2, consume/3, cancel/5, - handle_down/3, + handle_down/4, handle_event/3, module/2, deliver/3, @@ -42,7 +41,6 @@ credit/5, dequeue/5, fold_state/3, - find_name_from_pid/2, is_policy_applicable/2, is_server_named_allowed/1, arguments/1, @@ -51,7 +49,6 @@ ]). -type queue_name() :: rabbit_types:r(queue). --type queue_ref() :: queue_name() | atom(). -type queue_state() :: term(). -type msg_tag() :: term(). -type arguments() :: queue_arguments | consumer_arguments. @@ -66,13 +63,8 @@ %% TODO resolve all registered queue types from registry -define(QUEUE_TYPES, [rabbit_classic_queue, rabbit_quorum_queue, rabbit_stream_queue]). --define(QREF(QueueReference), - (is_tuple(QueueReference) andalso element(1, QueueReference) == resource) - orelse is_atom(QueueReference)). -%% anything that the host process needs to do on behalf of the queue type -%% session, like knowing when to notify on monitor down +%% anything that the host process needs to do on behalf of the queue type session -type action() :: - {monitor, Pid :: pid(), queue_ref()} | %% indicate to the queue type module that a message has been delivered %% fully to the queue {settled, Success :: boolean(), [msg_tag()]} | @@ -85,7 +77,6 @@ term(). -record(ctx, {module :: module(), - name :: queue_name(), %% "publisher confirm queue accounting" %% queue type implementation should emit a: %% {settle, Success :: boolean(), msg_tag()} @@ -97,8 +88,7 @@ state :: queue_state()}). --record(?STATE, {ctxs = #{} :: #{queue_ref() => #ctx{}}, - monitor_registry = #{} :: #{pid() => queue_ref()} +-record(?STATE, {ctxs = #{} :: #{queue_name() => #ctx{}} }). -opaque state() :: #?STATE{}. @@ -245,12 +235,6 @@ feature_flag_name(_) -> default() -> rabbit_classic_queue. -%% is the queue type API supported in the cluster -is_supported() -> - %% the stream_queue feature enables - %% the queue_type internal message API - rabbit_feature_flags:is_enabled(stream_queue). - %% is a specific queue type implementation enabled -spec is_enabled(module()) -> boolean(). is_enabled(Type) -> @@ -297,7 +281,7 @@ stat(Q) -> Mod = amqqueue:get_type(Q), Mod:stat(Q). --spec remove(queue_ref(), state()) -> state(). +-spec remove(queue_name(), state()) -> state(). remove(QRef, #?STATE{ctxs = Ctxs0} = State) -> case maps:take(QRef, Ctxs0) of error -> @@ -319,11 +303,6 @@ info(Q, Items) -> fold_state(Fun, Acc, #?STATE{ctxs = Ctxs}) -> maps:fold(Fun, Acc, Ctxs). -%% slight hack to help provide backwards compatibility in the channel -%% better than scanning the entire queue state -find_name_from_pid(Pid, #?STATE{monitor_registry = Mons}) -> - maps:get(Pid, Mons, undefined). - state_info(#ctx{state = S, module = Mod}) -> Mod:state_info(S); @@ -399,13 +378,13 @@ new(Q, State) when ?is_amqqueue(Q) -> set_ctx(Q, Ctx, State). -spec consume(amqqueue:amqqueue(), consume_spec(), state()) -> - {ok, state(), actions()} | {error, term()}. + {ok, state()} | {error, term()}. consume(Q, Spec, State) -> #ctx{state = CtxState0} = Ctx = get_ctx(Q, State), Mod = amqqueue:get_type(Q), case Mod:consume(Q, Spec, CtxState0) of - {ok, CtxState, Actions} -> - return_ok(set_ctx(Q, Ctx#ctx{state = CtxState}, State), Actions); + {ok, CtxState} -> + {ok, set_ctx(Q, Ctx#ctx{state = CtxState}, State)}; Err -> Err end. @@ -453,26 +432,20 @@ recover(VHost, Qs) -> {R0 ++ R, F0 ++ F} end, {[], []}, ByType). --spec handle_down(pid(), term(), state()) -> - {ok, state(), actions()} | {eol, state(), queue_ref()} | {error, term()}. -handle_down(Pid, Info, #?STATE{monitor_registry = Reg0} = State0) -> - %% lookup queue ref in monitor registry - case maps:take(Pid, Reg0) of - {QRef, Reg} -> - case handle_event(QRef, {down, Pid, Info}, State0) of - {ok, State, Actions} -> - {ok, State#?STATE{monitor_registry = Reg}, Actions}; - eol -> - {eol, State0#?STATE{monitor_registry = Reg}, QRef}; - Err -> - Err - end; - error -> - {ok, State0, []} +-spec handle_down(pid(), queue_name(), term(), state()) -> + {ok, state(), actions()} | {eol, state(), queue_name()} | {error, term()}. +handle_down(Pid, QName, Info, State0) -> + case handle_event(QName, {down, Pid, QName, Info}, State0) of + {ok, State, Actions} -> + {ok, State, Actions}; + eol -> + {eol, State0, QName}; + Err -> + Err end. %% messages sent from queues --spec handle_event(queue_ref(), term(), state()) -> +-spec handle_event(queue_name(), term(), state()) -> {ok, state(), actions()} | eol | {error, term()} | {protocol_error, Type :: atom(), Reason :: string(), Args :: term()}. handle_event(QRef, Evt, Ctxs) -> @@ -483,7 +456,7 @@ handle_event(QRef, Evt, Ctxs) -> state = State0} = Ctx -> case Mod:handle_event(Evt, State0) of {ok, State, Actions} -> - return_ok(set_ctx(QRef, Ctx#ctx{state = State}, Ctxs), Actions); + {ok, set_ctx(QRef, Ctx#ctx{state = State}, Ctxs), Actions}; Err -> Err end; @@ -491,7 +464,7 @@ handle_event(QRef, Evt, Ctxs) -> {ok, Ctxs, []} end. --spec module(queue_ref(), state()) -> +-spec module(queue_name(), state()) -> {ok, module()} | {error, not_found}. module(QRef, State) -> %% events can arrive after a queue state has been cleared up @@ -515,7 +488,7 @@ deliver(Qs, Delivery, State) -> end. deliver0(Qs, Delivery, stateless) -> - _ = lists:map(fun(Q) -> + lists:foreach(fun(Q) -> Mod = amqqueue:get_type(Q), _ = Mod:deliver([{Q, stateless}], Delivery) end, Qs), @@ -542,15 +515,13 @@ deliver0(Qs, Delivery, #?STATE{} = State0) -> Ctx = get_ctx_with(Q, Acc, S), set_ctx(qref(Q), Ctx#ctx{state = S}, Acc) end, State0, Xs), - return_ok(State, Actions). - + {ok, State, Actions}. --spec settle(queue_ref(), settle_op(), rabbit_types:ctag(), +-spec settle(queue_name(), settle_op(), rabbit_types:ctag(), [non_neg_integer()], state()) -> {ok, state(), actions()} | {'protocol_error', Type :: atom(), Reason :: string(), Args :: term()}. -settle(QRef, Op, CTag, MsgIds, Ctxs) - when ?QREF(QRef) -> +settle(#resource{kind = queue} = QRef, Op, CTag, MsgIds, Ctxs) -> case get_ctx(QRef, Ctxs, undefined) of undefined -> %% if we receive a settlement and there is no queue state it means @@ -566,7 +537,7 @@ settle(QRef, Op, CTag, MsgIds, Ctxs) end end. --spec credit(amqqueue:amqqueue() | queue_ref(), +-spec credit(amqqueue:amqqueue() | queue_name(), rabbit_types:ctag(), non_neg_integer(), boolean(), state()) -> {ok, state(), actions()}. credit(Q, CTag, Credit, Drain, Ctxs) -> @@ -609,24 +580,20 @@ get_ctx_with(Q, #?STATE{ctxs = Contexts}, InitState) _ when InitState == undefined -> %% not found and no initial state passed - initialize new state Mod = amqqueue:get_type(Q), - Name = amqqueue:get_name(Q), case Mod:init(Q) of {error, Reason} -> exit({Reason, Ref}); {ok, QState} -> #ctx{module = Mod, - name = Name, state = QState} end; _ -> %% not found - initialize with supplied initial state Mod = amqqueue:get_type(Q), - Name = amqqueue:get_name(Q), #ctx{module = Mod, - name = Name, state = InitState} end; -get_ctx_with(QRef, Contexts, undefined) when ?QREF(QRef) -> +get_ctx_with(#resource{kind = queue} = QRef, Contexts, undefined) -> case get_ctx(QRef, Contexts, undefined) of undefined -> exit({queue_context_not_found, QRef}); @@ -639,10 +606,6 @@ get_ctx(QRef, #?STATE{ctxs = Contexts}, Default) -> %% if we use a QRef it should always be initialised maps:get(Ref, Contexts, Default). -set_ctx(Q, Ctx, #?STATE{ctxs = Contexts} = State) - when ?is_amqqueue(Q) -> - Ref = qref(Q), - State#?STATE{ctxs = maps:put(Ref, Ctx, Contexts)}; set_ctx(QRef, Ctx, #?STATE{ctxs = Contexts} = State) -> Ref = qref(QRef), State#?STATE{ctxs = maps:put(Ref, Ctx, Contexts)}. @@ -651,27 +614,3 @@ qref(#resource{kind = queue} = QName) -> QName; qref(Q) when ?is_amqqueue(Q) -> amqqueue:get_name(Q). - -return_ok(State0, []) -> - {ok, State0, []}; -return_ok(State0, Actions0) -> - {State, Actions} = - lists:foldl( - fun({monitor, Pid, QRef}, - {#?STATE{monitor_registry = M0} = S0, A0}) -> - case M0 of - #{Pid := QRef} -> - %% already monitored by the qref - {S0, A0}; - #{Pid := _} -> - %% TODO: allow multiple Qrefs to monitor the same pid - exit(return_ok_duplicate_monitored_pid); - _ -> - _ = erlang:monitor(process, Pid), - M = M0#{Pid => QRef}, - {S0#?STATE{monitor_registry = M}, A0} - end; - (Act, {S, A0}) -> - {S, [Act | A0]} - end, {State0, []}, Actions0), - {ok, State, lists:reverse(Actions)}. diff --git a/deps/rabbit/src/rabbit_quorum_queue.erl b/deps/rabbit/src/rabbit_quorum_queue.erl index 88709c465156..eba9fdb34ea5 100644 --- a/deps/rabbit/src/rabbit_quorum_queue.erl +++ b/deps/rabbit/src/rabbit_quorum_queue.erl @@ -816,7 +816,7 @@ consume(Q, Spec, QState0) when ?amqqueue_is_quorum(Q) -> emit_consumer_created(ChPid, ConsumerTag, ExclusiveConsume, AckRequired, QName, Prefetch, Args, none, ActingUser), - {ok, QState, []}; + {ok, QState}; {error, Error} -> Error; {timeout, _} -> @@ -831,7 +831,7 @@ consume(Q, Spec, QState0) when ?amqqueue_is_quorum(Q) -> emit_consumer_created(ChPid, ConsumerTag, ExclusiveConsume, AckRequired, QName, Prefetch, Args, none, ActingUser), - {ok, QState, []} + {ok, QState} end. cancel(_Q, ConsumerTag, OkMsg, _ActingUser, State) -> @@ -893,7 +893,7 @@ deliver(QSs, #delivery{message = #basic_message{content = Content0} = Msg, case deliver(Confirm, Delivery, S0) of {reject_publish, S} -> Seq = Delivery#delivery.msg_seq_no, - QName = rabbit_fifo_client:cluster_name(S), + QName = rabbit_fifo_client:queue_name(S), {[{Q, S} | Qs], [{rejected, QName, [Seq]} | Actions]}; {_, S} -> {[{Q, S} | Qs], Actions} @@ -1325,18 +1325,8 @@ dlh(undefined, _, Strategy, _, QName) -> "because dead-letter-exchange is not configured.", [rabbit_misc:rs(QName), Strategy]), undefined; -dlh(Exchange, RoutingKey, <<"at-least-once">>, reject_publish, QName) -> - %% Feature flag stream_queue includes the rabbit_queue_type refactor - %% which is required by rabbit_fifo_dlx_worker. - case rabbit_queue_type:is_supported() of - true -> - at_least_once; - false -> - rabbit_log:warning("Falling back to dead-letter-strategy at-most-once for ~ts " - "because feature flag stream_queue is disabled.", - [rabbit_misc:rs(QName)]), - dlh_at_most_once(Exchange, RoutingKey, QName) - end; +dlh(_, _, <<"at-least-once">>, reject_publish, _) -> + at_least_once; dlh(Exchange, RoutingKey, <<"at-least-once">>, drop_head, QName) -> rabbit_log:warning("Falling back to dead-letter-strategy at-most-once for ~ts " "because configured dead-letter-strategy at-least-once is incompatible with " @@ -1593,7 +1583,7 @@ maybe_send_reply(_ChPid, undefined) -> ok; maybe_send_reply(ChPid, Msg) -> ok = rabbit_channel:send_command(ChPid, Msg). queue_name(RaFifoState) -> - rabbit_fifo_client:cluster_name(RaFifoState). + rabbit_fifo_client:queue_name(RaFifoState). get_default_quorum_initial_group_size(Arguments) -> case rabbit_misc:table_lookup(Arguments, <<"x-quorum-initial-group-size">>) of diff --git a/deps/rabbit/src/rabbit_stream_queue.erl b/deps/rabbit/src/rabbit_stream_queue.erl index 3b9698def214..dde9a3597869 100644 --- a/deps/rabbit/src/rabbit_stream_queue.erl +++ b/deps/rabbit/src/rabbit_stream_queue.erl @@ -321,9 +321,8 @@ begin_stream(#stream_client{name = QName, readers = Readers0} = State0, listening_offset = NextOffset, log = Seg0, max = Max}, - Actions = [], {ok, State#stream_client{local_pid = LocalPid, - readers = Readers0#{Tag => Str0}}, Actions} + readers = Readers0#{Tag => Str0}}} end. cancel(_Q, ConsumerTag, OkMsg, ActingUser, #stream_client{readers = Readers0, diff --git a/deps/rabbit/test/backing_queue_SUITE.erl b/deps/rabbit/test/backing_queue_SUITE.erl index a02eb959c278..65d7c8ffc2cc 100644 --- a/deps/rabbit/test/backing_queue_SUITE.erl +++ b/deps/rabbit/test/backing_queue_SUITE.erl @@ -1616,12 +1616,11 @@ wait_for_confirms(Unconfirmed) -> true -> ok; false -> receive - {'$gen_cast', - {queue_event, _QName, {confirm, Confirmed, _}}} -> + {'$gen_cast', {queue_event, _QName, {confirm, Confirmed, _}}} -> wait_for_confirms( sets:subtract( Unconfirmed, sets:from_list(Confirmed))); - {'$gen_cast', {confirm, Confirmed, _}} -> + {'$gen_cast', {queue_event, QName, {confirm, Confirmed, _, QName}}} -> wait_for_confirms( sets:subtract( Unconfirmed, sets:from_list(Confirmed))) diff --git a/deps/rabbit_common/src/rabbit_writer.erl b/deps/rabbit_common/src/rabbit_writer.erl index be306fac39e1..6d4610cd0978 100644 --- a/deps/rabbit_common/src/rabbit_writer.erl +++ b/deps/rabbit_common/src/rabbit_writer.erl @@ -58,7 +58,7 @@ %% data pending delivery (between socket %% flushes) pending, - %% defines how ofter gc will be executed + %% defines how often gc will be executed writer_gc_threshold }). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 3548c65647b6..0bb3c5950d95 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -11,8 +11,7 @@ -export([info/2, initial_state/2, initial_state/4, process_frame/2, serialise/2, send_will/1, terminate/1, handle_pre_hibernate/0, - handle_ra_event/2, handle_down/2, handle_queue_event/2, - handle_deprecated_delivery/2]). + handle_ra_event/2, handle_down/2, handle_queue_event/2]). %%TODO Use single queue per MQTT subscriber connection? %% * when publishing we store in x-mqtt-publish-qos header the publishing QoS @@ -958,7 +957,7 @@ consume(Q, QoS, #proc_state{ ok_msg => undefined, acting_user => Username}, case rabbit_queue_type:consume(Q, Spec, QStates0) of - {ok, QStates, _Actions = []} -> + {ok, QStates} -> % rabbit_global_counters:consumer_created(mqtt), PState = PState0#proc_state{queue_states = QStates}, {ok, PState}; @@ -1088,6 +1087,8 @@ publish_to_queues( deliver_to_queues(Delivery, RoutedToQNames, PState0 = #proc_state{queue_states = QStates0}) -> + %% TODO only lookup fields that are needed using ets:select / match? + %% TODO Use ETS continuations to be more space efficient Qs0 = rabbit_amqqueue:lookup(RoutedToQNames), Qs = rabbit_amqqueue:prepend_extra_bcc(Qs0), case rabbit_queue_type:deliver(Qs, Delivery, QStates0) of @@ -1210,10 +1211,9 @@ handle_ra_event(Evt, PState) -> rabbit_log:debug("unhandled ra_event: ~w ", [Evt]), PState. -handle_down({'DOWN', _MRef, process, QPid, Reason}, +handle_down({{'DOWN', QName}, _MRef, process, QPid, Reason}, PState0 = #proc_state{queue_states = QStates0}) -> - %% spike handles only QoS0 - case rabbit_queue_type:handle_down(QPid, Reason, QStates0) of + case rabbit_queue_type:handle_down(QPid, QName, Reason, QStates0) of {ok, QStates1, Actions} -> PState = PState0#proc_state{queue_states = QStates1}, handle_queue_actions(Actions, PState); @@ -1222,11 +1222,6 @@ handle_down({'DOWN', _MRef, process, QPid, Reason}, PState0#proc_state{queue_states = QStates} end. -%% Handle deprecated delivery from classic queue. This function is to be -%% removed when feature flag classic_queue_type_delivery_support becomes required. -handle_deprecated_delivery({deliver, ?CONSUMER_TAG, AckRequired, Msg}, PState) -> - {ok, deliver_one_to_client(Msg, AckRequired, PState)}. - handle_queue_event({queue_event, QName, Evt}, PState0 = #proc_state{queue_states = QStates0, unacked_client_pubs = U0}) -> diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index 35faa58fc142..9ef5c36d8749 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -128,9 +128,6 @@ handle_cast(QueueEvent = {queue_event, _, _}, State = #state{proc_state = PState}) -> callback_reply(State, rabbit_mqtt_processor:handle_queue_event(QueueEvent, PState)); -handle_cast(Delivery = {deliver, _, _, _}, State = #state{proc_state = PState}) -> - callback_reply(State, rabbit_mqtt_processor:handle_deprecated_delivery(Delivery, PState)); - handle_cast(Msg, State) -> {stop, {mqtt_unexpected_cast, Msg}, State}. @@ -207,7 +204,7 @@ handle_info({ra_event, _From, Evt}, PState = rabbit_mqtt_processor:handle_ra_event(Evt, PState0), {noreply, pstate(State, PState), ?HIBERNATE_AFTER}; -handle_info({'DOWN', _MRef, process, _Pid, _Reason} = Evt, +handle_info({{'DOWN', _QName}, _MRef, process, _Pid, _Reason} = Evt, #state{proc_state = PState0} = State) -> PState = rabbit_mqtt_processor:handle_down(Evt, PState0), {noreply, pstate(State, PState), ?HIBERNATE_AFTER}; diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index ac523fbd8cd1..3c41eb632a4f 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -36,9 +36,6 @@ keepalive :: rabbit_mqtt_keepalive:state() }). -%%TODO move from deprecated callback results to new callback results -%% see cowboy_websocket.erl - %%TODO call rabbit_networking:register_non_amqp_connection/1 so that we are notified %% when need to force load the 'connection_created' event for the management plugin, see %% https://github.com/rabbitmq/rabbitmq-management-agent/issues/58 @@ -170,10 +167,6 @@ websocket_info({'$gen_cast', QueueEvent = {queue_event, _, _}}, [State#state.conn_name, Reason]), stop(State#state{proc_state = PState}) end; -websocket_info({'$gen_cast', Delivery = {deliver, _, _, _}}, - State = #state{proc_state = PState0}) -> - {ok, PState} = rabbit_mqtt_processor:handle_deprecated_delivery(Delivery, PState0), - {[], State#state{proc_state = PState}, hibernate}; websocket_info({'$gen_cast', duplicate_id}, State = #state{ proc_state = ProcState, conn_name = ConnName }) -> rabbit_log_connection:warning("Web MQTT disconnecting a client with duplicate ID '~s' (~p)", diff --git a/rabbitmq.bzl b/rabbitmq.bzl index bed896eb8f15..8955c4b1b748 100644 --- a/rabbitmq.bzl +++ b/rabbitmq.bzl @@ -275,7 +275,9 @@ def rabbitmq_integration_suite( # user_limits # Starting from 3.12.0: # feature_flags_v2 - "RABBITMQ_FEATURE_FLAGS": "quorum_queue,implicit_default_bindings,virtual_host_metadata,maintenance_mode_status,user_limits,feature_flags_v2", + # stream_queue + # classic_queue_type_delivery_support + "RABBITMQ_FEATURE_FLAGS": "quorum_queue,implicit_default_bindings,virtual_host_metadata,maintenance_mode_status,user_limits,feature_flags_v2,stream_queue,classic_queue_type_delivery_support", "RABBITMQ_RUN": "$TEST_SRCDIR/$TEST_WORKSPACE/{}/rabbitmq-for-tests-run".format(package), "RABBITMQCTL": "$TEST_SRCDIR/$TEST_WORKSPACE/{}/broker-for-tests-home/sbin/rabbitmqctl".format(package), "RABBITMQ_PLUGINS": "$TEST_SRCDIR/$TEST_WORKSPACE/{}/broker-for-tests-home/sbin/rabbitmq-plugins".format(package), From 07ad410d816e479190af0680731f39ee198d6ff5 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Sun, 23 Oct 2022 17:06:01 +0000 Subject: [PATCH 022/118] Skip queue when MQTT QoS 0 This commit allows for huge fanouts if the MQTT subscriber connects with clean_session = true and QoS 0. Messages are not sent to a conventional queue. Instead, messages are forwarded directly from MQTT publisher connection process or channel to MQTT subscriber connection process. So, the queue process is skipped. The MQTT subscriber connection process acts as the queue process. Its mailbox is a superset of the queue. This new queue type is called rabbit_mqtt_qos0_queue. Given that the only current use case is MQTT, this queue type is currently defined in the MQTT plugin. The rabbit app is not aware that this new queue type exists. The new queue gets persisted as any other queue such that routing via the topic exchange contineues to work as usual. This allows routing across different protocols without any additional changes, e.g. huge fanout from AMQP client (or management UI) to all MQTT devices. The main benefit is that memory usage of the publishing process is kept at 0 MB once garbage collection kicked in (when hibernating the gen_server). This is achieved by having this queue type's client not maintain any state. Previously, without this new queue type, the publisher process maintained state of 500MB to all the 1 million destination queues even long after stopping sending messages to these queues. Another big benefit is that no queue process need to be created. Prior to this commit, with 1 million MQTT subscribers, 3 million Erlang processes got created: 1 million MQTT connection processes, 1 million classic queue processes, and 1 million classic queue supervisor processes. After this commit, only the 1 million MQTT connection processes get created. Hence, a few GBs of process memory will be saved. Yet another big benefit is that because the new queue type's client auto-settles the delivery when sending, the publishing process only awaits confirmation from queues who potentially have at-least-once consumers. So, the publishing process is not blocked on sending the confirm back to the publisher if 1 message is let's say routed to 1 million MQTT QoS 0 subscribers while 1 copy is routed to an important quorum queue or stream and while a single out of the million MQTT connection processes is down. Other benefits include: * Lower publisher confirm latency * Reduced inter-node network traffic In a certain sense, this commit allows RabbitMQ to act as a high scale and high throughput MQTT router (that obviously can lose messages at any time given the QoS is 0). For example, it allows use cases as using RabbitMQ to send messages cheaply and quickly to 1 million devices that happen to be online at the given time: e.g. send a notification to any online mobile device. --- deps/rabbit/src/rabbit_classic_queue.erl | 5 +- deps/rabbit/src/rabbit_queue_type.erl | 70 ++++--- deps/rabbit/src/rabbit_quorum_queue.erl | 9 +- deps/rabbit/src/rabbit_stream_queue.erl | 5 +- .../src/rabbit_mqtt_processor.erl | 141 ++++++++------ .../src/rabbit_mqtt_qos0_queue.erl | 183 ++++++++++++++++++ deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl | 20 +- deps/rabbitmq_mqtt/test/reader_SUITE.erl | 52 +++++ 8 files changed, 393 insertions(+), 92 deletions(-) create mode 100644 deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl diff --git a/deps/rabbit/src/rabbit_classic_queue.erl b/deps/rabbit/src/rabbit_classic_queue.erl index f842b214f201..33a1c92bfadd 100644 --- a/deps/rabbit/src/rabbit_classic_queue.erl +++ b/deps/rabbit/src/rabbit_classic_queue.erl @@ -47,7 +47,8 @@ info/2, state_info/1, capabilities/0, - notify_decorators/1 + notify_decorators/1, + is_stateful/0 ]). -export([delete_crashed/1, @@ -498,6 +499,8 @@ notify_decorators(Q) when ?is_amqqueue(Q) -> QPid = amqqueue:get_pid(Q), delegate:invoke_no_result(QPid, {gen_server2, cast, [notify_decorators]}). +is_stateful() -> true. + reject_seq_no(SeqNo, U0) -> reject_seq_no(SeqNo, U0, []). diff --git a/deps/rabbit/src/rabbit_queue_type.erl b/deps/rabbit/src/rabbit_queue_type.erl index 4feb912b31bd..dc5fcc5c5d10 100644 --- a/deps/rabbit/src/rabbit_queue_type.erl +++ b/deps/rabbit/src/rabbit_queue_type.erl @@ -138,20 +138,33 @@ rabbit_types:error(in_use | not_empty) | {protocol_error, Type :: atom(), Reason :: string(), Args :: term()}. --callback recover(rabbit_types:vhost(), [amqqueue:amqqueue()]) -> - {Recovered :: [amqqueue:amqqueue()], - Failed :: [amqqueue:amqqueue()]}. - %% checks if the queue should be recovered -callback is_recoverable(amqqueue:amqqueue()) -> boolean(). +-callback recover(rabbit_types:vhost(), [amqqueue:amqqueue()]) -> + {Recovered :: [amqqueue:amqqueue()], + Failed :: [amqqueue:amqqueue()]}. + -callback purge(amqqueue:amqqueue()) -> {ok, non_neg_integer()} | {error, term()}. -callback policy_changed(amqqueue:amqqueue()) -> ok. -%% stateful +-callback is_stateful() -> boolean(). + +%% stateful callbacks are optional +-optional_callbacks([init/1, + close/1, + update/2, + consume/3, + cancel/5, + handle_event/2, + settle/4, + credit/4, + dequeue/4, + state_info/1]). + %% intitialise and return a queue type specific session context -callback init(amqqueue:amqqueue()) -> {ok, queue_state()} | {error, Reason :: term()}. @@ -416,21 +429,20 @@ is_recoverable(Q) -> {Recovered :: [amqqueue:amqqueue()], Failed :: [amqqueue:amqqueue()]}. recover(VHost, Qs) -> - ByType0 = lists:map(fun(T) -> {T, []} end, ?QUEUE_TYPES), - ByType1 = maps:from_list(ByType0), + ByType0 = maps:from_keys(?QUEUE_TYPES, []), ByType = lists:foldl( fun (Q, Acc) -> T = amqqueue:get_type(Q), maps:update_with(T, fun (X) -> [Q | X] - end, Acc) - end, ByType1, Qs), - maps:fold(fun (Mod, Queues, {R0, F0}) -> - {Taken, {R, F}} = timer:tc(Mod, recover, [VHost, Queues]), - rabbit_log:info("Recovering ~b queues of type ~ts took ~bms", - [length(Queues), Mod, Taken div 1000]), - {R0 ++ R, F0 ++ F} - end, {[], []}, ByType). + end, [Q], Acc) + end, ByType0, Qs), + maps:fold(fun (Mod, Queues, {R0, F0}) -> + {Taken, {R, F}} = timer:tc(Mod, recover, [VHost, Queues]), + rabbit_log:info("Recovering ~b queues of type ~ts took ~bms", + [length(Queues), Mod, Taken div 1000]), + {R0 ++ R, F0 ++ F} + end, {[], []}, ByType). -spec handle_down(pid(), queue_name(), term(), state()) -> {ok, state(), actions()} | {eol, state(), queue_name()} | {error, term()}. @@ -496,15 +508,23 @@ deliver0(Qs, Delivery, stateless) -> deliver0(Qs, Delivery, #?STATE{} = State0) -> %% TODO: optimise single queue case? %% sort by queue type - then dispatch each group - ByType = lists:foldl( - fun (Q, Acc) -> - T = amqqueue:get_type(Q), - Ctx = get_ctx(Q, State0), - maps:update_with( - T, fun (A) -> - [{Q, Ctx#ctx.state} | A] - end, [{Q, Ctx#ctx.state}], Acc) - end, #{}, Qs), + {ByType, Actions0} = + lists:foldl( + fun (Q, {M, L}) -> + T = amqqueue:get_type(Q), + case T:is_stateful() of + true -> + Ctx = get_ctx(Q, State0), + {maps:update_with( + T, fun (A) -> + [{Q, Ctx#ctx.state} | A] + end, [{Q, Ctx#ctx.state}], M), + L}; + false -> + {[], DeliverActions} = T:deliver([{Q, stateless}], Delivery), + {M, DeliverActions ++ L} + end + end, {#{}, []}, Qs), %%% dispatch each group to queue type interface? {Xs, Actions} = maps:fold(fun(Mod, QSs, {X0, A0}) -> {X, A} = Mod:deliver(QSs, Delivery), @@ -515,7 +535,7 @@ deliver0(Qs, Delivery, #?STATE{} = State0) -> Ctx = get_ctx_with(Q, Acc, S), set_ctx(qref(Q), Ctx#ctx{state = S}, Acc) end, State0, Xs), - {ok, State, Actions}. + {ok, State, Actions0 ++ Actions}. -spec settle(queue_name(), settle_op(), rabbit_types:ctag(), [non_neg_integer()], state()) -> diff --git a/deps/rabbit/src/rabbit_quorum_queue.erl b/deps/rabbit/src/rabbit_quorum_queue.erl index eba9fdb34ea5..9f30bacde5e5 100644 --- a/deps/rabbit/src/rabbit_quorum_queue.erl +++ b/deps/rabbit/src/rabbit_quorum_queue.erl @@ -64,7 +64,8 @@ -export([is_enabled/0, is_compatible/3, - declare/2]). + declare/2, + is_stateful/0]). -import(rabbit_queue_type_util, [args_policy_lookup/3, qname_to_internal_name/1]). @@ -1702,8 +1703,8 @@ ets_lookup_element(Tbl, Key, Pos, Default) -> Default end. -erpc_call(Node, M, F, A, Timeout) - when is_integer(Timeout) andalso Node == node() -> +erpc_call(Node, M, F, A, _Timeout) + when Node =:= node() -> %% Only timeout 'infinity' optimises the local call in OTP 23-25 avoiding a new process being spawned: %% https://github.com/erlang/otp/blob/47f121af8ee55a0dbe2a8c9ab85031ba052bad6b/lib/kernel/src/erpc.erl#L121 try erpc:call(Node, M, F, A, infinity) of @@ -1727,4 +1728,4 @@ erpc_call(Node, M, F, A, Timeout) -> {error, noconnection} end. - +is_stateful() -> true. diff --git a/deps/rabbit/src/rabbit_stream_queue.erl b/deps/rabbit/src/rabbit_stream_queue.erl index dde9a3597869..ec7e5d713fc0 100644 --- a/deps/rabbit/src/rabbit_stream_queue.erl +++ b/deps/rabbit/src/rabbit_stream_queue.erl @@ -34,7 +34,8 @@ state_info/1, stat/1, capabilities/0, - notify_decorators/1]). + notify_decorators/1, + is_stateful/0]). -export([list_with_minimum_quorum/0]). @@ -1095,3 +1096,5 @@ list_with_minimum_quorum() -> RunningMembers = maps:filter(fun(_, {State, _}) -> State =/= undefined end, Members), map_size(RunningMembers) =< map_size(Members) div 2 + 1 end, rabbit_amqqueue:list_local_stream_queues()). + +is_stateful() -> true. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 0bb3c5950d95..6a44d1c3ce84 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -486,15 +486,11 @@ get_queue(QoS, PState) -> QName = queue_name(QoS, PState), rabbit_amqqueue:lookup(QName). -queue_name(QoS, #proc_state{auth_state = #auth_state{vhost = VHost}, - client_id = ClientId}) -> - QNameBin = queue_name_bin(QoS, ClientId), +queue_name(QoS, #proc_state{client_id = ClientId, + auth_state = #auth_state{vhost = VHost}}) -> + QNameBin = rabbit_mqtt_util:queue_name(ClientId, QoS), rabbit_misc:r(VHost, queue, QNameBin). -queue_name_bin(QoS, ClientId) -> - Names = rabbit_mqtt_util:queue_names(ClientId), - element(QoS + 1, Names). - hand_off_to_retainer(RetainerPid, Amqp2MqttFun, Topic0, #mqtt_msg{payload = <<"">>}) -> Topic1 = Amqp2MqttFun(Topic0), rabbit_mqtt_retainer:clear(RetainerPid, Topic1), @@ -864,7 +860,6 @@ delivery_mode(?QOS_0) -> 1; delivery_mode(?QOS_1) -> 2; delivery_mode(?QOS_2) -> 2. - ensure_queue(QoS, #proc_state{ client_id = ClientId, clean_sess = CleanSess, @@ -877,20 +872,26 @@ ensure_queue(QoS, #proc_state{ {ok, Q} -> {ok, Q}; {error, not_found} -> - QNameBin = queue_name_bin(QoS, ClientId), + QNameBin = rabbit_mqtt_util:queue_name(ClientId, QoS), QName = rabbit_misc:r(VHost, queue, QNameBin), %% configure access to queue required for queue.declare case check_resource_access(User, QName, configure, AuthzCtx) of ok -> - rabbit_core_metrics:queue_declared(QName), case rabbit_vhost_limit:is_over_queue_limit(VHost) of false -> - case rabbit_amqqueue:declare(QName, - _Durable = true, - _AutoDelete = false, - queue_args(QoS, CleanSess), - queue_owner(QoS, CleanSess), - Username) of + rabbit_core_metrics:queue_declared(QName), + QArgs = queue_args(QoS, CleanSess), + Q0 = amqqueue:new(QName, + self(), + _Durable = true, + _AutoDelete = false, + queue_owner(QoS, CleanSess), + QArgs, + VHost, + #{user => Username}, + queue_type(QoS, CleanSess, QArgs) + ), + case rabbit_queue_type:declare(Q0, node()) of {new, Q} when ?is_amqqueue(Q) -> rabbit_core_metrics:queue_created(QName), {ok, Q}; @@ -935,6 +936,11 @@ queue_args(_, _) -> Args end. +queue_type(?QOS_0, true, _) -> + rabbit_mqtt_qos0_queue; +queue_type(_, _, QArgs) -> + rabbit_amqqueue:get_queue_type(QArgs). + consume(Q, QoS, #proc_state{ queue_states = QStates0, auth_state = #auth_state{ @@ -946,25 +952,32 @@ consume(Q, QoS, #proc_state{ %% read access to queue required for basic.consume case check_resource_access(User, QName, read, AuthzCtx) of ok -> - Spec = #{no_ack => QoS =:= ?QOS_0, - channel_pid => self(), - limiter_pid => none, - limiter_active => false, - prefetch_count => Prefetch, - consumer_tag => ?CONSUMER_TAG, - exclusive_consume => false, - args => [], - ok_msg => undefined, - acting_user => Username}, - case rabbit_queue_type:consume(Q, Spec, QStates0) of - {ok, QStates} -> - % rabbit_global_counters:consumer_created(mqtt), - PState = PState0#proc_state{queue_states = QStates}, - {ok, PState}; - {error, Reason} = Err -> - rabbit_log:error("Failed to consume from ~s: ~p", - [rabbit_misc:rs(QName), Reason]), - Err + case amqqueue:get_type(Q) of + rabbit_mqtt_qos0_queue -> + %% Messages get delivered directly to our process without + %% explicitly calling rabbit_queue_type:consume/3. + {ok, PState0}; + _ -> + Spec = #{no_ack => QoS =:= ?QOS_0, + channel_pid => self(), + limiter_pid => none, + limiter_active => false, + prefetch_count => Prefetch, + consumer_tag => ?CONSUMER_TAG, + exclusive_consume => false, + args => [], + ok_msg => undefined, + acting_user => Username}, + case rabbit_queue_type:consume(Q, Spec, QStates0) of + {ok, QStates} -> + % rabbit_global_counters:consumer_created(mqtt), + PState = PState0#proc_state{queue_states = QStates}, + {ok, PState}; + {error, Reason} = Err -> + rabbit_log:error("Failed to consume from ~s: ~p", + [rabbit_misc:rs(QName), Reason]), + Err + end end; {error, access_refused} = Err -> Err @@ -1092,19 +1105,16 @@ deliver_to_queues(Delivery, Qs0 = rabbit_amqqueue:lookup(RoutedToQNames), Qs = rabbit_amqqueue:prepend_extra_bcc(Qs0), case rabbit_queue_type:deliver(Qs, Delivery, QStates0) of - {ok, QStates, _Actions = []} -> + {ok, QStates, Actions} -> rabbit_global_counters:messages_routed(mqtt, length(Qs)), - PState = process_routing_confirm(Delivery, Qs, PState0), + PState = process_routing_confirm(Delivery, Qs, + PState0#proc_state{queue_states = QStates}), %% Actions must be processed after registering confirms as actions may - %% contain rejections of publishes - %% TODO handle Actions: For example if the messages is rejected, MQTT 5 allows to send a NACK - %% back to the client (via PUBACK Reason Code). - % State = handle_queue_actions(Actions, State1#ch{queue_states = QueueStates}), - {ok, PState#proc_state{queue_states = QStates}}; + %% contain rejections of publishes. + {ok, handle_queue_actions(Actions, PState)}; {error, Reason} -> - rabbit_log:error("Failed to deliver message to queues " - "packet_id=~p, queues=~p, Reason=~p", - [Delivery#delivery.msg_seq_no, queue_names(Qs), Reason]), + rabbit_log:error("Failed to deliver message with packet_id=~p to queues: ~p", + [Delivery#delivery.msg_seq_no, Reason]), {error, Reason, PState0} end. @@ -1133,7 +1143,7 @@ process_routing_confirm(#delivery{confirm = true, process_routing_confirm(#delivery{confirm = true, msg_seq_no = MsgId}, Qs, PState = #proc_state{unacked_client_pubs = U0}) -> - QNames = queue_names(Qs), + QNames = lists:map(fun amqqueue:get_name/1, Qs), {ok, U} = rabbit_mqtt_confirms:insert(MsgId, QNames, U0), PState#proc_state{unacked_client_pubs = U}. @@ -1165,14 +1175,33 @@ serialise(Frame, #proc_state{proto_ver = ProtoVer}) -> terminate(#proc_state{client_id = undefined}) -> ok; -terminate(#proc_state{client_id = ClientId}) -> +terminate(PState = #proc_state{client_id = ClientId}) -> %% ignore any errors as we are shutting down case rabbit_mqtt_ff:track_client_id_in_ra() of true -> rabbit_mqtt_collector:unregister(ClientId, self()); false -> ok - end. + end, + delete_mqtt_qos0_queue(PState). + +delete_mqtt_qos0_queue(PState = #proc_state{clean_sess = true, + auth_state = #auth_state{username = Username}}) -> + case get_queue(?QOS_0, PState) of + {ok, Q} -> + %% double check we delete the right queue + case {amqqueue:get_type(Q), amqqueue:get_pid(Q)} of + {rabbit_mqtt_qos0_queue, Pid} + when Pid =:= self() -> + rabbit_queue_type:delete(Q, false, false, Username); + _ -> + ok + end; + {error, not_found} -> + ok + end; +delete_mqtt_qos0_queue(_) -> + ok. handle_pre_hibernate() -> erase(permission_cache), @@ -1222,6 +1251,9 @@ handle_down({{'DOWN', QName}, _MRef, process, QPid, Reason}, PState0#proc_state{queue_states = QStates} end. +handle_queue_event({queue_event, rabbit_mqtt_qos0_queue, Msg}, PState0) -> + PState = deliver_one_to_client(Msg, false, PState0), + {ok, PState}; handle_queue_event({queue_event, QName, Evt}, PState0 = #proc_state{queue_states = QStates0, unacked_client_pubs = U0}) -> @@ -1267,7 +1299,7 @@ deliver_to_client(Msgs, Ack, PState) -> deliver_one_to_client(Msg, Ack, S) end, PState, Msgs). -deliver_one_to_client(Msg = {QName, QPid, QMsgId, _Redelivered, +deliver_one_to_client(Msg = {QNameOrType, QPid, QMsgId, _Redelivered, #basic_message{content = #content{properties = #'P_basic'{headers = Headers}}}}, AckRequired, PState0) -> PublisherQoS = case rabbit_mqtt_util:table_lookup(Headers, <<"x-mqtt-publish-qos">>) of @@ -1285,13 +1317,13 @@ deliver_one_to_client(Msg = {QName, QPid, QMsgId, _Redelivered, end, QoS = effective_qos(PublisherQoS, SubscriberQoS), PState1 = maybe_publish_to_client(Msg, QoS, PState0), - PState = maybe_ack(AckRequired, QoS, QName, QMsgId, PState1), + PState = maybe_ack(AckRequired, QoS, QNameOrType, QMsgId, PState1), %%TODO GC % case GCThreshold of % undefined -> ok; % _ -> rabbit_basic:maybe_gc_large_msg(Content, GCThreshold) % end, - ok = maybe_notify_sent(QName, QPid, PState), + ok = maybe_notify_sent(QNameOrType, QPid, PState), PState. -spec effective_qos(qos(), qos()) -> qos(). @@ -1302,7 +1334,7 @@ effective_qos(PublisherQoS, SubscriberQoS) -> erlang:min(PublisherQoS, SubscriberQoS). maybe_publish_to_client({_, _, _, _Redelivered = true, _}, ?QOS_0, PState) -> - %% Do no redeliver to MQTT subscriber who gets message at most once. + %% Do not redeliver to MQTT subscriber who gets message at most once. PState; maybe_publish_to_client( {_QName, _QPid, QMsgId, Redelivered, @@ -1362,6 +1394,8 @@ maybe_ack(_AckRequired = true, ?QOS_0, QName, QMsgId, maybe_ack(_, _, _, _, PState) -> PState. +maybe_notify_sent(rabbit_mqtt_qos0_queue, _, _) -> + ok; maybe_notify_sent(QName, QPid, #proc_state{queue_states = QStates}) -> case rabbit_queue_type:module(QName, QStates) of {ok, rabbit_classic_queue} -> @@ -1478,6 +1512,3 @@ ssl_login_name(Sock) -> {error, no_peercert} -> none; nossl -> none end. - -queue_names(Queues) -> - lists:map(fun amqqueue:get_name/1, Queues). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl new file mode 100644 index 000000000000..c5260e6cd5da --- /dev/null +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl @@ -0,0 +1,183 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2018-2022 VMware, Inc. or its affiliates. All rights reserved. +%% + +%% This module is a pseudo queue type. +%% Messages are consumed directly by the declaring MQTT connection process. +%% In a certain sense, the declaring MQTT connection process' mailbox +%% represents a superset of this queue. +%% +%% This queue type allows only stateless clients such that clients (i.e. +%% MQTT publisher connection processes or AMQP 0.9.1 channels) can deliver +%% messages to millions of these queues without requiring a lot of memory. +%% +%% All messages are delivered at most once. + +%% TODO +% Write test for mixed version cluster where the new node declares new +% queue type and old node routes to new queue, but does not know +% the new queuey type module yet. This might either require a feature +% flag or dropping the MQTT at-most-once message during the rolling +% update from and old to a new node. + +-module(rabbit_mqtt_qos0_queue). +-behaviour(rabbit_queue_type). + +-include_lib("rabbit_common/include/rabbit.hrl"). + +%% rabbit_queue_type callbacks +-export([ + is_stateful/0, + declare/2, + delete/4, + deliver/2, + is_enabled/0, + is_compatible/3, + is_recoverable/1, + recover/2, + purge/1, + policy_changed/1, + info/2, + stat/1, + capabilities/0, + notify_decorators/1 + ]). + +-spec is_stateful() -> + boolean(). +is_stateful() -> + false. + +-spec declare(amqqueue:amqqueue(), node()) -> + {'new' | 'existing' | 'owner_died', amqqueue:amqqueue()} | + {'absent', amqqueue:amqqueue(), rabbit_queue_type:absent_reason()}. +declare(Q0, _Node) -> + %% The queue gets persisted such that routing to this + %% queue (via the topic exchange) works as usual. + case rabbit_amqqueue:internal_declare(Q0, false) of + {created, Q} -> + {new, Q}; + Other -> + Other + end. + +-spec delete(amqqueue:amqqueue(), + boolean(), + boolean(), + rabbit_types:username()) -> + rabbit_types:ok(non_neg_integer()). +delete(Q, _IfUnused, _IfEmpty, ActingUser) -> + QName = amqqueue:get_name(Q), + log_delete(QName, amqqueue:get_exclusive_owner(Q)), + ok = rabbit_amqqueue:internal_delete(QName, ActingUser), + {ok, 0}. + +%%TODO implement some form of flow control to not overwhelm the destination +%% MQTT connection process? +%% E.g. drop this message if destination queue is already long? +% erlang:process_info(Pid, message_queue_len) +%% ...if that's cheap. Alternatively, use credits? +-spec deliver([{amqqueue:amqqueue(), stateless}], Delivery :: term()) -> + {[], rabbit_queue_type:actions()}. +deliver([{Q, stateless}], Delivery = #delivery{message = BasicMessage}) -> + Pid = amqqueue:get_pid(Q), + Msg = {queue_event, ?MODULE, + {?MODULE, Pid, _QMsgId = none, _Redelivered = false, BasicMessage}}, + gen_server:cast(Pid, Msg), + Actions = confirm(Delivery, Q), + {[], Actions}. + +confirm(#delivery{confirm = false}, _) -> + []; +confirm(#delivery{confirm = true, + msg_seq_no = SeqNo}, Q) -> + %% We confirm the message directly here in the queue client. + %% Alternatively, we could have the target MQTT connection process confirm the message. + %% However, given that this message might be lost anyway between target MQTT connection + %% process and MQTT subscriber, and we know that the MQTT subscriber wants to receive + %% this message at most once, we confirm here directly. + %% Benefits: + %% 1. We do not block sending the confirmation back to the publishing client just because a single + %% (at-most-once) target queue out of potentially many (e.g. million) queues might be unavailable. + %% 2. Memory usage in this (publishing) process is kept lower because the target queue name can be + %% directly removed from rabbit_mqtt_confirms and rabbit_confirms. + %% 3. Reduced network traffic across RabbitMQ nodes. + %% 4. Lower latency of sending publisher confirmation back to the publishing client. + QName = amqqueue:get_name(Q), + [{settled, QName, [SeqNo]}]. + +-spec is_enabled() -> + boolean(). +is_enabled() -> + true. + +-spec is_compatible(boolean(), boolean(), boolean()) -> + boolean(). +is_compatible(_Durable = true, _Exclusive = true, _AutoDelete = false) -> + true; +is_compatible(_, _, _) -> + false. + +-spec is_recoverable(amqqueue:amqqueue()) -> + boolean(). +is_recoverable(Q) -> + Pid = amqqueue:get_pid(Q), + OwnerPid = amqqueue:get_exclusive_owner(Q), + node() =:= node(Pid) andalso + Pid =:= OwnerPid andalso + not is_process_alive(Pid). + +%% We (mis)use the recover callback to clean up our exclusive queues +%% which otherwise do not get cleaned up after a node crash. +-spec recover(rabbit_types:vhost(), [amqqueue:amqqueue()]) -> + {Recovered :: [amqqueue:amqqueue()], Failed :: [amqqueue:amqqueue()]}. +recover(_VHost, Queues) -> + lists:foreach( + fun(Q) -> + %% sanity check + true = is_recoverable(Q), + QName = amqqueue:get_name(Q), + log_delete(QName, amqqueue:get_exclusive_owner(Q)), + rabbit_amqqueue:internal_delete(QName, ?INTERNAL_USER) + end, Queues), + %% We mark the queue recovery as failed because these queues are not really + %% recovered, but deleted. + {[], Queues}. + +log_delete(QName, ConPid) -> + rabbit_log_queue:debug("Deleting ~s of type ~s because its declaring connection ~tp was closed", + [rabbit_misc:rs(QName), ?MODULE, ConPid]). + +-spec purge(amqqueue:amqqueue()) -> + {ok, non_neg_integer()}. +purge(_Q) -> + {ok, 0}. + +-spec policy_changed(amqqueue:amqqueue()) -> + ok. +policy_changed(_Q) -> + ok. + +%% general queue info +-spec info(amqqueue:amqqueue(), all_keys | rabbit_types:info_keys()) -> + rabbit_types:infos(). +info(_Q, _Items) -> + []. + +-spec notify_decorators(amqqueue:amqqueue()) -> + ok. +notify_decorators(_) -> + ok. + +-spec stat(amqqueue:amqqueue()) -> + {'ok', non_neg_integer(), non_neg_integer()}. +stat(_Q) -> + {ok, 0, 0}. + +-spec capabilities() -> + #{atom() := term()}. +capabilities() -> + #{}. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl index b2872deb138e..43eb920af710 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl @@ -8,8 +8,9 @@ -module(rabbit_mqtt_util). -include("rabbit_mqtt.hrl"). +-include("rabbit_mqtt_frame.hrl"). --export([queue_names/1, +-export([queue_name/2, gen_client_id/0, env/1, table_lookup/2, @@ -23,11 +24,18 @@ -define(MAX_TOPIC_TRANSLATION_CACHE_SIZE, 12). --spec queue_names(binary()) -> - {binary(), binary()}. -queue_names(ClientId) -> - Base = <<"mqtt-subscription-", ClientId/binary, "qos">>, - {<>, <>}. +-spec queue_name(binary(), qos()) -> + binary(). +queue_name(ClientId, QoS) -> + Prefix = <<"mqtt-subscription-", ClientId/binary, "qos">>, + queue_name0(Prefix, QoS). + +queue_name0(Prefix, ?QOS_0) -> + %%TODO consider shortening the QoS0 queue name to save memory + %%(can't change QoS1 name to not break rolling updates) + <>; +queue_name0(Prefix, ?QOS_1) -> + <>. cached(CacheName, Fun, Arg) -> Cache = diff --git a/deps/rabbitmq_mqtt/test/reader_SUITE.erl b/deps/rabbitmq_mqtt/test/reader_SUITE.erl index 34a9da26d9c3..7577c0ac9134 100644 --- a/deps/rabbitmq_mqtt/test/reader_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/reader_SUITE.erl @@ -27,6 +27,8 @@ groups() -> login_timeout, keepalive, stats, + clean_session_disconnect_client, + clean_session_kill_node, quorum_clean_session_false, quorum_clean_session_true, classic_clean_session_true, @@ -267,6 +269,56 @@ validate_durable_queue_type(Config, ClientName, CleanSession, ExpectedQueueType) ?assertEqual(ExpectedQueueType, get_durable_queue_type(Server, QNameBin)), ok = emqtt:disconnect(C). +clean_session_disconnect_client(Config) -> + P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), + {ok, C} = emqtt:start_link([{clean_start, true}, + {host, "localhost"}, + {port, P}, + {clientid, atom_to_binary(?FUNCTION_NAME)}, + {proto_ver, v4}]), + {ok, _Properties} = emqtt:connect(C), + + {ok, _, _} = emqtt:subscribe(C, <<"topic0">>, qos0), + L0 = rpc(Config, rabbit_amqqueue, list_by_type, [rabbit_mqtt_qos0_queue]), + ?assertEqual(1, length(L0)), + + {ok, _, _} = emqtt:subscribe(C, <<"topic1">>, qos1), + L1 = rpc(Config, rabbit_amqqueue, list_by_type, [rabbit_classic_queue]), + ?assertEqual(1, length(L1)), + + ok = emqtt:disconnect(C), + %% After terminating a clean session, we expect any session state to be cleaned up on the server. + timer:sleep(200), %% Give some time to clean up exclusive classic queue. + L = rpc(Config, rabbit_amqqueue, list, []), + ?assertEqual(0, length(L)). + +clean_session_kill_node(Config) -> + P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), + {ok, C} = emqtt:start_link([{clean_start, true}, + {host, "localhost"}, + {port, P}, + {clientid, atom_to_binary(?FUNCTION_NAME)}, + {proto_ver, v4}]), + {ok, _Properties} = emqtt:connect(C), + + {ok, _, _} = emqtt:subscribe(C, <<"topic0">>, qos0), + L0 = rpc(Config, rabbit_amqqueue, list_by_type, [rabbit_mqtt_qos0_queue]), + ?assertEqual(1, length(L0)), + + {ok, _, _} = emqtt:subscribe(C, <<"topic1">>, qos1), + L1 = rpc(Config, rabbit_amqqueue, list_by_type, [rabbit_classic_queue]), + ?assertEqual(1, length(L1)), + + ?assertEqual(2, rpc(Config, ets, info, [rabbit_durable_queue, size])), + + process_flag(trap_exit, true), + ok = rabbit_ct_broker_helpers:kill_node(Config, 0), + ok = rabbit_ct_broker_helpers:start_node(Config, 0), + + %% After terminating a clean session by a node crash, we expect any session + %% state to be cleaned up on the server once the server comes back up. + ?assertEqual(0, rpc(Config, ets, info, [rabbit_durable_queue, size])). + quorum_clean_session_false(Config) -> Default = rpc(Config, reader_SUITE, get_env, []), rpc(Config, reader_SUITE, set_env, [quorum]), From b74dea443595520c38174745eb49d98ba85e719e Mon Sep 17 00:00:00 2001 From: Chunyi Lyu Date: Thu, 27 Oct 2022 12:17:26 +0100 Subject: [PATCH 023/118] Send rabbit event declaring mqtt_qos0 queue --- deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl index c5260e6cd5da..8cfb582a4c48 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl @@ -59,6 +59,15 @@ declare(Q0, _Node) -> %% queue (via the topic exchange) works as usual. case rabbit_amqqueue:internal_declare(Q0, false) of {created, Q} -> + Opts = amqqueue:get_options(Q), + ActingUser = maps:get(user, Opts, ?UNKNOWN_USER), + rabbit_event:notify(queue_created, + [{name, amqqueue:get_name(Q0)}, + {durable, true}, + {auto_delete, false}, + {type, amqqueue:get_type(Q0)}, + {arguments, amqqueue:get_arguments(Q0)}, + {user_who_performed_action, ActingUser}]), {new, Q}; Other -> Other From 627ea8588a78829eeb9c9953f81be846194abbff Mon Sep 17 00:00:00 2001 From: David Ansari Date: Thu, 27 Oct 2022 17:07:43 +0000 Subject: [PATCH 024/118] Add rabbit_event tests for MQTT Add tests that MQTT plugin sends correct events to rabbit_event. Add event connection_closed. --- deps/rabbitmq_mqtt/BUILD.bazel | 30 ++- .../src/rabbit_mqtt_processor.erl | 49 +++-- .../src/rabbit_mqtt_qos0_queue.erl | 13 +- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 5 +- deps/rabbitmq_mqtt/test/event_recorder.erl | 20 ++ deps/rabbitmq_mqtt/test/integration_SUITE.erl | 174 ++++++++++++++++++ .../src/rabbit_web_mqtt_handler.erl | 8 +- 7 files changed, 263 insertions(+), 36 deletions(-) create mode 100644 deps/rabbitmq_mqtt/test/event_recorder.erl create mode 100644 deps/rabbitmq_mqtt/test/integration_SUITE.erl diff --git a/deps/rabbitmq_mqtt/BUILD.bazel b/deps/rabbitmq_mqtt/BUILD.bazel index 9a7f53ee4a4c..da617fa4891e 100644 --- a/deps/rabbitmq_mqtt/BUILD.bazel +++ b/deps/rabbitmq_mqtt/BUILD.bazel @@ -101,6 +101,16 @@ rabbitmq_test_helper( ], ) +rabbitmq_test_helper( + name = "event_recorder", + srcs = [ + "test/event_recorder.erl", + ], + deps = [ + "//deps/rabbit_common:erlang_app", + ], +) + rabbitmq_integration_suite( name = "auth_SUITE", additional_beam = [ @@ -175,6 +185,14 @@ rabbitmq_integration_suite( ], ) +rabbitmq_suite( + name = "util_SUITE", + size = "small", + data = [ + "test/rabbitmq_mqtt.app", + ], +) + rabbitmq_integration_suite( name = "ff_SUITE", runtime_deps = [ @@ -185,11 +203,13 @@ rabbitmq_integration_suite( ], ) -rabbitmq_suite( - name = "util_SUITE", - size = "small", - data = [ - "test/rabbitmq_mqtt.app", +rabbitmq_integration_suite( + name = "integration_SUITE", + runtime_deps = [ + "@emqtt//:erlang_app", + ], + additional_beam = [ + ":event_recorder", ], ) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 6a44d1c3ce84..62e87937598f 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -10,7 +10,7 @@ -export([info/2, initial_state/2, initial_state/4, process_frame/2, serialise/2, send_will/1, - terminate/1, handle_pre_hibernate/0, + terminate/2, handle_pre_hibernate/0, handle_ra_event/2, handle_down/2, handle_queue_event/2]). %%TODO Use single queue per MQTT subscriber connection? @@ -646,7 +646,7 @@ check_user_login(#{vhost := VHost, case rabbit_access_control:check_user_login( UsernameBin, AuthProps) of {ok, User = #user{username = Username}} -> - notify_auth_result(Username, user_authentication_success, []), + notify_auth_result(user_authentication_success, Username, PState), {ok, maps:put(user, User, In), PState}; {refused, Username, Msg, Args} -> rabbit_log_connection:error( @@ -654,17 +654,21 @@ check_user_login(#{vhost := VHost, "access refused for user '~s' in vhost '~s' " ++ Msg, [self(), Username, VHost] ++ Args), - notify_auth_result(Username, - user_authentication_failure, - [{error, rabbit_misc:format(Msg, Args)}]), + notify_auth_result(user_authentication_failure, Username, PState), {error, ?CONNACK_BAD_CREDENTIALS} end. -notify_auth_result(Username, AuthResult, ExtraProps) -> - EventProps = [{connection_type, mqtt}, - {name, case Username of none -> ''; _ -> Username end}] ++ - ExtraProps, - rabbit_event:notify(AuthResult, [P || {_, V} = P <- EventProps, V =/= '']). +notify_auth_result(AuthResult, Username, #proc_state{conn_name = ConnName}) -> + rabbit_event:notify( + AuthResult, + [ + {name, case Username of + none -> ''; + _ -> Username + end}, + {connection_name, ConnName}, + {connection_type, network} + ]). check_user_connection_limit(#{user := #user{username = Username}}) -> case rabbit_auth_backend_internal:is_over_connection_limit(Username) of @@ -1173,20 +1177,27 @@ serialise_and_send_to_client(Frame, #proc_state{proto_ver = ProtoVer, socket = S serialise(Frame, #proc_state{proto_ver = ProtoVer}) -> rabbit_mqtt_frame:serialise(Frame, ProtoVer). -terminate(#proc_state{client_id = undefined}) -> - ok; -terminate(PState = #proc_state{client_id = ClientId}) -> - %% ignore any errors as we are shutting down +terminate(PState, ConnName) -> + rabbit_event:notify(connection_closed, [{name, ConnName}, + {node, node()}, + {pid, self()}]), + maybe_unregister_client(PState), + maybe_delete_mqtt_qos0_queue(PState). + +maybe_unregister_client(#proc_state{client_id = ClientId}) + when ClientId =/= undefined -> case rabbit_mqtt_ff:track_client_id_in_ra() of true -> + %% ignore any errors as we are shutting down rabbit_mqtt_collector:unregister(ClientId, self()); false -> ok - end, - delete_mqtt_qos0_queue(PState). + end; +maybe_unregister_client(_) -> + ok. -delete_mqtt_qos0_queue(PState = #proc_state{clean_sess = true, - auth_state = #auth_state{username = Username}}) -> +maybe_delete_mqtt_qos0_queue(PState = #proc_state{clean_sess = true, + auth_state = #auth_state{username = Username}}) -> case get_queue(?QOS_0, PState) of {ok, Q} -> %% double check we delete the right queue @@ -1200,7 +1211,7 @@ delete_mqtt_qos0_queue(PState = #proc_state{clean_sess = true, {error, not_found} -> ok end; -delete_mqtt_qos0_queue(_) -> +maybe_delete_mqtt_qos0_queue(_) -> ok. handle_pre_hibernate() -> diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl index 8cfb582a4c48..05439158d1c7 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl @@ -62,12 +62,13 @@ declare(Q0, _Node) -> Opts = amqqueue:get_options(Q), ActingUser = maps:get(user, Opts, ?UNKNOWN_USER), rabbit_event:notify(queue_created, - [{name, amqqueue:get_name(Q0)}, - {durable, true}, - {auto_delete, false}, - {type, amqqueue:get_type(Q0)}, - {arguments, amqqueue:get_arguments(Q0)}, - {user_who_performed_action, ActingUser}]), + [{name, amqqueue:get_name(Q0)}, + {durable, true}, + {auto_delete, false}, + {exclusive, true}, + {type, amqqueue:get_type(Q0)}, + {arguments, amqqueue:get_arguments(Q0)}, + {user_who_performed_action, ActingUser}]), {new, Q}; Other -> Other diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index 9ef5c36d8749..ae9a43feb163 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -212,11 +212,12 @@ handle_info({{'DOWN', _QName}, _MRef, process, _Pid, _Reason} = Evt, handle_info(Msg, State) -> {stop, {mqtt_unexpected_msg, Msg}, State}. -terminate(Reason, State = #state{keepalive = KState0, +terminate(Reason, State = #state{conn_name = ConnName, + keepalive = KState0, proc_state = PState}) -> KState = rabbit_mqtt_keepalive:cancel_timer(KState0), maybe_emit_stats(State#state{keepalive = KState}), - rabbit_mqtt_processor:terminate(PState), + rabbit_mqtt_processor:terminate(PState, ConnName), log_terminate(Reason, State). log_terminate({network_error, {ssl_upgrade_error, closed}, ConnStr}, _State) -> diff --git a/deps/rabbitmq_mqtt/test/event_recorder.erl b/deps/rabbitmq_mqtt/test/event_recorder.erl new file mode 100644 index 000000000000..a9e219f476f1 --- /dev/null +++ b/deps/rabbitmq_mqtt/test/event_recorder.erl @@ -0,0 +1,20 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2007-2022 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(event_recorder). +-behaviour(gen_event). +-export([init/1, handle_event/2, handle_call/2]). +-define(INIT_STATE, []). + +init(_) -> + {ok, ?INIT_STATE}. + +handle_event(Event, State) -> + {ok, [Event | State]}. + +handle_call(take_state, State) -> + {ok, lists:reverse(State), ?INIT_STATE}. diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/integration_SUITE.erl new file mode 100644 index 000000000000..6aeddd0488ca --- /dev/null +++ b/deps/rabbitmq_mqtt/test/integration_SUITE.erl @@ -0,0 +1,174 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2007-2022 VMware, Inc. or its affiliates. All rights reserved. +%% +-module(integration_SUITE). +-compile([export_all, + nowarn_export_all]). + +-include_lib("common_test/include/ct.hrl"). +-include_lib("eunit/include/eunit.hrl"). +-include_lib("rabbit_common/include/rabbit.hrl"). + +all() -> + [ + {group, cluster_size_1} + ]. + +groups() -> + [ + {cluster_size_1, [], [events, + event_authentication_failure]} + ]. + +suite() -> + [{timetrap, {seconds, 60}}]. + +%% ------------------------------------------------------------------- +%% Testsuite setup/teardown. +%% ------------------------------------------------------------------- + +init_per_suite(Config) -> + rabbit_ct_helpers:log_environment(), + rabbit_ct_helpers:run_setup_steps(Config). + +end_per_suite(Config) -> + rabbit_ct_helpers:run_teardown_steps(Config). + +init_per_group(cluster_size_1, Config0) -> + Config = rabbit_ct_helpers:set_config(Config0, [{rmq_nodes_count, 1}]), + init_per_group0(Config). + +init_per_group0(Config0) -> + Config = rabbit_ct_helpers:set_config( + Config0, + [{rmq_nodename_suffix, ?MODULE}, + {rmq_extra_tcp_ports, [tcp_port_mqtt_extra, + tcp_port_mqtt_tls_extra]}]), + rabbit_ct_helpers:run_steps( + Config, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps()). + +end_per_group(_, Config) -> + rabbit_ct_helpers:run_teardown_steps( + Config, + rabbit_ct_client_helpers:teardown_steps() ++ + rabbit_ct_broker_helpers:teardown_steps()). + +init_per_testcase(Testcase, Config) -> + rabbit_ct_helpers:testcase_started(Config, Testcase). + +end_per_testcase(Testcase, Config) -> + rabbit_ct_helpers:testcase_finished(Config, Testcase). + +%% ------------------------------------------------------------------- +%% Testsuite cases +%% ------------------------------------------------------------------- + +events(Config) -> + P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), + ClientId = atom_to_binary(?FUNCTION_NAME), + {ok, C} = emqtt:start_link([{host, "localhost"}, + {port, P}, + {clientid, ClientId}, + {proto_ver, v4}]), + true = unlink(C), + + ok = rabbit_ct_broker_helpers:add_code_path_to_all_nodes(Config, event_recorder), + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + ok = gen_event:add_handler({rabbit_event, Server}, event_recorder, []), + + {ok, _Properties} = emqtt:connect(C), + + [E0, E1] = get_events(Server), + assert_event_type(user_authentication_success, E0), + assert_event_prop([{name, <<"guest">>}, + {connection_type, network}], + E0), + assert_event_type(connection_created, E1), + assert_event_prop({protocol, {'MQTT', "3.1.1"}}, E1), + + {ok, _, _} = emqtt:subscribe(C, <<"TopicA">>, qos0), + + [E2, E3] = get_events(Server), + assert_event_type(queue_created, E2), + QueueNameBin = <<"mqtt-subscription-", ClientId/binary, "qos0">>, + QueueName = {resource, <<"/">>, queue, QueueNameBin}, + assert_event_prop([{name, QueueName}, + {durable, true}, + {auto_delete, false}, + {exclusive, true}, + {type, rabbit_mqtt_qos0_queue}, + {arguments, []}], + E2), + assert_event_type(binding_created, E3), + assert_event_prop([{source_name, <<"amq.topic">>}, + {source_kind, exchange}, + {destination_name, QueueNameBin}, + {destination_kind, queue}, + {routing_key, <<"TopicA">>}, + {arguments, []}], + E3), + + {ok, _, _} = emqtt:unsubscribe(C, <<"TopicA">>), + + [E4] = get_events(Server), + assert_event_type(binding_deleted, E4), + + ok = emqtt:disconnect(C), + + [E5, E6] = get_events(Server), + assert_event_type(connection_closed, E5), + assert_event_type(queue_deleted, E6), + assert_event_prop({name, QueueName}, E6), + + ok = gen_event:delete_handler({rabbit_event, Server}, event_recorder, []). + +event_authentication_failure(Config) -> + P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), + ClientId = atom_to_binary(?FUNCTION_NAME), + {ok, C} = emqtt:start_link([{username, <<"Trudy">>}, + {password, <<"fake-password">>}, + {host, "localhost"}, + {port, P}, + {clientid, ClientId}, + {proto_ver, v4}]), + true = unlink(C), + + ok = rabbit_ct_broker_helpers:add_code_path_to_all_nodes(Config, event_recorder), + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + ok = gen_event:add_handler({rabbit_event, Server}, event_recorder, []), + + ?assertMatch({error, _}, emqtt:connect(C)), + + [E, _ConnectionClosedEvent] = get_events(Server), + assert_event_type(user_authentication_failure, E), + assert_event_prop([{name, <<"Trudy">>}, + {connection_type, network}], + E), + + ok = gen_event:delete_handler({rabbit_event, Server}, event_recorder, []). + +%% ------------------------------------------------------------------- +%% Internal helpers +%% ------------------------------------------------------------------- + +get_events(Node) -> + timer:sleep(100), %% events are sent and processed asynchronously + Result = gen_event:call({rabbit_event, Node}, event_recorder, take_state), + ?assert(is_list(Result)), + Result. + +assert_event_type(ExpectedType, #event{type = ActualType}) -> + ?assertEqual(ExpectedType, ActualType). + +assert_event_prop(ExpectedProp = {Key, _Value}, #event{props = Props}) -> + ?assertEqual(ExpectedProp, lists:keyfind(Key, 1, Props)); +assert_event_prop(ExpectedProps, Event) + when is_list(ExpectedProps) -> + lists:foreach(fun(P) -> + assert_event_prop(P, Event) + end, ExpectedProps). diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index 3c41eb632a4f..ba59869cb3f9 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -210,7 +210,7 @@ terminate(_Reason, _Request, maybe_emit_stats(State), rabbit_mqtt_keepalive:cancel_timer(KState), ok = file_handle_cache:release(), - stop_rabbit_mqtt_processor(PState). + stop_rabbit_mqtt_processor(PState, ConnName). %% Internal. @@ -272,11 +272,11 @@ stop(State, CloseCode, Error0) -> Error1 = rabbit_data_coercion:to_binary(Error0), {[{close, CloseCode, Error1}], State}. -stop_rabbit_mqtt_processor(undefined) -> +stop_rabbit_mqtt_processor(undefined, _) -> ok; -stop_rabbit_mqtt_processor(PState) -> +stop_rabbit_mqtt_processor(PState, ConnName) -> rabbit_mqtt_processor:send_will(PState), - rabbit_mqtt_processor:terminate(PState). + rabbit_mqtt_processor:terminate(PState, ConnName). handle_credits(State0) -> case control_throttle(State0) of From 8126925617386b632c30cb7a7d8e33e44a3a9f8f Mon Sep 17 00:00:00 2001 From: Chunyi Lyu Date: Fri, 28 Oct 2022 16:24:30 +0100 Subject: [PATCH 025/118] Implement format_status for mqtt reader - truncate queue type state from mqtt proc_state, which could be huge with many destination queues. Instead, format_status now returns number of destination queues. --- deps/rabbit/src/rabbit_queue_type.erl | 4 ++ .../src/rabbit_mqtt_processor.erl | 37 ++++++++++++++++++- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 35 +++++++++++++++++- 3 files changed, 74 insertions(+), 2 deletions(-) diff --git a/deps/rabbit/src/rabbit_queue_type.erl b/deps/rabbit/src/rabbit_queue_type.erl index dc5fcc5c5d10..9b05c2ac21bb 100644 --- a/deps/rabbit/src/rabbit_queue_type.erl +++ b/deps/rabbit/src/rabbit_queue_type.erl @@ -27,6 +27,7 @@ remove/2, info/2, state_info/1, + format_status/1, info_down/2, info_down/3, %% stateful client API @@ -322,6 +323,9 @@ state_info(#ctx{state = S, state_info(_) -> #{}. +format_status(#?STATE{ctxs = Ctxs}) -> + #{num_queue_clients => maps:size(Ctxs)}. + down_keys() -> ?DOWN_KEYS. info_down(Q, DownReason) -> diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 62e87937598f..10301ed86077 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -11,7 +11,7 @@ -export([info/2, initial_state/2, initial_state/4, process_frame/2, serialise/2, send_will/1, terminate/2, handle_pre_hibernate/0, - handle_ra_event/2, handle_down/2, handle_queue_event/2]). + handle_ra_event/2, handle_down/2, handle_queue_event/2, format_status/1]). %%TODO Use single queue per MQTT subscriber connection? %% * when publishing we store in x-mqtt-publish-qos header the publishing QoS @@ -1523,3 +1523,38 @@ ssl_login_name(Sock) -> {error, no_peercert} -> none; nossl -> none end. + +format_status(#proc_state{queue_states = QState, + proto_ver = ProtoVersion, + subscriptions = Sub, + unacked_client_pubs = UnackClientPubs, + unacked_server_pubs = UnackSerPubs, + packet_id = PackID, + client_id = ClientID, + clean_sess = CleanSess, + will_msg = WillMsg, + exchange = Exchange, + ssl_login_name = SSLLoginName, + retainer_pid = RetainerPid, + auth_state = AuthState, + peer_addr = PeerAddr, + register_state = RegisterState, + conn_name = ConnName, + info = Info}) -> + #{queue_states => rabbit_queue_type:format_status(QState), + proto_ver => ProtoVersion, + subscriptions => Sub, + unacked_client_pubs => UnackClientPubs, + unacked_server_pubs => UnackSerPubs, + packet_id => PackID, + client_id => ClientID, + clean_sess => CleanSess, + will_msg_defined => WillMsg =/= undefined, + exchange => Exchange, + ssl_login_name => SSLLoginName, + retainer_pid => RetainerPid, + auth_state => AuthState, + peer_addr => PeerAddr, + register_state => RegisterState, + conn_name => ConnName, + info => Info}. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index ae9a43feb163..dbb344661c8d 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -12,7 +12,7 @@ -export([start_link/3]). -export([init/1, handle_call/3, handle_cast/2, handle_info/2, - code_change/3, terminate/2]). + code_change/3, terminate/2, format_status/1]). %%TODO check where to best 'hibernate' when returning from callback %%TODO use rabbit_global_counters for MQTT protocol @@ -488,3 +488,36 @@ ssl_info(F, #state{socket = Sock, proxy_socket = ProxySock}) -> mac := H} = proplists:get_value(selected_cipher_suite, Items), F({P, {K, C, H}}) end. + +-spec format_status(map()) -> map(). +format_status(Status) -> + maps:map( + fun(state,State) -> + format_state(State); + (_,Value) -> + Value + end, Status). + +format_state(#state{proc_state = PState, + socket = Socket, + proxy_socket = ProxySock, + conn_name = ConnName, + await_recv = AwaitRecv, + deferred_recv = DeferredRecv, + received_connect_frame = ReceivedConnectFrame, + connection_state = ConnectionState, + conserve = Conserve, + stats_timer = StatsTimer, + keepalive = Keepalive, + parse_state = _}) -> + #{proc_state => rabbit_mqtt_processor:format_status(PState), + socket => Socket, + proxy_socket => ProxySock, + conn_name => ConnName, + await_recv => AwaitRecv, + deferred_recv => DeferredRecv, + received_connect_frame => ReceivedConnectFrame, + connection_state => ConnectionState, + conserve => Conserve, + stats_timer => StatsTimer, + keepalive => Keepalive}. From 33bf2150a5aeb3d0fb148cf6900fb5d4358321b4 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Fri, 28 Oct 2022 16:28:47 +0000 Subject: [PATCH 026/118] Add test for publishing via MQTT to different queue types --- deps/rabbitmq_mqtt/BUILD.bazel | 2 + .../src/rabbit_mqtt_confirms.erl | 4 +- deps/rabbitmq_mqtt/test/integration_SUITE.erl | 144 +++++++++++++++--- .../java/com/rabbitmq/mqtt/test/MqttTest.java | 2 - deps/rabbitmq_mqtt/test/reader_SUITE.erl | 31 +--- deps/rabbitmq_mqtt/test/util.erl | 23 ++- 6 files changed, 158 insertions(+), 48 deletions(-) diff --git a/deps/rabbitmq_mqtt/BUILD.bazel b/deps/rabbitmq_mqtt/BUILD.bazel index da617fa4891e..a933cf0d2b3b 100644 --- a/deps/rabbitmq_mqtt/BUILD.bazel +++ b/deps/rabbitmq_mqtt/BUILD.bazel @@ -205,11 +205,13 @@ rabbitmq_integration_suite( rabbitmq_integration_suite( name = "integration_SUITE", + size = "large", runtime_deps = [ "@emqtt//:erlang_app", ], additional_beam = [ ":event_recorder", + ":util", ], ) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl index 79ad1f22c591..c7146dca4934 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl @@ -30,10 +30,10 @@ size(State) -> maps:size(State). -spec insert(packet_id(), [queue_name()], state()) -> - {ok, state()} | {error, duplicate_packet_id}. + {ok, state()} | {error, {duplicate_packet_id, packet_id()}}. insert(PktId, _, State) when is_map_key(PktId, State) -> - {error, duplicate_packet_id}; + {error, {duplicate_packet_id, PktId}}; insert(PktId, QNames, State) when is_integer(PktId) andalso PktId > 0 -> QMap = maps:from_keys(QNames, ok), diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/integration_SUITE.erl index 6aeddd0488ca..78ec2f8e0292 100644 --- a/deps/rabbitmq_mqtt/test/integration_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/integration_SUITE.erl @@ -10,21 +10,34 @@ -include_lib("common_test/include/ct.hrl"). -include_lib("eunit/include/eunit.hrl"). --include_lib("rabbit_common/include/rabbit.hrl"). +-include_lib("amqp_client/include/amqp_client.hrl"). +-include_lib("rabbitmq_ct_helpers/include/rabbit_assert.hrl"). + +-import(rabbit_ct_broker_helpers, [rabbitmqctl_list/3]). +-import(util, [all_connection_pids/1, + publish_qos1/4]). all() -> [ - {group, cluster_size_1} + {group, cluster_size_1}, + {group, cluster_size_3} ]. groups() -> [ - {cluster_size_1, [], [events, - event_authentication_failure]} + {cluster_size_1, [], tests()}, + {cluster_size_3, [], tests()} + ]. + +tests() -> + [publish_to_all_queue_types_qos0 + ,publish_to_all_queue_types_qos1 + ,events + ,event_authentication_failure ]. suite() -> - [{timetrap, {seconds, 60}}]. + [{timetrap, {minutes, 5}}]. %% ------------------------------------------------------------------- %% Testsuite setup/teardown. @@ -37,14 +50,17 @@ init_per_suite(Config) -> end_per_suite(Config) -> rabbit_ct_helpers:run_teardown_steps(Config). -init_per_group(cluster_size_1, Config0) -> - Config = rabbit_ct_helpers:set_config(Config0, [{rmq_nodes_count, 1}]), - init_per_group0(Config). +init_per_group(cluster_size_1 = Group, Config0) -> + init_per_group0(Group, + rabbit_ct_helpers:set_config(Config0, [{rmq_nodes_count, 1}])); +init_per_group(cluster_size_3 = Group, Config0) -> + init_per_group0(Group, + rabbit_ct_helpers:set_config(Config0, [{rmq_nodes_count, 3}])). -init_per_group0(Config0) -> +init_per_group0(Group, Config0) -> Config = rabbit_ct_helpers:set_config( Config0, - [{rmq_nodename_suffix, ?MODULE}, + [{rmq_nodename_suffix, Group}, {rmq_extra_tcp_ports, [tcp_port_mqtt_extra, tcp_port_mqtt_tls_extra]}]), rabbit_ct_helpers:run_steps( @@ -68,20 +84,67 @@ end_per_testcase(Testcase, Config) -> %% Testsuite cases %% ------------------------------------------------------------------- -events(Config) -> - P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), - ClientId = atom_to_binary(?FUNCTION_NAME), - {ok, C} = emqtt:start_link([{host, "localhost"}, - {port, P}, - {clientid, ClientId}, - {proto_ver, v4}]), - true = unlink(C), +publish_to_all_queue_types_qos0(Config) -> + publish_to_all_queue_types(Config, qos0). + +publish_to_all_queue_types_qos1(Config) -> + publish_to_all_queue_types(Config, qos1). + +publish_to_all_queue_types(Config, QoS) -> + {_Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0), + + CQ = <<"classic-queue">>, + CMQ = <<"classic-mirrored-queue">>, + QQ = <<"quorum-queue">>, + SQ = <<"stream-queue">>, + Topic = <<"mytopic">>, + + declare_queue(Ch, CQ, []), + bind(Ch, CQ, Topic), + + ok = rabbit_ct_broker_helpers:set_ha_policy(Config, 0, CMQ, <<"all">>), + declare_queue(Ch, CMQ, []), + bind(Ch, CMQ, Topic), + + declare_queue(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}]), + bind(Ch, QQ, Topic), + + declare_queue(Ch, SQ, [{<<"x-queue-type">>, longstr, <<"stream">>}]), + bind(Ch, SQ, Topic), + + {C, _} = connect(?FUNCTION_NAME, Config), + lists:foreach(fun(_N) -> + case QoS of + qos0 -> + ok = emqtt:publish(C, Topic, <<"hello">>); + qos1 -> + {ok, _} = publish_qos1(C, Topic, <<"hello">>, 1000) + end + end, lists:seq(1, 2000)), + Expected = lists:sort([[CQ, <<"2000">>], + [CMQ, <<"2000">>], + [QQ, <<"2000">>], + [SQ, <<"2000">>] + ]), + ?awaitMatch(Expected, + lists:sort(rabbitmqctl_list(Config, 0, ["list_queues", "--no-table-headers", + "name", "messages_ready"])), + 20_000, 1000), + + delete_queue(Ch, [CQ, CMQ, QQ, SQ]), + ok = rabbit_ct_broker_helpers:clear_policy(Config, 0, CMQ), + ok = emqtt:disconnect(C), + ?awaitMatch([], all_connection_pids(Config), 10_000, 1000). + +%%TODO add test where target quorum queue rejects message +events(Config) -> ok = rabbit_ct_broker_helpers:add_code_path_to_all_nodes(Config, event_recorder), Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), ok = gen_event:add_handler({rabbit_event, Server}, event_recorder, []), - {ok, _Properties} = emqtt:connect(C), + ClientId = atom_to_binary(?FUNCTION_NAME), + {C, _} = connect(ClientId, Config), [E0, E1] = get_events(Server), assert_event_type(user_authentication_success, E0), @@ -156,6 +219,49 @@ event_authentication_failure(Config) -> %% Internal helpers %% ------------------------------------------------------------------- +connect(ClientId, Config) -> + connect(ClientId, Config, []). + +connect(ClientId, Config, AdditionalOpts) -> + P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), + Options = [{host, "localhost"}, + {port, P}, + {clientid, rabbit_data_coercion:to_binary(ClientId)}, + {proto_ver, v4} + ] ++ AdditionalOpts, + {ok, C} = emqtt:start_link(Options), + {ok, _Properties} = emqtt:connect(C), + true = unlink(C), + MRef = monitor(process, C), + {C, MRef}. + +declare_queue(Ch, QueueName, Args) + when is_pid(Ch), is_binary(QueueName), is_list(Args) -> + #'queue.declare_ok'{} = amqp_channel:call( + Ch, #'queue.declare'{ + queue = QueueName, + durable = true, + arguments = Args}). + +delete_queue(Ch, QueueNames) + when is_pid(Ch), is_list(QueueNames) -> + lists:foreach( + fun(Q) -> + delete_queue(Ch, Q) + end, QueueNames); +delete_queue(Ch, QueueName) + when is_pid(Ch), is_binary(QueueName) -> + #'queue.delete_ok'{} = amqp_channel:call( + Ch, #'queue.delete'{ + queue = QueueName}). + +bind(Ch, QueueName, Topic) + when is_pid(Ch), is_binary(QueueName), is_binary(Topic) -> + #'queue.bind_ok'{} = amqp_channel:call( + Ch, #'queue.bind'{queue = QueueName, + exchange = <<"amq.topic">>, + routing_key = Topic}). + get_events(Node) -> timer:sleep(100), %% events are sent and processed asynchronously Result = gen_event:call({rabbit_event, Node}, event_recorder, take_state), diff --git a/deps/rabbitmq_mqtt/test/java_SUITE_data/src/test/java/com/rabbitmq/mqtt/test/MqttTest.java b/deps/rabbitmq_mqtt/test/java_SUITE_data/src/test/java/com/rabbitmq/mqtt/test/MqttTest.java index c7ac646cfae6..f31497d1c705 100644 --- a/deps/rabbitmq_mqtt/test/java_SUITE_data/src/test/java/com/rabbitmq/mqtt/test/MqttTest.java +++ b/deps/rabbitmq_mqtt/test/java_SUITE_data/src/test/java/com/rabbitmq/mqtt/test/MqttTest.java @@ -34,8 +34,6 @@ import static org.junit.jupiter.api.Assertions.*; -//TODO Add quorum queue test - /*** * MQTT v3.1 tests * diff --git a/deps/rabbitmq_mqtt/test/reader_SUITE.erl b/deps/rabbitmq_mqtt/test/reader_SUITE.erl index 7577c0ac9134..8de3fa28d74b 100644 --- a/deps/rabbitmq_mqtt/test/reader_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/reader_SUITE.erl @@ -12,6 +12,8 @@ -include_lib("eunit/include/eunit.hrl"). -import(rabbit_ct_broker_helpers, [rpc/5]). +-import(util, [all_connection_pids/1, + publish_qos1/4]). all() -> [ @@ -99,15 +101,15 @@ block(Config) -> {ok, _, _} = emqtt:unsubscribe(C, <<"TopicA">>), {ok, _, _} = emqtt:subscribe(C, <<"Topic1">>), - {ok, _} = publish_qos1(C, <<"Topic1">>, <<"Not blocked yet">>), + {ok, _} = publish_qos1(C, <<"Topic1">>, <<"Not blocked yet">>, 1000), ok = rpc(Config, vm_memory_monitor, set_vm_memory_high_watermark, [0.00000001]), % %% Let it block timer:sleep(100), %% Blocked, but still will publish - puback_timeout = publish_qos1(C, <<"Topic1">>, <<"Now blocked">>), - puback_timeout = publish_qos1(C, <<"Topic1">>, <<"Still blocked">>), + puback_timeout = publish_qos1(C, <<"Topic1">>, <<"Now blocked">>, 1000), + puback_timeout = publish_qos1(C, <<"Topic1">>, <<"Still blocked">>, 1000), %% Unblock rpc(Config, vm_memory_monitor, set_vm_memory_high_watermark, [0.4]), @@ -158,7 +160,7 @@ block_connack_timeout(Config) -> ct:fail("missing peername_not_known from server") end, %% Ensure that our client is not registered. - ?assertEqual([], util:all_connection_pids(Config)), + ?assertEqual([], all_connection_pids(Config)), ok. handle_invalid_frames(Config) -> @@ -229,7 +231,7 @@ stats(Config) -> %% Wait for stats being emitted (every 100ms) timer:sleep(300), %% Retrieve the connection Pid - [Reader] = util:all_connection_pids(Config), + [Reader] = all_connection_pids(Config), [{_, Pid}] = rpc(Config, rabbit_mqtt_reader, info, [Reader, [connection]]), %% Verify the content of the metrics, garbage_collection must be present [{Pid, Props}] = rpc(Config, ets, lookup, [connection_metrics, Pid]), @@ -351,22 +353,3 @@ expect_publishes(Topic, [Payload|Rest]) -> rpc(Config, M, F, A) -> rabbit_ct_broker_helpers:rpc(Config, 0, M, F, A). - -publish_qos1(Client, Topic, Payload) -> - Mref = erlang:monitor(process, Client), - ok = emqtt:publish_async(Client, Topic, #{}, Payload, [{qos, 1}], infinity, - {fun ?MODULE:sync_publish_result/3, [self(), Mref]}), - receive - {Mref, Reply} -> - erlang:demonitor(Mref, [flush]), - Reply; - {'DOWN', Mref, process, Client, Reason} -> - ct:fail("client is down: ~tp", [Reason]) - after - 1000 -> - erlang:demonitor(Mref, [flush]), - puback_timeout - end. - -sync_publish_result(Caller, Mref, Result) -> - erlang:send(Caller, {Mref, Result}). diff --git a/deps/rabbitmq_mqtt/test/util.erl b/deps/rabbitmq_mqtt/test/util.erl index 80d33584cb25..4a3034f1c871 100644 --- a/deps/rabbitmq_mqtt/test/util.erl +++ b/deps/rabbitmq_mqtt/test/util.erl @@ -1,6 +1,8 @@ -module(util). --export([all_connection_pids/1]). +-export([all_connection_pids/1, + publish_qos1/4, + sync_publish_result/3]). all_connection_pids(Config) -> Nodes = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), @@ -10,3 +12,22 @@ all_connection_pids(Config) -> (_, Acc) -> Acc end, [], Result). + +publish_qos1(Client, Topic, Payload, Timeout) -> + Mref = erlang:monitor(process, Client), + ok = emqtt:publish_async(Client, Topic, #{}, Payload, [{qos, 1}], infinity, + {fun ?MODULE:sync_publish_result/3, [self(), Mref]}), + receive + {Mref, Reply} -> + erlang:demonitor(Mref, [flush]), + Reply; + {'DOWN', Mref, process, Client, Reason} -> + ct:fail("client is down: ~tp", [Reason]) + after + Timeout -> + erlang:demonitor(Mref, [flush]), + puback_timeout + end. + +sync_publish_result(Caller, Mref, Result) -> + erlang:send(Caller, {Mref, Result}). From 816fedf080e547bb2724778b9b4093a3ac894db1 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Tue, 1 Nov 2022 17:47:40 +0100 Subject: [PATCH 027/118] Enable flow control to target classic queue --- deps/rabbit_common/src/credit_flow.erl | 5 ++- deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl | 4 ++- .../src/rabbit_mqtt_processor.erl | 13 +++++-- deps/rabbitmq_mqtt/test/integration_SUITE.erl | 34 ++++++++++++++----- .../src/rabbit_web_mqtt_handler.erl | 4 +++ 5 files changed, 44 insertions(+), 16 deletions(-) diff --git a/deps/rabbit_common/src/credit_flow.erl b/deps/rabbit_common/src/credit_flow.erl index 33c345315b74..b77b5116267b 100644 --- a/deps/rabbit_common/src/credit_flow.erl +++ b/deps/rabbit_common/src/credit_flow.erl @@ -88,9 +88,8 @@ end). %% If current process was blocked by credit flow in the last -%% STATE_CHANGE_INTERVAL milliseconds, state/0 will report it as "in -%% flow". --define(STATE_CHANGE_INTERVAL, 1000000). +%% STATE_CHANGE_INTERVAL microseconds, state/0 will report it as "in flow". +-define(STATE_CHANGE_INTERVAL, 1_000_000). -ifdef(CREDIT_FLOW_TRACING). -define(TRACE_BLOCKED(SELF, FROM), rabbit_event:notify(credit_flow_blocked, diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl index a2b8638c7498..f4112984e493 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl @@ -55,7 +55,9 @@ amqp2mqtt_fun, register_state, conn_name, - info}). + info, + delivery_flow :: flow | noflow + }). -type proc_state() :: #proc_state{}. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 10301ed86077..9678b87da76e 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -43,13 +43,18 @@ initial_state(Socket, ConnectionName) -> initial_state(Socket, ConnectionName, SendFun, PeerAddr) -> {ok, {mqtt2amqp_fun, M2A}, {amqp2mqtt_fun, A2M}} = rabbit_mqtt_util:get_topic_translation_funs(), + Flow = case rabbit_misc:get_env(rabbit, mirroring_flow_control, true) of + true -> flow; + false -> noflow + end, #proc_state{socket = Socket, conn_name = ConnectionName, ssl_login_name = ssl_login_name(Socket), peer_addr = PeerAddr, send_fun = SendFun, mqtt2amqp_fun = M2A, - amqp2mqtt_fun = A2M}. + amqp2mqtt_fun = A2M, + delivery_flow = Flow}. process_frame(#mqtt_frame{fixed = #mqtt_frame_fixed{type = Type}}, PState = #proc_state{auth_state = undefined}) @@ -1061,7 +1066,8 @@ publish_to_queues( message_id = MessageId, payload = Payload}, #proc_state{exchange = ExchangeName, - mqtt2amqp_fun = Mqtt2AmqpFun} = PState) -> + mqtt2amqp_fun = Mqtt2AmqpFun, + delivery_flow = Flow} = PState) -> RoutingKey = Mqtt2AmqpFun(Topic), Confirm = Qos > ?QOS_0, Headers = [{<<"x-mqtt-publish-qos">>, byte, Qos}, @@ -1090,7 +1096,7 @@ publish_to_queues( sender = self(), message = BasicMessage, msg_seq_no = MessageId, - flow = noflow %%TODO enable flow control + flow = Flow }, case rabbit_exchange:lookup(ExchangeName) of {ok, Exchange} -> @@ -1253,6 +1259,7 @@ handle_ra_event(Evt, PState) -> handle_down({{'DOWN', QName}, _MRef, process, QPid, Reason}, PState0 = #proc_state{queue_states = QStates0}) -> + credit_flow:peer_down(QPid), case rabbit_queue_type:handle_down(QPid, QName, Reason, QStates0) of {ok, QStates1, Actions} -> PState = PState0#proc_state{queue_states = QStates1}, diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/integration_SUITE.erl index 78ec2f8e0292..237b3ccef247 100644 --- a/deps/rabbitmq_mqtt/test/integration_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/integration_SUITE.erl @@ -13,7 +13,8 @@ -include_lib("amqp_client/include/amqp_client.hrl"). -include_lib("rabbitmq_ct_helpers/include/rabbit_assert.hrl"). --import(rabbit_ct_broker_helpers, [rabbitmqctl_list/3]). +-import(rabbit_ct_broker_helpers, [rabbitmqctl_list/3, + rpc_all/4]). -import(util, [all_connection_pids/1, publish_qos1/4]). @@ -91,6 +92,13 @@ publish_to_all_queue_types_qos1(Config) -> publish_to_all_queue_types(Config, qos1). publish_to_all_queue_types(Config, QoS) -> + %% Give only 1/10 of the default credits. + %% We want to test whether sending many messages work when MQTT connection sometimes gets blocked. + Result = rpc_all(Config, application, set_env, [rabbit, credit_flow_default_credit, {40, 20}]), + Result = rpc_all(Config, application, set_env, [rabbit, quorum_commands_soft_limit, 3]), + Result = rpc_all(Config, application, set_env, [rabbit, stream_messages_soft_limit, 25]), + ?assert(lists:all(fun(R) -> R =:= ok end, Result)), + {_Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0), CQ = <<"classic-queue">>, @@ -112,19 +120,22 @@ publish_to_all_queue_types(Config, QoS) -> declare_queue(Ch, SQ, [{<<"x-queue-type">>, longstr, <<"stream">>}]), bind(Ch, SQ, Topic), + NumMsgs = 2000, + NumMsgsBin = integer_to_binary(NumMsgs), {C, _} = connect(?FUNCTION_NAME, Config), lists:foreach(fun(_N) -> case QoS of qos0 -> - ok = emqtt:publish(C, Topic, <<"hello">>); + ok = emqtt:publish(C, Topic, <<"m">>); qos1 -> - {ok, _} = publish_qos1(C, Topic, <<"hello">>, 1000) + {ok, _} = publish_qos1(C, Topic, <<"m">>, 1000) end - end, lists:seq(1, 2000)), - Expected = lists:sort([[CQ, <<"2000">>], - [CMQ, <<"2000">>], - [QQ, <<"2000">>], - [SQ, <<"2000">>] + end, lists:seq(1, NumMsgs)), + + Expected = lists:sort([[CQ, NumMsgsBin], + [CMQ, NumMsgsBin], + [QQ, NumMsgsBin], + [SQ, NumMsgsBin] ]), ?awaitMatch(Expected, lists:sort(rabbitmqctl_list(Config, 0, ["list_queues", "--no-table-headers", @@ -134,7 +145,12 @@ publish_to_all_queue_types(Config, QoS) -> delete_queue(Ch, [CQ, CMQ, QQ, SQ]), ok = rabbit_ct_broker_helpers:clear_policy(Config, 0, CMQ), ok = emqtt:disconnect(C), - ?awaitMatch([], all_connection_pids(Config), 10_000, 1000). + ?awaitMatch([], all_connection_pids(Config), 10_000, 1000), + + Result = rpc_all(Config, application, unset_env, [rabbit, credit_flow_default_credit]), + Result = rpc_all(Config, application, unset_env, [rabbit, quorum_commands_soft_limit]), + Result = rpc_all(Config, application, unset_env, [rabbit, stream_messages_soft_limit]), + ok. %%TODO add test where target quorum queue rejects message diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index ba59869cb3f9..4addb54c3f4b 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -197,6 +197,10 @@ websocket_info({ra_event, _From, Evt}, #state{proc_state = PState0} = State) -> PState = rabbit_mqtt_processor:handle_ra_event(Evt, PState0), {[], State#state{proc_state = PState}, hibernate}; +websocket_info({{'DOWN', _QName}, _MRef, process, _Pid, _Reason} = Evt, + State = #state{proc_state = PState0}) -> + PState = rabbit_mqtt_processor:handle_down(Evt, PState0), + {[], State#state{proc_state = PState}, hibernate}; websocket_info(Msg, State) -> rabbit_log_connection:warning("Web MQTT: unexpected message ~p", [Msg]), {[], State, hibernate}. From 14f59f138035a9801b925565fef6bbea8d03f746 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Tue, 1 Nov 2022 20:52:40 +0100 Subject: [PATCH 028/118] Handle soft limit exceeded as queue action Instead of performing credit_flow within quorum queue and stream queue clients, return new {block | unblock, QueueName} actions. The queue client process can then decide what to do. For example, the channel continues to use credit_flow such that the channel gets blocked sending any more credits to rabbit_reader. However, the MQTT connection process does not use credit_flow. It instead blocks its reader directly. --- deps/rabbit/src/rabbit_channel.erl | 27 ++++--- deps/rabbit/src/rabbit_fifo_client.erl | 63 +++++---------- deps/rabbit/src/rabbit_fifo_dlx_worker.erl | 4 + deps/rabbit/src/rabbit_queue_type.erl | 3 +- deps/rabbit/src/rabbit_quorum_queue.erl | 18 ++--- deps/rabbit/src/rabbit_stream_queue.erl | 37 +++++---- deps/rabbit/test/rabbit_fifo_int_SUITE.erl | 77 ++++++++++--------- deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl | 4 +- .../src/rabbit_mqtt_processor.erl | 18 ++++- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 24 +++--- .../src/rabbit_web_mqtt_handler.erl | 20 +++-- 11 files changed, 148 insertions(+), 147 deletions(-) diff --git a/deps/rabbit/src/rabbit_channel.erl b/deps/rabbit/src/rabbit_channel.erl index c6d342442bf8..f138b8b9f735 100644 --- a/deps/rabbit/src/rabbit_channel.erl +++ b/deps/rabbit/src/rabbit_channel.erl @@ -2846,22 +2846,27 @@ handle_queue_actions(Actions, #ch{} = State0) -> confirm(MsgSeqNos, QRef, S0); ({rejected, _QRef, MsgSeqNos}, S0) -> {U, Rej} = - lists:foldr( - fun(SeqNo, {U1, Acc}) -> - case rabbit_confirms:reject(SeqNo, U1) of - {ok, MX, U2} -> - {U2, [MX | Acc]}; - {error, not_found} -> - {U1, Acc} - end - end, {S0#ch.unconfirmed, []}, MsgSeqNos), + lists:foldr( + fun(SeqNo, {U1, Acc}) -> + case rabbit_confirms:reject(SeqNo, U1) of + {ok, MX, U2} -> + {U2, [MX | Acc]}; + {error, not_found} -> + {U1, Acc} + end + end, {S0#ch.unconfirmed, []}, MsgSeqNos), S = S0#ch{unconfirmed = U}, record_rejects(Rej, S); ({deliver, CTag, AckRequired, Msgs}, S0) -> handle_deliver(CTag, AckRequired, Msgs, S0); ({queue_down, QRef}, S0) -> - handle_consuming_queue_down_or_eol(QRef, S0) - + handle_consuming_queue_down_or_eol(QRef, S0); + ({block, QName}, S0) -> + credit_flow:block(QName), + S0; + ({unblock, QName}, S0) -> + credit_flow:unblock(QName), + S0 end, State0, Actions). maybe_increase_global_publishers(#ch{publishing_mode = true} = State0) -> diff --git a/deps/rabbit/src/rabbit_fifo_client.erl b/deps/rabbit/src/rabbit_fifo_client.erl index c00d33b5326f..bb4a1980d4ab 100644 --- a/deps/rabbit/src/rabbit_fifo_client.erl +++ b/deps/rabbit/src/rabbit_fifo_client.erl @@ -14,7 +14,6 @@ -export([ init/2, init/3, - init/5, checkout/5, cancel_checkout/2, enqueue/2, @@ -44,7 +43,8 @@ -type seq() :: non_neg_integer(). -type action() :: {send_credit_reply, Available :: non_neg_integer()} | {send_drained, CTagCredit :: - {rabbit_fifo:consumer_tag(), non_neg_integer()}}. + {rabbit_fifo:consumer_tag(), non_neg_integer()}} | + rabbit_queue_type:action(). -type actions() :: [action()]. -type queue_name() :: rabbit_types:r(queue). @@ -56,8 +56,6 @@ -record(cfg, {queue_name :: queue_name(), servers = [] :: [ra:server_id()], soft_limit = ?SOFT_LIMIT :: non_neg_integer(), - block_handler = fun() -> ok end :: fun(() -> term()), - unblock_handler = fun() -> ok end :: fun(() -> ok), timeout :: non_neg_integer(), version = 0 :: non_neg_integer()}). @@ -108,33 +106,18 @@ init(QueueName = #resource{}, Servers, SoftLimit) -> soft_limit = SoftLimit, timeout = Timeout * 1000}}. --spec init(queue_name(), [ra:server_id()], non_neg_integer(), fun(() -> ok), - fun(() -> ok)) -> state(). -init(QueueName = #resource{}, Servers, SoftLimit, BlockFun, UnblockFun) -> - %% net ticktime is in seconds - Timeout = application:get_env(kernel, net_ticktime, 60) + 5, - #state{cfg = #cfg{queue_name = QueueName, - servers = Servers, - block_handler = BlockFun, - unblock_handler = UnblockFun, - soft_limit = SoftLimit, - timeout = Timeout * 1000}}. - - %% @doc Enqueues a message. %% @param Correlation an arbitrary erlang term used to correlate this %% command when it has been applied. %% @param Msg an arbitrary erlang term representing the message. %% @param State the current {@module} state. %% @returns -%% `{ok | slow, State}' if the command was successfully sent. If the return -%% tag is `slow' it means the limit is approaching and it is time to slow down -%% the sending rate. +%% `{ok, State, Actions}' if the command was successfully sent %% {@module} assigns a sequence number to every raft command it issues. The %% SequenceNumber can be correlated to the applied sequence numbers returned %% by the {@link handle_ra_event/2. handle_ra_event/2} function. -spec enqueue(Correlation :: term(), Msg :: term(), State :: state()) -> - {ok | slow | reject_publish, state()}. + {ok, state(), actions()} | {reject_publish, state()}. enqueue(Correlation, Msg, #state{queue_status = undefined, next_enqueue_seq = 1, @@ -182,8 +165,7 @@ enqueue(Correlation, Msg, queue_status = go, next_seq = Seq, next_enqueue_seq = EnqueueSeq, - cfg = #cfg{soft_limit = SftLmt, - block_handler = BlockFun}} = State0) -> + cfg = #cfg{soft_limit = SftLmt}} = State0) -> Server = pick_server(State0), % by default there is no correlation id Cmd = rabbit_fifo:make_enqueue(self(), EnqueueSeq, Msg), @@ -198,25 +180,22 @@ enqueue(Correlation, Msg, slow = Tag == slow}, case Tag of slow when not Slow -> - BlockFun(), - {slow, set_timer(State)}; + {ok, set_timer(State), [{block, cluster_name(State)}]}; _ -> - {ok, State} + {ok, State, []} end. %% @doc Enqueues a message. %% @param Msg an arbitrary erlang term representing the message. %% @param State the current {@module} state. %% @returns -%% `{ok | slow, State}' if the command was successfully sent. If the return -%% tag is `slow' it means the limit is approaching and it is time to slow down -%% the sending rate. +%% `{ok, State, Actions}' if the command was successfully sent. %% {@module} assigns a sequence number to every raft command it issues. The %% SequenceNumber can be correlated to the applied sequence numbers returned %% by the {@link handle_ra_event/2. handle_ra_event/2} function. %% -spec enqueue(Msg :: term(), State :: state()) -> - {ok | slow | reject_publish, state()}. + {ok, state(), actions()} | {reject_publish, state()}. enqueue(Msg, State) -> enqueue(undefined, Msg, State). @@ -274,10 +253,6 @@ add_delivery_count_header(Msg, _Count) -> %% @param ConsumerTag the tag uniquely identifying the consumer. %% @param MsgIds the message ids received with the {@link rabbit_fifo:delivery/0.} %% @param State the {@module} state -%% @returns -%% `{ok | slow, State}' if the command was successfully sent. If the return -%% tag is `slow' it means the limit is approaching and it is time to slow down -%% the sending rate. %% -spec settle(rabbit_fifo:consumer_tag(), [rabbit_fifo:msg_id()], state()) -> {state(), list()}. @@ -305,9 +280,7 @@ settle(ConsumerTag, [_|_] = MsgIds, %% from {@link rabbit_fifo:delivery/0.} %% @param State the {@module} state %% @returns -%% `{State, list()}' if the command was successfully sent. If the return -%% tag is `slow' it means the limit is approaching and it is time to slow down -%% the sending rate. +%% `{State, list()}' if the command was successfully sent. %% -spec return(rabbit_fifo:consumer_tag(), [rabbit_fifo:msg_id()], state()) -> {state(), list()}. @@ -334,10 +307,6 @@ return(ConsumerTag, [_|_] = MsgIds, %% @param MsgIds the message ids to discard %% from {@link rabbit_fifo:delivery/0.} %% @param State the {@module} state -%% @returns -%% `{ok | slow, State}' if the command was successfully sent. If the return -%% tag is `slow' it means the limit is approaching and it is time to slow down -%% the sending rate. -spec discard(rabbit_fifo:consumer_tag(), [rabbit_fifo:msg_id()], state()) -> {state(), list()}. discard(ConsumerTag, [_|_] = MsgIds, #state{slow = false} = State0) -> @@ -562,8 +531,8 @@ update_machine_state(Server, Conf) -> {rabbit_fifo:client_msg(), state()} | eol. handle_ra_event(From, {applied, Seqs}, #state{cfg = #cfg{queue_name = QRef, - soft_limit = SftLmt, - unblock_handler = UnblockFun}} = State0) -> + soft_limit = SftLmt + }} = State0) -> {Corrs, Actions0, State1} = lists:foldl(fun seq_applied/2, {[], [], State0#state{leader = From}}, @@ -600,8 +569,7 @@ handle_ra_event(From, {applied, Seqs}, send_command(Node, undefined, C, normal, S0) end, State2, Commands), - UnblockFun(), - {ok, State, Actions}; + {ok, State, [{unblock, cluster_name(State)} | Actions]}; _ -> {ok, State1, Actions} end; @@ -950,3 +918,8 @@ find_leader([Server | Servers]) -> qref({Ref, _}) -> Ref; qref(Ref) -> Ref. + +-spec cluster_name(state()) -> + atom(). +cluster_name(#state{cfg = #cfg{servers = [{Name, _Node} | _]}}) -> + Name. diff --git a/deps/rabbit/src/rabbit_fifo_dlx_worker.erl b/deps/rabbit/src/rabbit_fifo_dlx_worker.erl index 9e46c0996e40..ff61bbe5e5c1 100644 --- a/deps/rabbit/src/rabbit_fifo_dlx_worker.erl +++ b/deps/rabbit/src/rabbit_fifo_dlx_worker.erl @@ -247,6 +247,10 @@ handle_queue_actions(Actions, State0) -> handle_rejected(QRef, MsgSeqs, S0); ({queue_down, _QRef}, S0) -> %% target classic queue is down, but not deleted + S0; + ({block, _QName}, S0) -> + S0; + ({unblock, _QName}, S0) -> S0 end, State0, Actions). diff --git a/deps/rabbit/src/rabbit_queue_type.erl b/deps/rabbit/src/rabbit_queue_type.erl index 9b05c2ac21bb..d61f8db41444 100644 --- a/deps/rabbit/src/rabbit_queue_type.erl +++ b/deps/rabbit/src/rabbit_queue_type.erl @@ -69,7 +69,8 @@ %% indicate to the queue type module that a message has been delivered %% fully to the queue {settled, Success :: boolean(), [msg_tag()]} | - {deliver, rabbit_types:ctag(), boolean(), [rabbit_amqqueue:qmsg()]}. + {deliver, rabbit_types:ctag(), boolean(), [rabbit_amqqueue:qmsg()]} | + {block | unblock, QueueName :: term()}. -type actions() :: [action()]. diff --git a/deps/rabbit/src/rabbit_quorum_queue.erl b/deps/rabbit/src/rabbit_quorum_queue.erl index 9f30bacde5e5..a61c64fdfe9f 100644 --- a/deps/rabbit/src/rabbit_quorum_queue.erl +++ b/deps/rabbit/src/rabbit_quorum_queue.erl @@ -25,7 +25,7 @@ -export([settle/4, dequeue/4, consume/3, cancel/5]). -export([credit/4]). -export([purge/1]). --export([stateless_deliver/2, deliver/3, deliver/2]). +-export([stateless_deliver/2, deliver/2]). -export([dead_letter_publish/5]). -export([queue_name/1]). -export([cluster_state/1, status/2]). @@ -137,9 +137,7 @@ init(Q) when ?is_amqqueue(Q) -> %% Ensure the leader is listed first Servers0 = [{Name, N} || N <- Nodes], Servers = [Leader | lists:delete(Leader, Servers0)], - {ok, rabbit_fifo_client:init(QName, Servers, SoftLimit, - fun() -> credit_flow:block(Name) end, - fun() -> credit_flow:unblock(Name), ok end)}. + {ok, rabbit_fifo_client:init(QName, Servers, SoftLimit)}. -spec close(rabbit_fifo_client:state()) -> ok. close(_State) -> @@ -866,14 +864,14 @@ stateless_deliver(ServerId, Delivery) -> -spec deliver(Confirm :: boolean(), rabbit_types:delivery(), rabbit_fifo_client:state()) -> - {ok | slow, rabbit_fifo_client:state()} | + {ok, rabbit_fifo_client:state(), rabbit_queue_type:actions()} | {reject_publish, rabbit_fifo_client:state()}. deliver(false, Delivery, QState0) -> case rabbit_fifo_client:enqueue(Delivery#delivery.message, QState0) of - {ok, _} = Res -> Res; - {slow, _} = Res -> Res; + {ok, _State, _Actions} = Res -> + Res; {reject_publish, State} -> - {ok, State} + {ok, State, []} end; deliver(true, Delivery, QState0) -> rabbit_fifo_client:enqueue(Delivery#delivery.msg_seq_no, @@ -896,8 +894,8 @@ deliver(QSs, #delivery{message = #basic_message{content = Content0} = Msg, Seq = Delivery#delivery.msg_seq_no, QName = rabbit_fifo_client:queue_name(S), {[{Q, S} | Qs], [{rejected, QName, [Seq]} | Actions]}; - {_, S} -> - {[{Q, S} | Qs], Actions} + {ok, S, As} -> + {[{Q, S} | Qs], As ++ Actions} end end, {[], []}, QSs). diff --git a/deps/rabbit/src/rabbit_stream_queue.erl b/deps/rabbit/src/rabbit_stream_queue.erl index ec7e5d713fc0..9afb918493b1 100644 --- a/deps/rabbit/src/rabbit_stream_queue.erl +++ b/deps/rabbit/src/rabbit_stream_queue.erl @@ -378,18 +378,18 @@ deliver(QSs, #delivery{confirm = Confirm} = Delivery) -> %% [QRef], Delivery#delivery.message), {Qs, Actions}; ({Q, S0}, {Qs, Actions}) -> - S = deliver(Confirm, Delivery, S0), - {[{Q, S} | Qs], Actions} + {S, As} = deliver(Confirm, Delivery, S0), + {[{Q, S} | Qs], As ++ Actions} end, {[], []}, QSs). deliver(_Confirm, #delivery{message = Msg, msg_seq_no = MsgId}, - #stream_client{name = Name, - leader = LeaderPid, - writer_id = WriterId, - next_seq = Seq, - correlation = Correlation0, - soft_limit = SftLmt, - slow = Slow0} = State) -> + #stream_client{name = Name, + leader = LeaderPid, + writer_id = WriterId, + next_seq = Seq, + correlation = Correlation0, + soft_limit = SftLmt, + slow = Slow0} = State) -> ok = osiris:write(LeaderPid, WriterId, Seq, msg_to_iodata(Msg)), Correlation = case MsgId of undefined -> @@ -397,16 +397,15 @@ deliver(_Confirm, #delivery{message = Msg, msg_seq_no = MsgId}, _ when is_number(MsgId) -> Correlation0#{Seq => {MsgId, Msg}} end, - Slow = case maps:size(Correlation) >= SftLmt of - true when not Slow0 -> - credit_flow:block(Name), - true; - Bool -> - Bool - end, - State#stream_client{next_seq = Seq + 1, - correlation = Correlation, - slow = Slow}. + {Slow, Actions} = case maps:size(Correlation) >= SftLmt of + true when not Slow0 -> + {true, [{block, Name}]}; + Bool -> + {Bool, []} + end, + {State#stream_client{next_seq = Seq + 1, + correlation = Correlation, + slow = Slow}, Actions}. -spec dequeue(_, _, _, client()) -> no_return(). dequeue(_, _, _, #stream_client{name = Name}) -> diff --git a/deps/rabbit/test/rabbit_fifo_int_SUITE.erl b/deps/rabbit/test/rabbit_fifo_int_SUITE.erl index 74881791f4eb..756d7fc2ace6 100644 --- a/deps/rabbit/test/rabbit_fifo_int_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_int_SUITE.erl @@ -97,7 +97,7 @@ basics(Config) -> % create segment the segment will trigger a snapshot timer:sleep(1000), - {ok, FState2} = rabbit_fifo_client:enqueue(one, FState1), + {ok, FState2, []} = rabbit_fifo_client:enqueue(one, FState1), DeliverFun = fun DeliverFun(S0, F) -> receive @@ -135,7 +135,7 @@ basics(Config) -> exit(leader_change_timeout) end, - {ok, FState6} = rabbit_fifo_client:enqueue(two, FState5b), + {ok, FState6, []} = rabbit_fifo_client:enqueue(two, FState5b), _FState8 = DeliverFun(FState6, return), rabbit_quorum_queue:stop_server(ServerId), @@ -147,8 +147,8 @@ return(Config) -> ok = start_cluster(ClusterName, [ServerId]), F00 = rabbit_fifo_client:init(ClusterName, [ServerId]), - {ok, F0} = rabbit_fifo_client:enqueue(1, msg1, F00), - {ok, F1} = rabbit_fifo_client:enqueue(2, msg2, F0), + {ok, F0, []} = rabbit_fifo_client:enqueue(1, msg1, F00), + {ok, F1, []} = rabbit_fifo_client:enqueue(2, msg2, F0), {_, _, F2} = process_ra_events(receive_ra_events(2, 0), F1), {ok, _, {_, _, MsgId, _, _}, F} = rabbit_fifo_client:dequeue(<<"tag">>, unsettled, F2), _F2 = rabbit_fifo_client:return(<<"tag">>, [MsgId], F), @@ -161,7 +161,7 @@ rabbit_fifo_returns_correlation(Config) -> ServerId = ?config(node_id, Config), ok = start_cluster(ClusterName, [ServerId]), F0 = rabbit_fifo_client:init(ClusterName, [ServerId]), - {ok, F1} = rabbit_fifo_client:enqueue(corr1, msg1, F0), + {ok, F1, []} = rabbit_fifo_client:enqueue(corr1, msg1, F0), receive {ra_event, Frm, E} -> case rabbit_fifo_client:handle_ra_event(Frm, E, F1) of @@ -182,7 +182,7 @@ duplicate_delivery(Config) -> ok = start_cluster(ClusterName, [ServerId]), F0 = rabbit_fifo_client:init(ClusterName, [ServerId]), {ok, F1} = rabbit_fifo_client:checkout(<<"tag">>, 10, simple_prefetch, #{}, F0), - {ok, F2} = rabbit_fifo_client:enqueue(corr1, msg1, F1), + {ok, F2, []} = rabbit_fifo_client:enqueue(corr1, msg1, F1), Fun = fun Loop(S0) -> receive {ra_event, Frm, E} = Evt -> @@ -217,8 +217,8 @@ usage(Config) -> ok = start_cluster(ClusterName, [ServerId]), F0 = rabbit_fifo_client:init(ClusterName, [ServerId]), {ok, F1} = rabbit_fifo_client:checkout(<<"tag">>, 10, simple_prefetch, #{}, F0), - {ok, F2} = rabbit_fifo_client:enqueue(corr1, msg1, F1), - {ok, F3} = rabbit_fifo_client:enqueue(corr2, msg2, F2), + {ok, F2, []} = rabbit_fifo_client:enqueue(corr1, msg1, F1), + {ok, F3, []} = rabbit_fifo_client:enqueue(corr2, msg2, F2), {_, _, _} = process_ra_events(receive_ra_events(2, 2), F3), % force tick and usage stats emission ServerId ! tick_timeout, @@ -236,12 +236,12 @@ resends_lost_command(Config) -> ok = meck:new(ra, [passthrough]), F0 = rabbit_fifo_client:init(ClusterName, [ServerId]), - {ok, F1} = rabbit_fifo_client:enqueue(msg1, F0), + {ok, F1, []} = rabbit_fifo_client:enqueue(msg1, F0), % lose the enqueue meck:expect(ra, pipeline_command, fun (_, _, _) -> ok end), - {ok, F2} = rabbit_fifo_client:enqueue(msg2, F1), + {ok, F2, []} = rabbit_fifo_client:enqueue(msg2, F1), meck:unload(ra), - {ok, F3} = rabbit_fifo_client:enqueue(msg3, F2), + {ok, F3, []} = rabbit_fifo_client:enqueue(msg3, F2), {_, _, F4} = process_ra_events(receive_ra_events(2, 0), F3), {ok, _, {_, _, _, _, msg1}, F5} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F4), {ok, _, {_, _, _, _, msg2}, F6} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F5), @@ -256,7 +256,7 @@ two_quick_enqueues(Config) -> F0 = rabbit_fifo_client:init(ClusterName, [ServerId]), F1 = element(2, rabbit_fifo_client:enqueue(msg1, F0)), - {ok, F2} = rabbit_fifo_client:enqueue(msg2, F1), + {ok, F2, []} = rabbit_fifo_client:enqueue(msg2, F1), _ = process_ra_events(receive_ra_events(2, 0), F2), rabbit_quorum_queue:stop_server(ServerId), ok. @@ -267,11 +267,11 @@ detects_lost_delivery(Config) -> ok = start_cluster(ClusterName, [ServerId]), F000 = rabbit_fifo_client:init(ClusterName, [ServerId]), - {ok, F00} = rabbit_fifo_client:enqueue(msg1, F000), + {ok, F00, []} = rabbit_fifo_client:enqueue(msg1, F000), {_, _, F0} = process_ra_events(receive_ra_events(1, 0), F00), {ok, F1} = rabbit_fifo_client:checkout(<<"tag">>, 10, simple_prefetch, #{}, F0), - {ok, F2} = rabbit_fifo_client:enqueue(msg2, F1), - {ok, F3} = rabbit_fifo_client:enqueue(msg3, F2), + {ok, F2, []} = rabbit_fifo_client:enqueue(msg2, F1), + {ok, F3, []} = rabbit_fifo_client:enqueue(msg3, F2), % lose first delivery receive {ra_event, _, {machine, {delivery, _, [{_, {_, msg1}}]}}} -> @@ -291,7 +291,7 @@ returns_after_down(Config) -> ok = start_cluster(ClusterName, [ServerId]), F0 = rabbit_fifo_client:init(ClusterName, [ServerId]), - {ok, F1} = rabbit_fifo_client:enqueue(msg1, F0), + {ok, F1, []} = rabbit_fifo_client:enqueue(msg1, F0), {_, _, F2} = process_ra_events(receive_ra_events(1, 0), F1), % start a consumer in a separate processes % that exits after checkout @@ -316,9 +316,9 @@ resends_after_lost_applied(Config) -> ok = start_cluster(ClusterName, [ServerId]), F0 = rabbit_fifo_client:init(ClusterName, [ServerId]), - {ok, F1} = rabbit_fifo_client:enqueue(msg1, F0), + {ok, F1, []} = rabbit_fifo_client:enqueue(msg1, F0), {_, _, F2} = process_ra_events(receive_ra_events(1, 0), F1), - {ok, F3} = rabbit_fifo_client:enqueue(msg2, F2), + {ok, F3, []} = rabbit_fifo_client:enqueue(msg2, F2), % lose an applied event receive {ra_event, _, {applied, _}} -> @@ -327,7 +327,7 @@ resends_after_lost_applied(Config) -> exit(await_ra_event_timeout) end, % send another message - {ok, F4} = rabbit_fifo_client:enqueue(msg3, F3), + {ok, F4, []} = rabbit_fifo_client:enqueue(msg3, F3), {_, _, F5} = process_ra_events(receive_ra_events(1, 0), F4), {ok, _, {_, _, _, _, msg1}, F6} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F5), {ok, _, {_, _, _, _, msg2}, F7} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F6), @@ -350,7 +350,7 @@ handles_reject_notification(Config) -> #{})), % reverse order - should try the first node in the list first F0 = rabbit_fifo_client:init(ClusterName, [ServerId2, ServerId1]), - {ok, F1} = rabbit_fifo_client:enqueue(one, F0), + {ok, F1, []} = rabbit_fifo_client:enqueue(one, F0), timer:sleep(500), @@ -381,7 +381,7 @@ discard(Config) -> F0 = rabbit_fifo_client:init(ClusterName, [ServerId]), {ok, F1} = rabbit_fifo_client:checkout(<<"tag">>, 10, simple_prefetch, #{}, F0), - {ok, F2} = rabbit_fifo_client:enqueue(msg1, F1), + {ok, F2, []} = rabbit_fifo_client:enqueue(msg1, F1), F3 = discard_next_delivery(F2, 5000), {empty, _F4} = rabbit_fifo_client:dequeue(<<"tag1">>, settled, F3), receive @@ -401,7 +401,7 @@ cancel_checkout(Config) -> ServerId = ?config(node_id, Config), ok = start_cluster(ClusterName, [ServerId]), F0 = rabbit_fifo_client:init(ClusterName, [ServerId], 4), - {ok, F1} = rabbit_fifo_client:enqueue(m1, F0), + {ok, F1, []} = rabbit_fifo_client:enqueue(m1, F0), {ok, F2} = rabbit_fifo_client:checkout(<<"tag">>, 10, simple_prefetch, #{}, F1), {_, _, F3} = process_ra_events(receive_ra_events(1, 1), F2, [], [], fun (_, S) -> S end), {ok, F4} = rabbit_fifo_client:cancel_checkout(<<"tag">>, F3), @@ -414,7 +414,7 @@ lost_delivery(Config) -> ServerId = ?config(node_id, Config), ok = start_cluster(ClusterName, [ServerId]), F0 = rabbit_fifo_client:init(ClusterName, [ServerId], 4), - {ok, F1} = rabbit_fifo_client:enqueue(m1, F0), + {ok, F1, []} = rabbit_fifo_client:enqueue(m1, F0), {_, _, F2} = process_ra_events( receive_ra_events(1, 0), F1, [], [], fun (_, S) -> S end), {ok, F3} = rabbit_fifo_client:checkout(<<"tag">>, 10, simple_prefetch, #{}, F2), @@ -427,7 +427,7 @@ lost_delivery(Config) -> exit(await_ra_event_timeout) end, % send another message - {ok, F4} = rabbit_fifo_client:enqueue(m2, F3), + {ok, F4, []} = rabbit_fifo_client:enqueue(m2, F3), %% this hsould trigger the fifo client to fetch any missing messages %% from the server {_, _, _F5} = process_ra_events( @@ -446,8 +446,8 @@ credit(Config) -> ServerId = ?config(node_id, Config), ok = start_cluster(ClusterName, [ServerId]), F0 = rabbit_fifo_client:init(ClusterName, [ServerId], 4), - {ok, F1} = rabbit_fifo_client:enqueue(m1, F0), - {ok, F2} = rabbit_fifo_client:enqueue(m2, F1), + {ok, F1, []} = rabbit_fifo_client:enqueue(m1, F0), + {ok, F2, []} = rabbit_fifo_client:enqueue(m2, F1), {_, _, F3} = process_ra_events(receive_ra_events(2, 0), F2), %% checkout with 0 prefetch {ok, F4} = rabbit_fifo_client:checkout(<<"tag">>, 0, credited, #{}, F3), @@ -469,7 +469,7 @@ credit(Config) -> %% enqueue another message - at this point the consumer credit should be %% all used up due to the drain - {ok, F10} = rabbit_fifo_client:enqueue(m3, F9), + {ok, F10, []} = rabbit_fifo_client:enqueue(m3, F9), %% assert no deliveries {_, _, F11} = process_ra_events(receive_ra_events(), F10, [], [], fun @@ -495,15 +495,16 @@ untracked_enqueue(Config) -> flow(Config) -> ClusterName = ?config(cluster_name, Config), - ServerId = ?config(node_id, Config), + {Name, _Node} = ServerId = ?config(node_id, Config), ok = start_cluster(ClusterName, [ServerId]), F0 = rabbit_fifo_client:init(ClusterName, [ServerId], 3), - {ok, F1} = rabbit_fifo_client:enqueue(m1, F0), - {ok, F2} = rabbit_fifo_client:enqueue(m2, F1), - {ok, F3} = rabbit_fifo_client:enqueue(m3, F2), - {slow, F4} = rabbit_fifo_client:enqueue(m4, F3), - {_, _, F5} = process_ra_events(receive_ra_events(4, 0), F4), - {ok, _} = rabbit_fifo_client:enqueue(m5, F5), + {ok, F1, []} = rabbit_fifo_client:enqueue(m1, F0), + {ok, F2, []} = rabbit_fifo_client:enqueue(m2, F1), + {ok, F3, []} = rabbit_fifo_client:enqueue(m3, F2), + {ok, F4, [{block, Name}]} = rabbit_fifo_client:enqueue(m4, F3), + {_, Actions, F5} = process_ra_events(receive_ra_events(4, 0), F4), + true = lists:member({unblock, Name}, Actions), + {ok, _, []} = rabbit_fifo_client:enqueue(m5, F5), rabbit_quorum_queue:stop_server(ServerId), ok. @@ -515,8 +516,8 @@ test_queries(Config) -> Self = self(), P = spawn(fun () -> F0 = rabbit_fifo_client:init(ClusterName, [ServerId], 4), - {ok, F1} = rabbit_fifo_client:enqueue(m1, F0), - {ok, F2} = rabbit_fifo_client:enqueue(m2, F1), + {ok, F1, []} = rabbit_fifo_client:enqueue(m1, F0), + {ok, F2, []} = rabbit_fifo_client:enqueue(m2, F1), process_ra_events(receive_ra_events(2, 0), F2), Self ! ready, receive stop -> ok end @@ -552,12 +553,12 @@ dequeue(Config) -> ok = start_cluster(ClusterName, [ServerId]), F1 = rabbit_fifo_client:init(ClusterName, [ServerId]), {empty, F1b} = rabbit_fifo_client:dequeue(Tag, settled, F1), - {ok, F2_} = rabbit_fifo_client:enqueue(msg1, F1b), + {ok, F2_, []} = rabbit_fifo_client:enqueue(msg1, F1b), {_, _, F2} = process_ra_events(receive_ra_events(1, 0), F2_), % {ok, {{0, {_, msg1}}, _}, F3} = rabbit_fifo_client:dequeue(Tag, settled, F2), {ok, _, {_, _, 0, _, msg1}, F3} = rabbit_fifo_client:dequeue(Tag, settled, F2), - {ok, F4_} = rabbit_fifo_client:enqueue(msg2, F3), + {ok, F4_, []} = rabbit_fifo_client:enqueue(msg2, F3), {_, _, F4} = process_ra_events(receive_ra_events(1, 0), F4_), {ok, _, {_, _, MsgId, _, msg2}, F5} = rabbit_fifo_client:dequeue(Tag, unsettled, F4), {_F6, _A} = rabbit_fifo_client:settle(Tag, [MsgId], F5), diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl index f4112984e493..b27518d87bb4 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl @@ -56,7 +56,9 @@ register_state, conn_name, info, - delivery_flow :: flow | noflow + delivery_flow :: flow | noflow, + %% quorum queues and streams whose soft limit has been exceeded + soft_limit_exceeded = sets:new([{version, 2}]) :: sets:set() }). -type proc_state() :: #proc_state{}. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 9678b87da76e..f19b1b7180b3 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -11,7 +11,8 @@ -export([info/2, initial_state/2, initial_state/4, process_frame/2, serialise/2, send_will/1, terminate/2, handle_pre_hibernate/0, - handle_ra_event/2, handle_down/2, handle_queue_event/2, format_status/1]). + handle_ra_event/2, handle_down/2, handle_queue_event/2, + soft_limit_exceeded/1, format_status/1]). %%TODO Use single queue per MQTT subscriber connection? %% * when publishing we store in x-mqtt-publish-qos header the publishing QoS @@ -1309,7 +1310,11 @@ handle_queue_actions(Actions, #proc_state{} = PState0) -> {error, not_found} -> Acc0 end end, U0, MsgIds), - S#proc_state{unacked_client_pubs = U} + S#proc_state{unacked_client_pubs = U}; + ({block, QName}, S = #proc_state{soft_limit_exceeded = SLE}) -> + S#proc_state{soft_limit_exceeded = sets:add_element(QName, SLE)}; + ({unblock, QName}, S = #proc_state{soft_limit_exceeded = SLE}) -> + S#proc_state{soft_limit_exceeded = sets:del_element(QName, SLE)} end, PState0, Actions). deliver_to_client(Msgs, Ack, PState) -> @@ -1547,7 +1552,8 @@ format_status(#proc_state{queue_states = QState, peer_addr = PeerAddr, register_state = RegisterState, conn_name = ConnName, - info = Info}) -> + info = Info + } = PState) -> #{queue_states => rabbit_queue_type:format_status(QState), proto_ver => ProtoVersion, subscriptions => Sub, @@ -1564,4 +1570,8 @@ format_status(#proc_state{queue_states = QState, peer_addr => PeerAddr, register_state => RegisterState, conn_name => ConnName, - info => Info}. + info => Info, + soft_limit_exceeded => soft_limit_exceeded(PState)}. + +soft_limit_exceeded(#proc_state{soft_limit_exceeded = SLE}) -> + not sets:is_empty(SLE). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index dbb344661c8d..021e570ce9ce 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -125,8 +125,13 @@ handle_cast({close_connection, Reason}, {stop, {shutdown, server_initiated_close}, State}; handle_cast(QueueEvent = {queue_event, _, _}, - State = #state{proc_state = PState}) -> - callback_reply(State, rabbit_mqtt_processor:handle_queue_event(QueueEvent, PState)); + State = #state{proc_state = PState0}) -> + case rabbit_mqtt_processor:handle_queue_event(QueueEvent, PState0) of + {ok, PState} -> + maybe_process_deferred_recv(control_throttle(pstate(State, PState))); + {error, Reason, PState} -> + {stop, Reason, pstate(State, PState)} + end; handle_cast(Msg, State) -> {stop, {mqtt_unexpected_cast, Msg}, State}. @@ -207,7 +212,7 @@ handle_info({ra_event, _From, Evt}, handle_info({{'DOWN', _QName}, _MRef, process, _Pid, _Reason} = Evt, #state{proc_state = PState0} = State) -> PState = rabbit_mqtt_processor:handle_down(Evt, PState0), - {noreply, pstate(State, PState), ?HIBERNATE_AFTER}; + maybe_process_deferred_recv(control_throttle(pstate(State, PState))); handle_info(Msg, State) -> {stop, {mqtt_unexpected_msg, Msg}, State}. @@ -339,11 +344,6 @@ process_received_bytes(Bytes, {stop, {shutdown, Error}, State} end. -callback_reply(State, {ok, ProcState}) -> - {noreply, pstate(State, ProcState), ?HIBERNATE_AFTER}; -callback_reply(State, {error, Reason, ProcState}) -> - {stop, Reason, pstate(State, ProcState)}. - pstate(State = #state {}, PState = #proc_state{}) -> State #state{ proc_state = PState }. @@ -399,8 +399,12 @@ run_socket(State = #state{ socket = Sock }) -> control_throttle(State = #state{connection_state = Flow, conserve = Conserve, - keepalive = KState}) -> - case {Flow, Conserve orelse credit_flow:blocked()} of + keepalive = KState, + proc_state = PState}) -> + Throttle = Conserve orelse + rabbit_mqtt_processor:soft_limit_exceeded(PState) orelse + credit_flow:blocked(), + case {Flow, Throttle} of {running, true} -> State#state{connection_state = blocked, keepalive = rabbit_mqtt_keepalive:cancel_timer(KState)}; diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index 4addb54c3f4b..f61c59c0ad04 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -148,11 +148,10 @@ websocket_handle(Frame, State) -> {cowboy_websocket:commands(), State, hibernate}. websocket_info({conserve_resources, Conserve}, State) -> NewState = State#state{conserve_resources = Conserve}, - handle_credits(control_throttle(NewState)); + handle_credits(NewState); websocket_info({bump_credit, Msg}, State) -> credit_flow:handle_bump_msg(Msg), - handle_credits(control_throttle(State)); - %%TODO return hibernate? + handle_credits(State); websocket_info({reply, Data}, State) -> {[{binary, Data}], State, hibernate}; websocket_info({'EXIT', _, _}, State) -> @@ -161,7 +160,7 @@ websocket_info({'$gen_cast', QueueEvent = {queue_event, _, _}}, State = #state{proc_state = PState0}) -> case rabbit_mqtt_processor:handle_queue_event(QueueEvent, PState0) of {ok, PState} -> - {[], State#state{proc_state = PState}, hibernate}; + handle_credits(State#state{proc_state = PState}); {error, Reason, PState} -> rabbit_log_connection:error("Web MQTT connection ~p failed to handle queue event: ~p", [State#state.conn_name, Reason]), @@ -200,7 +199,7 @@ websocket_info({ra_event, _From, Evt}, websocket_info({{'DOWN', _QName}, _MRef, process, _Pid, _Reason} = Evt, State = #state{proc_state = PState0}) -> PState = rabbit_mqtt_processor:handle_down(Evt, PState0), - {[], State#state{proc_state = PState}, hibernate}; + handle_credits(State#state{proc_state = PState}); websocket_info(Msg, State) -> rabbit_log_connection:warning("Web MQTT: unexpected message ~p", [Msg]), {[], State, hibernate}. @@ -283,6 +282,7 @@ stop_rabbit_mqtt_processor(PState, ConnName) -> rabbit_mqtt_processor:terminate(PState, ConnName). handle_credits(State0) -> + %%TODO return hibernate? case control_throttle(State0) of State = #state{state = running} -> {[{active, true}], State}; @@ -291,9 +291,13 @@ handle_credits(State0) -> end. control_throttle(State = #state{state = CS, - conserve_resources = Mem, - keepalive = KState}) -> - case {CS, Mem orelse credit_flow:blocked()} of + conserve_resources = Conserve, + keepalive = KState, + proc_state = PState}) -> + Throttle = Conserve orelse + rabbit_mqtt_processor:soft_limit_exceeded(PState) orelse + credit_flow:blocked(), + case {CS, Throttle} of {running, true} -> State#state{state = blocked, keepalive = rabbit_mqtt_keepalive:cancel_timer(KState)}; From 645531bc95941d62fc132a20275c8033a1744e36 Mon Sep 17 00:00:00 2001 From: Chunyi Lyu Date: Wed, 2 Nov 2022 17:52:31 +0000 Subject: [PATCH 029/118] Register mqtt connections in case event refresh --- deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl | 2 ++ deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 6 ------ 2 files changed, 2 insertions(+), 6 deletions(-) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index f19b1b7180b3..d0118a4d9609 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -412,6 +412,7 @@ notify_connection_created( ], rabbit_core_metrics:connection_created(self(), Infos), rabbit_event:notify(connection_created, Infos), + rabbit_networking:register_non_amqp_connection(self()), {ok, PState#proc_state{ %% We won't need conn_name anymore. Use less memmory by setting to undefined. conn_name = undefined}}. @@ -1188,6 +1189,7 @@ terminate(PState, ConnName) -> rabbit_event:notify(connection_closed, [{name, ConnName}, {node, node()}, {pid, self()}]), + rabbit_networking:unregister_non_amqp_connection(self()), maybe_unregister_client(PState), maybe_delete_mqtt_qos0_queue(PState). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index 021e570ce9ce..e70b2a73b354 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -62,12 +62,6 @@ init(Ref) -> LoginTimeout = application:get_env(rabbitmq_mqtt, login_timeout, 10_000), erlang:send_after(LoginTimeout, self(), login_timeout), ProcessorState = rabbit_mqtt_processor:initial_state(RealSocket, ConnStr), - %%TODO call rabbit_networking:register_non_amqp_connection/1 so that we are notified - %% when need to force load the 'connection_created' event for the management plugin, see - %% https://github.com/rabbitmq/rabbitmq-management-agent/issues/58 - %% https://github.com/rabbitmq/rabbitmq-server/blob/90cc0e2abf944141feedaf3190d7b6d8b4741b11/deps/rabbitmq_stream/src/rabbit_stream_reader.erl#L536 - %% https://github.com/rabbitmq/rabbitmq-server/blob/90cc0e2abf944141feedaf3190d7b6d8b4741b11/deps/rabbitmq_stream/src/rabbit_stream_reader.erl#L189 - %% https://github.com/rabbitmq/rabbitmq-server/blob/7eb4084cf879fe6b1d26dd3c837bd180cb3a8546/deps/rabbitmq_management_agent/src/rabbit_mgmt_db_handler.erl#L72 gen_server:enter_loop(?MODULE, [], rabbit_event:init_stats_timer( control_throttle( From 1925862b1e26ad230650b8bd0f9ef6a3a0a820e0 Mon Sep 17 00:00:00 2001 From: Chunyi Lyu Date: Thu, 3 Nov 2022 11:25:35 +0000 Subject: [PATCH 030/118] Test drain closing connections in web mqtt - web mqtt connections are tracked by rabbitmq_mqtt and will be closed by mqtt event handler at node drain. --- deps/rabbitmq_web_mqtt/test/system_SUITE.erl | 24 ++++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl index 0bf59e9c3263..a5bc97e5dc24 100644 --- a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl +++ b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl @@ -27,6 +27,7 @@ groups() -> , last_will_disabled , disconnect , keepalive + , maintenance ]} ]. @@ -256,6 +257,24 @@ keepalive(Config) -> ?assertEqual({close, {1000, <<"MQTT keepalive timeout">>}}, rfc6455_client:recv(WS, ceil(3 * 0.75 * KeepaliveMs))). +maintenance(Config) -> + Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt), + PortStr = integer_to_list(Port), + WS = rfc6455_client:new("ws://127.0.0.1:" ++ PortStr ++ "/ws", self()), + {ok, _} = rfc6455_client:open(WS), + ok = raw_send(WS, + ?CONNECT_PACKET(#mqtt_packet_connect{ + clean_sess = true, + client_id = <<"node-drain-test">>, + username = <<"guest">>, + password = <<"guest">>})), + + ?assertEqual(1, num_mqtt_connections(Config, 0)), + ok = rabbit_ct_broker_helpers:drain_node(Config, 0), + + ?assertEqual(0, num_mqtt_connections(Config, 0)), + ok = rabbit_ct_broker_helpers:revive_node(Config, 0). + raw_send(WS, Packet) -> Frame = emqttc_serialiser:serialise(Packet), rfc6455_client:send_binary(WS, Frame). @@ -270,3 +289,8 @@ raw_recv(WS, Timeout) -> {error, timeout} -> {error, timeout} end. + +%% Web mqtt connections are tracked together with mqtt connections +num_mqtt_connections(Config, Node) -> + length(rabbit_ct_broker_helpers:rpc( + Config, Node, rabbit_mqtt,local_connection_pids,[])). \ No newline at end of file From df491d3489c5306d83fab354479f5c0deac3aab2 Mon Sep 17 00:00:00 2001 From: Chunyi Lyu Date: Fri, 4 Nov 2022 13:40:56 +0000 Subject: [PATCH 031/118] Use eqmtt client in web mqtt tests --- deps/rabbitmq_web_mqtt/BUILD.bazel | 4 + deps/rabbitmq_web_mqtt/Makefile | 6 +- deps/rabbitmq_web_mqtt/test/system_SUITE.erl | 116 +++++++++---------- 3 files changed, 64 insertions(+), 62 deletions(-) diff --git a/deps/rabbitmq_web_mqtt/BUILD.bazel b/deps/rabbitmq_web_mqtt/BUILD.bazel index 7210fd0ec120..4f83efffa1de 100644 --- a/deps/rabbitmq_web_mqtt/BUILD.bazel +++ b/deps/rabbitmq_web_mqtt/BUILD.bazel @@ -90,6 +90,10 @@ rabbitmq_integration_suite( name = "system_SUITE", additional_beam = [":test_utils"], additional_hdrs = glob(["test/src/*.hrl"]), + runtime_deps = [ + "@emqtt//:erlang_app", + "@gun//:erlang_app", + ], ) assert_suites() diff --git a/deps/rabbitmq_web_mqtt/Makefile b/deps/rabbitmq_web_mqtt/Makefile index e31f164954af..27d82360fb72 100644 --- a/deps/rabbitmq_web_mqtt/Makefile +++ b/deps/rabbitmq_web_mqtt/Makefile @@ -13,8 +13,12 @@ define PROJECT_ENV ] endef +# We do not need QUIC as dependency of emqtt. +BUILD_WITHOUT_QUIC=1 +export BUILD_WITHOUT_QUIC + DEPS = rabbit_common rabbit cowboy rabbitmq_mqtt -TEST_DEPS = rabbitmq_ct_helpers rabbitmq_ct_client_helpers +TEST_DEPS = emqtt rabbitmq_ct_helpers rabbitmq_ct_client_helpers # FIXME: Add Ranch as a BUILD_DEPS to be sure the correct version is picked. # See rabbitmq-components.mk. diff --git a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl index a5bc97e5dc24..98ea25c91901 100644 --- a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl +++ b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl @@ -62,73 +62,67 @@ end_per_testcase(Testcase, Config) -> connection(Config) -> Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt), - PortStr = integer_to_list(Port), - WS = rfc6455_client:new("ws://127.0.0.1:" ++ PortStr ++ "/ws", self()), - {ok, _} = rfc6455_client:open(WS), - {close, _} = rfc6455_client:close(WS), - ok. + {ok, C} = emqtt:start_link([{host, "127.0.0.1"}, + {username, "guest"}, + {password, "guest"}, + {ws_path, "/ws"}, + {port, Port}]), + {ok, _} = emqtt:ws_connect(C), + ok = emqtt:disconnect(C). pubsub_shared_connection(Config) -> Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt), - PortStr = integer_to_list(Port), - WS = rfc6455_client:new("ws://127.0.0.1:" ++ PortStr ++ "/ws", self()), - {ok, _} = rfc6455_client:open(WS), - ok = raw_send(WS, - ?CONNECT_PACKET(#mqtt_packet_connect{ - clean_sess = true, - client_id = <<"web-mqtt-tests-pubsub">>, - username = <<"guest">>, - password = <<"guest">>})), - - {ok, ?CONNACK_PACKET(?CONNACK_ACCEPT), _} = raw_recv(WS), - + {ok, C} = emqtt:start_link([{host, "127.0.0.1"}, + {username, "guest"}, + {password, "guest"}, + {ws_path, "/ws"}, + {client_id, ?FUNCTION_NAME}, + {clean_start, true}, + {port, Port}]), + {ok, _} = emqtt:ws_connect(C), Dst = <<"/topic/test-web-mqtt">>, - - ok = raw_send(WS, ?SUBSCRIBE_PACKET(1, [{Dst, ?QOS_1}])), - {ok, ?SUBACK_PACKET(_, _), _} = raw_recv(WS), + {ok, _, [1]} = emqtt:subscribe(C, Dst, qos1), Payload = <<"a\x00a">>, + {ok, PubReply} = emqtt:publish(C, Dst, Payload, [{qos, 1}]), + ?assertMatch(#{packet_id := _, + reason_code := 0, + reason_code_name := success + }, PubReply), - ok = raw_send(WS, ?PUBLISH_PACKET(?QOS_1, Dst, 2, Payload)), - {ok, ?PUBLISH_PACKET(_, Dst, _, Payload), _} = raw_recv(WS), - - {close, _} = rfc6455_client:close(WS), - ok. + ok = emqtt:disconnect(C). pubsub_separate_connections(Config) -> Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt), - PortStr = integer_to_list(Port), - WS1 = rfc6455_client:new("ws://127.0.0.1:" ++ PortStr ++ "/ws", self()), - {ok, _} = rfc6455_client:open(WS1), - ok = raw_send(WS1, - ?CONNECT_PACKET(#mqtt_packet_connect{ - clean_sess = true, - client_id = <<"web-mqtt-tests-publisher">>, - username = <<"guest">>, - password = <<"guest">>})), - {ok, ?CONNACK_PACKET(?CONNACK_ACCEPT), _} = raw_recv(WS1), - - WS2 = rfc6455_client:new("ws://127.0.0.1:" ++ PortStr ++ "/ws", self()), - {ok, _} = rfc6455_client:open(WS2), - ok = raw_send(WS2, - ?CONNECT_PACKET(#mqtt_packet_connect{ - clean_sess = true, - client_id = <<"web-mqtt-tests-consumer">>, - username = <<"guest">>, - password = <<"guest">>})), - {ok, ?CONNACK_PACKET(?CONNACK_ACCEPT), _} = raw_recv(WS2), + {ok, C1} = emqtt:start_link([{host, "127.0.0.1"}, + {username, "guest"}, + {password, "guest"}, + {ws_path, "/ws"}, + {client_id, "web-mqtt-tests-consumer"}, + {clean_start, true}, + {port, Port}]), + {ok, _} = emqtt:ws_connect(C1), + {ok, C2} = emqtt:start_link([{host, "127.0.0.1"}, + {username, "guest"}, + {password, "guest"}, + {ws_path, "/ws"}, + {client_id, "web-mqtt-tests-consumer"}, + {clean_start, true}, + {port, Port}]), + {ok, _} = emqtt:ws_connect(C2), Dst = <<"/topic/test-web-mqtt">>, - ok = raw_send(WS2, ?SUBSCRIBE_PACKET(1, [{Dst, ?QOS_1}])), - {ok, ?SUBACK_PACKET(_, _), _} = raw_recv(WS2), + {ok, _, [1]} = emqtt:subscribe(C2, Dst, qos1), Payload = <<"a\x00a">>, - ok = raw_send(WS1, ?PUBLISH_PACKET(?QOS_1, Dst, 2, Payload)), - {ok, ?PUBLISH_PACKET(_, Dst, _, Payload), _} = raw_recv(WS2), + {ok, PubReply} = emqtt:publish(C1, Dst, Payload, [{qos, 1}]), + ?assertMatch(#{packet_id := _, + reason_code := 0, + reason_code_name := success + }, PubReply), - {close, _} = rfc6455_client:close(WS1), - {close, _} = rfc6455_client:close(WS2), - ok. + ok = emqtt:disconnect(C1), + ok = emqtt:disconnect(C2). last_will_enabled(Config) -> Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt), @@ -259,15 +253,15 @@ keepalive(Config) -> maintenance(Config) -> Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt), - PortStr = integer_to_list(Port), - WS = rfc6455_client:new("ws://127.0.0.1:" ++ PortStr ++ "/ws", self()), - {ok, _} = rfc6455_client:open(WS), - ok = raw_send(WS, - ?CONNECT_PACKET(#mqtt_packet_connect{ - clean_sess = true, - client_id = <<"node-drain-test">>, - username = <<"guest">>, - password = <<"guest">>})), + {ok, C} = emqtt:start_link([{host, "127.0.0.1"}, + {username, "guest"}, + {password, "guest"}, + {ws_path, "/ws"}, + {client_id, "node-drain-test"}, + {clean_start, true}, + {port, Port}]), + {ok, _} = emqtt:ws_connect(C), + unlink(C), ?assertEqual(1, num_mqtt_connections(Config, 0)), ok = rabbit_ct_broker_helpers:drain_node(Config, 0), From 4c15299196825a585298f78e09e0a259d1bd62d6 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Mon, 7 Nov 2022 11:26:12 +0000 Subject: [PATCH 032/118] Delete old emqttc client Instead use latest emqtt client for Web MQTT tests. --- .../src/rabbit_ct_broker_helpers.erl | 4 + deps/rabbitmq_mqtt/test/reader_SUITE.erl | 2 +- deps/rabbitmq_web_mqtt/BUILD.bazel | 3 - .../src/rabbit_web_mqtt_handler.erl | 4 +- .../test/proxy_protocol_SUITE.erl | 16 +- .../test/src/emqttc_packet.hrl | 240 -------------- .../test/src/emqttc_parser.erl | 215 ------------- .../test/src/emqttc_serialiser.erl | 157 --------- .../test/src/rabbit_ws_test_util.erl | 31 +- deps/rabbitmq_web_mqtt/test/system_SUITE.erl | 297 ++++++++---------- 10 files changed, 166 insertions(+), 803 deletions(-) delete mode 100644 deps/rabbitmq_web_mqtt/test/src/emqttc_packet.hrl delete mode 100644 deps/rabbitmq_web_mqtt/test/src/emqttc_parser.erl delete mode 100644 deps/rabbitmq_web_mqtt/test/src/emqttc_serialiser.erl diff --git a/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl b/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl index 7c957c80d4a1..142a048099eb 100644 --- a/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl +++ b/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl @@ -189,6 +189,7 @@ tcp_port_mqtt, tcp_port_mqtt_tls, tcp_port_web_mqtt, + tcp_port_web_mqtt_tls, tcp_port_stomp, tcp_port_stomp_tls, tcp_port_web_stomp, @@ -513,6 +514,9 @@ update_tcp_ports_in_rmq_config(NodeConfig, [tcp_port_mqtt_tls = Key | Rest]) -> NodeConfig1 = rabbit_ct_helpers:merge_app_env(NodeConfig, {rabbitmq_mqtt, [{ssl_listeners, [?config(Key, NodeConfig)]}]}), update_tcp_ports_in_rmq_config(NodeConfig1, Rest); +update_tcp_ports_in_rmq_config(NodeConfig, [tcp_port_web_mqtt_tls | Rest]) -> + %% Skip this one, because we need more than just a port to configure + update_tcp_ports_in_rmq_config(NodeConfig, Rest); update_tcp_ports_in_rmq_config(NodeConfig, [tcp_port_web_mqtt = Key | Rest]) -> NodeConfig1 = rabbit_ct_helpers:merge_app_env(NodeConfig, {rabbitmq_web_mqtt, [{tcp_config, [{port, ?config(Key, NodeConfig)}]}]}), diff --git a/deps/rabbitmq_mqtt/test/reader_SUITE.erl b/deps/rabbitmq_mqtt/test/reader_SUITE.erl index 8de3fa28d74b..e69f68fb59dc 100644 --- a/deps/rabbitmq_mqtt/test/reader_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/reader_SUITE.erl @@ -348,7 +348,7 @@ expect_publishes(Topic, [Payload|Rest]) -> payload := Payload}} -> expect_publishes(Topic, Rest) after 5000 -> - throw({publish_not_delivered, Payload}) + throw({publish_not_received, Payload}) end. rpc(Config, M, F, A) -> diff --git a/deps/rabbitmq_web_mqtt/BUILD.bazel b/deps/rabbitmq_web_mqtt/BUILD.bazel index 4f83efffa1de..c6fcea47136f 100644 --- a/deps/rabbitmq_web_mqtt/BUILD.bazel +++ b/deps/rabbitmq_web_mqtt/BUILD.bazel @@ -69,7 +69,6 @@ broker_for_integration_suites() rabbitmq_test_helper( name = "test_utils", srcs = glob(["test/src/*.erl"]), - hdrs = glob(["test/src/*.hrl"]), ) rabbitmq_integration_suite( @@ -79,7 +78,6 @@ rabbitmq_integration_suite( rabbitmq_integration_suite( name = "proxy_protocol_SUITE", additional_beam = [":test_utils"], - additional_hdrs = glob(["test/src/*.hrl"]), erlc_opts = [ "-I", "deps/rabbitmq_web_mqtt/test", @@ -89,7 +87,6 @@ rabbitmq_integration_suite( rabbitmq_integration_suite( name = "system_SUITE", additional_beam = [":test_utils"], - additional_hdrs = glob(["test/src/*.hrl"]), runtime_deps = [ "@emqtt//:erlang_app", "@gun//:erlang_app", diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index f61c59c0ad04..02b7d660378e 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -272,8 +272,8 @@ stop(State) -> stop(State, 1000, "MQTT died"). stop(State, CloseCode, Error0) -> - Error1 = rabbit_data_coercion:to_binary(Error0), - {[{close, CloseCode, Error1}], State}. + Error = rabbit_data_coercion:to_binary(Error0), + {[{close, CloseCode, Error}], State}. stop_rabbit_mqtt_processor(undefined, _) -> ok; diff --git a/deps/rabbitmq_web_mqtt/test/proxy_protocol_SUITE.erl b/deps/rabbitmq_web_mqtt/test/proxy_protocol_SUITE.erl index bd49cec10d8b..1750eba3cce6 100644 --- a/deps/rabbitmq_web_mqtt/test/proxy_protocol_SUITE.erl +++ b/deps/rabbitmq_web_mqtt/test/proxy_protocol_SUITE.erl @@ -8,11 +8,10 @@ -module(proxy_protocol_SUITE). --compile(export_all). +-compile([export_all, nowarn_export_all]). -include_lib("common_test/include/ct.hrl"). -include_lib("eunit/include/eunit.hrl"). --include("src/emqttc_packet.hrl"). suite() -> [ @@ -21,14 +20,14 @@ suite() -> ]. all() -> - [{group, http_tests}].%, -% {group, https_tests}]. + [{group, http_tests}, + {group, https_tests}]. groups() -> Tests = [ proxy_protocol ], - [%{https_tests, [], Tests}, + [{https_tests, [], Tests}, {http_tests, [], Tests}]. init_per_suite(Config) -> @@ -85,12 +84,7 @@ proxy_protocol(Config) -> WS = rfc6455_client:new(Protocol ++ "://127.0.0.1:" ++ PortStr ++ "/ws", self(), undefined, [], "PROXY TCP4 192.168.1.1 192.168.1.2 80 81\r\n"), {ok, _} = rfc6455_client:open(WS), - Frame = emqttc_serialiser:serialise( - ?CONNECT_PACKET(#mqtt_packet_connect{ - client_id = <<"web-mqtt-tests-proxy-protocol">>, - username = <<"guest">>, - password = <<"guest">>})), - rfc6455_client:send_binary(WS, Frame), + rfc6455_client:send_binary(WS, rabbit_ws_test_util:mqtt_3_1_1_connect_frame()), {binary, _P} = rfc6455_client:recv(WS), ConnectionName = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, connection_name, []), diff --git a/deps/rabbitmq_web_mqtt/test/src/emqttc_packet.hrl b/deps/rabbitmq_web_mqtt/test/src/emqttc_packet.hrl deleted file mode 100644 index f1063fdcff43..000000000000 --- a/deps/rabbitmq_web_mqtt/test/src/emqttc_packet.hrl +++ /dev/null @@ -1,240 +0,0 @@ -%%%----------------------------------------------------------------------------- -%%% @Copyright (C) 2012-2015, Feng Lee -%%% -%%% Permission is hereby granted, free of charge, to any person obtaining a copy -%%% of this software and associated documentation files (the "Software"), to deal -%%% in the Software without restriction, including without limitation the rights -%%% to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -%%% copies of the Software, and to permit persons to whom the Software is -%%% furnished to do so, subject to the following conditions: -%%% -%%% The above copyright notice and this permission notice shall be included in all -%%% copies or substantial portions of the Software. -%%% -%%% THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -%%% IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -%%% FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -%%% AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -%%% LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE -%%% SOFTWARE. -%%%----------------------------------------------------------------------------- -%%% @doc -%%% emqttc packet header. -%%% @end -%%%----------------------------------------------------------------------------- - -%%------------------------------------------------------------------------------ -%% MQTT Protocol Version and Levels -%%------------------------------------------------------------------------------ --define(MQTT_PROTO_V31, 3). --define(MQTT_PROTO_V311, 4). - --define(PROTOCOL_NAMES, [ - {?MQTT_PROTO_V31, <<"MQIsdp">>}, - {?MQTT_PROTO_V311, <<"MQTT">>}]). - --type mqtt_vsn() :: ?MQTT_PROTO_V31 | ?MQTT_PROTO_V311. - -%%------------------------------------------------------------------------------ -%% QoS Levels -%%------------------------------------------------------------------------------ - --define(QOS_0, 0). --define(QOS_1, 1). --define(QOS_2, 2). - --define(IS_QOS(I), (I >= ?QOS_0 andalso I =< ?QOS_2)). - --type mqtt_qos() :: ?QOS_0 | ?QOS_1 | ?QOS_2. - -%%------------------------------------------------------------------------------ -%% Default Keepalive Timeout(secs) -%%------------------------------------------------------------------------------ --define(KEEPALIVE, 90). - -%%------------------------------------------------------------------------------ -%% Max ClientId Length. Why 1024? NiDongDe! -%%------------------------------------------------------------------------------ --define(MAX_CLIENTID_LEN, 1024). - -%%------------------------------------------------------------------------------ -%% MQTT Control Packet Types -%%------------------------------------------------------------------------------ --define(RESERVED, 0). %% Reserved --define(CONNECT, 1). %% Client request to connect to Server --define(CONNACK, 2). %% Server to Client: Connect acknowledgment --define(PUBLISH, 3). %% Publish message --define(PUBACK, 4). %% Publish acknowledgment --define(PUBREC, 5). %% Publish received (assured delivery part 1) --define(PUBREL, 6). %% Publish release (assured delivery part 2) --define(PUBCOMP, 7). %% Publish complete (assured delivery part 3) --define(SUBSCRIBE, 8). %% Client subscribe request --define(SUBACK, 9). %% Server Subscribe acknowledgment --define(UNSUBSCRIBE, 10). %% Unsubscribe request --define(UNSUBACK, 11). %% Unsubscribe acknowledgment --define(PINGREQ, 12). %% PING request --define(PINGRESP, 13). %% PING response --define(DISCONNECT, 14). %% Client is disconnecting - --define(TYPE_NAMES, [ - 'CONNECT', - 'CONNACK', - 'PUBLISH', - 'PUBACK', - 'PUBREC', - 'PUBREL', - 'PUBCOMP', - 'SUBSCRIBE', - 'SUBACK', - 'UNSUBSCRIBE', - 'UNSUBACK', - 'PINGREQ', - 'PINGRESP', - 'DISCONNECT']). - --type mqtt_packet_type() :: ?RESERVED..?DISCONNECT. - -%%------------------------------------------------------------------------------ -%% MQTT Connect Return Codes -%%------------------------------------------------------------------------------ --define(CONNACK_ACCEPT, 0). %% Connection accepted --define(CONNACK_PROTO_VER, 1). %% Unacceptable protocol version --define(CONNACK_INVALID_ID, 2). %% Client Identifier is correct UTF-8 but not allowed by the Server --define(CONNACK_SERVER, 3). %% Server unavailable --define(CONNACK_CREDENTIALS, 4). %% Username or password is malformed --define(CONNACK_AUTH, 5). %% Client is not authorized to connect - --type mqtt_connack() :: ?CONNACK_ACCEPT..?CONNACK_AUTH. - -%%------------------------------------------------------------------------------ -%% MQTT Parser and Serialiser -%%------------------------------------------------------------------------------ --define(MAX_LEN, 16#fffffff). --define(HIGHBIT, 2#10000000). --define(LOWBITS, 2#01111111). - -%%------------------------------------------------------------------------------ -%% MQTT Packet Fixed Header -%%------------------------------------------------------------------------------ --record(mqtt_packet_header, { - type = ?RESERVED :: mqtt_packet_type(), - dup = false :: boolean(), - qos = ?QOS_0 :: mqtt_qos(), - retain = false :: boolean()}). - -%%------------------------------------------------------------------------------ -%% MQTT Packets -%%------------------------------------------------------------------------------ --type mqtt_packet_id() :: 1..16#ffff | undefined. - --record(mqtt_packet_connect, { - client_id = <<>> :: binary(), - proto_ver = ?MQTT_PROTO_V311 :: mqtt_vsn(), - proto_name = <<"MQTT">> :: binary(), - will_retain = false :: boolean(), - will_qos = ?QOS_0 :: mqtt_qos(), - will_flag = false :: boolean(), - clean_sess = false :: boolean(), - keep_alive = 60 :: non_neg_integer(), - will_topic = undefined :: undefined | binary(), - will_msg = undefined :: undefined | binary(), - username = undefined :: undefined | binary(), - password = undefined :: undefined | binary()}). - --record(mqtt_packet_connack, { - ack_flags = ?RESERVED :: 0 | 1, - return_code :: mqtt_connack() }). - --record(mqtt_packet_publish, { - topic_name :: binary(), - packet_id :: mqtt_packet_id() }). - --record(mqtt_packet_puback, { - packet_id :: mqtt_packet_id() }). - --record(mqtt_packet_subscribe, { - packet_id :: mqtt_packet_id(), - topic_table :: list({binary(), mqtt_qos()}) }). - --record(mqtt_packet_unsubscribe, { - packet_id :: mqtt_packet_id(), - topics :: list(binary()) }). - --record(mqtt_packet_suback, { - packet_id :: mqtt_packet_id(), - qos_table :: list(mqtt_qos() | 128) }). - --record(mqtt_packet_unsuback, { - packet_id :: mqtt_packet_id() }). - -%%------------------------------------------------------------------------------ -%% MQTT Control Packet -%%------------------------------------------------------------------------------ --record(mqtt_packet, { - header :: #mqtt_packet_header{}, - variable :: #mqtt_packet_connect{} | #mqtt_packet_connack{} - | #mqtt_packet_publish{} | #mqtt_packet_puback{} - | #mqtt_packet_subscribe{} | #mqtt_packet_suback{} - | #mqtt_packet_unsubscribe{} | #mqtt_packet_unsuback{} - | mqtt_packet_id() | undefined, - payload :: binary() | undefined }). - --type mqtt_packet() :: #mqtt_packet{}. - -%%------------------------------------------------------------------------------ -%% MQTT Packet Match -%%------------------------------------------------------------------------------ --define(CONNECT_PACKET(Packet), - #mqtt_packet{header = #mqtt_packet_header{type = ?CONNECT}, variable = Packet}). - --define(CONNACK_PACKET(ReturnCode), - #mqtt_packet{header = #mqtt_packet_header{type = ?CONNACK}, - variable = #mqtt_packet_connack{return_code = ReturnCode}}). - --define(PUBLISH_PACKET(Qos, Topic, PacketId, Payload), - #mqtt_packet{header = #mqtt_packet_header{type = ?PUBLISH, - qos = Qos}, - variable = #mqtt_packet_publish{topic_name = Topic, - packet_id = PacketId}, - payload = Payload}). - --define(PUBACK_PACKET(Type, PacketId), - #mqtt_packet{header = #mqtt_packet_header{type = Type}, - variable = #mqtt_packet_puback{packet_id = PacketId}}). - --define(PUBREL_PACKET(PacketId), - #mqtt_packet{header = #mqtt_packet_header{type = ?PUBREL, qos = ?QOS_1}, - variable = #mqtt_packet_puback{packet_id = PacketId}}). - --define(SUBSCRIBE_PACKET(PacketId, TopicTable), - #mqtt_packet{header = #mqtt_packet_header{type = ?SUBSCRIBE, qos = ?QOS_1}, - variable = #mqtt_packet_subscribe{packet_id = PacketId, - topic_table = TopicTable}}). --define(SUBACK_PACKET(PacketId, QosTable), - #mqtt_packet{header = #mqtt_packet_header{type = ?SUBACK}, - variable = #mqtt_packet_suback{packet_id = PacketId, - qos_table = QosTable}}). --define(UNSUBSCRIBE_PACKET(PacketId, Topics), - #mqtt_packet{header = #mqtt_packet_header{type = ?UNSUBSCRIBE, qos = ?QOS_1}, - variable = #mqtt_packet_unsubscribe{packet_id = PacketId, - topics = Topics}}). --define(UNSUBACK_PACKET(PacketId), - #mqtt_packet{header = #mqtt_packet_header{type = ?UNSUBACK}, - variable = #mqtt_packet_unsuback{packet_id = PacketId}}). - --define(PACKET(Type), - #mqtt_packet{header = #mqtt_packet_header{type = Type}}). - -%%------------------------------------------------------------------------------ -%% MQTT Message -%%------------------------------------------------------------------------------ --record(mqtt_message, { - qos = ?QOS_0 :: mqtt_qos(), - retain = false :: boolean(), - dup = false :: boolean(), - msgid :: mqtt_packet_id(), - topic :: binary(), - payload :: binary()}). - --type mqtt_message() :: #mqtt_message{}. diff --git a/deps/rabbitmq_web_mqtt/test/src/emqttc_parser.erl b/deps/rabbitmq_web_mqtt/test/src/emqttc_parser.erl deleted file mode 100644 index ff800c1e1ff6..000000000000 --- a/deps/rabbitmq_web_mqtt/test/src/emqttc_parser.erl +++ /dev/null @@ -1,215 +0,0 @@ -%%%----------------------------------------------------------------------------- -%%% Copyright (c) 2012-2015 eMQTT.IO, All Rights Reserved. -%%% -%%% Permission is hereby granted, free of charge, to any person obtaining a copy -%%% of this software and associated documentation files (the "Software"), to deal -%%% in the Software without restriction, including without limitation the rights -%%% to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -%%% copies of the Software, and to permit persons to whom the Software is -%%% furnished to do so, subject to the following conditions: -%%% -%%% The above copyright notice and this permission notice shall be included in all -%%% copies or substantial portions of the Software. -%%% -%%% THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -%%% IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -%%% FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -%%% AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -%%% LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE -%%% SOFTWARE. -%%%----------------------------------------------------------------------------- -%%% @doc -%%% emqttc received packet parser. -%%% -%%% @end -%%%----------------------------------------------------------------------------- --module(emqttc_parser). - --author("feng@emqtt.io"). - --include("emqttc_packet.hrl"). - -%% API --export([new/0, parse/2]). - - -%%%----------------------------------------------------------------------------- -%% @doc Initialize a parser. -%% @end -%%%----------------------------------------------------------------------------- --spec new() -> none. -new() -> none. - -%%%----------------------------------------------------------------------------- -%% @doc Parse MQTT Packet. -%% @end -%%%----------------------------------------------------------------------------- --spec parse(binary(), none | fun()) -> {ok, mqtt_packet()} | {error, any()} | {more, fun()}. -parse(<<>>, none) -> - {more, fun(Bin) -> parse(Bin, none) end}; -parse(<>, none) -> - parse_remaining_len(Rest, #mqtt_packet_header{type = PacketType, - dup = bool(Dup), - qos = QoS, - retain = bool(Retain)}); -parse(Bin, Cont) -> Cont(Bin). - -parse_remaining_len(<<>>, Header) -> - {more, fun(Bin) -> parse_remaining_len(Bin, Header) end}; -parse_remaining_len(Rest, Header) -> - parse_remaining_len(Rest, Header, 1, 0). - -parse_remaining_len(_Bin, _Header, _Multiplier, Length) - when Length > ?MAX_LEN -> - {error, invalid_mqtt_frame_len}; -parse_remaining_len(<<>>, Header, Multiplier, Length) -> - {more, fun(Bin) -> parse_remaining_len(Bin, Header, Multiplier, Length) end}; -parse_remaining_len(<<1:1, Len:7, Rest/binary>>, Header, Multiplier, Value) -> - parse_remaining_len(Rest, Header, Multiplier * ?HIGHBIT, Value + Len * Multiplier); -parse_remaining_len(<<0:1, Len:7, Rest/binary>>, Header, Multiplier, Value) -> - parse_frame(Rest, Header, Value + Len * Multiplier). - -parse_frame(Bin, #mqtt_packet_header{type = Type, - qos = Qos} = Header, Length) -> - case {Type, Bin} of - %{?CONNECT, <>} -> - % {ProtoName, Rest1} = parse_utf(FrameBin), - % <> = Rest1, - % <> = Rest2, - % {ClientId, Rest4} = parse_utf(Rest3), - % {WillTopic, Rest5} = parse_utf(Rest4, WillFlag), - % {WillMsg, Rest6} = parse_msg(Rest5, WillFlag), - % {UserName, Rest7} = parse_utf(Rest6, UsernameFlag), - % {PasssWord, <<>>} = parse_utf(Rest7, PasswordFlag), - % case protocol_name_approved(ProtoVersion, ProtoName) of - % true -> - % wrap(Header, - % #mqtt_packet_connect{ - % proto_ver = ProtoVersion, - % proto_name = ProtoName, - % will_retain = bool(WillRetain), - % will_qos = WillQos, - % will_flag = bool(WillFlag), - % clean_sess = bool(CleanSession), - % keep_alive = KeepAlive, - % client_id = ClientId, - % will_topic = WillTopic, - % will_msg = WillMsg, - % username = UserName, - % password = PasssWord}, Rest); - % false -> - % {error, protocol_header_corrupt} - % end; - {?CONNACK, <>} -> - <<_Reserved:7, SP:1, ReturnCode:8>> = FrameBin, - wrap(Header, #mqtt_packet_connack{ack_flags = SP, - return_code = ReturnCode }, Rest); - {?PUBLISH, <>} -> - {TopicName, Rest1} = parse_utf(FrameBin), - {PacketId, Payload} = case Qos of - 0 -> {undefined, Rest1}; - _ -> <> = Rest1, - {Id, R} - end, - wrap(Header, #mqtt_packet_publish{topic_name = TopicName, - packet_id = PacketId}, - Payload, Rest); - {?PUBACK, <>} -> - <> = FrameBin, - wrap(Header, #mqtt_packet_puback{packet_id = PacketId}, Rest); - {?PUBREC, <>} -> - <> = FrameBin, - wrap(Header, #mqtt_packet_puback{packet_id = PacketId}, Rest); - {?PUBREL, <>} -> - 1 = Qos, - <> = FrameBin, - wrap(Header, #mqtt_packet_puback{packet_id = PacketId}, Rest); - {?PUBCOMP, <>} -> - <> = FrameBin, - wrap(Header, #mqtt_packet_puback{packet_id = PacketId}, Rest); - %{?SUBSCRIBE, <>} -> - % 1 = Qos, - % <> = FrameBin, - % TopicTable = parse_topics(?SUBSCRIBE, Rest1, []), - % wrap(Header, #mqtt_packet_subscribe{packet_id = PacketId, - % topic_table = TopicTable}, Rest); - {?SUBACK, <>} -> - <> = FrameBin, - wrap(Header, #mqtt_packet_suback{packet_id = PacketId, - qos_table = parse_qos(Rest1, [])}, Rest); - %{?UNSUBSCRIBE, <>} -> - % 1 = Qos, - % <> = FrameBin, - % Topics = parse_topics(?UNSUBSCRIBE, Rest1, []), - % wrap(Header, #mqtt_packet_unsubscribe{packet_id = PacketId, - % topics = Topics}, Rest); - {?UNSUBACK, <>} -> - <> = FrameBin, - wrap(Header, #mqtt_packet_unsuback{packet_id = PacketId}, Rest); - %{?PINGREQ, Rest} -> - % Length = 0, - % wrap(Header, Rest); - {?PINGRESP, Rest} -> - Length = 0, - wrap(Header, Rest); - %{?DISCONNECT, Rest} -> - % Length = 0, - % wrap(Header, Rest); - {_, TooShortBin} -> - {more, fun(BinMore) -> - parse_frame(<>, - Header, Length) - end} - end. - -wrap(Header, Variable, Payload, Rest) -> - {ok, #mqtt_packet{header = Header, variable = Variable, payload = Payload}, Rest}. -wrap(Header, Variable, Rest) -> - {ok, #mqtt_packet{header = Header, variable = Variable}, Rest}. -wrap(Header, Rest) -> - {ok, #mqtt_packet{header = Header}, Rest}. - -parse_qos(<<>>, Acc) -> - lists:reverse(Acc); -parse_qos(<>, Acc) -> - parse_qos(Rest, [QoS | Acc]). - -% server function -%parse_topics(_, <<>>, Topics) -> -% Topics; -%parse_topics(?SUBSCRIBE = Sub, Bin, Topics) -> -% {Name, <<_:6, QoS:2, Rest/binary>>} = parse_utf(Bin), -% parse_topics(Sub, Rest, [{Name, QoS}| Topics]); -%parse_topics(?UNSUBSCRIBE = Sub, Bin, Topics) -> -% {Name, <>} = parse_utf(Bin), -% parse_topics(Sub, Rest, [Name | Topics]). - -%parse_utf(Bin, 0) -> -% {undefined, Bin}; -%parse_utf(Bin, _) -> -% parse_utf(Bin). - -parse_utf(<>) -> - {Str, Rest}. - -% server function -%parse_msg(Bin, 0) -> -% {undefined, Bin}; -%parse_msg(<>, _) -> -% {Msg, Rest}. - -bool(0) -> false; -bool(1) -> true. - -%protocol_name_approved(Ver, Name) -> -% lists:member({Ver, Name}, ?PROTOCOL_NAMES). - diff --git a/deps/rabbitmq_web_mqtt/test/src/emqttc_serialiser.erl b/deps/rabbitmq_web_mqtt/test/src/emqttc_serialiser.erl deleted file mode 100644 index 5654de221233..000000000000 --- a/deps/rabbitmq_web_mqtt/test/src/emqttc_serialiser.erl +++ /dev/null @@ -1,157 +0,0 @@ -%%%----------------------------------------------------------------------------- -%%% Copyright (c) 2012-2015 eMQTT.IO, All Rights Reserved. -%%% -%%% Permission is hereby granted, free of charge, to any person obtaining a copy -%%% of this software and associated documentation files (the "Software"), to deal -%%% in the Software without restriction, including without limitation the rights -%%% to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -%%% copies of the Software, and to permit persons to whom the Software is -%%% furnished to do so, subject to the following conditions: -%%% -%%% The above copyright notice and this permission notice shall be included in all -%%% copies or substantial portions of the Software. -%%% -%%% THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -%%% IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -%%% FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -%%% AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -%%% LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE -%%% SOFTWARE. -%%%----------------------------------------------------------------------------- -%%% @doc -%%% emqttc packet serialiser. -%%% -%%% @end -%%%----------------------------------------------------------------------------- --module(emqttc_serialiser). - --author("feng@emqtt.io"). - --include("emqttc_packet.hrl"). - -%% API --export([serialise/1]). - - -serialise(#mqtt_packet{header = Header = #mqtt_packet_header{type = Type}, - variable = Variable, - payload = Payload}) -> - serialise_header(Header, - serialise_variable(Type, Variable, - serialise_payload(Payload))). - -serialise_header(#mqtt_packet_header{type = Type, - dup = Dup, - qos = Qos, - retain = Retain}, - {VariableBin, PayloadBin}) - when is_integer(Type) andalso ?CONNECT =< Type andalso Type =< ?DISCONNECT -> - Len = size(VariableBin) + size(PayloadBin), - true = (Len =< ?MAX_LEN), - LenBin = serialise_len(Len), - <>. - -serialise_variable(?CONNECT, #mqtt_packet_connect{client_id = ClientId, - proto_ver = ProtoVer, - proto_name = ProtoName, - will_retain = WillRetain, - will_qos = WillQos, - will_flag = WillFlag, - clean_sess = CleanSess, - keep_alive = KeepAlive, - will_topic = WillTopic, - will_msg = WillMsg, - username = Username, - password = Password }, undefined) -> - VariableBin = <<(size(ProtoName)):16/big-unsigned-integer, - ProtoName/binary, - ProtoVer:8, - (opt(Username)):1, - (opt(Password)):1, - (opt(WillRetain)):1, - WillQos:2, - (opt(WillFlag)):1, - (opt(CleanSess)):1, - 0:1, - KeepAlive:16/big-unsigned-integer>>, - PayloadBin = serialise_utf(ClientId), - PayloadBin1 = case WillFlag of - true -> <>; - false -> PayloadBin - end, - UserPasswd = << <<(serialise_utf(B))/binary>> || B <- [Username, Password], B =/= undefined >>, - {VariableBin, <>}; - -serialise_variable(?CONNACK, #mqtt_packet_connack{ack_flags = AckFlags, - return_code = ReturnCode}, - undefined) -> - {<>, <<>>}; - -serialise_variable(?SUBSCRIBE, #mqtt_packet_subscribe{packet_id = PacketId, - topic_table = Topics }, undefined) -> - {<>, serialise_topics(Topics)}; - -serialise_variable(?SUBACK, #mqtt_packet_suback {packet_id = PacketId, - qos_table = QosTable}, - undefined) -> - {<>, << <> || Q <- QosTable >>}; - -serialise_variable(?UNSUBSCRIBE, #mqtt_packet_unsubscribe{ - packet_id = PacketId, topics = Topics }, undefined) -> - {<>, serialise_topics(Topics)}; - -serialise_variable(?UNSUBACK, #mqtt_packet_suback {packet_id = PacketId}, - undefined) -> - {<>, <<>>}; - -serialise_variable(?PUBLISH, #mqtt_packet_publish { topic_name = TopicName, - packet_id = PacketId }, PayloadBin) -> - TopicBin = serialise_utf(TopicName), - PacketIdBin = if - PacketId =:= undefined -> <<>>; - true -> <> - end, - {<>, PayloadBin}; - -serialise_variable(PubAck, #mqtt_packet_puback { packet_id = PacketId }, _Payload) - when PubAck =:= ?PUBACK; PubAck =:= ?PUBREC; PubAck =:= ?PUBREL; PubAck =:= ?PUBCOMP -> - {<>, <<>>}; - -serialise_variable(?PINGREQ, undefined, undefined) -> - {<<>>, <<>>}; - -serialise_variable(?DISCONNECT, undefined, undefined) -> - {<<>>, <<>>}. - -serialise_payload(undefined) -> - undefined; -serialise_payload(Bin) when is_binary(Bin) -> - Bin. - -serialise_topics([{_Topic, _Qos}|_] = Topics) -> - << <<(serialise_utf(Topic))/binary, ?RESERVED:6, Qos:2>> || {Topic, Qos} <- Topics >>; - -serialise_topics([H|_] = Topics) when is_binary(H) -> - << <<(serialise_utf(Topic))/binary>> || Topic <- Topics >>. - -serialise_utf(String) -> - StringBin = unicode:characters_to_binary(String), - Len = size(StringBin), - true = (Len =< 16#ffff), - <>. - -serialise_len(N) when N =< ?LOWBITS -> - <<0:1, N:7>>; -serialise_len(N) -> - <<1:1, (N rem ?HIGHBIT):7, (serialise_len(N div ?HIGHBIT))/binary>>. - -opt(undefined) -> ?RESERVED; -opt(false) -> 0; -opt(true) -> 1; -opt(X) when is_integer(X) -> X; -opt(B) when is_binary(B) -> 1. diff --git a/deps/rabbitmq_web_mqtt/test/src/rabbit_ws_test_util.erl b/deps/rabbitmq_web_mqtt/test/src/rabbit_ws_test_util.erl index 697370c583b4..6aa74f13b847 100644 --- a/deps/rabbitmq_web_mqtt/test/src/rabbit_ws_test_util.erl +++ b/deps/rabbitmq_web_mqtt/test/src/rabbit_ws_test_util.erl @@ -7,7 +7,8 @@ -module(rabbit_ws_test_util). --export([update_app_env/3, get_web_mqtt_port_str/1]). +-export([update_app_env/3, get_web_mqtt_port_str/1, + mqtt_3_1_1_connect_frame/0]). update_app_env(Config, Key, Val) -> ok = rabbit_ct_broker_helpers:rpc(Config, 0, @@ -28,3 +29,31 @@ get_web_mqtt_port_str(Config) -> rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt_tls) end, integer_to_list(Port). + +mqtt_3_1_1_connect_frame() -> + <<16, + 24, + 0, + 4, + 77, + 81, + 84, + 84, + 4, + 2, + 0, + 60, + 0, + 12, + 84, + 101, + 115, + 116, + 67, + 111, + 110, + 115, + 117, + 109, + 101, + 114>>. diff --git a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl index 98ea25c91901..f71aa8e543fe 100644 --- a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl +++ b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl @@ -8,10 +8,11 @@ -module(system_SUITE). -include_lib("eunit/include/eunit.hrl"). --include("src/emqttc_packet.hrl"). -compile([export_all, nowarn_export_all]). +-import(rabbit_ct_broker_helpers, [rpc/5]). + all() -> [ {group, non_parallel_tests} @@ -24,7 +25,6 @@ groups() -> , pubsub_shared_connection , pubsub_separate_connections , last_will_enabled - , last_will_disabled , disconnect , keepalive , maintenance @@ -57,9 +57,6 @@ init_per_testcase(Testcase, Config) -> end_per_testcase(Testcase, Config) -> rabbit_ct_helpers:testcase_finished(Config, Testcase). --define(DEFAULT_TIMEOUT, 15_000). - - connection(Config) -> Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt), {ok, C} = emqtt:start_link([{host, "127.0.0.1"}, @@ -76,180 +73,139 @@ pubsub_shared_connection(Config) -> {username, "guest"}, {password, "guest"}, {ws_path, "/ws"}, - {client_id, ?FUNCTION_NAME}, + {clientid, atom_to_binary(?FUNCTION_NAME)}, {clean_start, true}, {port, Port}]), {ok, _} = emqtt:ws_connect(C), - Dst = <<"/topic/test-web-mqtt">>, - {ok, _, [1]} = emqtt:subscribe(C, Dst, qos1), + Topic = <<"/topic/test-web-mqtt">>, + {ok, _, [1]} = emqtt:subscribe(C, Topic, qos1), Payload = <<"a\x00a">>, - {ok, PubReply} = emqtt:publish(C, Dst, Payload, [{qos, 1}]), - ?assertMatch(#{packet_id := _, - reason_code := 0, - reason_code_name := success - }, PubReply), - + ?assertMatch({ok, #{packet_id := _, + reason_code := 0, + reason_code_name := success + }}, + emqtt:publish(C, Topic, Payload, [{qos, 1}])), + ok = expect_publishes(C, Topic, [Payload]), ok = emqtt:disconnect(C). pubsub_separate_connections(Config) -> Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt), - {ok, C1} = emqtt:start_link([{host, "127.0.0.1"}, - {username, "guest"}, - {password, "guest"}, - {ws_path, "/ws"}, - {client_id, "web-mqtt-tests-consumer"}, - {clean_start, true}, - {port, Port}]), - {ok, _} = emqtt:ws_connect(C1), - {ok, C2} = emqtt:start_link([{host, "127.0.0.1"}, - {username, "guest"}, - {password, "guest"}, - {ws_path, "/ws"}, - {client_id, "web-mqtt-tests-consumer"}, - {clean_start, true}, - {port, Port}]), - {ok, _} = emqtt:ws_connect(C2), - - Dst = <<"/topic/test-web-mqtt">>, - {ok, _, [1]} = emqtt:subscribe(C2, Dst, qos1), + {ok, Publisher} = emqtt:start_link([{host, "127.0.0.1"}, + {username, "guest"}, + {password, "guest"}, + {ws_path, "/ws"}, + {clientid, <<(atom_to_binary(?FUNCTION_NAME))/binary, "_publisher">>}, + {clean_start, true}, + {port, Port}]), + {ok, _} = emqtt:ws_connect(Publisher), + {ok, Consumer} = emqtt:start_link([{host, "127.0.0.1"}, + {username, "guest"}, + {password, "guest"}, + {ws_path, "/ws"}, + {clientid, <<(atom_to_binary(?FUNCTION_NAME))/binary, "_consumer">>}, + {clean_start, true}, + {port, Port}]), + {ok, _} = emqtt:ws_connect(Consumer), + + Topic = <<"/topic/test-web-mqtt">>, + {ok, _, [1]} = emqtt:subscribe(Consumer, Topic, qos1), Payload = <<"a\x00a">>, - {ok, PubReply} = emqtt:publish(C1, Dst, Payload, [{qos, 1}]), - ?assertMatch(#{packet_id := _, - reason_code := 0, - reason_code_name := success - }, PubReply), - - ok = emqtt:disconnect(C1), - ok = emqtt:disconnect(C2). + ?assertMatch({ok, #{packet_id := _, + reason_code := 0, + reason_code_name := success + }}, + emqtt:publish(Publisher, Topic, Payload, [{qos, 1}])), + ok = expect_publishes(Consumer, Topic, [Payload]), + ok = emqtt:disconnect(Publisher), + ok = emqtt:disconnect(Consumer). last_will_enabled(Config) -> Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt), - PortStr = integer_to_list(Port), - - LastWillDst = <<"/topic/web-mqtt-tests-ws1-last-will">>, + LastWillTopic = <<"/topic/web-mqtt-tests-ws1-last-will">>, LastWillMsg = <<"a last will and testament message">>, - - WS1 = rfc6455_client:new("ws://127.0.0.1:" ++ PortStr ++ "/ws", self()), - {ok, _} = rfc6455_client:open(WS1), - ok = raw_send(WS1, - ?CONNECT_PACKET(#mqtt_packet_connect{ - clean_sess = true, - client_id = <<"web-mqtt-tests-last-will-ws1">>, - will_flag = true, - will_qos = ?QOS_1, - will_topic = LastWillDst, - will_msg = LastWillMsg, - username = <<"guest">>, - password = <<"guest">>})), - {ok, ?CONNACK_PACKET(?CONNACK_ACCEPT), _} = raw_recv(WS1), - - WS2 = rfc6455_client:new("ws://127.0.0.1:" ++ PortStr ++ "/ws", self()), - {ok, _} = rfc6455_client:open(WS2), - ok = raw_send(WS2, - ?CONNECT_PACKET(#mqtt_packet_connect{ - clean_sess = true, - client_id = <<"web-mqtt-tests-last-will-ws2">>, - username = <<"guest">>, - password = <<"guest">>})), - {ok, ?CONNACK_PACKET(?CONNACK_ACCEPT), _} = raw_recv(WS2), - - ok = raw_send(WS2, ?SUBSCRIBE_PACKET(1, [{LastWillDst, ?QOS_1}])), - {ok, ?SUBACK_PACKET(_, _), _} = raw_recv(WS2), - - {close, _} = rfc6455_client:close(WS1), - ?assertMatch({ok, ?PUBLISH_PACKET(_, LastWillDst, _, LastWillMsg), _}, raw_recv(WS2, 5000)), - - {close, _} = rfc6455_client:close(WS2), - ok. - -last_will_disabled(Config) -> - Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt), - PortStr = integer_to_list(Port), - - LastWillDst = <<"/topic/web-mqtt-tests-ws1-last-will-disabled">>, - LastWillMsg = <<"a last will and testament message">>, - - WS1 = rfc6455_client:new("ws://127.0.0.1:" ++ PortStr ++ "/ws", self()), - {ok, _} = rfc6455_client:open(WS1), - ok = raw_send(WS1, - ?CONNECT_PACKET(#mqtt_packet_connect{ - clean_sess = true, - client_id = <<"web-mqtt-tests-last-will-ws1-disabled">>, - will_flag = false, - will_qos = ?QOS_1, - will_topic = LastWillDst, - will_msg = LastWillMsg, - username = <<"guest">>, - password = <<"guest">>})), - {ok, ?CONNACK_PACKET(?CONNACK_ACCEPT), _} = raw_recv(WS1), - - WS2 = rfc6455_client:new("ws://127.0.0.1:" ++ PortStr ++ "/ws", self()), - {ok, _} = rfc6455_client:open(WS2), - ok = raw_send(WS2, - ?CONNECT_PACKET(#mqtt_packet_connect{ - clean_sess = true, - client_id = <<"web-mqtt-tests-last-will-ws2-disabled">>, - username = <<"guest">>, - password = <<"guest">>})), - {ok, ?CONNACK_PACKET(?CONNACK_ACCEPT), _} = raw_recv(WS2), - - ok = raw_send(WS2, ?SUBSCRIBE_PACKET(1, [{LastWillDst, ?QOS_1}])), - ?assertMatch({ok, ?SUBACK_PACKET(_, _), _}, raw_recv(WS2)), - - {close, _} = rfc6455_client:close(WS1), - ?assertEqual({error, timeout}, raw_recv(WS2, 3000)), - - {close, _} = rfc6455_client:close(WS2), - ok. + {ok, Publisher} = emqtt:start_link([{host, "127.0.0.1"}, + {username, "guest"}, + {password, "guest"}, + {ws_path, "/ws"}, + {clientid, <<(atom_to_binary(?FUNCTION_NAME))/binary, "_publisher">>}, + {clean_start, true}, + {port, Port}, + {will_topic, LastWillTopic}, + {will_payload, LastWillMsg}, + {will_qos, 1} + ]), + {ok, _} = emqtt:ws_connect(Publisher), + {ok, Consumer} = emqtt:start_link([{host, "127.0.0.1"}, + {username, "guest"}, + {password, "guest"}, + {ws_path, "/ws"}, + {clientid, <<(atom_to_binary(?FUNCTION_NAME))/binary, "_consumer">>}, + {clean_start, true}, + {port, Port} + ]), + {ok, _} = emqtt:ws_connect(Consumer), + {ok, _, [1]} = emqtt:subscribe(Consumer, LastWillTopic, qos1), + ok = emqtt:disconnect(Publisher), + ok = expect_publishes(Consumer, LastWillTopic, [LastWillMsg]), + ok = emqtt:disconnect(Consumer). disconnect(Config) -> Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt), - PortStr = integer_to_list(Port), - WS = rfc6455_client:new("ws://127.0.0.1:" ++ PortStr ++ "/ws", self()), - {ok, _} = rfc6455_client:open(WS), - ok = raw_send(WS, - ?CONNECT_PACKET(#mqtt_packet_connect{ - clean_sess = true, - client_id = <<"web-mqtt-tests-disconnect">>, - username = <<"guest">>, - password = <<"guest">>})), - - {ok, ?CONNACK_PACKET(?CONNACK_ACCEPT), _} = raw_recv(WS), - - ok = raw_send(WS, ?PACKET(?DISCONNECT)), - {close, {1000, _}} = rfc6455_client:recv(WS), - - ok. + {ok, C} = emqtt:start_link([{host, "127.0.0.1"}, + {username, "guest"}, + {password, "guest"}, + {ws_path, "/ws"}, + {clientid, atom_to_binary(?FUNCTION_NAME)}, + {clean_start, true}, + {port, Port}]), + process_flag(trap_exit, true), + {ok, _} = emqtt:ws_connect(C), + ?assertEqual(1, num_mqtt_connections(Config, 0)), + ok = emqtt:disconnect(C), + receive + {'EXIT', C, normal} -> + ok + after 5000 -> + ct:fail("disconnect didn't terminate client") + end, + ?assertEqual(0, num_mqtt_connections(Config, 0)). keepalive(Config) -> - Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt), - PortStr = integer_to_list(Port), - WS = rfc6455_client:new("ws://127.0.0.1:" ++ PortStr ++ "/ws", self()), - {ok, _} = rfc6455_client:open(WS), - KeepaliveSecs = 1, KeepaliveMs = timer:seconds(KeepaliveSecs), - ok = raw_send(WS, - ?CONNECT_PACKET( - #mqtt_packet_connect{ - keep_alive = KeepaliveSecs, - clean_sess = true, - client_id = <<"web-mqtt-tests-disconnect">>, - username = <<"guest">>, - password = <<"guest">>})), - {ok, ?CONNACK_PACKET(?CONNACK_ACCEPT), _} = raw_recv(WS), + Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt), + {ok, C} = emqtt:start_link([{keepalive, KeepaliveSecs}, + {host, "127.0.0.1"}, + {username, "guest"}, + {password, "guest"}, + {ws_path, "/ws"}, + {clientid, atom_to_binary(?FUNCTION_NAME)}, + {clean_start, true}, + {port, Port}]), + {ok, _} = emqtt:ws_connect(C), - %% Sanity check that MQTT ping request and ping response work. + %% Connection should stay up when client sends PING requests. timer:sleep(KeepaliveMs), - ok = raw_send(WS, #mqtt_packet{header = #mqtt_packet_header{type = ?PINGREQ}}), - {ok, #mqtt_packet{header = #mqtt_packet_header{type = ?PINGRESP}}, <<>>} = raw_recv(WS), - %% Stop sending any data to the server (including ping requests). - %% The server should disconnect us. - ?assertEqual({close, {1000, <<"MQTT keepalive timeout">>}}, - rfc6455_client:recv(WS, ceil(3 * 0.75 * KeepaliveMs))). + %% Mock the server socket to not have received any bytes. + rabbit_ct_broker_helpers:setup_meck(Config), + Mod = rabbit_net, + ok = rpc(Config, 0, meck, new, [Mod, [no_link, passthrough]]), + ok = rpc(Config, 0, meck, expect, [Mod, getstat, 2, {ok, [{recv_oct, 999}]} ]), + + process_flag(trap_exit, true), + receive + {'EXIT', C, _Reason} -> + ok + after + ceil(3 * 0.75 * KeepaliveMs) -> + ct:fail("server did not respect keepalive") + end, + + true = rpc(Config, 0, meck, validate, [Mod]), + ok = rpc(Config, 0, meck, unload, [Mod]). maintenance(Config) -> Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt), @@ -257,7 +213,7 @@ maintenance(Config) -> {username, "guest"}, {password, "guest"}, {ws_path, "/ws"}, - {client_id, "node-drain-test"}, + {clientid, atom_to_binary(?FUNCTION_NAME)}, {clean_start, true}, {port, Port}]), {ok, _} = emqtt:ws_connect(C), @@ -265,26 +221,21 @@ maintenance(Config) -> ?assertEqual(1, num_mqtt_connections(Config, 0)), ok = rabbit_ct_broker_helpers:drain_node(Config, 0), - ?assertEqual(0, num_mqtt_connections(Config, 0)), ok = rabbit_ct_broker_helpers:revive_node(Config, 0). -raw_send(WS, Packet) -> - Frame = emqttc_serialiser:serialise(Packet), - rfc6455_client:send_binary(WS, Frame). - -raw_recv(WS) -> - raw_recv(WS, ?DEFAULT_TIMEOUT). - -raw_recv(WS, Timeout) -> - case rfc6455_client:recv(WS, Timeout) of - {binary, P} -> - emqttc_parser:parse(P, emqttc_parser:new()); - {error, timeout} -> - {error, timeout} - end. - %% Web mqtt connections are tracked together with mqtt connections num_mqtt_connections(Config, Node) -> - length(rabbit_ct_broker_helpers:rpc( - Config, Node, rabbit_mqtt,local_connection_pids,[])). \ No newline at end of file + length(rpc(Config, Node, rabbit_mqtt, local_connection_pids, [])). + +expect_publishes(_ClientPid, _Topic, []) -> + ok; +expect_publishes(ClientPid, Topic, [Payload|Rest]) -> + receive + {publish, #{client_pid := ClientPid, + topic := Topic, + payload := Payload}} -> + expect_publishes(ClientPid, Topic, Rest) + after 5000 -> + throw({publish_not_received, Payload}) + end. From 319af3872e60bcf53df601a6c1752e8d53f3d312 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Tue, 8 Nov 2022 11:59:16 +0000 Subject: [PATCH 033/118] Handle duplicate packet IDs "If a Client re-sends a particular Control Packet, then it MUST use the same Packet Identifier in subsequent re-sends of that packet." A client can re-send a PUBLISH packet with the same packet ID. If the MQTT connection process already received the original packet and sent it to destination queues, it will ignore this re-send. The packet ID will be acked to the publisher once a confirmation from all target queues is received. There should be no risk of "stuck" messages within the MQTT connection process because quorum and stream queue clients re-send the message and classic queues will send a monitor DOWN message in case they are down. --- deps/rabbit/src/rabbit_confirms.erl | 2 +- .../src/rabbit_mqtt_confirms.erl | 19 +++--- .../src/rabbit_mqtt_processor.erl | 61 +++++++++++-------- deps/rabbitmq_mqtt/test/integration_SUITE.erl | 30 +++++---- 4 files changed, 66 insertions(+), 46 deletions(-) diff --git a/deps/rabbit/src/rabbit_confirms.erl b/deps/rabbit/src/rabbit_confirms.erl index 2fe032d1f1a6..5479b358c3ad 100644 --- a/deps/rabbit/src/rabbit_confirms.erl +++ b/deps/rabbit/src/rabbit_confirms.erl @@ -44,7 +44,7 @@ insert(SeqNo, QNames, #resource{kind = exchange} = XName, unconfirmed = U0} = State) when is_integer(SeqNo) andalso is_list(QNames) - andalso is_map_key(SeqNo, U0) == false -> + andalso not is_map_key(SeqNo, U0) -> U = U0#{SeqNo => {XName, maps:from_list([{Q, ok} || Q <- QNames])}}, S = case S0 of undefined -> SeqNo; diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl index c7146dca4934..8c3d87898039 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl @@ -15,7 +15,8 @@ confirm/3, reject/2, remove_queue/2, - size/1]). + size/1, + contains/2]). -type queue_name() :: rabbit_amqqueue:name(). -opaque state() :: #{packet_id() => #{queue_name() => ok}}. @@ -29,15 +30,17 @@ init() -> size(State) -> maps:size(State). --spec insert(packet_id(), [queue_name()], state()) -> - {ok, state()} | {error, {duplicate_packet_id, packet_id()}}. -insert(PktId, _, State) - when is_map_key(PktId, State) -> - {error, {duplicate_packet_id, PktId}}; +-spec contains(packet_id(), state()) -> boolean(). +contains(PktId, State) -> + maps:is_key(PktId, State). + +-spec insert(packet_id(), [queue_name()], state()) -> state(). insert(PktId, QNames, State) - when is_integer(PktId) andalso PktId > 0 -> + when is_integer(PktId) andalso + PktId > 0 andalso + not is_map_key(PktId, State) -> QMap = maps:from_keys(QNames, ok), - {ok, maps:put(PktId, QMap, State)}. + maps:put(PktId, QMap, State). -spec confirm([packet_id()], queue_name(), state()) -> {[packet_id()], state()}. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index d0118a4d9609..9e859d1463b5 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -7,7 +7,6 @@ -module(rabbit_mqtt_processor). - -export([info/2, initial_state/2, initial_state/4, process_frame/2, serialise/2, send_will/1, terminate/2, handle_pre_hibernate/0, @@ -115,34 +114,44 @@ process_request(?PUBLISH, message_id = MessageId }, payload = Payload}, PState = #proc_state{retainer_pid = RPid, - amqp2mqtt_fun = Amqp2MqttFun}) -> + amqp2mqtt_fun = Amqp2MqttFun, + unacked_client_pubs = U}) -> rabbit_global_counters:messages_received(mqtt, 1), + Publish = fun() -> + Msg = #mqtt_msg{retain = Retain, + qos = Qos, + topic = Topic, + dup = Dup, + message_id = MessageId, + payload = Payload}, + case publish_to_queues(Msg, PState) of + {ok, _} = Ok -> + case Retain of + false -> + ok; + true -> + hand_off_to_retainer(RPid, Amqp2MqttFun, Topic, Msg) + end, + Ok; + Error -> + Error + end + end, case Qos of N when N > ?QOS_0 -> - rabbit_global_counters:messages_received_confirm(mqtt, 1); + rabbit_global_counters:messages_received_confirm(mqtt, 1), + case rabbit_mqtt_confirms:contains(MessageId, U) of + false -> + publish_to_queues_with_checks(Topic, Publish, PState); + true -> + %% Client re-sent this PUBLISH packet. + %% We already sent this message to target queues awaiting confirmations. + %% Hence, we ignore this re-send. + {ok, PState} + end; _ -> - ok - end, - publish_to_queues_with_checks( - Topic, - fun() -> - Msg = #mqtt_msg{retain = Retain, - qos = Qos, - topic = Topic, - dup = Dup, - message_id = MessageId, - payload = Payload}, - case publish_to_queues(Msg, PState) of - {ok, _} = Ok -> - case Retain of - false -> ok; - true -> hand_off_to_retainer(RPid, Amqp2MqttFun, Topic, Msg) - end, - Ok; - Error -> - Error - end - end, PState); + publish_to_queues_with_checks(Topic, Publish, PState) + end; process_request(?SUBSCRIBE, #mqtt_frame{ @@ -1156,7 +1165,7 @@ process_routing_confirm(#delivery{confirm = true, msg_seq_no = MsgId}, Qs, PState = #proc_state{unacked_client_pubs = U0}) -> QNames = lists:map(fun amqqueue:get_name/1, Qs), - {ok, U} = rabbit_mqtt_confirms:insert(MsgId, QNames, U0), + U = rabbit_mqtt_confirms:insert(MsgId, QNames, U0), PState#proc_state{unacked_client_pubs = U}. send_puback(MsgIds, PState) diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/integration_SUITE.erl index 237b3ccef247..29ae9b0fa5e9 100644 --- a/deps/rabbitmq_mqtt/test/integration_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/integration_SUITE.erl @@ -15,6 +15,7 @@ -import(rabbit_ct_broker_helpers, [rabbitmqctl_list/3, rpc_all/4]). +-import(rabbit_ct_helpers, [eventually/3]). -import(util, [all_connection_pids/1, publish_qos1/4]). @@ -121,8 +122,7 @@ publish_to_all_queue_types(Config, QoS) -> bind(Ch, SQ, Topic), NumMsgs = 2000, - NumMsgsBin = integer_to_binary(NumMsgs), - {C, _} = connect(?FUNCTION_NAME, Config), + {C, _} = connect(?FUNCTION_NAME, Config, [{retry_interval, 2}]), lists:foreach(fun(_N) -> case QoS of qos0 -> @@ -132,15 +132,23 @@ publish_to_all_queue_types(Config, QoS) -> end end, lists:seq(1, NumMsgs)), - Expected = lists:sort([[CQ, NumMsgsBin], - [CMQ, NumMsgsBin], - [QQ, NumMsgsBin], - [SQ, NumMsgsBin] - ]), - ?awaitMatch(Expected, - lists:sort(rabbitmqctl_list(Config, 0, ["list_queues", "--no-table-headers", - "name", "messages_ready"])), - 20_000, 1000), + eventually(?_assert( + begin + L = rabbitmqctl_list(Config, 0, ["list_queues", "messages", "--no-table-headers"]), + length(L) =:= 4 andalso + lists:all(fun([Bin]) -> + N = binary_to_integer(Bin), + case QoS of + qos0 -> + N =:= NumMsgs; + qos1 -> + %% Allow for some duplicates when client resends + %% a message that gets acked at roughly the same time. + N >= NumMsgs andalso + N < NumMsgs * 2 + end + end, L) + end), 2000, 10), delete_queue(Ch, [CQ, CMQ, QQ, SQ]), ok = rabbit_ct_broker_helpers:clear_policy(Config, 0, CMQ), From 96854a8c4cd3537105b729627b34dfc900530de1 Mon Sep 17 00:00:00 2001 From: Chunyi Lyu Date: Tue, 8 Nov 2022 14:08:51 +0000 Subject: [PATCH 034/118] Use emqtt:publish in mqtt tests - rename publish_qos1 to publish_qos1_timeout since it's only been used for handling publisher timeout more gracefully in tests --- deps/rabbitmq_mqtt/test/integration_SUITE.erl | 5 ++--- deps/rabbitmq_mqtt/test/reader_SUITE.erl | 8 ++++---- deps/rabbitmq_mqtt/test/util.erl | 4 ++-- 3 files changed, 8 insertions(+), 9 deletions(-) diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/integration_SUITE.erl index 29ae9b0fa5e9..b423ada6213d 100644 --- a/deps/rabbitmq_mqtt/test/integration_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/integration_SUITE.erl @@ -16,8 +16,7 @@ -import(rabbit_ct_broker_helpers, [rabbitmqctl_list/3, rpc_all/4]). -import(rabbit_ct_helpers, [eventually/3]). --import(util, [all_connection_pids/1, - publish_qos1/4]). +-import(util, [all_connection_pids/1]). all() -> [ @@ -128,7 +127,7 @@ publish_to_all_queue_types(Config, QoS) -> qos0 -> ok = emqtt:publish(C, Topic, <<"m">>); qos1 -> - {ok, _} = publish_qos1(C, Topic, <<"m">>, 1000) + {ok, _} = emqtt:publish(C, Topic, <<"m">>, [{qos, 1}]) end end, lists:seq(1, NumMsgs)), diff --git a/deps/rabbitmq_mqtt/test/reader_SUITE.erl b/deps/rabbitmq_mqtt/test/reader_SUITE.erl index e69f68fb59dc..970f468822f4 100644 --- a/deps/rabbitmq_mqtt/test/reader_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/reader_SUITE.erl @@ -13,7 +13,7 @@ -import(rabbit_ct_broker_helpers, [rpc/5]). -import(util, [all_connection_pids/1, - publish_qos1/4]). + publish_qos1_timeout/4]). all() -> [ @@ -101,15 +101,15 @@ block(Config) -> {ok, _, _} = emqtt:unsubscribe(C, <<"TopicA">>), {ok, _, _} = emqtt:subscribe(C, <<"Topic1">>), - {ok, _} = publish_qos1(C, <<"Topic1">>, <<"Not blocked yet">>, 1000), + {ok, _} = emqtt:publish(C, <<"Topic1">>, <<"Not blocked yet">>, [{qos, 1}]), ok = rpc(Config, vm_memory_monitor, set_vm_memory_high_watermark, [0.00000001]), % %% Let it block timer:sleep(100), %% Blocked, but still will publish - puback_timeout = publish_qos1(C, <<"Topic1">>, <<"Now blocked">>, 1000), - puback_timeout = publish_qos1(C, <<"Topic1">>, <<"Still blocked">>, 1000), + puback_timeout = publish_qos1_timeout(C, <<"Topic1">>, <<"Now blocked">>, 1000), + puback_timeout = publish_qos1_timeout(C, <<"Topic1">>, <<"Still blocked">>, 1000), %% Unblock rpc(Config, vm_memory_monitor, set_vm_memory_high_watermark, [0.4]), diff --git a/deps/rabbitmq_mqtt/test/util.erl b/deps/rabbitmq_mqtt/test/util.erl index 4a3034f1c871..4bedb80a5ebb 100644 --- a/deps/rabbitmq_mqtt/test/util.erl +++ b/deps/rabbitmq_mqtt/test/util.erl @@ -1,7 +1,7 @@ -module(util). -export([all_connection_pids/1, - publish_qos1/4, + publish_qos1_timeout/4, sync_publish_result/3]). all_connection_pids(Config) -> @@ -13,7 +13,7 @@ all_connection_pids(Config) -> Acc end, [], Result). -publish_qos1(Client, Topic, Payload, Timeout) -> +publish_qos1_timeout(Client, Topic, Payload, Timeout) -> Mref = erlang:monitor(process, Client), ok = emqtt:publish_async(Client, Topic, #{}, Payload, [{qos, 1}], infinity, {fun ?MODULE:sync_publish_result/3, [self(), Mref]}), From 9fd5704e30cc2a91b699d668d6fe681233b7cf9b Mon Sep 17 00:00:00 2001 From: David Ansari Date: Tue, 8 Nov 2022 15:46:37 +0000 Subject: [PATCH 035/118] Fix mixed version Web MQTT system tests In mixed verion tests all non-required feature flags are disabled. Therefore, MQTT client ID tracking happens in Ra. The Ra client sends commands asynchronously when registering and deregistering the client ID. Also, add more tests. --- .../src/rabbit_mqtt_confirms.erl | 2 +- deps/rabbitmq_mqtt/test/integration_SUITE.erl | 42 +++++++++++++++---- deps/rabbitmq_web_mqtt/test/system_SUITE.erl | 10 +++-- 3 files changed, 40 insertions(+), 14 deletions(-) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl index 8c3d87898039..939c616606e0 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl @@ -54,7 +54,7 @@ confirm(PktIds, QName, State0) -> reject(PktId, State0) when is_integer(PktId) -> case maps:take(PktId, State0) of - {_QMap, State} -> + {_, State} -> {ok, State}; error -> {error, not_found} diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/integration_SUITE.erl index b423ada6213d..214590a11540 100644 --- a/deps/rabbitmq_mqtt/test/integration_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/integration_SUITE.erl @@ -31,7 +31,8 @@ groups() -> ]. tests() -> - [publish_to_all_queue_types_qos0 + [quorum_queue_rejects + ,publish_to_all_queue_types_qos0 ,publish_to_all_queue_types_qos1 ,events ,event_authentication_failure @@ -85,6 +86,35 @@ end_per_testcase(Testcase, Config) -> %% Testsuite cases %% ------------------------------------------------------------------- +quorum_queue_rejects(Config) -> + {_Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0), + Name = atom_to_binary(?FUNCTION_NAME), + + ok = rabbit_ct_broker_helpers:set_policy( + Config, 0, <<"qq-policy">>, Name, <<"queues">>, [{<<"max-length">>, 1}, + {<<"overflow">>, <<"reject-publish">>}]), + declare_queue(Ch, Name, [{<<"x-queue-type">>, longstr, <<"quorum">>}]), + bind(Ch, Name, Name), + + {C, _} = connect(Name, Config, [{retry_interval, 1}]), + {ok, _} = emqtt:publish(C, Name, <<"m1">>, qos1), + {ok, _} = emqtt:publish(C, Name, <<"m2">>, qos1), + %% We expect m3 to be rejected and dropped. + ?assertEqual(puback_timeout, util:publish_qos1_timeout(C, Name, <<"m3">>, 700)), + + ?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = <<"m1">>}}, + amqp_channel:call(Ch, #'basic.get'{queue = Name, no_ack = true})), + ?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = <<"m2">>}}, + amqp_channel:call(Ch, #'basic.get'{queue = Name, no_ack = true})), + %% m3 is re-sent by emqtt. + ?awaitMatch({#'basic.get_ok'{}, #amqp_msg{payload = <<"m3">>}}, + amqp_channel:call(Ch, #'basic.get'{queue = Name, no_ack = true}), + 2000, 200), + + ok = emqtt:disconnect(C), + delete_queue(Ch, Name), + ok = rabbit_ct_broker_helpers:clear_policy(Config, 0, <<"qq-policy">>). + publish_to_all_queue_types_qos0(Config) -> publish_to_all_queue_types(Config, qos0). @@ -99,7 +129,7 @@ publish_to_all_queue_types(Config, QoS) -> Result = rpc_all(Config, application, set_env, [rabbit, stream_messages_soft_limit, 25]), ?assert(lists:all(fun(R) -> R =:= ok end, Result)), - {_Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0), + {Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0), CQ = <<"classic-queue">>, CMQ = <<"classic-mirrored-queue">>, @@ -153,13 +183,7 @@ publish_to_all_queue_types(Config, QoS) -> ok = rabbit_ct_broker_helpers:clear_policy(Config, 0, CMQ), ok = emqtt:disconnect(C), ?awaitMatch([], all_connection_pids(Config), 10_000, 1000), - - Result = rpc_all(Config, application, unset_env, [rabbit, credit_flow_default_credit]), - Result = rpc_all(Config, application, unset_env, [rabbit, quorum_commands_soft_limit]), - Result = rpc_all(Config, application, unset_env, [rabbit, stream_messages_soft_limit]), - ok. - -%%TODO add test where target quorum queue rejects message + ok = rabbit_ct_client_helpers:close_connection_and_channel(Conn, Ch). events(Config) -> ok = rabbit_ct_broker_helpers:add_code_path_to_all_nodes(Config, event_recorder), diff --git a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl index f71aa8e543fe..8c8cfe342fdb 100644 --- a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl +++ b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl @@ -12,6 +12,7 @@ -compile([export_all, nowarn_export_all]). -import(rabbit_ct_broker_helpers, [rpc/5]). +-import(rabbit_ct_helpers, [eventually/1]). all() -> [ @@ -162,7 +163,7 @@ disconnect(Config) -> {port, Port}]), process_flag(trap_exit, true), {ok, _} = emqtt:ws_connect(C), - ?assertEqual(1, num_mqtt_connections(Config, 0)), + eventually(?_assertEqual(1, num_mqtt_connections(Config, 0))), ok = emqtt:disconnect(C), receive {'EXIT', C, normal} -> @@ -170,7 +171,8 @@ disconnect(Config) -> after 5000 -> ct:fail("disconnect didn't terminate client") end, - ?assertEqual(0, num_mqtt_connections(Config, 0)). + eventually(?_assertEqual(0, num_mqtt_connections(Config, 0))), + ok. keepalive(Config) -> KeepaliveSecs = 1, @@ -219,9 +221,9 @@ maintenance(Config) -> {ok, _} = emqtt:ws_connect(C), unlink(C), - ?assertEqual(1, num_mqtt_connections(Config, 0)), + eventually(?_assertEqual(1, num_mqtt_connections(Config, 0))), ok = rabbit_ct_broker_helpers:drain_node(Config, 0), - ?assertEqual(0, num_mqtt_connections(Config, 0)), + eventually(?_assertEqual(0, num_mqtt_connections(Config, 0))), ok = rabbit_ct_broker_helpers:revive_node(Config, 0). %% Web mqtt connections are tracked together with mqtt connections From 17c5dffe7aba58aaac91024736e6c677f8c3ff76 Mon Sep 17 00:00:00 2001 From: Chunyi Lyu Date: Thu, 10 Nov 2022 15:52:46 +0000 Subject: [PATCH 036/118] Set common global counters for mqtt - protocols are set to mqtt301 or mqtt311 depends on protocal version set by client connection - added boolean isPublisher in proc state to track if connection was ever used to publish. This is used to set publisher_create and publisher_delete global counters. - tests added in integration_SUITE --- deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl | 1 + deps/rabbitmq_mqtt/src/rabbit_mqtt.erl | 3 +- .../src/rabbit_mqtt_processor.erl | 93 ++++++++++++++++--- deps/rabbitmq_mqtt/test/integration_SUITE.erl | 65 ++++++++++++- 4 files changed, 145 insertions(+), 17 deletions(-) diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl index b27518d87bb4..ce99d508c623 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl @@ -43,6 +43,7 @@ clean_sess :: boolean(), will_msg, exchange :: rabbit_exchange:name(), + isPublisher :: boolean(), ssl_login_name, %% Retained messages handler. See rabbit_mqtt_retainer_sup %% and rabbit_mqtt_retainer. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl index ac42da990e2b..0322ea30447e 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl @@ -19,7 +19,8 @@ local_connection_pids/0]). start(normal, []) -> - rabbit_global_counters:init([{protocol, mqtt}]), + rabbit_global_counters:init([{protocol, mqtt301}]), + rabbit_global_counters:init([{protocol, mqtt311}]), {ok, Listeners} = application:get_env(tcp_listeners), {ok, SslListeners} = application:get_env(ssl_listeners), case rabbit_mqtt_ff:track_client_id_in_ra() of diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 9e859d1463b5..1a6ffaf42157 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -113,10 +113,18 @@ process_request(?PUBLISH, variable = #mqtt_frame_publish{topic_name = Topic, message_id = MessageId }, payload = Payload}, - PState = #proc_state{retainer_pid = RPid, + PState = #proc_state{isPublisher = IsPublisher, + retainer_pid = RPid, amqp2mqtt_fun = Amqp2MqttFun, unacked_client_pubs = U}) -> - rabbit_global_counters:messages_received(mqtt, 1), + counters_messages_received(PState), + PState1 = case IsPublisher of + true -> + PState; + _ -> + counters_publisher_created(PState), + PState#proc_state{isPublisher = true} + end, Publish = fun() -> Msg = #mqtt_msg{retain = Retain, qos = Qos, @@ -124,7 +132,7 @@ process_request(?PUBLISH, dup = Dup, message_id = MessageId, payload = Payload}, - case publish_to_queues(Msg, PState) of + case publish_to_queues(Msg, PState1) of {ok, _} = Ok -> case Retain of false -> @@ -139,18 +147,18 @@ process_request(?PUBLISH, end, case Qos of N when N > ?QOS_0 -> - rabbit_global_counters:messages_received_confirm(mqtt, 1), + counters_messages_received_confirm(PState1), case rabbit_mqtt_confirms:contains(MessageId, U) of false -> - publish_to_queues_with_checks(Topic, Publish, PState); + publish_to_queues_with_checks(Topic, Publish, PState1); true -> %% Client re-sent this PUBLISH packet. %% We already sent this message to target queues awaiting confirmations. %% Hence, we ignore this re-send. - {ok, PState} + {ok, PState1} end; _ -> - publish_to_queues_with_checks(Topic, Publish, PState) + publish_to_queues_with_checks(Topic, Publish, PState1) end; process_request(?SUBSCRIBE, @@ -243,6 +251,7 @@ process_request(?UNSUBSCRIBE, #mqtt_frame{fixed = #mqtt_frame_fixed {type = ?UNSUBACK}, variable = #mqtt_frame_suback{message_id = MessageId}}, PState), + counters_consumer_deleted(PState), {ok, PState}; process_request(?PINGREQ, #mqtt_frame{}, PState = #proc_state{send_fun = SendFun}) -> @@ -976,6 +985,7 @@ consume(Q, QoS, #proc_state{ rabbit_mqtt_qos0_queue -> %% Messages get delivered directly to our process without %% explicitly calling rabbit_queue_type:consume/3. + counters_consumer_created(PState0), {ok, PState0}; _ -> Spec = #{no_ack => QoS =:= ?QOS_0, @@ -990,7 +1000,7 @@ consume(Q, QoS, #proc_state{ acting_user => Username}, case rabbit_queue_type:consume(Q, Spec, QStates0) of {ok, QStates} -> - % rabbit_global_counters:consumer_created(mqtt), + counters_consumer_created(PState0), PState = PState0#proc_state{queue_states = QStates}, {ok, PState}; {error, Reason} = Err -> @@ -1127,7 +1137,7 @@ deliver_to_queues(Delivery, Qs = rabbit_amqqueue:prepend_extra_bcc(Qs0), case rabbit_queue_type:deliver(Qs, Delivery, QStates0) of {ok, QStates, Actions} -> - rabbit_global_counters:messages_routed(mqtt, length(Qs)), + counters_messages_routed(PState0, length(Qs)), PState = process_routing_confirm(Delivery, Qs, PState0#proc_state{queue_states = QStates}), %% Actions must be processed after registering confirms as actions may @@ -1140,16 +1150,16 @@ deliver_to_queues(Delivery, end. process_routing_confirm(#delivery{confirm = false}, [], PState) -> - rabbit_global_counters:messages_unroutable_dropped(mqtt, 1), + counters_messages_unroutable_dropped(PState), PState; process_routing_confirm(#delivery{confirm = true, msg_seq_no = undefined}, [], PState) -> %% unroutable will message with QoS > 0 - rabbit_global_counters:messages_unroutable_dropped(mqtt, 1), + counters_messages_unroutable_dropped(PState), PState; process_routing_confirm(#delivery{confirm = true, msg_seq_no = MsgId}, [], PState) -> - rabbit_global_counters:messages_unroutable_returned(mqtt, 1), + counters_messages_unroutable_returned(PState), %% MQTT 5 spec: %% If the Server knows that there are no matching subscribers, it MAY use %% Reason Code 0x10 (No matching subscribers) instead of 0x00 (Success). @@ -1174,7 +1184,7 @@ send_puback(MsgIds, PState) send_puback(Id, PState) end, MsgIds); send_puback(MsgId, PState = #proc_state{send_fun = SendFun}) -> - rabbit_global_counters:messages_confirmed(mqtt, 1), + counters_messages_confirmed(PState), SendFun( #mqtt_frame{fixed = #mqtt_frame_fixed{type = ?PUBACK}, variable = #mqtt_frame_publish{message_id = MsgId}}, @@ -1200,6 +1210,8 @@ terminate(PState, ConnName) -> {pid, self()}]), rabbit_networking:unregister_non_amqp_connection(self()), maybe_unregister_client(PState), + counters_consumer_deleted(PState), + counters_publisher_deleted(PState), maybe_delete_mqtt_qos0_queue(PState). maybe_unregister_client(#proc_state{client_id = ClientId}) @@ -1586,3 +1598,58 @@ format_status(#proc_state{queue_states = QState, soft_limit_exceeded(#proc_state{soft_limit_exceeded = SLE}) -> not sets:is_empty(SLE). + +%% Handle global counters +counters_messages_received(#proc_state{proto_ver = ?MQTT_PROTO_V3}) -> + rabbit_global_counters:messages_received(mqtt301, 1); +counters_messages_received(#proc_state{proto_ver = ?MQTT_PROTO_V4}) -> + rabbit_global_counters:messages_received(mqtt311, 1). + +counters_messages_received_confirm(#proc_state{proto_ver = ?MQTT_PROTO_V3}) -> + rabbit_global_counters:messages_received_confirm(mqtt301, 1); +counters_messages_received_confirm(#proc_state{proto_ver = ?MQTT_PROTO_V4}) -> + rabbit_global_counters:messages_received_confirm(mqtt311, 1). + +counters_messages_confirmed(#proc_state{proto_ver = ?MQTT_PROTO_V3}) -> + rabbit_global_counters:messages_confirmed(mqtt301, 1); +counters_messages_confirmed(#proc_state{proto_ver = ?MQTT_PROTO_V4}) -> + rabbit_global_counters:messages_confirmed(mqtt311, 1). + +counters_consumer_created(#proc_state{proto_ver = ?MQTT_PROTO_V3}) -> + rabbit_global_counters:consumer_created(mqtt301); +counters_consumer_created(#proc_state{proto_ver = ?MQTT_PROTO_V4}) -> + rabbit_global_counters:consumer_created(mqtt311). + +counters_consumer_deleted(#proc_state{proto_ver = ?MQTT_PROTO_V3, subscriptions = Sub}) when Sub /= #{} -> + rabbit_global_counters:consumer_deleted(mqtt301); +counters_consumer_deleted(#proc_state{proto_ver = ?MQTT_PROTO_V4, subscriptions = Sub}) when Sub /= #{} -> + rabbit_global_counters:consumer_deleted(mqtt311); +counters_consumer_deleted(_) -> + ok. + +counters_publisher_created(#proc_state{proto_ver = ?MQTT_PROTO_V3}) -> + rabbit_global_counters:publisher_created(mqtt301); +counters_publisher_created(#proc_state{proto_ver = ?MQTT_PROTO_V4}) -> + rabbit_global_counters:publisher_created(mqtt311). + +counters_publisher_deleted(#proc_state{proto_ver = ?MQTT_PROTO_V3, isPublisher = true}) -> + rabbit_global_counters:publisher_deleted(mqtt301); +counters_publisher_deleted(#proc_state{proto_ver = ?MQTT_PROTO_V4, isPublisher = true}) -> + rabbit_global_counters:publisher_deleted(mqtt311); +counters_publisher_deleted(_) -> + ok. + +counters_messages_routed(#proc_state{proto_ver = ?MQTT_PROTO_V3}, Num) -> + rabbit_global_counters:messages_routed(mqtt301, Num); +counters_messages_routed(#proc_state{proto_ver = ?MQTT_PROTO_V4}, Num) -> + rabbit_global_counters:messages_routed(mqtt311, Num). + +counters_messages_unroutable_dropped(#proc_state{proto_ver = ?MQTT_PROTO_V3}) -> + rabbit_global_counters:messages_unroutable_dropped(mqtt301, 1); +counters_messages_unroutable_dropped(#proc_state{proto_ver = ?MQTT_PROTO_V4}) -> + rabbit_global_counters:messages_unroutable_dropped(mqtt311, 1). + +counters_messages_unroutable_returned(#proc_state{proto_ver = ?MQTT_PROTO_V3}) -> + rabbit_global_counters:messages_unroutable_returned(mqtt301, 1); +counters_messages_unroutable_returned(#proc_state{proto_ver = ?MQTT_PROTO_V4}) -> + rabbit_global_counters:messages_unroutable_returned(mqtt311, 1). \ No newline at end of file diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/integration_SUITE.erl index 214590a11540..fa474a39e0c1 100644 --- a/deps/rabbitmq_mqtt/test/integration_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/integration_SUITE.erl @@ -21,13 +21,15 @@ all() -> [ {group, cluster_size_1}, - {group, cluster_size_3} + {group, cluster_size_3}, + {group, single_node} ]. groups() -> [ {cluster_size_1, [], tests()}, - {cluster_size_3, [], tests()} + {cluster_size_3, [], tests()}, + {single_node, [], [test_global_counters_v3, test_global_counters_v4]} ]. tests() -> @@ -57,7 +59,10 @@ init_per_group(cluster_size_1 = Group, Config0) -> rabbit_ct_helpers:set_config(Config0, [{rmq_nodes_count, 1}])); init_per_group(cluster_size_3 = Group, Config0) -> init_per_group0(Group, - rabbit_ct_helpers:set_config(Config0, [{rmq_nodes_count, 3}])). + rabbit_ct_helpers:set_config(Config0, [{rmq_nodes_count, 3}])); +init_per_group(single_node = Group, Config0) -> + init_per_group0(Group, + rabbit_ct_helpers:set_config(Config0, [{rmq_nodes_count, 1}])). init_per_group0(Group, Config0) -> Config = rabbit_ct_helpers:set_config( @@ -262,6 +267,45 @@ event_authentication_failure(Config) -> ok = gen_event:delete_handler({rabbit_event, Server}, event_recorder, []). +test_global_counters_v3(Config) -> + test_global_counters(Config, v3). +test_global_counters_v4(Config) -> + test_global_counters(Config, v4). +test_global_counters(Config, ProtoVer) -> + Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), + {ok, C} = emqtt:start_link([{host, "localhost"}, + {port, Port}, + {proto_ver, ProtoVer}, + {clientid, atom_to_binary(?FUNCTION_NAME)}]), + {ok, _Properties} = emqtt:connect(C), + + Topic = <<"test-topic">>, + {ok, _, [1]} = emqtt:subscribe(C, Topic, [{qos, 1}]), + {ok, _} = emqtt:publish(C, Topic, <<"testm">>, [{qos, 1}]), + {ok, _} = emqtt:publish(C, Topic, <<"testm">>, [{qos, 1}]), + + ?assertEqual(#{publishers => 1, + consumers => 1, + messages_confirmed_total => 2, + messages_received_confirm_total => 2, + messages_received_total => 2, + messages_routed_total => 2, + messages_unroutable_dropped_total => 0, + messages_unroutable_returned_total => 0}, + get_global_counters(Config, ProtoVer)), + + ok = emqtt:disconnect(C), + + ?assertEqual(#{publishers => 0, + consumers => 0, + messages_confirmed_total => 2, + messages_received_confirm_total => 2, + messages_received_total => 2, + messages_routed_total => 2, + messages_unroutable_dropped_total => 0, + messages_unroutable_returned_total => 0}, + get_global_counters(Config, ProtoVer)). + %% ------------------------------------------------------------------- %% Internal helpers %% ------------------------------------------------------------------- @@ -325,3 +369,18 @@ assert_event_prop(ExpectedProps, Event) lists:foreach(fun(P) -> assert_event_prop(P, Event) end, ExpectedProps). + +get_global_counters(Config, v3) -> + maps:get([{protocol, mqtt301}], + rabbit_ct_broker_helpers:rpc(Config, + 0, + rabbit_global_counters, + overview, + [])); +get_global_counters(Config, v4) -> + maps:get([{protocol, mqtt311}], + rabbit_ct_broker_helpers:rpc(Config, + 0, + rabbit_global_counters, + overview, + [])). \ No newline at end of file From 38e5e20bb8ef879780734f9e61f811146ac3cc96 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Thu, 10 Nov 2022 18:55:47 +0000 Subject: [PATCH 037/118] Add tests --- .../src/rabbit_mqtt_processor.erl | 11 ++- deps/rabbitmq_mqtt/test/integration_SUITE.erl | 69 ++++++++++++------- deps/rabbitmq_mqtt/test/reader_SUITE.erl | 26 +++++++ 3 files changed, 80 insertions(+), 26 deletions(-) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 1a6ffaf42157..0cc995a753cb 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -1324,8 +1324,8 @@ handle_queue_actions(Actions, #proc_state{} = PState0) -> send_puback(ConfirmMsgIds, S), S#proc_state{unacked_client_pubs = U}; ({rejected, _QName, MsgIds}, S = #proc_state{unacked_client_pubs = U0}) -> - %% Negative acks are supported in MQTT 5 only. - %% Therefore, in MQTT 3 we ignore rejected messages. + %% Negative acks are supported in MQTT v5 only. + %% Therefore, in MQTT v3 and v4 we ignore rejected messages. U = lists:foldl( fun(MsgId, Acc0) -> case rabbit_mqtt_confirms:reject(MsgId, Acc0) of @@ -1337,7 +1337,12 @@ handle_queue_actions(Actions, #proc_state{} = PState0) -> ({block, QName}, S = #proc_state{soft_limit_exceeded = SLE}) -> S#proc_state{soft_limit_exceeded = sets:add_element(QName, SLE)}; ({unblock, QName}, S = #proc_state{soft_limit_exceeded = SLE}) -> - S#proc_state{soft_limit_exceeded = sets:del_element(QName, SLE)} + S#proc_state{soft_limit_exceeded = sets:del_element(QName, SLE)}; + ({queue_down, _QName}, S) -> + %% classic queue is down, but not deleted + %% TODO if we were consuming from that queue: + %% remove subscription? recover if we support classic mirrored queues? see channel + S end, PState0, Actions). deliver_to_client(Msgs, Ack, PState) -> diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/integration_SUITE.erl index fa474a39e0c1..54e710491ba1 100644 --- a/deps/rabbitmq_mqtt/test/integration_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/integration_SUITE.erl @@ -21,15 +21,18 @@ all() -> [ {group, cluster_size_1}, - {group, cluster_size_3}, - {group, single_node} + {group, cluster_size_3} ]. groups() -> [ - {cluster_size_1, [], tests()}, - {cluster_size_3, [], tests()}, - {single_node, [], [test_global_counters_v3, test_global_counters_v4]} + {cluster_size_1, [], + [global_counters_v3, global_counters_v4] + ++ tests() + }, + {cluster_size_3, [], + [queue_down_qos1]} + ++ tests() ]. tests() -> @@ -59,10 +62,7 @@ init_per_group(cluster_size_1 = Group, Config0) -> rabbit_ct_helpers:set_config(Config0, [{rmq_nodes_count, 1}])); init_per_group(cluster_size_3 = Group, Config0) -> init_per_group0(Group, - rabbit_ct_helpers:set_config(Config0, [{rmq_nodes_count, 3}])); -init_per_group(single_node = Group, Config0) -> - init_per_group0(Group, - rabbit_ct_helpers:set_config(Config0, [{rmq_nodes_count, 1}])). + rabbit_ct_helpers:set_config(Config0, [{rmq_nodes_count, 3}])). init_per_group0(Group, Config0) -> Config = rabbit_ct_helpers:set_config( @@ -267,11 +267,13 @@ event_authentication_failure(Config) -> ok = gen_event:delete_handler({rabbit_event, Server}, event_recorder, []). -test_global_counters_v3(Config) -> - test_global_counters(Config, v3). -test_global_counters_v4(Config) -> - test_global_counters(Config, v4). -test_global_counters(Config, ProtoVer) -> +global_counters_v3(Config) -> + global_counters(Config, v3). + +global_counters_v4(Config) -> + global_counters(Config, v4). + +global_counters(Config, ProtoVer) -> Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), {ok, C} = emqtt:start_link([{host, "localhost"}, {port, Port}, @@ -306,6 +308,29 @@ test_global_counters(Config, ProtoVer) -> messages_unroutable_returned_total => 0}, get_global_counters(Config, ProtoVer)). +queue_down_qos1(Config) -> + {Conn1, Ch1} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 1), + CQ = Topic = atom_to_binary(?FUNCTION_NAME), + declare_queue(Ch1, CQ, []), + bind(Ch1, CQ, Topic), + ok = rabbit_ct_client_helpers:close_connection_and_channel(Conn1, Ch1), + ok = rabbit_ct_broker_helpers:stop_node(Config, 1), + + {C, _} = connect(?FUNCTION_NAME, Config, [{retry_interval, 2}]), + %% classic queue is down, therefore message is rejected + ?assertEqual(puback_timeout, util:publish_qos1_timeout(C, Topic, <<"msg">>, 500)), + + ok = rabbit_ct_broker_helpers:start_node(Config, 1), + %% classic queue is up, therefore message should arrive + eventually(?_assertEqual([[<<"1">>]], + rabbitmqctl_list(Config, 1, ["list_queues", "messages", "--no-table-headers"])), + 500, 20), + + {Conn0, Ch0} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0), + delete_queue(Ch0, CQ), + ok = rabbit_ct_client_helpers:close_connection_and_channel(Conn0, Ch0), + ok = emqtt:disconnect(C). + %% ------------------------------------------------------------------- %% Internal helpers %% ------------------------------------------------------------------- @@ -371,16 +396,14 @@ assert_event_prop(ExpectedProps, Event) end, ExpectedProps). get_global_counters(Config, v3) -> - maps:get([{protocol, mqtt301}], + get_global_counters0(Config, mqtt301); +get_global_counters(Config, v4) -> + get_global_counters0(Config, mqtt311). + +get_global_counters0(Config, Proto) -> + maps:get([{protocol, Proto}], rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_global_counters, overview, - [])); -get_global_counters(Config, v4) -> - maps:get([{protocol, mqtt311}], - rabbit_ct_broker_helpers:rpc(Config, - 0, - rabbit_global_counters, - overview, - [])). \ No newline at end of file + [])). diff --git a/deps/rabbitmq_mqtt/test/reader_SUITE.erl b/deps/rabbitmq_mqtt/test/reader_SUITE.erl index 970f468822f4..00d7fc35d0cf 100644 --- a/deps/rabbitmq_mqtt/test/reader_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/reader_SUITE.erl @@ -12,6 +12,7 @@ -include_lib("eunit/include/eunit.hrl"). -import(rabbit_ct_broker_helpers, [rpc/5]). +-import(rabbit_ct_helpers, [consistently/1]). -import(util, [all_connection_pids/1, publish_qos1_timeout/4]). @@ -28,6 +29,7 @@ groups() -> handle_invalid_frames, login_timeout, keepalive, + keepalive_turned_off, stats, clean_session_disconnect_client, clean_session_kill_node, @@ -221,6 +223,30 @@ keepalive(Config) -> true = rpc(Config, 0, meck, validate, [Mod]), ok = rpc(Config, 0, meck, unload, [Mod]). +keepalive_turned_off(Config) -> + %% "A Keep Alive value of zero (0) has the effect of turning off the keep alive mechanism." + KeepaliveSecs = 0, + P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), + {ok, C} = emqtt:start_link([{keepalive, KeepaliveSecs}, + {host, "localhost"}, + {port, P}, + {clientid, atom_to_binary(?FUNCTION_NAME)}, + {proto_ver, v4} + ]), + {ok, _Properties} = emqtt:connect(C), + + %% Mock the server socket to not have received any bytes. + rabbit_ct_broker_helpers:setup_meck(Config), + Mod = rabbit_net, + ok = rpc(Config, 0, meck, new, [Mod, [no_link, passthrough]]), + ok = rpc(Config, 0, meck, expect, [Mod, getstat, 2, {ok, [{recv_oct, 999}]} ]), + + consistently(?_assert(erlang:is_process_alive(C))), + + true = rpc(Config, 0, meck, validate, [Mod]), + ok = rpc(Config, 0, meck, unload, [Mod]), + ok = emqtt:disconnect(C). + stats(Config) -> P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), {ok, C} = emqtt:start_link([{host, "localhost"}, From de984d026b40edb9b4adfbd27fca56e16b440c79 Mon Sep 17 00:00:00 2001 From: Chunyi Lyu Date: Fri, 11 Nov 2022 12:26:55 +0000 Subject: [PATCH 038/118] Subs from 1 connection counts as 1 consumer in global counter - rename proc state isPublisher to has_published - create macro for v3 and v4 mqtt protocol name for global counters - sub groups in integration suite --- deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl | 4 +- deps/rabbitmq_mqtt/src/rabbit_mqtt.erl | 4 +- .../src/rabbit_mqtt_processor.erl | 82 +++++++++++-------- deps/rabbitmq_mqtt/test/integration_SUITE.erl | 46 ++++++----- 4 files changed, 78 insertions(+), 58 deletions(-) diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl index ce99d508c623..5a88ef117680 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl @@ -7,6 +7,8 @@ -define(CLIENT_ID_MAXLEN, 23). -define(PG_SCOPE, pg_scope_rabbitmq_mqtt_clientid). +-define(V3_GLOBAL_COUNTER_PROTO, mqtt310). +-define(V4_GLOBAL_COUNTER_PROTO, mqtt311). -include("rabbit_mqtt_types.hrl"). @@ -43,7 +45,7 @@ clean_sess :: boolean(), will_msg, exchange :: rabbit_exchange:name(), - isPublisher :: boolean(), + has_published :: boolean(), ssl_login_name, %% Retained messages handler. See rabbit_mqtt_retainer_sup %% and rabbit_mqtt_retainer. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl index 0322ea30447e..1c06278bf4e5 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl @@ -19,8 +19,8 @@ local_connection_pids/0]). start(normal, []) -> - rabbit_global_counters:init([{protocol, mqtt301}]), - rabbit_global_counters:init([{protocol, mqtt311}]), + rabbit_global_counters:init([{protocol, ?V3_GLOBAL_COUNTER_PROTO}]), + rabbit_global_counters:init([{protocol, ?V4_GLOBAL_COUNTER_PROTO}]), {ok, Listeners} = application:get_env(tcp_listeners), {ok, SslListeners} = application:get_env(ssl_listeners), case rabbit_mqtt_ff:track_client_id_in_ra() of diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 0cc995a753cb..784957718480 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -113,18 +113,11 @@ process_request(?PUBLISH, variable = #mqtt_frame_publish{topic_name = Topic, message_id = MessageId }, payload = Payload}, - PState = #proc_state{isPublisher = IsPublisher, - retainer_pid = RPid, + PState = #proc_state{retainer_pid = RPid, amqp2mqtt_fun = Amqp2MqttFun, unacked_client_pubs = U}) -> counters_messages_received(PState), - PState1 = case IsPublisher of - true -> - PState; - _ -> - counters_publisher_created(PState), - PState#proc_state{isPublisher = true} - end, + PState1 = maybe_increment_publisher(PState), Publish = fun() -> Msg = #mqtt_msg{retain = Retain, qos = Qos, @@ -193,6 +186,7 @@ process_request(?SUBSCRIBE, %% for the same queue case consume(Q, QoS, S2) of {ok, S} -> + maybe_increment_consumer(S1), {[QoS | L], S}; {error, _Reason} -> {[?SUBACK_FAILURE | L], S2} @@ -251,7 +245,7 @@ process_request(?UNSUBSCRIBE, #mqtt_frame{fixed = #mqtt_frame_fixed {type = ?UNSUBACK}, variable = #mqtt_frame_suback{message_id = MessageId}}, PState), - counters_consumer_deleted(PState), + maybe_decrement_consumer(PState0, PState), {ok, PState}; process_request(?PINGREQ, #mqtt_frame{}, PState = #proc_state{send_fun = SendFun}) -> @@ -985,7 +979,6 @@ consume(Q, QoS, #proc_state{ rabbit_mqtt_qos0_queue -> %% Messages get delivered directly to our process without %% explicitly calling rabbit_queue_type:consume/3. - counters_consumer_created(PState0), {ok, PState0}; _ -> Spec = #{no_ack => QoS =:= ?QOS_0, @@ -1000,7 +993,6 @@ consume(Q, QoS, #proc_state{ acting_user => Username}, case rabbit_queue_type:consume(Q, Spec, QStates0) of {ok, QStates} -> - counters_consumer_created(PState0), PState = PState0#proc_state{queue_states = QStates}, {ok, PState}; {error, Reason} = Err -> @@ -1606,55 +1598,75 @@ soft_limit_exceeded(#proc_state{soft_limit_exceeded = SLE}) -> %% Handle global counters counters_messages_received(#proc_state{proto_ver = ?MQTT_PROTO_V3}) -> - rabbit_global_counters:messages_received(mqtt301, 1); + rabbit_global_counters:messages_received(?V3_GLOBAL_COUNTER_PROTO, 1); counters_messages_received(#proc_state{proto_ver = ?MQTT_PROTO_V4}) -> - rabbit_global_counters:messages_received(mqtt311, 1). + rabbit_global_counters:messages_received(?V4_GLOBAL_COUNTER_PROTO, 1). counters_messages_received_confirm(#proc_state{proto_ver = ?MQTT_PROTO_V3}) -> - rabbit_global_counters:messages_received_confirm(mqtt301, 1); + rabbit_global_counters:messages_received_confirm(?V3_GLOBAL_COUNTER_PROTO, 1); counters_messages_received_confirm(#proc_state{proto_ver = ?MQTT_PROTO_V4}) -> - rabbit_global_counters:messages_received_confirm(mqtt311, 1). + rabbit_global_counters:messages_received_confirm(?V4_GLOBAL_COUNTER_PROTO, 1). counters_messages_confirmed(#proc_state{proto_ver = ?MQTT_PROTO_V3}) -> - rabbit_global_counters:messages_confirmed(mqtt301, 1); + rabbit_global_counters:messages_confirmed(?V3_GLOBAL_COUNTER_PROTO, 1); counters_messages_confirmed(#proc_state{proto_ver = ?MQTT_PROTO_V4}) -> - rabbit_global_counters:messages_confirmed(mqtt311, 1). + rabbit_global_counters:messages_confirmed(?V4_GLOBAL_COUNTER_PROTO, 1). +%% one or multiple subscriptions from the same connection count as one consumer +maybe_increment_consumer(PState = #proc_state{subscriptions = Sub}) when map_size(Sub) =:= 0 -> + counters_consumer_created(PState); +maybe_increment_consumer(_) -> + ok. counters_consumer_created(#proc_state{proto_ver = ?MQTT_PROTO_V3}) -> - rabbit_global_counters:consumer_created(mqtt301); + rabbit_global_counters:consumer_created(?V3_GLOBAL_COUNTER_PROTO); counters_consumer_created(#proc_state{proto_ver = ?MQTT_PROTO_V4}) -> - rabbit_global_counters:consumer_created(mqtt311). - -counters_consumer_deleted(#proc_state{proto_ver = ?MQTT_PROTO_V3, subscriptions = Sub}) when Sub /= #{} -> - rabbit_global_counters:consumer_deleted(mqtt301); -counters_consumer_deleted(#proc_state{proto_ver = ?MQTT_PROTO_V4, subscriptions = Sub}) when Sub /= #{} -> - rabbit_global_counters:consumer_deleted(mqtt311); + rabbit_global_counters:consumer_created(?V4_GLOBAL_COUNTER_PROTO). + +%% only decrement consumer count if there is no other subscription from this connection +maybe_decrement_consumer(PState0 = #proc_state{subscriptions = Sub0}, + #proc_state{subscriptions = Sub}) + when map_size(Sub0) =/= 0, map_size(Sub) =:= 0 -> + counters_consumer_deleted(PState0); +maybe_decrement_consumer(_, _) -> + ok. +counters_consumer_deleted(#proc_state{proto_ver = ?MQTT_PROTO_V3, subscriptions = Sub}) when map_size(Sub) =/= 0 -> + rabbit_global_counters:consumer_deleted(?V3_GLOBAL_COUNTER_PROTO); +counters_consumer_deleted(#proc_state{proto_ver = ?MQTT_PROTO_V4, subscriptions = Sub}) when map_size(Sub) =/= 0 -> + rabbit_global_counters:consumer_deleted(?V4_GLOBAL_COUNTER_PROTO); counters_consumer_deleted(_) -> ok. +maybe_increment_publisher(PState = #proc_state{has_published = HasPublished}) -> + case HasPublished of + true -> + PState; + _ -> + counters_publisher_created(PState), + PState#proc_state{has_published = true} + end. counters_publisher_created(#proc_state{proto_ver = ?MQTT_PROTO_V3}) -> - rabbit_global_counters:publisher_created(mqtt301); + rabbit_global_counters:publisher_created(?V3_GLOBAL_COUNTER_PROTO); counters_publisher_created(#proc_state{proto_ver = ?MQTT_PROTO_V4}) -> - rabbit_global_counters:publisher_created(mqtt311). + rabbit_global_counters:publisher_created(?V4_GLOBAL_COUNTER_PROTO). -counters_publisher_deleted(#proc_state{proto_ver = ?MQTT_PROTO_V3, isPublisher = true}) -> - rabbit_global_counters:publisher_deleted(mqtt301); -counters_publisher_deleted(#proc_state{proto_ver = ?MQTT_PROTO_V4, isPublisher = true}) -> - rabbit_global_counters:publisher_deleted(mqtt311); +counters_publisher_deleted(#proc_state{proto_ver = ?MQTT_PROTO_V3, has_published = true}) -> + rabbit_global_counters:publisher_deleted(?V3_GLOBAL_COUNTER_PROTO); +counters_publisher_deleted(#proc_state{proto_ver = ?MQTT_PROTO_V4, has_published = true}) -> + rabbit_global_counters:publisher_deleted(?V4_GLOBAL_COUNTER_PROTO); counters_publisher_deleted(_) -> ok. counters_messages_routed(#proc_state{proto_ver = ?MQTT_PROTO_V3}, Num) -> - rabbit_global_counters:messages_routed(mqtt301, Num); + rabbit_global_counters:messages_routed(?V3_GLOBAL_COUNTER_PROTO, Num); counters_messages_routed(#proc_state{proto_ver = ?MQTT_PROTO_V4}, Num) -> - rabbit_global_counters:messages_routed(mqtt311, Num). + rabbit_global_counters:messages_routed(?V4_GLOBAL_COUNTER_PROTO, Num). counters_messages_unroutable_dropped(#proc_state{proto_ver = ?MQTT_PROTO_V3}) -> - rabbit_global_counters:messages_unroutable_dropped(mqtt301, 1); + rabbit_global_counters:messages_unroutable_dropped(?V3_GLOBAL_COUNTER_PROTO, 1); counters_messages_unroutable_dropped(#proc_state{proto_ver = ?MQTT_PROTO_V4}) -> rabbit_global_counters:messages_unroutable_dropped(mqtt311, 1). counters_messages_unroutable_returned(#proc_state{proto_ver = ?MQTT_PROTO_V3}) -> - rabbit_global_counters:messages_unroutable_returned(mqtt301, 1); + rabbit_global_counters:messages_unroutable_returned(?V3_GLOBAL_COUNTER_PROTO, 1); counters_messages_unroutable_returned(#proc_state{proto_ver = ?MQTT_PROTO_V4}) -> rabbit_global_counters:messages_unroutable_returned(mqtt311, 1). \ No newline at end of file diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/integration_SUITE.erl index 54e710491ba1..415d906b0f89 100644 --- a/deps/rabbitmq_mqtt/test/integration_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/integration_SUITE.erl @@ -8,10 +8,10 @@ -compile([export_all, nowarn_export_all]). --include_lib("common_test/include/ct.hrl"). -include_lib("eunit/include/eunit.hrl"). -include_lib("amqp_client/include/amqp_client.hrl"). -include_lib("rabbitmq_ct_helpers/include/rabbit_assert.hrl"). +-include("rabbit_mqtt.hrl"). -import(rabbit_ct_broker_helpers, [rabbitmqctl_list/3, rpc_all/4]). @@ -27,15 +27,16 @@ all() -> groups() -> [ {cluster_size_1, [], - [global_counters_v3, global_counters_v4] - ++ tests() - }, + [ + {global_counters, [], [global_counters_v3, global_counters_v4]}, % separate RMQ so global counters start from 0 + {common_tests, [], common_tests()} + ]}, {cluster_size_3, [], - [queue_down_qos1]} - ++ tests() + [queue_down_qos1] + ++ common_tests()} ]. -tests() -> +common_tests() -> [quorum_queue_rejects ,publish_to_all_queue_types_qos0 ,publish_to_all_queue_types_qos1 @@ -57,12 +58,14 @@ init_per_suite(Config) -> end_per_suite(Config) -> rabbit_ct_helpers:run_teardown_steps(Config). -init_per_group(cluster_size_1 = Group, Config0) -> +init_per_group(cluster_size_1, Config) -> + rabbit_ct_helpers:set_config(Config, [{rmq_nodes_count, 1}]); +init_per_group(cluster_size_3 = Group, Config) -> init_per_group0(Group, - rabbit_ct_helpers:set_config(Config0, [{rmq_nodes_count, 1}])); -init_per_group(cluster_size_3 = Group, Config0) -> - init_per_group0(Group, - rabbit_ct_helpers:set_config(Config0, [{rmq_nodes_count, 3}])). + rabbit_ct_helpers:set_config(Config, [{rmq_nodes_count, 3}])); + +init_per_group(Group, Config) when Group =:= global_counters orelse Group =:= common_tests -> + init_per_group0(Group,Config). init_per_group0(Group, Config0) -> Config = rabbit_ct_helpers:set_config( @@ -281,10 +284,12 @@ global_counters(Config, ProtoVer) -> {clientid, atom_to_binary(?FUNCTION_NAME)}]), {ok, _Properties} = emqtt:connect(C), - Topic = <<"test-topic">>, - {ok, _, [1]} = emqtt:subscribe(C, Topic, [{qos, 1}]), - {ok, _} = emqtt:publish(C, Topic, <<"testm">>, [{qos, 1}]), - {ok, _} = emqtt:publish(C, Topic, <<"testm">>, [{qos, 1}]), + Topic0 = <<"test-topic0">>, + Topic1 = <<"test-topic1">>, + {ok, _, [1]} = emqtt:subscribe(C, Topic0, qos1), + {ok, _, [1]} = emqtt:subscribe(C, Topic1, qos1), + {ok, _} = emqtt:publish(C, Topic0, <<"testm">>, qos1), + {ok, _} = emqtt:publish(C, Topic1, <<"testm">>, qos1), ?assertEqual(#{publishers => 1, consumers => 1, @@ -296,8 +301,10 @@ global_counters(Config, ProtoVer) -> messages_unroutable_returned_total => 0}, get_global_counters(Config, ProtoVer)), - ok = emqtt:disconnect(C), + {ok, _, _} = emqtt:unsubscribe(C, Topic1), + ?assertEqual(1, maps:get(consumers, get_global_counters(Config, ProtoVer))), + ok = emqtt:disconnect(C), ?assertEqual(#{publishers => 0, consumers => 0, messages_confirmed_total => 2, @@ -396,10 +403,9 @@ assert_event_prop(ExpectedProps, Event) end, ExpectedProps). get_global_counters(Config, v3) -> - get_global_counters0(Config, mqtt301); + get_global_counters0(Config, ?V3_GLOBAL_COUNTER_PROTO); get_global_counters(Config, v4) -> - get_global_counters0(Config, mqtt311). - + get_global_counters0(Config, ?V4_GLOBAL_COUNTER_PROTO). get_global_counters0(Config, Proto) -> maps:get([{protocol, Proto}], rabbit_ct_broker_helpers:rpc(Config, From ab5007a53be7266605dcf6e6344dac228012be1e Mon Sep 17 00:00:00 2001 From: David Ansari Date: Fri, 11 Nov 2022 17:56:32 +0000 Subject: [PATCH 039/118] Handle queue deletion Handle deletion of queues correctly that an MQTT connection is publishing to with QoS 1. --- deps/rabbit/src/rabbit_fifo_client.erl | 5 + .../src/rabbit_mqtt_confirms.erl | 16 ++- .../src/rabbit_mqtt_processor.erl | 21 +++- deps/rabbitmq_mqtt/test/integration_SUITE.erl | 115 +++++++++++++++--- 4 files changed, 129 insertions(+), 28 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo_client.erl b/deps/rabbit/src/rabbit_fifo_client.erl index bb4a1980d4ab..2e916f7fb5b4 100644 --- a/deps/rabbit/src/rabbit_fifo_client.erl +++ b/deps/rabbit/src/rabbit_fifo_client.erl @@ -541,6 +541,11 @@ handle_ra_event(From, {applied, Seqs}, [] -> lists:reverse(Actions0); _ -> + %%TODO consider using lists:foldr/3 above because + %% Corrs is returned in the wrong order here. + %% The wrong order does not matter much because the channel sorts the + %% sequence numbers before confirming to the client. But rabbit_fifo_client + %% is sequence numer agnostic: it handles any correlation terms. [{settled, QRef, Corrs} | lists:reverse(Actions0)] end, diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl index 939c616606e0..0cfd899a8a4b 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl @@ -18,8 +18,12 @@ size/1, contains/2]). +%% As done in OTP's sets module: +%% Empty list is cheaper to serialize than atom. +-define(VALUE, []). + -type queue_name() :: rabbit_amqqueue:name(). --opaque state() :: #{packet_id() => #{queue_name() => ok}}. +-opaque state() :: #{packet_id() => #{queue_name() => ?VALUE}}. -export_type([state/0]). -spec init() -> state(). @@ -39,15 +43,17 @@ insert(PktId, QNames, State) when is_integer(PktId) andalso PktId > 0 andalso not is_map_key(PktId, State) -> - QMap = maps:from_keys(QNames, ok), + QMap = maps:from_keys(QNames, ?VALUE), maps:put(PktId, QMap, State). -spec confirm([packet_id()], queue_name(), state()) -> {[packet_id()], state()}. confirm(PktIds, QName, State0) -> - lists:foldr(fun(PktId, Acc) -> - confirm_one(PktId, QName, Acc) - end, {[], State0}, PktIds). + {L0, State} = lists:foldl(fun(PktId, Acc) -> + confirm_one(PktId, QName, Acc) + end, {[], State0}, PktIds), + L = lists:reverse(L0), + {L, State}. -spec reject(packet_id(), state()) -> {ok, state()} | {error, not_found}. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 784957718480..fcbaeb67fa59 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -1170,8 +1170,11 @@ process_routing_confirm(#delivery{confirm = true, U = rabbit_mqtt_confirms:insert(MsgId, QNames, U0), PState#proc_state{unacked_client_pubs = U}. -send_puback(MsgIds, PState) - when is_list(MsgIds) -> +send_puback(MsgIds0, PState) + when is_list(MsgIds0) -> + %% Classic queues confirm messages unordered. + %% Let's sort them here assuming most MQTT clients send with an increasing packet identifier. + MsgIds = lists:usort(MsgIds0), lists:foreach(fun(Id) -> send_puback(Id, PState) end, MsgIds); @@ -1274,15 +1277,20 @@ handle_ra_event(Evt, PState) -> PState. handle_down({{'DOWN', QName}, _MRef, process, QPid, Reason}, - PState0 = #proc_state{queue_states = QStates0}) -> + PState0 = #proc_state{queue_states = QStates0, + unacked_client_pubs = U0}) -> credit_flow:peer_down(QPid), case rabbit_queue_type:handle_down(QPid, QName, Reason, QStates0) of {ok, QStates1, Actions} -> PState = PState0#proc_state{queue_states = QStates1}, handle_queue_actions(Actions, PState); {eol, QStates1, QRef} -> + {ConfirmMsgIds, U} = rabbit_mqtt_confirms:remove_queue(QRef, U0), QStates = rabbit_queue_type:remove(QRef, QStates1), - PState0#proc_state{queue_states = QStates} + PState = PState0#proc_state{queue_states = QStates, + unacked_client_pubs = U}, + send_puback(ConfirmMsgIds, PState), + PState end. handle_queue_event({queue_event, rabbit_mqtt_qos0_queue, Msg}, PState0) -> @@ -1300,7 +1308,9 @@ handle_queue_event({queue_event, QName, Evt}, %%TODO handle consuming queue down % State1 = handle_consuming_queue_down_or_eol(QRef, State0), {ConfirmMsgIds, U} = rabbit_mqtt_confirms:remove_queue(QName, U0), - PState = PState0#proc_state{unacked_client_pubs = U}, + QStates = rabbit_queue_type:remove(QName, QStates0), + PState = PState0#proc_state{queue_states = QStates, + unacked_client_pubs = U}, send_puback(ConfirmMsgIds, PState), {ok, PState}; {protocol_error, _Type, _Reason, _ReasonArgs} = Error -> @@ -1334,6 +1344,7 @@ handle_queue_actions(Actions, #proc_state{} = PState0) -> %% classic queue is down, but not deleted %% TODO if we were consuming from that queue: %% remove subscription? recover if we support classic mirrored queues? see channel + % State1 = handle_consuming_queue_down_or_eol(QRef, State0), S end, PState0, Actions). diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/integration_SUITE.erl index 415d906b0f89..e8701a160c3c 100644 --- a/deps/rabbitmq_mqtt/test/integration_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/integration_SUITE.erl @@ -28,8 +28,9 @@ groups() -> [ {cluster_size_1, [], [ - {global_counters, [], [global_counters_v3, global_counters_v4]}, % separate RMQ so global counters start from 0 - {common_tests, [], common_tests()} + %% separate RMQ so global counters start from 0 + {global_counters, [], [global_counters_v3, global_counters_v4]}, + {common_tests, [], common_tests()} ]}, {cluster_size_3, [], [queue_down_qos1] @@ -37,7 +38,8 @@ groups() -> ]. common_tests() -> - [quorum_queue_rejects + [delete_create_queue + ,quorum_queue_rejects ,publish_to_all_queue_types_qos0 ,publish_to_all_queue_types_qos1 ,events @@ -68,15 +70,18 @@ init_per_group(Group, Config) when Group =:= global_counters orelse Group =:= co init_per_group0(Group,Config). init_per_group0(Group, Config0) -> - Config = rabbit_ct_helpers:set_config( - Config0, - [{rmq_nodename_suffix, Group}, - {rmq_extra_tcp_ports, [tcp_port_mqtt_extra, - tcp_port_mqtt_tls_extra]}]), - rabbit_ct_helpers:run_steps( - Config, - rabbit_ct_broker_helpers:setup_steps() ++ - rabbit_ct_client_helpers:setup_steps()). + Config1 = rabbit_ct_helpers:set_config( + Config0, + [{rmq_nodename_suffix, Group}, + {rmq_extra_tcp_ports, [tcp_port_mqtt_extra, + tcp_port_mqtt_tls_extra]}]), + Config = rabbit_ct_helpers:run_steps( + Config1, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps()), + Result = rpc_all(Config, application, set_env, [rabbit, classic_queue_default_version, 2]), + ?assert(lists:all(fun(R) -> R =:= ok end, Result)), + Config. end_per_group(_, Config) -> rabbit_ct_helpers:run_teardown_steps( @@ -160,12 +165,14 @@ publish_to_all_queue_types(Config, QoS) -> NumMsgs = 2000, {C, _} = connect(?FUNCTION_NAME, Config, [{retry_interval, 2}]), - lists:foreach(fun(_N) -> - case QoS of - qos0 -> - ok = emqtt:publish(C, Topic, <<"m">>); - qos1 -> - {ok, _} = emqtt:publish(C, Topic, <<"m">>, [{qos, 1}]) + lists:foreach(fun(N) -> + case emqtt:publish(C, Topic, integer_to_binary(N), QoS) of + ok -> + ok; + {ok, _} -> + ok; + Other -> + ct:fail("Failed to publish: ~p", [Other]) end end, lists:seq(1, NumMsgs)), @@ -338,6 +345,78 @@ queue_down_qos1(Config) -> ok = rabbit_ct_client_helpers:close_connection_and_channel(Conn0, Ch0), ok = emqtt:disconnect(C). +delete_create_queue(Config) -> + {Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0), + CQ1 = <<"classic-queue-1-delete-create">>, + CQ2 = <<"classic-queue-2-delete-create">>, + QQ = <<"quorum-queue-delete-create">>, + Topic = atom_to_binary(?FUNCTION_NAME), + + DeclareQueues = fun() -> + declare_queue(Ch, CQ1, []), + bind(Ch, CQ1, Topic), + declare_queue(Ch, CQ2, []), + bind(Ch, CQ2, Topic), + declare_queue(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}]), + bind(Ch, QQ, Topic) + end, + DeclareQueues(), + + %% some large retry_interval to avoid re-sending + {C, _} = connect(?FUNCTION_NAME, Config, [{retry_interval, 300}]), + NumMsgs = 50, + TestPid = self(), + spawn( + fun() -> + lists:foreach( + fun(N) -> + ok = emqtt:publish_async(C, Topic, integer_to_binary(N), qos1, + {fun(N0, {ok, #{reason_code_name := success}}) -> + TestPid ! {self(), N0} + end, [N]}) + end, lists:seq(1, NumMsgs)) + end), + + %% Delete queues while sending to them. + %% We want to test the path where a queue is deleted while confirms are outstanding. + timer:sleep(2), + delete_queue(Ch, [CQ1, QQ]), + %% Give queues some time to be fully deleted + timer:sleep(2000), + + %% We expect all confirms in the right order (because emqtt publishes with increasing packet ID). + %% Confirm here does not mean that messages made it ever to the deleted queues. + ok = await_confirms(C, 1, NumMsgs), + + %% Recreate the same queues. + DeclareQueues(), + + %% Sending a message to each of them should work. + {ok, _} = emqtt:publish(C, Topic, <<"m">>, qos1), + eventually(?_assertEqual(lists:sort([[CQ1, <<"1">>], + %% This queue should have all messages because we did not delete it. + [CQ2, integer_to_binary(NumMsgs + 1)], + [QQ, <<"1">>]]), + lists:sort(rabbitmqctl_list(Config, 0, ["list_queues", "name", "messages", "--no-table-headers"]))), + 1000, 10), + + delete_queue(Ch, [CQ1, CQ2, QQ]), + ok = rabbit_ct_client_helpers:close_connection_and_channel(Conn, Ch), + ok = emqtt:disconnect(C). + +await_confirms(_, To, To) -> + ok; +await_confirms(From, N, To) -> + Expected = {From, N}, + receive + Expected -> + await_confirms(From, N + 1, To); + Got -> + ct:fail("Received unexpected message. Expected: ~p Got: ~p", [Expected, Got]) + after 10_000 -> + ct:fail("Did not receive expected message: ~p", [Expected]) + end. + %% ------------------------------------------------------------------- %% Internal helpers %% ------------------------------------------------------------------- From 6533532039dd0237abecf441fbabbbf699b63ede Mon Sep 17 00:00:00 2001 From: David Ansari Date: Sun, 13 Nov 2022 16:23:03 +0000 Subject: [PATCH 040/118] Simplify counters by storing mqtt310 and mqtt311 atoms directly in the processor state. --- deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl | 6 +- .../include/rabbit_mqtt_frame.hrl | 6 +- deps/rabbitmq_mqtt/src/rabbit_mqtt.erl | 5 +- .../rabbit_mqtt_internal_event_handler.erl | 45 +++-- .../src/rabbit_mqtt_processor.erl | 156 +++++++----------- deps/rabbitmq_mqtt/test/integration_SUITE.erl | 17 +- 6 files changed, 104 insertions(+), 131 deletions(-) diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl index 5a88ef117680..23bf304ba8b0 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl @@ -7,8 +7,6 @@ -define(CLIENT_ID_MAXLEN, 23). -define(PG_SCOPE, pg_scope_rabbitmq_mqtt_clientid). --define(V3_GLOBAL_COUNTER_PROTO, mqtt310). --define(V4_GLOBAL_COUNTER_PROTO, mqtt311). -include("rabbit_mqtt_types.hrl"). @@ -30,7 +28,7 @@ %% processor state -record(proc_state, {socket, - proto_ver :: 3 | 4, + proto_ver :: mqtt310 | mqtt311, queue_states = rabbit_queue_type:init() :: rabbit_queue_type:state(), subscriptions = #{} :: #{Topic :: binary() => QoS :: 0..2}, %% Packet IDs published to queues but not yet confirmed. @@ -45,7 +43,7 @@ clean_sess :: boolean(), will_msg, exchange :: rabbit_exchange:name(), - has_published :: boolean(), + has_published = false :: boolean(), ssl_login_name, %% Retained messages handler. See rabbit_mqtt_retainer_sup %% and rabbit_mqtt_retainer. diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt_frame.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt_frame.hrl index 168d1c8d15bf..5fe78c26e7ff 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt_frame.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt_frame.hrl @@ -5,9 +5,9 @@ %% Copyright (c) 2020-2023 VMware, Inc. or its affiliates. All rights reserved. %% --define(MQTT_PROTO_V3, 3). --define(MQTT_PROTO_V4, 4). --define(PROTOCOL_NAMES, [{?MQTT_PROTO_V3, "MQIsdp"}, {?MQTT_PROTO_V4, "MQTT"}]). +-define(PROTOCOL_NAMES, [{3, "MQIsdp"}, {4, "MQTT"}]). +-define(MQTT_PROTO_V3, mqtt310). +-define(MQTT_PROTO_V4, mqtt311). %% frame types diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl index 1c06278bf4e5..27823b9d5b75 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl @@ -8,6 +8,7 @@ -module(rabbit_mqtt). -include("rabbit_mqtt.hrl"). +-include("rabbit_mqtt_frame.hrl"). -behaviour(application). @@ -19,8 +20,8 @@ local_connection_pids/0]). start(normal, []) -> - rabbit_global_counters:init([{protocol, ?V3_GLOBAL_COUNTER_PROTO}]), - rabbit_global_counters:init([{protocol, ?V4_GLOBAL_COUNTER_PROTO}]), + rabbit_global_counters:init([{protocol, ?MQTT_PROTO_V3}]), + rabbit_global_counters:init([{protocol, ?MQTT_PROTO_V4}]), {ok, Listeners} = application:get_env(tcp_listeners), {ok, SslListeners} = application:get_env(ssl_listeners), case rabbit_mqtt_ff:track_client_id_in_ra() of diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_internal_event_handler.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_internal_event_handler.erl index 6209bb32188e..711c065accf2 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_internal_event_handler.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_internal_event_handler.erl @@ -9,37 +9,30 @@ -behaviour(gen_event). --export([init/1, handle_event/2, handle_call/2, handle_info/2, terminate/2, code_change/3]). +-export([init/1, handle_event/2, handle_call/2]). -import(rabbit_misc, [pget/2]). +-define(STATE, []). + init([]) -> - {ok, []}. - -handle_event({event, vhost_created, Info, _, _}, State) -> - Name = pget(name, Info), - rabbit_mqtt_retainer_sup:child_for_vhost(Name), - {ok, State}; -handle_event({event, vhost_deleted, Info, _, _}, State) -> - Name = pget(name, Info), - rabbit_mqtt_retainer_sup:delete_child(Name), - {ok, State}; -handle_event({event, maintenance_connections_closed, _Info, _, _}, State) -> + {ok, ?STATE}. + +handle_event({event, vhost_created, Info, _, _}, ?STATE) -> + Name = pget(name, Info), + rabbit_mqtt_retainer_sup:child_for_vhost(Name), + {ok, ?STATE}; +handle_event({event, vhost_deleted, Info, _, _}, ?STATE) -> + Name = pget(name, Info), + rabbit_mqtt_retainer_sup:delete_child(Name), + {ok, ?STATE}; +handle_event({event, maintenance_connections_closed, _Info, _, _}, ?STATE) -> %% we should close our connections {ok, NConnections} = rabbit_mqtt:close_local_client_connections("node is being put into maintenance mode"), rabbit_log:warning("Closed ~b local MQTT client connections", [NConnections]), - {ok, State}; -handle_event(_Event, State) -> - {ok, State}. - -handle_call(_Request, State) -> - {ok, State}. - -handle_info(_Info, State) -> - {ok, State}. - -terminate(_Reason, _State) -> - ok. + {ok, ?STATE}; +handle_event(_Event, ?STATE) -> + {ok, ?STATE}. -code_change(_OldVsn, State, _Extra) -> - {ok, State}. +handle_call(_Request, ?STATE) -> + {ok, ok, ?STATE}. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index fcbaeb67fa59..96055dbe7362 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -113,11 +113,12 @@ process_request(?PUBLISH, variable = #mqtt_frame_publish{topic_name = Topic, message_id = MessageId }, payload = Payload}, - PState = #proc_state{retainer_pid = RPid, - amqp2mqtt_fun = Amqp2MqttFun, - unacked_client_pubs = U}) -> - counters_messages_received(PState), - PState1 = maybe_increment_publisher(PState), + PState0 = #proc_state{retainer_pid = RPid, + amqp2mqtt_fun = Amqp2MqttFun, + unacked_client_pubs = U, + proto_ver = ProtoVer}) -> + rabbit_global_counters:messages_received(ProtoVer, 1), + PState = maybe_increment_publisher(PState0), Publish = fun() -> Msg = #mqtt_msg{retain = Retain, qos = Qos, @@ -125,7 +126,7 @@ process_request(?PUBLISH, dup = Dup, message_id = MessageId, payload = Payload}, - case publish_to_queues(Msg, PState1) of + case publish_to_queues(Msg, PState) of {ok, _} = Ok -> case Retain of false -> @@ -140,18 +141,18 @@ process_request(?PUBLISH, end, case Qos of N when N > ?QOS_0 -> - counters_messages_received_confirm(PState1), + rabbit_global_counters:messages_received_confirm(ProtoVer, 1), case rabbit_mqtt_confirms:contains(MessageId, U) of false -> - publish_to_queues_with_checks(Topic, Publish, PState1); + publish_to_queues_with_checks(Topic, Publish, PState); true -> %% Client re-sent this PUBLISH packet. %% We already sent this message to target queues awaiting confirmations. %% Hence, we ignore this re-send. - {ok, PState1} + {ok, PState} end; _ -> - publish_to_queues_with_checks(Topic, Publish, PState1) + publish_to_queues_with_checks(Topic, Publish, PState) end; process_request(?SUBSCRIBE, @@ -370,7 +371,7 @@ register_client(Frame = #mqtt_frame_connect{proto_ver = ProtoVersion}, will_msg = make_will_msg(Frame), retainer_pid = RetainerPid, register_state = RegisterState, - proto_ver = ProtoVersion, + proto_ver = protocol_integer_to_atom(ProtoVersion), info = #info{prefetch = Prefetch, peer_host = PeerHost, peer_port = PeerPort, @@ -1122,14 +1123,15 @@ publish_to_queues( deliver_to_queues(Delivery, RoutedToQNames, - PState0 = #proc_state{queue_states = QStates0}) -> + PState0 = #proc_state{queue_states = QStates0, + proto_ver = ProtoVer}) -> %% TODO only lookup fields that are needed using ets:select / match? %% TODO Use ETS continuations to be more space efficient Qs0 = rabbit_amqqueue:lookup(RoutedToQNames), Qs = rabbit_amqqueue:prepend_extra_bcc(Qs0), case rabbit_queue_type:deliver(Qs, Delivery, QStates0) of {ok, QStates, Actions} -> - counters_messages_routed(PState0, length(Qs)), + rabbit_global_counters:messages_routed(ProtoVer, length(Qs)), PState = process_routing_confirm(Delivery, Qs, PState0#proc_state{queue_states = QStates}), %% Actions must be processed after registering confirms as actions may @@ -1141,17 +1143,20 @@ deliver_to_queues(Delivery, {error, Reason, PState0} end. -process_routing_confirm(#delivery{confirm = false}, [], PState) -> - counters_messages_unroutable_dropped(PState), +process_routing_confirm(#delivery{confirm = false}, + [], PState = #proc_state{proto_ver = ProtoVer}) -> + rabbit_global_counters:messages_unroutable_dropped(ProtoVer, 1), PState; process_routing_confirm(#delivery{confirm = true, - msg_seq_no = undefined}, [], PState) -> + msg_seq_no = undefined}, + [], PState = #proc_state{proto_ver = ProtoVer}) -> %% unroutable will message with QoS > 0 - counters_messages_unroutable_dropped(PState), + rabbit_global_counters:messages_unroutable_dropped(ProtoVer, 1), PState; process_routing_confirm(#delivery{confirm = true, - msg_seq_no = MsgId}, [], PState) -> - counters_messages_unroutable_returned(PState), + msg_seq_no = MsgId}, + [], PState = #proc_state{proto_ver = ProtoVer}) -> + rabbit_global_counters:messages_unroutable_returned(ProtoVer, 1), %% MQTT 5 spec: %% If the Server knows that there are no matching subscribers, it MAY use %% Reason Code 0x10 (No matching subscribers) instead of 0x00 (Success). @@ -1178,8 +1183,9 @@ send_puback(MsgIds0, PState) lists:foreach(fun(Id) -> send_puback(Id, PState) end, MsgIds); -send_puback(MsgId, PState = #proc_state{send_fun = SendFun}) -> - counters_messages_confirmed(PState), +send_puback(MsgId, PState = #proc_state{send_fun = SendFun, + proto_ver = ProtoVer}) -> + rabbit_global_counters:messages_confirmed(ProtoVer, 1), SendFun( #mqtt_frame{fixed = #mqtt_frame_fixed{type = ?PUBACK}, variable = #mqtt_frame_publish{message_id = MsgId}}, @@ -1205,8 +1211,8 @@ terminate(PState, ConnName) -> {pid, self()}]), rabbit_networking:unregister_non_amqp_connection(self()), maybe_unregister_client(PState), - counters_consumer_deleted(PState), - counters_publisher_deleted(PState), + maybe_decrement_consumer(PState), + maybe_decrement_publisher(PState), maybe_delete_mqtt_qos0_queue(PState). maybe_unregister_client(#proc_state{client_id = ClientId}) @@ -1607,77 +1613,43 @@ format_status(#proc_state{queue_states = QState, soft_limit_exceeded(#proc_state{soft_limit_exceeded = SLE}) -> not sets:is_empty(SLE). -%% Handle global counters -counters_messages_received(#proc_state{proto_ver = ?MQTT_PROTO_V3}) -> - rabbit_global_counters:messages_received(?V3_GLOBAL_COUNTER_PROTO, 1); -counters_messages_received(#proc_state{proto_ver = ?MQTT_PROTO_V4}) -> - rabbit_global_counters:messages_received(?V4_GLOBAL_COUNTER_PROTO, 1). - -counters_messages_received_confirm(#proc_state{proto_ver = ?MQTT_PROTO_V3}) -> - rabbit_global_counters:messages_received_confirm(?V3_GLOBAL_COUNTER_PROTO, 1); -counters_messages_received_confirm(#proc_state{proto_ver = ?MQTT_PROTO_V4}) -> - rabbit_global_counters:messages_received_confirm(?V4_GLOBAL_COUNTER_PROTO, 1). - -counters_messages_confirmed(#proc_state{proto_ver = ?MQTT_PROTO_V3}) -> - rabbit_global_counters:messages_confirmed(?V3_GLOBAL_COUNTER_PROTO, 1); -counters_messages_confirmed(#proc_state{proto_ver = ?MQTT_PROTO_V4}) -> - rabbit_global_counters:messages_confirmed(?V4_GLOBAL_COUNTER_PROTO, 1). - -%% one or multiple subscriptions from the same connection count as one consumer -maybe_increment_consumer(PState = #proc_state{subscriptions = Sub}) when map_size(Sub) =:= 0 -> - counters_consumer_created(PState); -maybe_increment_consumer(_) -> - ok. -counters_consumer_created(#proc_state{proto_ver = ?MQTT_PROTO_V3}) -> - rabbit_global_counters:consumer_created(?V3_GLOBAL_COUNTER_PROTO); -counters_consumer_created(#proc_state{proto_ver = ?MQTT_PROTO_V4}) -> - rabbit_global_counters:consumer_created(?V4_GLOBAL_COUNTER_PROTO). - -%% only decrement consumer count if there is no other subscription from this connection -maybe_decrement_consumer(PState0 = #proc_state{subscriptions = Sub0}, - #proc_state{subscriptions = Sub}) - when map_size(Sub0) =/= 0, map_size(Sub) =:= 0 -> - counters_consumer_deleted(PState0); -maybe_decrement_consumer(_, _) -> - ok. -counters_consumer_deleted(#proc_state{proto_ver = ?MQTT_PROTO_V3, subscriptions = Sub}) when map_size(Sub) =/= 0 -> - rabbit_global_counters:consumer_deleted(?V3_GLOBAL_COUNTER_PROTO); -counters_consumer_deleted(#proc_state{proto_ver = ?MQTT_PROTO_V4, subscriptions = Sub}) when map_size(Sub) =/= 0 -> - rabbit_global_counters:consumer_deleted(?V4_GLOBAL_COUNTER_PROTO); -counters_consumer_deleted(_) -> - ok. +protocol_integer_to_atom(3) -> + ?MQTT_PROTO_V3; +protocol_integer_to_atom(4) -> + ?MQTT_PROTO_V4. -maybe_increment_publisher(PState = #proc_state{has_published = HasPublished}) -> - case HasPublished of - true -> - PState; - _ -> - counters_publisher_created(PState), - PState#proc_state{has_published = true} - end. -counters_publisher_created(#proc_state{proto_ver = ?MQTT_PROTO_V3}) -> - rabbit_global_counters:publisher_created(?V3_GLOBAL_COUNTER_PROTO); -counters_publisher_created(#proc_state{proto_ver = ?MQTT_PROTO_V4}) -> - rabbit_global_counters:publisher_created(?V4_GLOBAL_COUNTER_PROTO). - -counters_publisher_deleted(#proc_state{proto_ver = ?MQTT_PROTO_V3, has_published = true}) -> - rabbit_global_counters:publisher_deleted(?V3_GLOBAL_COUNTER_PROTO); -counters_publisher_deleted(#proc_state{proto_ver = ?MQTT_PROTO_V4, has_published = true}) -> - rabbit_global_counters:publisher_deleted(?V4_GLOBAL_COUNTER_PROTO); -counters_publisher_deleted(_) -> +maybe_increment_publisher(PState = #proc_state{has_published = false, + proto_ver = ProtoVer}) -> + rabbit_global_counters:publisher_created(ProtoVer), + PState#proc_state{has_published = true}; +maybe_increment_publisher(PState) -> + PState. + +maybe_decrement_publisher(#proc_state{has_published = true, + proto_ver = ProtoVer}) -> + rabbit_global_counters:publisher_deleted(ProtoVer); +maybe_decrement_publisher(_) -> ok. -counters_messages_routed(#proc_state{proto_ver = ?MQTT_PROTO_V3}, Num) -> - rabbit_global_counters:messages_routed(?V3_GLOBAL_COUNTER_PROTO, Num); -counters_messages_routed(#proc_state{proto_ver = ?MQTT_PROTO_V4}, Num) -> - rabbit_global_counters:messages_routed(?V4_GLOBAL_COUNTER_PROTO, Num). +%% multiple subscriptions from the same connection count as one consumer +maybe_increment_consumer(#proc_state{subscriptions = OldSubs, + proto_ver = ProtoVer}) + when map_size(OldSubs) =:= 0 -> + rabbit_global_counters:consumer_created(ProtoVer); +maybe_increment_consumer(_) -> + ok. -counters_messages_unroutable_dropped(#proc_state{proto_ver = ?MQTT_PROTO_V3}) -> - rabbit_global_counters:messages_unroutable_dropped(?V3_GLOBAL_COUNTER_PROTO, 1); -counters_messages_unroutable_dropped(#proc_state{proto_ver = ?MQTT_PROTO_V4}) -> - rabbit_global_counters:messages_unroutable_dropped(mqtt311, 1). +maybe_decrement_consumer(#proc_state{subscriptions = Sub, + proto_ver = ProtoVer}) + when map_size(Sub) =/= 0 -> + rabbit_global_counters:consumer_deleted(ProtoVer); +maybe_decrement_consumer(_) -> + ok. -counters_messages_unroutable_returned(#proc_state{proto_ver = ?MQTT_PROTO_V3}) -> - rabbit_global_counters:messages_unroutable_returned(?V3_GLOBAL_COUNTER_PROTO, 1); -counters_messages_unroutable_returned(#proc_state{proto_ver = ?MQTT_PROTO_V4}) -> - rabbit_global_counters:messages_unroutable_returned(mqtt311, 1). \ No newline at end of file +maybe_decrement_consumer(#proc_state{subscriptions = OldSubs}, + #proc_state{subscriptions = NewSubs, + proto_ver = ProtoVer}) + when map_size(OldSubs) =/= 0, map_size(NewSubs) =:= 0 -> + rabbit_global_counters:consumer_deleted(ProtoVer); +maybe_decrement_consumer(_, _) -> + ok. diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/integration_SUITE.erl index e8701a160c3c..0b9e84c2a782 100644 --- a/deps/rabbitmq_mqtt/test/integration_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/integration_SUITE.erl @@ -11,7 +11,7 @@ -include_lib("eunit/include/eunit.hrl"). -include_lib("amqp_client/include/amqp_client.hrl"). -include_lib("rabbitmq_ct_helpers/include/rabbit_assert.hrl"). --include("rabbit_mqtt.hrl"). +-include("rabbit_mqtt_frame.hrl"). -import(rabbit_ct_broker_helpers, [rabbitmqctl_list/3, rpc_all/4]). @@ -44,6 +44,7 @@ common_tests() -> ,publish_to_all_queue_types_qos1 ,events ,event_authentication_failure + ,internal_event_handler ]. suite() -> @@ -66,7 +67,9 @@ init_per_group(cluster_size_3 = Group, Config) -> init_per_group0(Group, rabbit_ct_helpers:set_config(Config, [{rmq_nodes_count, 3}])); -init_per_group(Group, Config) when Group =:= global_counters orelse Group =:= common_tests -> +init_per_group(Group, Config) + when Group =:= global_counters orelse + Group =:= common_tests -> init_per_group0(Group,Config). init_per_group0(Group, Config0) -> @@ -83,6 +86,8 @@ init_per_group0(Group, Config0) -> ?assert(lists:all(fun(R) -> R =:= ok end, Result)), Config. +end_per_group(cluster_size_1, Config) -> + Config; end_per_group(_, Config) -> rabbit_ct_helpers:run_teardown_steps( Config, @@ -277,6 +282,10 @@ event_authentication_failure(Config) -> ok = gen_event:delete_handler({rabbit_event, Server}, event_recorder, []). +internal_event_handler(Config) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + ok = gen_event:call({rabbit_event, Server}, rabbit_mqtt_internal_event_handler, ignored_request, 1000). + global_counters_v3(Config) -> global_counters(Config, v3). @@ -482,9 +491,9 @@ assert_event_prop(ExpectedProps, Event) end, ExpectedProps). get_global_counters(Config, v3) -> - get_global_counters0(Config, ?V3_GLOBAL_COUNTER_PROTO); + get_global_counters0(Config, ?MQTT_PROTO_V3); get_global_counters(Config, v4) -> - get_global_counters0(Config, ?V4_GLOBAL_COUNTER_PROTO). + get_global_counters0(Config, ?MQTT_PROTO_V4). get_global_counters0(Config, Proto) -> maps:get([{protocol, Proto}], rabbit_ct_broker_helpers:rpc(Config, From 7fc2234117cc1d72c6814b3078344b1515ad454f Mon Sep 17 00:00:00 2001 From: David Ansari Date: Mon, 14 Nov 2022 10:37:41 +0000 Subject: [PATCH 041/118] Test ETS and NOOP retained message stores --- .../src/rabbit_mqtt_processor.erl | 2 + deps/rabbitmq_mqtt/test/retainer_SUITE.erl | 88 ++++++++++++------- 2 files changed, 57 insertions(+), 33 deletions(-) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 96055dbe7362..08813ae2c434 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -1194,6 +1194,8 @@ send_puback(MsgId, PState = #proc_state{send_fun = SendFun, serialise_and_send_to_client(Frame, #proc_state{proto_ver = ProtoVer, socket = Sock}) -> %%TODO Test sending large frames at high speed: %% Will we need garbage collection as done in rabbit_writer:maybe_gc_large_msg/1? + %%TODO batch to fill up MTU if there are messages in the Erlang mailbox? + %% Check rabbit_writer.erl Data = rabbit_mqtt_frame:serialise(Frame, ProtoVer), try rabbit_net:port_command(Sock, Data) catch _:Error -> diff --git a/deps/rabbitmq_mqtt/test/retainer_SUITE.erl b/deps/rabbitmq_mqtt/test/retainer_SUITE.erl index 1bee53413ad1..9e78f919dc67 100644 --- a/deps/rabbitmq_mqtt/test/retainer_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/retainer_SUITE.erl @@ -11,21 +11,28 @@ all() -> [ - {group, non_parallel_tests} + {group, dets}, + {group, ets}, + {group, noop} ]. groups() -> [ - {non_parallel_tests, [], [ - coerce_configuration_data, - should_translate_amqp2mqtt_on_publish, - should_translate_amqp2mqtt_on_retention, - should_translate_amqp2mqtt_on_retention_search - ]} + {dets, [], tests()}, + {ets, [], tests()}, + {noop, [], [does_not_retain]} + ]. + +tests() -> + [ + coerce_configuration_data, + should_translate_amqp2mqtt_on_publish, + should_translate_amqp2mqtt_on_retention, + should_translate_amqp2mqtt_on_retention_search ]. suite() -> - [{timetrap, {seconds, 600}}]. + [{timetrap, {minutes, 2}}]. %% ------------------------------------------------------------------- %% Testsuite setup/teardown. @@ -33,36 +40,38 @@ suite() -> init_per_suite(Config) -> rabbit_ct_helpers:log_environment(), - Config1 = rabbit_ct_helpers:set_config(Config, [ - {rmq_nodename_suffix, ?MODULE}, - {rmq_extra_tcp_ports, [tcp_port_mqtt_extra, - tcp_port_mqtt_tls_extra]} - ]), - % see https://github.com/rabbitmq/rabbitmq-mqtt/issues/86 - RabbitConfig = {rabbit, [ - {default_user, "guest"}, - {default_pass, "guest"}, - {default_vhost, "/"}, - {default_permissions, [".*", ".*", ".*"]} - ]}, - rabbit_ct_helpers:run_setup_steps(Config1, - [ fun(Conf) -> merge_app_env(RabbitConfig, Conf) end ] ++ - rabbit_ct_broker_helpers:setup_steps() ++ - rabbit_ct_client_helpers:setup_steps()). - -merge_app_env(MqttConfig, Config) -> - rabbit_ct_helpers:merge_app_env(Config, MqttConfig). + Config. end_per_suite(Config) -> - rabbit_ct_helpers:run_teardown_steps(Config, - rabbit_ct_client_helpers:teardown_steps() ++ - rabbit_ct_broker_helpers:teardown_steps()). - -init_per_group(_, Config) -> Config. +init_per_group(Group, Config0) -> + Config = rabbit_ct_helpers:set_config( + Config0, + [ + {rmq_nodename_suffix, Group}, + {rmq_extra_tcp_ports, [tcp_port_mqtt_extra, + tcp_port_mqtt_tls_extra]} + ]), + Mod = list_to_atom("rabbit_mqtt_retained_msg_store_" ++ atom_to_list(Group)), + Env = [{rabbitmq_mqtt, [{retained_message_store, Mod}]}, + {rabbit, [ + {default_user, "guest"}, + {default_pass, "guest"}, + {default_vhost, "/"}, + {default_permissions, [".*", ".*", ".*"]} + ]}], + rabbit_ct_helpers:run_setup_steps( + Config, + [fun(Conf) -> rabbit_ct_helpers:merge_app_env(Conf, Env) end] ++ + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps()). + end_per_group(_, Config) -> - Config. + rabbit_ct_helpers:run_teardown_steps( + Config, + rabbit_ct_client_helpers:teardown_steps() ++ + rabbit_ct_broker_helpers:teardown_steps()). init_per_testcase(Testcase, Config) -> rabbit_ct_helpers:testcase_started(Config, Testcase). @@ -126,6 +135,19 @@ should_translate_amqp2mqtt_on_retention_search(Config) -> expect_publishes(<<"TopicA/Device/Field">>, [<<"Payload">>]), ok = emqtt:disconnect(C). +does_not_retain(Config) -> + P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), + C = connect(P), + ok = emqtt:publish(C, <<"TopicA/Device.Field">>, #{}, <<"Payload">>, [{retain, true}]), + {ok, _, _} = emqtt:subscribe(C, <<"TopicA/Device.Field">>, qos1), + receive + Unexpected -> + ct:fail("Unexpected message: ~p", [Unexpected]) + after 1000 -> + ok + end, + ok = emqtt:disconnect(C). + connect(Port) -> {ok, C} = emqtt:start_link( [{host, "localhost"}, From 16b5ec5659434f519f68abf1af3184a8f7405fb6 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Mon, 14 Nov 2022 15:57:39 +0000 Subject: [PATCH 042/118] Add missing unblock stream queue action Test flow control for all queue types. --- deps/rabbit/src/rabbit_stream_queue.erl | 15 ++--- deps/rabbitmq_mqtt/test/integration_SUITE.erl | 64 ++++++++++++++++--- 2 files changed, 61 insertions(+), 18 deletions(-) diff --git a/deps/rabbit/src/rabbit_stream_queue.erl b/deps/rabbit/src/rabbit_stream_queue.erl index 9afb918493b1..70c4769d5918 100644 --- a/deps/rabbit/src/rabbit_stream_queue.erl +++ b/deps/rabbit/src/rabbit_stream_queue.erl @@ -423,15 +423,14 @@ handle_event({osiris_written, From, _WriterId, Corrs}, end, [], maps:with(Corrs, Correlation0))), Correlation = maps:without(Corrs, Correlation0), - Slow = case maps:size(Correlation) < SftLmt of - true when Slow0 -> - credit_flow:unblock(Name), - false; - _ -> - Slow0 - end, + {Slow, Actions} = case maps:size(Correlation) < SftLmt of + true when Slow0 -> + {false, [{unblock, Name}]}; + _ -> + {Slow0, []} + end, {ok, State#stream_client{correlation = Correlation, - slow = Slow}, [{settled, From, MsgIds}]}; + slow = Slow}, [{settled, From, MsgIds} | Actions]}; handle_event({osiris_offset, _From, _Offs}, State = #stream_client{local_pid = LocalPid, readers = Readers0, diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/integration_SUITE.erl index 0b9e84c2a782..9754a9d95375 100644 --- a/deps/rabbitmq_mqtt/test/integration_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/integration_SUITE.erl @@ -33,8 +33,12 @@ groups() -> {common_tests, [], common_tests()} ]}, {cluster_size_3, [], - [queue_down_qos1] - ++ common_tests()} + [queue_down_qos1] ++ + common_tests() ++ + [flow_classic_mirrored_queue, + flow_quorum_queue, + flow_stream] + } ]. common_tests() -> @@ -140,13 +144,6 @@ publish_to_all_queue_types_qos1(Config) -> publish_to_all_queue_types(Config, qos1). publish_to_all_queue_types(Config, QoS) -> - %% Give only 1/10 of the default credits. - %% We want to test whether sending many messages work when MQTT connection sometimes gets blocked. - Result = rpc_all(Config, application, set_env, [rabbit, credit_flow_default_credit, {40, 20}]), - Result = rpc_all(Config, application, set_env, [rabbit, quorum_commands_soft_limit, 3]), - Result = rpc_all(Config, application, set_env, [rabbit, stream_messages_soft_limit, 25]), - ?assert(lists:all(fun(R) -> R =:= ok end, Result)), - {Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0), CQ = <<"classic-queue">>, @@ -202,7 +199,54 @@ publish_to_all_queue_types(Config, QoS) -> delete_queue(Ch, [CQ, CMQ, QQ, SQ]), ok = rabbit_ct_broker_helpers:clear_policy(Config, 0, CMQ), ok = emqtt:disconnect(C), - ?awaitMatch([], all_connection_pids(Config), 10_000, 1000), + ?awaitMatch([], + all_connection_pids(Config), 10_000, 1000), + ok = rabbit_ct_client_helpers:close_connection_and_channel(Conn, Ch). + +flow_classic_mirrored_queue(Config) -> + QueueName = <<"flow">>, + ok = rabbit_ct_broker_helpers:set_ha_policy(Config, 0, QueueName, <<"all">>), + flow(Config, [rabbit, credit_flow_default_credit, {2, 1}], <<"classic">>), + ok = rabbit_ct_broker_helpers:clear_policy(Config, 0, QueueName). + +flow_quorum_queue(Config) -> + flow(Config, [rabbit, quorum_commands_soft_limit, 1], <<"quorum">>). + +flow_stream(Config) -> + flow(Config, [rabbit, stream_messages_soft_limit, 1], <<"stream">>). + +flow(Config, Env, QueueType) + when is_list(Env), is_binary(QueueType) -> + Result = rpc_all(Config, application, set_env, Env), + ?assert(lists:all(fun(R) -> R =:= ok end, Result)), + + {Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0), + QueueName = Topic = atom_to_binary(?FUNCTION_NAME), + declare_queue(Ch, QueueName, [{<<"x-queue-type">>, longstr, QueueType}]), + bind(Ch, QueueName, Topic), + + NumMsgs = 1000, + {C, _} = connect(?FUNCTION_NAME, Config, [{retry_interval, 600}, + {max_inflight, NumMsgs}]), + TestPid = self(), + lists:foreach( + fun(N) -> + %% Publish async all messages at once to trigger flow control + ok = emqtt:publish_async(C, Topic, integer_to_binary(N), qos1, + {fun(N0, {ok, #{reason_code_name := success}}) -> + TestPid ! {self(), N0} + end, [N]}) + end, lists:seq(1, NumMsgs)), + ok = await_confirms(C, 1, NumMsgs), + eventually(?_assertEqual( + [[integer_to_binary(NumMsgs)]], + rabbitmqctl_list(Config, 0, ["list_queues", "messages", "--no-table-headers"]) + ), 1000, 10), + + delete_queue(Ch, QueueName), + ok = emqtt:disconnect(C), + ?awaitMatch([], + all_connection_pids(Config), 10_000, 1000), ok = rabbit_ct_client_helpers:close_connection_and_channel(Conn, Ch). events(Config) -> From b97006c4b9d7242829864e4583a92618d63943ab Mon Sep 17 00:00:00 2001 From: David Ansari Date: Tue, 15 Nov 2022 15:25:59 +0000 Subject: [PATCH 043/118] Output username in connection closed event --- deps/rabbitmq_mqtt/BUILD.bazel | 3 +++ deps/rabbitmq_mqtt/src/mqtt_node.erl | 1 - .../src/rabbit_mqtt_processor.erl | 19 +++++++++++--- deps/rabbitmq_mqtt/test/integration_SUITE.erl | 25 ++++++++----------- deps/rabbitmq_mqtt/test/reader_SUITE.erl | 25 ++++++++++--------- deps/rabbitmq_mqtt/test/util.erl | 17 ++++++++++++- 6 files changed, 58 insertions(+), 32 deletions(-) diff --git a/deps/rabbitmq_mqtt/BUILD.bazel b/deps/rabbitmq_mqtt/BUILD.bazel index a933cf0d2b3b..ea7e7940ed36 100644 --- a/deps/rabbitmq_mqtt/BUILD.bazel +++ b/deps/rabbitmq_mqtt/BUILD.bazel @@ -99,6 +99,9 @@ rabbitmq_test_helper( deps = [ "//deps/rabbit_common:erlang_app", ], + hdrs = [ + "include/rabbit_mqtt_frame.hrl" + ], ) rabbitmq_test_helper( diff --git a/deps/rabbitmq_mqtt/src/mqtt_node.erl b/deps/rabbitmq_mqtt/src/mqtt_node.erl index a9c253e4a923..e8baa149b642 100644 --- a/deps/rabbitmq_mqtt/src/mqtt_node.erl +++ b/deps/rabbitmq_mqtt/src/mqtt_node.erl @@ -43,7 +43,6 @@ start(Delay, AttemptsLeft) -> undefined -> case Nodes of [] -> - %%TODO use global lock instead %% Since cluster members are not known ahead of time and initial boot can be happening in parallel, %% we wait and check a few times (up to a few seconds) to see if we can discover any peers to %% join before forming a cluster. This reduces the probability of N independent clusters being diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 08813ae2c434..7eab96e2b973 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -1208,15 +1208,28 @@ serialise(Frame, #proc_state{proto_ver = ProtoVer}) -> rabbit_mqtt_frame:serialise(Frame, ProtoVer). terminate(PState, ConnName) -> - rabbit_event:notify(connection_closed, [{name, ConnName}, - {node, node()}, - {pid, self()}]), + Infos = [{name, ConnName}, + {node, node()}, + {pid, self()}, + {disconnected_at, os:system_time(milli_seconds)} + ] ++ additional_connection_closed_info(PState), + rabbit_event:notify(connection_closed, Infos), rabbit_networking:unregister_non_amqp_connection(self()), maybe_unregister_client(PState), maybe_decrement_consumer(PState), maybe_decrement_publisher(PState), maybe_delete_mqtt_qos0_queue(PState). +additional_connection_closed_info( + #proc_state{info = #info{proto_human = {ProtoName, ProtoVsn}}, + auth_state = #auth_state{vhost = VHost, + username = Username}}) -> + [{protocol, {ProtoName, binary_to_list(ProtoVsn)}}, + {vhost, VHost}, + {user, Username}]; +additional_connection_closed_info(_) -> + []. + maybe_unregister_client(#proc_state{client_id = ClientId}) when ClientId =/= undefined -> case rabbit_mqtt_ff:track_client_id_in_ra() of diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/integration_SUITE.erl index 9754a9d95375..5121de55d109 100644 --- a/deps/rabbitmq_mqtt/test/integration_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/integration_SUITE.erl @@ -11,12 +11,12 @@ -include_lib("eunit/include/eunit.hrl"). -include_lib("amqp_client/include/amqp_client.hrl"). -include_lib("rabbitmq_ct_helpers/include/rabbit_assert.hrl"). --include("rabbit_mqtt_frame.hrl"). -import(rabbit_ct_broker_helpers, [rabbitmqctl_list/3, rpc_all/4]). -import(rabbit_ct_helpers, [eventually/3]). --import(util, [all_connection_pids/1]). +-import(util, [all_connection_pids/1, + get_global_counters/2]). all() -> [ @@ -263,7 +263,13 @@ events(Config) -> {connection_type, network}], E0), assert_event_type(connection_created, E1), - assert_event_prop({protocol, {'MQTT', "3.1.1"}}, E1), + [ConnectionPid] = all_connection_pids(Config), + ExpectedConnectionProps = [{protocol, {'MQTT', "3.1.1"}}, + {node, Server}, + {vhost, <<"/">>}, + {user, <<"guest">>}, + {pid, ConnectionPid}], + assert_event_prop(ExpectedConnectionProps, E1), {ok, _, _} = emqtt:subscribe(C, <<"TopicA">>, qos0), @@ -296,6 +302,7 @@ events(Config) -> [E5, E6] = get_events(Server), assert_event_type(connection_closed, E5), + assert_event_prop(ExpectedConnectionProps, E5), assert_event_type(queue_deleted, E6), assert_event_prop({name, QueueName}, E6), @@ -533,15 +540,3 @@ assert_event_prop(ExpectedProps, Event) lists:foreach(fun(P) -> assert_event_prop(P, Event) end, ExpectedProps). - -get_global_counters(Config, v3) -> - get_global_counters0(Config, ?MQTT_PROTO_V3); -get_global_counters(Config, v4) -> - get_global_counters0(Config, ?MQTT_PROTO_V4). -get_global_counters0(Config, Proto) -> - maps:get([{protocol, Proto}], - rabbit_ct_broker_helpers:rpc(Config, - 0, - rabbit_global_counters, - overview, - [])). diff --git a/deps/rabbitmq_mqtt/test/reader_SUITE.erl b/deps/rabbitmq_mqtt/test/reader_SUITE.erl index 00d7fc35d0cf..1fb88050d493 100644 --- a/deps/rabbitmq_mqtt/test/reader_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/reader_SUITE.erl @@ -12,7 +12,8 @@ -include_lib("eunit/include/eunit.hrl"). -import(rabbit_ct_broker_helpers, [rpc/5]). --import(rabbit_ct_helpers, [consistently/1]). +-import(rabbit_ct_helpers, [consistently/1, + eventually/3]). -import(util, [all_connection_pids/1, publish_qos1_timeout/4]). @@ -192,33 +193,32 @@ keepalive(Config) -> KeepaliveSecs = 1, KeepaliveMs = timer:seconds(KeepaliveSecs), P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), + ProtoVer = v4, {ok, C} = emqtt:start_link([{keepalive, KeepaliveSecs}, {host, "localhost"}, {port, P}, {clientid, atom_to_binary(?FUNCTION_NAME)}, - {proto_ver, v4} + {proto_ver, ProtoVer} ]), {ok, _Properties} = emqtt:connect(C), + ok = emqtt:publish(C, <<"ignored">>, <<"msg">>), %% Connection should stay up when client sends PING requests. timer:sleep(KeepaliveMs), + ?assertMatch(#{publishers := 1}, + util:get_global_counters(Config, ProtoVer)), %% Mock the server socket to not have received any bytes. rabbit_ct_broker_helpers:setup_meck(Config), Mod = rabbit_net, ok = rpc(Config, 0, meck, new, [Mod, [no_link, passthrough]]), ok = rpc(Config, 0, meck, expect, [Mod, getstat, 2, {ok, [{recv_oct, 999}]} ]), - process_flag(trap_exit, true), - receive - {'EXIT', C, {shutdown, tcp_closed}} -> - ok - after - ceil(3 * 0.75 * KeepaliveMs) -> - ct:fail("server did not respect keepalive") - end, - %%TODO Introduce Prometheus counter for number of connections closed - %% due to keepalive timeout and assert here that this counter is 1. + + %% We expect the server to respect the keepalive closing the connection. + eventually(?_assertMatch(#{publishers := 0}, + util:get_global_counters(Config, ProtoVer)), + KeepaliveMs, 3 * KeepaliveSecs), true = rpc(Config, 0, meck, validate, [Mod]), ok = rpc(Config, 0, meck, unload, [Mod]). @@ -234,6 +234,7 @@ keepalive_turned_off(Config) -> {proto_ver, v4} ]), {ok, _Properties} = emqtt:connect(C), + ok = emqtt:publish(C, <<"TopicB">>, <<"Payload">>), %% Mock the server socket to not have received any bytes. rabbit_ct_broker_helpers:setup_meck(Config), diff --git a/deps/rabbitmq_mqtt/test/util.erl b/deps/rabbitmq_mqtt/test/util.erl index 4bedb80a5ebb..0e59be57b412 100644 --- a/deps/rabbitmq_mqtt/test/util.erl +++ b/deps/rabbitmq_mqtt/test/util.erl @@ -1,8 +1,11 @@ -module(util). +-include("rabbit_mqtt_frame.hrl"). + -export([all_connection_pids/1, publish_qos1_timeout/4, - sync_publish_result/3]). + sync_publish_result/3, + get_global_counters/2]). all_connection_pids(Config) -> Nodes = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), @@ -31,3 +34,15 @@ publish_qos1_timeout(Client, Topic, Payload, Timeout) -> sync_publish_result(Caller, Mref, Result) -> erlang:send(Caller, {Mref, Result}). + +get_global_counters(Config, v3) -> + get_global_counters(Config, ?MQTT_PROTO_V3); +get_global_counters(Config, v4) -> + get_global_counters(Config, ?MQTT_PROTO_V4); +get_global_counters(Config, Proto) -> + maps:get([{protocol, Proto}], + rabbit_ct_broker_helpers:rpc(Config, + 0, + rabbit_global_counters, + overview, + [])). From bda52dbf64965531f5e8272484162b995cd25cd3 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Wed, 16 Nov 2022 16:20:17 +0000 Subject: [PATCH 044/118] Support consuming classic mirrored queue failover Some users use classic mirrored queues for MQTT queues by applying a policy. Given that classic mirrored queues are deprecated, but still supported in RabbitMQ 3.x, native MQTT must support classic mirrored queues. --- .../src/rabbit_mqtt_processor.erl | 71 +++++++++++++------ .../src/rabbit_mqtt_qos0_queue.erl | 3 +- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 14 +++- deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl | 27 +++++-- deps/rabbitmq_mqtt/test/integration_SUITE.erl | 58 ++++++++++++++- deps/rabbitmq_mqtt/test/reader_SUITE.erl | 13 +--- deps/rabbitmq_mqtt/test/util.erl | 14 +++- .../src/rabbit_web_mqtt_handler.erl | 26 +++++-- 8 files changed, 177 insertions(+), 49 deletions(-) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 7eab96e2b973..0cc8368b3e98 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -508,7 +508,7 @@ get_queue(QoS, PState) -> queue_name(QoS, #proc_state{client_id = ClientId, auth_state = #auth_state{vhost = VHost}}) -> - QNameBin = rabbit_mqtt_util:queue_name(ClientId, QoS), + QNameBin = rabbit_mqtt_util:queue_name_bin(ClientId, QoS), rabbit_misc:r(VHost, queue, QNameBin). hand_off_to_retainer(RetainerPid, Amqp2MqttFun, Topic0, #mqtt_msg{payload = <<"">>}) -> @@ -896,7 +896,7 @@ ensure_queue(QoS, #proc_state{ {ok, Q} -> {ok, Q}; {error, not_found} -> - QNameBin = rabbit_mqtt_util:queue_name(ClientId, QoS), + QNameBin = rabbit_mqtt_util:queue_name_bin(ClientId, QoS), QName = rabbit_misc:r(VHost, queue, QNameBin), %% configure access to queue required for queue.declare case check_resource_access(User, QName, configure, AuthzCtx) of @@ -992,15 +992,19 @@ consume(Q, QoS, #proc_state{ args => [], ok_msg => undefined, acting_user => Username}, - case rabbit_queue_type:consume(Q, Spec, QStates0) of - {ok, QStates} -> - PState = PState0#proc_state{queue_states = QStates}, - {ok, PState}; - {error, Reason} = Err -> - rabbit_log:error("Failed to consume from ~s: ~p", - [rabbit_misc:rs(QName), Reason]), - Err - end + rabbit_amqqueue:with( + QName, + fun(Q1) -> + case rabbit_queue_type:consume(Q1, Spec, QStates0) of + {ok, QStates} -> + PState = PState0#proc_state{queue_states = QStates}, + {ok, PState}; + {error, Reason} = Err -> + rabbit_log:error("Failed to consume from ~s: ~p", + [rabbit_misc:rs(QName), Reason]), + Err + end + end) end; {error, access_refused} = Err -> Err @@ -1195,7 +1199,7 @@ serialise_and_send_to_client(Frame, #proc_state{proto_ver = ProtoVer, socket = S %%TODO Test sending large frames at high speed: %% Will we need garbage collection as done in rabbit_writer:maybe_gc_large_msg/1? %%TODO batch to fill up MTU if there are messages in the Erlang mailbox? - %% Check rabbit_writer.erl + %% Check rabbit_writer:maybe_flush/1 Data = rabbit_mqtt_frame:serialise(Frame, ProtoVer), try rabbit_net:port_command(Sock, Data) catch _:Error -> @@ -1303,15 +1307,20 @@ handle_down({{'DOWN', QName}, _MRef, process, QPid, Reason}, credit_flow:peer_down(QPid), case rabbit_queue_type:handle_down(QPid, QName, Reason, QStates0) of {ok, QStates1, Actions} -> - PState = PState0#proc_state{queue_states = QStates1}, - handle_queue_actions(Actions, PState); + PState1 = PState0#proc_state{queue_states = QStates1}, + try handle_queue_actions(Actions, PState1) of + PState -> + {ok, PState} + catch throw:consuming_queue_down -> + {error, consuming_queue_down} + end; {eol, QStates1, QRef} -> {ConfirmMsgIds, U} = rabbit_mqtt_confirms:remove_queue(QRef, U0), QStates = rabbit_queue_type:remove(QRef, QStates1), PState = PState0#proc_state{queue_states = QStates, unacked_client_pubs = U}, send_puback(ConfirmMsgIds, PState), - PState + {ok, PState} end. handle_queue_event({queue_event, rabbit_mqtt_qos0_queue, Msg}, PState0) -> @@ -1361,14 +1370,34 @@ handle_queue_actions(Actions, #proc_state{} = PState0) -> S#proc_state{soft_limit_exceeded = sets:add_element(QName, SLE)}; ({unblock, QName}, S = #proc_state{soft_limit_exceeded = SLE}) -> S#proc_state{soft_limit_exceeded = sets:del_element(QName, SLE)}; - ({queue_down, _QName}, S) -> - %% classic queue is down, but not deleted - %% TODO if we were consuming from that queue: - %% remove subscription? recover if we support classic mirrored queues? see channel - % State1 = handle_consuming_queue_down_or_eol(QRef, State0), - S + ({queue_down, QName}, S) -> + handle_queue_down(QName, S) end, PState0, Actions). +handle_queue_down(QName, PState0 = #proc_state{client_id = ClientId}) -> + %% Classic queue is down. + case rabbit_amqqueue:lookup(QName) of + {ok, Q} -> + case rabbit_mqtt_util:qos_from_queue_name(QName, ClientId) of + no_consuming_queue -> + PState0; + QoS -> + %% Consuming classic queue is down. + %% Let's try to re-consume: HA failover for classic mirrored queues. + case consume(Q, QoS, PState0) of + {ok, PState} -> + PState; + {error, Reason} -> + rabbit_log:info("Terminating MQTT connection because consuming ~s " + "is down and could not re-consume: ~p", + [rabbit_misc:rs(QName), Reason]), + throw(consuming_queue_down) + end + end; + {error, not_found} -> + PState0 + end. + deliver_to_client(Msgs, Ack, PState) -> lists:foldl(fun(Msg, S) -> deliver_one_to_client(Msg, Ack, S) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl index 05439158d1c7..7ae4ea77b1d8 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl @@ -89,7 +89,8 @@ delete(Q, _IfUnused, _IfEmpty, ActingUser) -> %% MQTT connection process? %% E.g. drop this message if destination queue is already long? % erlang:process_info(Pid, message_queue_len) -%% ...if that's cheap. Alternatively, use credits? +%% However that seems to be expensive due to locking (see lcnt). +%% Alternatively, use credits? use rabbit_amqqueue_common:notify_sent/2 on the consuming side? -spec deliver([{amqqueue:amqqueue(), stateless}], Delivery :: term()) -> {[], rabbit_queue_type:actions()}. deliver([{Q, stateless}], Delivery = #delivery{message = BasicMessage}) -> diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index e70b2a73b354..5ae1a99d790c 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -183,7 +183,7 @@ handle_info({keepalive, Req}, State = #state{keepalive = KState0, end; handle_info(login_timeout, State = #state{received_connect_frame = true}) -> - {noreply, State}; + {noreply, State, ?HIBERNATE_AFTER}; handle_info(login_timeout, State = #state{conn_name = ConnStr}) -> %% The connection is also closed if the CONNECT frame happens to %% be already in the `deferred_recv' buffer. This can happen while @@ -205,8 +205,16 @@ handle_info({ra_event, _From, Evt}, handle_info({{'DOWN', _QName}, _MRef, process, _Pid, _Reason} = Evt, #state{proc_state = PState0} = State) -> - PState = rabbit_mqtt_processor:handle_down(Evt, PState0), - maybe_process_deferred_recv(control_throttle(pstate(State, PState))); + case rabbit_mqtt_processor:handle_down(Evt, PState0) of + {ok, PState} -> + maybe_process_deferred_recv(control_throttle(pstate(State, PState))); + {error, Reason} -> + send_will_and_terminate({shutdown, Reason}, State) + end; + +handle_info({'DOWN', _MRef, process, QPid, _Reason}, State) -> + rabbit_amqqueue_common:notify_sent_queue_down(QPid), + {noreply, State, ?HIBERNATE_AFTER}; handle_info(Msg, State) -> {stop, {mqtt_unexpected_msg, Msg}, State}. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl index 43eb920af710..aecabc7d83ef 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl @@ -7,10 +7,12 @@ -module(rabbit_mqtt_util). +-include_lib("rabbit_common/include/resource.hrl"). -include("rabbit_mqtt.hrl"). -include("rabbit_mqtt_frame.hrl"). --export([queue_name/2, +-export([queue_name_bin/2, + qos_from_queue_name/2, gen_client_id/0, env/1, table_lookup/2, @@ -24,10 +26,10 @@ -define(MAX_TOPIC_TRANSLATION_CACHE_SIZE, 12). --spec queue_name(binary(), qos()) -> +-spec queue_name_bin(binary(), qos()) -> binary(). -queue_name(ClientId, QoS) -> - Prefix = <<"mqtt-subscription-", ClientId/binary, "qos">>, +queue_name_bin(ClientId, QoS) -> + Prefix = queue_name_prefix(ClientId), queue_name0(Prefix, QoS). queue_name0(Prefix, ?QOS_0) -> @@ -37,6 +39,23 @@ queue_name0(Prefix, ?QOS_0) -> queue_name0(Prefix, ?QOS_1) -> <>. +-spec qos_from_queue_name(rabbit_amqqueue:name(), binary()) -> + qos() | no_consuming_queue. +qos_from_queue_name(#resource{name = Name}, ClientId) -> + Prefix = queue_name_prefix(ClientId), + PrefixSize = erlang:byte_size(Prefix), + case Name of + <> -> + ?QOS_0; + <> -> + ?QOS_1; + _ -> + no_consuming_queue + end. + +queue_name_prefix(ClientId) -> + <<"mqtt-subscription-", ClientId/binary, "qos">>. + cached(CacheName, Fun, Arg) -> Cache = case get(CacheName) of diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/integration_SUITE.erl index 5121de55d109..c81dc316a680 100644 --- a/deps/rabbitmq_mqtt/test/integration_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/integration_SUITE.erl @@ -16,7 +16,8 @@ rpc_all/4]). -import(rabbit_ct_helpers, [eventually/3]). -import(util, [all_connection_pids/1, - get_global_counters/2]). + get_global_counters/2, + expect_publishes/2]). all() -> [ @@ -33,7 +34,8 @@ groups() -> {common_tests, [], common_tests()} ]}, {cluster_size_3, [], - [queue_down_qos1] ++ + [queue_down_qos1, + consuming_classic_mirrored_queue_down] ++ common_tests() ++ [flow_classic_mirrored_queue, flow_quorum_queue, @@ -405,6 +407,58 @@ queue_down_qos1(Config) -> ok = rabbit_ct_client_helpers:close_connection_and_channel(Conn0, Ch0), ok = emqtt:disconnect(C). +%% Even though classic mirrored queues are deprecated, we know that some users have set up +%% a policy to mirror MQTT queues. So, we need to support that use case in RabbitMQ 3.x +%% and failover consumption when the classic mirrored queue leader fails. +consuming_classic_mirrored_queue_down(Config) -> + [Server1, Server2, _Server3] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Topic = PolicyName = atom_to_binary(?FUNCTION_NAME), + + ok = rabbit_ct_broker_helpers:set_policy( + Config, Server1, PolicyName, <<".*">>, <<"queues">>, + [{<<"ha-mode">>, <<"all">>}, + {<<"queue-master-locator">>, <<"client-local">>}]), + + %% Declare queue leader on Server1. + {C1, _} = connect(?FUNCTION_NAME, Config, [{clean_start, false}]), + {ok, _, _} = emqtt:subscribe(C1, Topic, qos1), + ok = emqtt:disconnect(C1), + + %% Consume from Server2. + Options = [{host, "localhost"}, + {port, rabbit_ct_broker_helpers:get_node_config(Config, Server2, tcp_port_mqtt)}, + {clientid, atom_to_binary(?FUNCTION_NAME)}, + {proto_ver, v4}], + {ok, C2} = emqtt:start_link([{clean_start, false} | Options]), + {ok, _} = emqtt:connect(C2), + + %% Sanity check that consumption works. + {ok, _} = emqtt:publish(C2, Topic, <<"m1">>, qos1), + expect_publishes(Topic, [<<"m1">>]), + + %% Let's stop the queue leader node. + ok = rabbit_ct_broker_helpers:stop_node(Config, Server1), + + %% Consumption should continue to work. + {ok, _} = emqtt:publish(C2, Topic, <<"m2">>, qos1), + expect_publishes(Topic, [<<"m2">>]), + + %% Cleanup + ok = emqtt:disconnect(C2), + ok = rabbit_ct_broker_helpers:start_node(Config, Server1), + ?assertMatch([_Q], + rabbit_ct_broker_helpers:rpc(Config, Server1, rabbit_amqqueue, list, [])), + %% "When a Client has determined that it has no further use for the session it should do a + %% final connect with CleanSession set to 1 and then disconnect." + {ok, C3} = emqtt:start_link([{clean_start, true} | Options]), + {ok, _} = emqtt:connect(C3), + ok = emqtt:disconnect(C3), + ?assertEqual([], + rabbit_ct_broker_helpers:rpc(Config, Server1, rabbit_amqqueue, list, [])), + ok = rabbit_ct_broker_helpers:clear_policy(Config, Server1, PolicyName). + +%%TODO add test where consuming classic queue on different node goes down + delete_create_queue(Config) -> {Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0), CQ1 = <<"classic-queue-1-delete-create">>, diff --git a/deps/rabbitmq_mqtt/test/reader_SUITE.erl b/deps/rabbitmq_mqtt/test/reader_SUITE.erl index 1fb88050d493..33d03b912359 100644 --- a/deps/rabbitmq_mqtt/test/reader_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/reader_SUITE.erl @@ -15,7 +15,8 @@ -import(rabbit_ct_helpers, [consistently/1, eventually/3]). -import(util, [all_connection_pids/1, - publish_qos1_timeout/4]). + publish_qos1_timeout/4, + expect_publishes/2]). all() -> [ @@ -368,15 +369,5 @@ classic_clean_session_true(Config) -> classic_clean_session_false(Config) -> validate_durable_queue_type(Config, <<"classicCleanSessionFalse">>, false, rabbit_classic_queue). -expect_publishes(_Topic, []) -> ok; -expect_publishes(Topic, [Payload|Rest]) -> - receive - {publish, #{topic := Topic, - payload := Payload}} -> - expect_publishes(Topic, Rest) - after 5000 -> - throw({publish_not_received, Payload}) - end. - rpc(Config, M, F, A) -> rabbit_ct_broker_helpers:rpc(Config, 0, M, F, A). diff --git a/deps/rabbitmq_mqtt/test/util.erl b/deps/rabbitmq_mqtt/test/util.erl index 0e59be57b412..ea720b674f7b 100644 --- a/deps/rabbitmq_mqtt/test/util.erl +++ b/deps/rabbitmq_mqtt/test/util.erl @@ -5,7 +5,8 @@ -export([all_connection_pids/1, publish_qos1_timeout/4, sync_publish_result/3, - get_global_counters/2]). + get_global_counters/2, + expect_publishes/2]). all_connection_pids(Config) -> Nodes = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), @@ -32,6 +33,17 @@ publish_qos1_timeout(Client, Topic, Payload, Timeout) -> puback_timeout end. +expect_publishes(_Topic, []) -> + ok; +expect_publishes(Topic, [Payload|Rest]) -> + receive + {publish, #{topic := Topic, + payload := Payload}} -> + expect_publishes(Topic, Rest) + after 5000 -> + ct:fail({publish_not_received, Payload}) + end. + sync_publish_result(Caller, Mref, Result) -> erlang:send(Caller, {Mref, Result}). diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index 02b7d660378e..234dc73e977e 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -36,6 +36,11 @@ keepalive :: rabbit_mqtt_keepalive:state() }). +%% Close frame status codes as defined in https://www.rfc-editor.org/rfc/rfc6455#section-7.4.1 +-define(CLOSE_NORMAL, 1000). +-define(CLOSE_PROTOCOL_ERROR, 1002). +-define(CLOSE_INCONSISTENT_MSG_TYPE, 1007). + %%TODO call rabbit_networking:register_non_amqp_connection/1 so that we are notified %% when need to force load the 'connection_created' event for the management plugin, see %% https://github.com/rabbitmq/rabbitmq-management-agent/issues/58 @@ -184,7 +189,7 @@ websocket_info({keepalive, Req}, State = #state{keepalive = KState0, {error, timeout} -> rabbit_log_connection:error("keepalive timeout in Web MQTT connection ~p", [ConnName]), - stop(State, 1000, <<"MQTT keepalive timeout">>); + stop(State, ?CLOSE_NORMAL, <<"MQTT keepalive timeout">>); {error, Reason} -> rabbit_log_connection:error("keepalive error in Web MQTT connection ~p: ~p", [ConnName, Reason]), @@ -198,8 +203,15 @@ websocket_info({ra_event, _From, Evt}, {[], State#state{proc_state = PState}, hibernate}; websocket_info({{'DOWN', _QName}, _MRef, process, _Pid, _Reason} = Evt, State = #state{proc_state = PState0}) -> - PState = rabbit_mqtt_processor:handle_down(Evt, PState0), - handle_credits(State#state{proc_state = PState}); + case rabbit_mqtt_processor:handle_down(Evt, PState0) of + {ok, PState} -> + handle_credits(State#state{proc_state = PState}); + {error, Reason} -> + stop(State, ?CLOSE_NORMAL, Reason) + end; +websocket_info({'DOWN', _MRef, process, QPid, _Reason}, State) -> + rabbit_amqqueue_common:notify_sent_queue_down(QPid), + {[], State, hibernate}; websocket_info(Msg, State) -> rabbit_log_connection:warning("Web MQTT: unexpected message ~p", [Msg]), {[], State, hibernate}. @@ -254,7 +266,9 @@ handle_data1(Data, State = #state{ parse_state = ParseState, {error, Reason, _} -> rabbit_log_connection:info("MQTT protocol error ~tp for connection ~tp", [Reason, ConnStr]), - stop(State, 1002, Reason); + stop(State, ?CLOSE_PROTOCOL_ERROR, Reason); + {error, Error} -> + stop_with_framing_error(State, Error, ConnStr); {stop, _} -> stop(State) end; @@ -266,10 +280,10 @@ stop_with_framing_error(State, Error0, ConnStr) -> Error1 = rabbit_misc:format("~tp", [Error0]), rabbit_log_connection:error("MQTT detected framing error '~ts' for connection ~tp", [Error1, ConnStr]), - stop(State, 1007, Error1). + stop(State, ?CLOSE_INCONSISTENT_MSG_TYPE, Error1). stop(State) -> - stop(State, 1000, "MQTT died"). + stop(State, ?CLOSE_NORMAL, "MQTT died"). stop(State, CloseCode, Error0) -> Error = rabbit_data_coercion:to_binary(Error0), From aad7e1cdf6632ed43c4bbcc280c53d99695dec32 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Thu, 17 Nov 2022 10:22:31 +0000 Subject: [PATCH 045/118] Add test for consuming MQTT classic queue going down --- .../src/rabbit_mqtt_processor.erl | 7 ++- deps/rabbitmq_mqtt/test/integration_SUITE.erl | 54 +++++++++++++++++-- deps/rabbitmq_mqtt/test/util.erl | 20 +++---- 3 files changed, 64 insertions(+), 17 deletions(-) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 0cc8368b3e98..385107ff2f67 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -1387,10 +1387,9 @@ handle_queue_down(QName, PState0 = #proc_state{client_id = ClientId}) -> case consume(Q, QoS, PState0) of {ok, PState} -> PState; - {error, Reason} -> - rabbit_log:info("Terminating MQTT connection because consuming ~s " - "is down and could not re-consume: ~p", - [rabbit_misc:rs(QName), Reason]), + {error, _Reason} -> + rabbit_log:info("Terminating MQTT connection because consuming ~s is down.", + [rabbit_misc:rs(QName)]), throw(consuming_queue_down) end end; diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/integration_SUITE.erl index c81dc316a680..5a35236108ce 100644 --- a/deps/rabbitmq_mqtt/test/integration_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/integration_SUITE.erl @@ -14,9 +14,11 @@ -import(rabbit_ct_broker_helpers, [rabbitmqctl_list/3, rpc_all/4]). --import(rabbit_ct_helpers, [eventually/3]). +-import(rabbit_ct_helpers, [eventually/3, + eventually/1]). -import(util, [all_connection_pids/1, get_global_counters/2, + get_global_counters/3, expect_publishes/2]). all() -> @@ -35,7 +37,8 @@ groups() -> ]}, {cluster_size_3, [], [queue_down_qos1, - consuming_classic_mirrored_queue_down] ++ + consuming_classic_mirrored_queue_down, + consuming_classic_queue_down] ++ common_tests() ++ [flow_classic_mirrored_queue, flow_quorum_queue, @@ -457,7 +460,52 @@ consuming_classic_mirrored_queue_down(Config) -> rabbit_ct_broker_helpers:rpc(Config, Server1, rabbit_amqqueue, list, [])), ok = rabbit_ct_broker_helpers:clear_policy(Config, Server1, PolicyName). -%%TODO add test where consuming classic queue on different node goes down +%% Consuming classic queue on a different node goes down. +consuming_classic_queue_down(Config) -> + [Server1, Server2, _Server3] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + ClientId = Topic = atom_to_binary(?FUNCTION_NAME), + + %% Declare classic queue on Server1. + {C1, _} = connect(?FUNCTION_NAME, Config, [{clean_start, false}]), + {ok, _, _} = emqtt:subscribe(C1, Topic, qos1), + ok = emqtt:disconnect(C1), + + ProtoVer = v4, + %% Consume from Server2. + Options = [{host, "localhost"}, + {port, rabbit_ct_broker_helpers:get_node_config(Config, Server2, tcp_port_mqtt)}, + {clientid, ClientId}, + {proto_ver, ProtoVer}], + {ok, C2} = emqtt:start_link([{clean_start, false} | Options]), + {ok, _} = emqtt:connect(C2), + + %%TODO uncomment below 2 lines once consumers counter works for clean_sess = false + % ?assertMatch(#{consumers := 1}, + % get_global_counters(Config, ProtoVer, Server2)), + + %% Let's stop the queue leader node. + process_flag(trap_exit, true), + ok = rabbit_ct_broker_helpers:stop_node(Config, Server1), + + %% When the dedicated MQTT connection (non-mirrored classic) queue goes down, it is reasonable + %% that the server closes the MQTT connection because the MQTT client cannot consume anymore. + eventually(?_assertMatch(#{consumers := 0}, + get_global_counters(Config, ProtoVer, Server2))), + receive + {'EXIT', C2, _} -> + ok + after 3000 -> + ct:fail("MQTT connection should have been closed") + end, + + %% Cleanup + ok = rabbit_ct_broker_helpers:start_node(Config, Server1), + {ok, C3} = emqtt:start_link([{clean_start, true} | Options]), + {ok, _} = emqtt:connect(C3), + ok = emqtt:disconnect(C3), + ?assertEqual([], + rabbit_ct_broker_helpers:rpc(Config, Server1, rabbit_amqqueue, list, [])), + ok. delete_create_queue(Config) -> {Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0), diff --git a/deps/rabbitmq_mqtt/test/util.erl b/deps/rabbitmq_mqtt/test/util.erl index ea720b674f7b..7ab8de96ade6 100644 --- a/deps/rabbitmq_mqtt/test/util.erl +++ b/deps/rabbitmq_mqtt/test/util.erl @@ -6,6 +6,7 @@ publish_qos1_timeout/4, sync_publish_result/3, get_global_counters/2, + get_global_counters/3, expect_publishes/2]). all_connection_pids(Config) -> @@ -47,14 +48,13 @@ expect_publishes(Topic, [Payload|Rest]) -> sync_publish_result(Caller, Mref, Result) -> erlang:send(Caller, {Mref, Result}). -get_global_counters(Config, v3) -> - get_global_counters(Config, ?MQTT_PROTO_V3); -get_global_counters(Config, v4) -> - get_global_counters(Config, ?MQTT_PROTO_V4); -get_global_counters(Config, Proto) -> +get_global_counters(Config, ProtoVer) -> + get_global_counters(Config, ProtoVer, 0). + +get_global_counters(Config, v3, Node) -> + get_global_counters(Config, ?MQTT_PROTO_V3, Node); +get_global_counters(Config, v4, Node) -> + get_global_counters(Config, ?MQTT_PROTO_V4, Node); +get_global_counters(Config, Proto, Node) -> maps:get([{protocol, Proto}], - rabbit_ct_broker_helpers:rpc(Config, - 0, - rabbit_global_counters, - overview, - [])). + rabbit_ct_broker_helpers:rpc(Config, Node, rabbit_global_counters, overview, [])). From 0b43f002f51b9c6b2c2c6013099faa93dc77f548 Mon Sep 17 00:00:00 2001 From: Chunyi Lyu Date: Thu, 17 Nov 2022 10:22:06 +0000 Subject: [PATCH 046/118] Remove subscriptions map from proc state in mqtt - subscriptions information can be retrieved directly from mnesia - when unsubscribe, we check if there is binding between topic name and queue (check for both qos0 queue name and qos1 queue name) to unbind - added a boolean value has_subs in proc state which will indicate if connection has any active subscriptions. Used for setting consumer global counter --- deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl | 3 +- .../src/rabbit_mqtt_processor.erl | 112 +++++++++++------- deps/rabbitmq_mqtt/test/cluster_SUITE.erl | 17 +-- deps/rabbitmq_mqtt/test/integration_SUITE.erl | 31 ++++- deps/rabbitmq_mqtt/test/reader_SUITE.erl | 6 +- deps/rabbitmq_mqtt/test/retainer_SUITE.erl | 21 +--- deps/rabbitmq_mqtt/test/util.erl | 2 +- 7 files changed, 110 insertions(+), 82 deletions(-) diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl index 23bf304ba8b0..430e756de821 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl @@ -30,7 +30,6 @@ {socket, proto_ver :: mqtt310 | mqtt311, queue_states = rabbit_queue_type:init() :: rabbit_queue_type:state(), - subscriptions = #{} :: #{Topic :: binary() => QoS :: 0..2}, %% Packet IDs published to queues but not yet confirmed. unacked_client_pubs = rabbit_mqtt_confirms:init() :: rabbit_mqtt_confirms:state(), %% Packet IDs published to MQTT subscribers but not yet acknowledged. @@ -43,6 +42,7 @@ clean_sess :: boolean(), will_msg, exchange :: rabbit_exchange:name(), + has_subs = false :: boolean(), has_published = false :: boolean(), ssl_login_name, %% Retained messages handler. See rabbit_mqtt_retainer_sup @@ -107,7 +107,6 @@ will_msg, retainer_pid, exchange, - subscriptions, prefetch, messages_unconfirmed, messages_unacknowledged diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 385107ff2f67..679024ddae34 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -180,17 +180,16 @@ process_request(?SUBSCRIBE, {ok, Q} -> QName = amqqueue:get_name(Q), case bind(QName, TopicName, S0) of - {ok, _Output, S1 = #proc_state{subscriptions = Subs0}} -> - Subs = maps:put(TopicName, QoS, Subs0), - S2 = S1#proc_state{subscriptions = Subs}, + {ok, _Output, S1} -> %%TODO check what happens if we basic.consume multiple times %% for the same queue - case consume(Q, QoS, S2) of - {ok, S} -> - maybe_increment_consumer(S1), - {[QoS | L], S}; + case consume(Q, QoS, S1) of + {ok, S2} -> + S3 = S2#proc_state{has_subs = true}, + maybe_increment_consumer(S3, S2), + {[QoS | L], S3}; {error, _Reason} -> - {[?SUBACK_FAILURE | L], S2} + {[?SUBACK_FAILURE | L], S1} end; {error, Reason, S} -> rabbit_log:error("Failed to bind ~s with topic ~s: ~p", @@ -225,29 +224,40 @@ process_request(?UNSUBSCRIBE, PState0 = #proc_state{send_fun = SendFun}) -> rabbit_log_connection:debug("Received an UNSUBSCRIBE for topic(s) ~p", [Topics]), PState = lists:foldl( - fun(#mqtt_topic{name = TopicName}, - #proc_state{subscriptions = Subs0} = S0) -> - case maps:take(TopicName, Subs0) of - {QoS, Subs} -> - QName = queue_name(QoS, S0), - case unbind(QName, TopicName, S0) of - {ok, _Output, S} -> - S#proc_state{subscriptions = Subs}; - {error, Reason, S} -> - rabbit_log:error("Failed to unbind ~s with topic ~s: ~p", - [rabbit_misc:rs(QName), TopicName, Reason]), - S - end; - error -> - S0 - end - end, PState0, Topics), + fun(#mqtt_topic{name = TopicName}, #proc_state{} = S0) -> + case find_queue_name(TopicName, S0) of + {ok, QName} -> + case unbind(QName, TopicName, S0) of + {ok, _, _} -> + PState0; + {error, Reason, State} -> + rabbit_log:error("Failed to unbind ~s with topic ~s: ~p", + [rabbit_misc:rs(QName), TopicName, Reason]), + State + end; + {not_found, _} -> + PState0 + end + end, PState0, Topics), SendFun( #mqtt_frame{fixed = #mqtt_frame_fixed {type = ?UNSUBACK}, variable = #mqtt_frame_suback{message_id = MessageId}}, PState), - maybe_decrement_consumer(PState0, PState), - {ok, PState}; + + PState3 = case rabbit_binding:list_for_destination(queue_name(?QOS_0, PState)) of + [] -> + case rabbit_binding:list_for_destination(queue_name(?QOS_1, PState)) of + [] -> + PState2 = #proc_state{has_subs = false}, + maybe_decrement_consumer(PState, PState2), + PState2; + _ -> + PState + end; + _ -> + PState + end, + {ok, PState3}; process_request(?PINGREQ, #mqtt_frame{}, PState = #proc_state{send_fun = SendFun}) -> rabbit_log_connection:debug("Received a PINGREQ"), @@ -511,6 +521,29 @@ queue_name(QoS, #proc_state{client_id = ClientId, QNameBin = rabbit_mqtt_util:queue_name_bin(ClientId, QoS), rabbit_misc:r(VHost, queue, QNameBin). +find_queue_name(TopicName, #proc_state{exchange = Exchange, + mqtt2amqp_fun = Mqtt2AmqpFun} = PState) -> + RoutingKey = Mqtt2AmqpFun(TopicName), + QName0 = queue_name(?QOS_0, PState), + case lookup_binding(Exchange, QName0, RoutingKey) of + true -> + {ok, QName0}; + false -> + QName1 = queue_name(?QOS_1, PState), + case lookup_binding(Exchange, QName1, RoutingKey) of + true -> + {ok, QName1}; + false -> + {not_found, []} + end + end. + +lookup_binding(Exchange, QueueName, RoutingKey) -> + B= #binding{source = Exchange, + destination = QueueName, + key = RoutingKey}, + lists:member(B, rabbit_binding:list_for_source_and_destination(Exchange, QueueName)). + hand_off_to_retainer(RetainerPid, Amqp2MqttFun, Topic0, #mqtt_msg{payload = <<"">>}) -> Topic1 = Amqp2MqttFun(Topic0), rabbit_mqtt_retainer:clear(RetainerPid, Topic1), @@ -1594,8 +1627,6 @@ info(clean_sess, #proc_state{clean_sess = Val}) -> Val; info(will_msg, #proc_state{will_msg = Val}) -> Val; info(retainer_pid, #proc_state{retainer_pid = Val}) -> Val; info(exchange, #proc_state{exchange = #resource{name = Val}}) -> Val; -info(subscriptions, #proc_state{subscriptions = Val}) -> - maps:keys(Val); info(prefetch, #proc_state{info = #info{prefetch = Val}}) -> Val; info(messages_unconfirmed, #proc_state{unacked_client_pubs = Val}) -> rabbit_mqtt_confirms:size(Val); @@ -1618,7 +1649,7 @@ ssl_login_name(Sock) -> format_status(#proc_state{queue_states = QState, proto_ver = ProtoVersion, - subscriptions = Sub, + has_subs = HasSubs, unacked_client_pubs = UnackClientPubs, unacked_server_pubs = UnackSerPubs, packet_id = PackID, @@ -1636,7 +1667,7 @@ format_status(#proc_state{queue_states = QState, } = PState) -> #{queue_states => rabbit_queue_type:format_status(QState), proto_ver => ProtoVersion, - subscriptions => Sub, + has_subs => HasSubs, unacked_client_pubs => UnackClientPubs, unacked_server_pubs => UnackSerPubs, packet_id => PackID, @@ -1675,24 +1706,21 @@ maybe_decrement_publisher(_) -> ok. %% multiple subscriptions from the same connection count as one consumer -maybe_increment_consumer(#proc_state{subscriptions = OldSubs, - proto_ver = ProtoVer}) - when map_size(OldSubs) =:= 0 -> +maybe_increment_consumer(#proc_state{has_subs = true, proto_ver = ProtoVer}, + #proc_state{has_subs = false}) -> rabbit_global_counters:consumer_created(ProtoVer); -maybe_increment_consumer(_) -> +maybe_increment_consumer(_, _) -> ok. -maybe_decrement_consumer(#proc_state{subscriptions = Sub, - proto_ver = ProtoVer}) - when map_size(Sub) =/= 0 -> +maybe_decrement_consumer(#proc_state{has_subs = true, + proto_ver = ProtoVer}) -> rabbit_global_counters:consumer_deleted(ProtoVer); maybe_decrement_consumer(_) -> ok. -maybe_decrement_consumer(#proc_state{subscriptions = OldSubs}, - #proc_state{subscriptions = NewSubs, - proto_ver = ProtoVer}) - when map_size(OldSubs) =/= 0, map_size(NewSubs) =:= 0 -> +% when there were subscriptions but not anymore +maybe_decrement_consumer(#proc_state{has_subs = false, proto_ver = ProtoVer}, + #proc_state{has_subs = true}) -> rabbit_global_counters:consumer_deleted(ProtoVer); maybe_decrement_consumer(_, _) -> ok. diff --git a/deps/rabbitmq_mqtt/test/cluster_SUITE.erl b/deps/rabbitmq_mqtt/test/cluster_SUITE.erl index a5505d53549e..d1c7dd8d69d9 100644 --- a/deps/rabbitmq_mqtt/test/cluster_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/cluster_SUITE.erl @@ -9,6 +9,7 @@ -include_lib("common_test/include/ct.hrl"). -include_lib("eunit/include/eunit.hrl"). +-import(util, [expect_publishes/2]). -import(rabbit_ct_broker_helpers, [setup_steps/0, @@ -139,7 +140,7 @@ connection_id_tracking(Config) -> {ok, MRef1, C1} = connect_to_node(Config, 0, ID), {ok, _, _} = emqtt:subscribe(C1, <<"TopicA">>, qos0), ok = emqtt:publish(C1, <<"TopicA">>, <<"Payload">>), - expect_publishes(<<"TopicA">>, [<<"Payload">>]), + ok = expect_publishes(<<"TopicA">>, [<<"Payload">>]), %% there's one connection assert_connection_count(Config, 4, 2, 1), @@ -162,7 +163,7 @@ connection_id_tracking_on_nodedown(Config) -> {ok, MRef, C} = connect_to_node(Config, 0, <<"simpleClient">>), {ok, _, _} = emqtt:subscribe(C, <<"TopicA">>, qos0), ok = emqtt:publish(C, <<"TopicA">>, <<"Payload">>), - expect_publishes(<<"TopicA">>, [<<"Payload">>]), + ok = expect_publishes(<<"TopicA">>, [<<"Payload">>]), assert_connection_count(Config, 4, 2, 1), ok = stop_node(Config, Server), await_disconnection(MRef), @@ -178,7 +179,7 @@ connection_id_tracking_with_decommissioned_node(Config) -> {ok, MRef, C} = connect_to_node(Config, 0, <<"simpleClient">>), {ok, _, _} = emqtt:subscribe(C, <<"TopicA">>, qos0), ok = emqtt:publish(C, <<"TopicA">>, <<"Payload">>), - expect_publishes(<<"TopicA">>, [<<"Payload">>]), + ok = expect_publishes(<<"TopicA">>, [<<"Payload">>]), assert_connection_count(Config, 4, 2, 1), {ok, _} = rabbitmqctl(Config, 0, ["decommission_mqtt_node", Server]), @@ -226,13 +227,3 @@ await_disconnection(Ref) -> after 20_000 -> exit(missing_down_message) end. - -expect_publishes(_Topic, []) -> ok; -expect_publishes(Topic, [Payload|Rest]) -> - receive - {publish, #{topic := Topic, - payload := Payload}} -> - expect_publishes(Topic, Rest) - after 5000 -> - throw({publish_not_delivered, Payload}) - end. diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/integration_SUITE.erl index 5a35236108ce..e50197ac1f96 100644 --- a/deps/rabbitmq_mqtt/test/integration_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/integration_SUITE.erl @@ -54,6 +54,7 @@ common_tests() -> ,events ,event_authentication_failure ,internal_event_handler + ,non_clean_sess_disconnect ]. suite() -> @@ -437,14 +438,14 @@ consuming_classic_mirrored_queue_down(Config) -> %% Sanity check that consumption works. {ok, _} = emqtt:publish(C2, Topic, <<"m1">>, qos1), - expect_publishes(Topic, [<<"m1">>]), + ok = expect_publishes(Topic, [<<"m1">>]), %% Let's stop the queue leader node. ok = rabbit_ct_broker_helpers:stop_node(Config, Server1), %% Consumption should continue to work. {ok, _} = emqtt:publish(C2, Topic, <<"m2">>, qos1), - expect_publishes(Topic, [<<"m2">>]), + ok = expect_publishes(Topic, [<<"m2">>]), %% Cleanup ok = emqtt:disconnect(C2), @@ -566,6 +567,28 @@ delete_create_queue(Config) -> ok = rabbit_ct_client_helpers:close_connection_and_channel(Conn, Ch), ok = emqtt:disconnect(C). +non_clean_sess_disconnect(Config) -> + {C1, _} = connect(?FUNCTION_NAME, Config, [{clean_start, false}]), + Topic = <<"test-topic1">>, + {ok, _, [1]} = emqtt:subscribe(C1, Topic, qos1), + ok = emqtt:disconnect(C1), + + {C2, _} = connect(?FUNCTION_NAME, Config, [{clean_start, false}]), + + %% shouldn't receive message after unsubscribe + {ok, _, _} = emqtt:unsubscribe(C2, Topic), + Msg = <<"msg">>, + {ok, _} = emqtt:publish(C2, Topic, Msg, qos1), + {publish_not_received, Msg} = expect_publishes(Topic, [Msg]), + + %% connect with clean sess true to clean up + {C3, _} = connect(?FUNCTION_NAME, Config, [{clean_start, true}]), + ok = emqtt:disconnect(C3). + +%% ------------------------------------------------------------------- +%% Internal helpers +%% ------------------------------------------------------------------- + await_confirms(_, To, To) -> ok; await_confirms(From, N, To) -> @@ -579,10 +602,6 @@ await_confirms(From, N, To) -> ct:fail("Did not receive expected message: ~p", [Expected]) end. -%% ------------------------------------------------------------------- -%% Internal helpers -%% ------------------------------------------------------------------- - connect(ClientId, Config) -> connect(ClientId, Config, []). diff --git a/deps/rabbitmq_mqtt/test/reader_SUITE.erl b/deps/rabbitmq_mqtt/test/reader_SUITE.erl index 33d03b912359..579373d644ae 100644 --- a/deps/rabbitmq_mqtt/test/reader_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/reader_SUITE.erl @@ -101,7 +101,7 @@ block(Config) -> %% Only here to ensure the connection is really up {ok, _, _} = emqtt:subscribe(C, <<"TopicA">>), ok = emqtt:publish(C, <<"TopicA">>, <<"Payload">>), - expect_publishes(<<"TopicA">>, [<<"Payload">>]), + ok = expect_publishes(<<"TopicA">>, [<<"Payload">>]), {ok, _, _} = emqtt:unsubscribe(C, <<"TopicA">>), {ok, _, _} = emqtt:subscribe(C, <<"Topic1">>), @@ -117,7 +117,7 @@ block(Config) -> %% Unblock rpc(Config, vm_memory_monitor, set_vm_memory_high_watermark, [0.4]), - expect_publishes(<<"Topic1">>, [<<"Not blocked yet">>, + ok = expect_publishes(<<"Topic1">>, [<<"Not blocked yet">>, <<"Now blocked">>, <<"Still blocked">>]), ok = emqtt:disconnect(C). @@ -291,7 +291,7 @@ validate_durable_queue_type(Config, ClientName, CleanSession, ExpectedQueueType) {ok, _Properties} = emqtt:connect(C), {ok, _, _} = emqtt:subscribe(C, <<"TopicB">>, qos1), ok = emqtt:publish(C, <<"TopicB">>, <<"Payload">>), - expect_publishes(<<"TopicB">>, [<<"Payload">>]), + ok = expect_publishes(<<"TopicB">>, [<<"Payload">>]), {ok, _, _} = emqtt:unsubscribe(C, <<"TopicB">>), Prefix = <<"mqtt-subscription-">>, Suffix = <<"qos1">>, diff --git a/deps/rabbitmq_mqtt/test/retainer_SUITE.erl b/deps/rabbitmq_mqtt/test/retainer_SUITE.erl index 9e78f919dc67..2c0afd20c04d 100644 --- a/deps/rabbitmq_mqtt/test/retainer_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/retainer_SUITE.erl @@ -8,6 +8,7 @@ -compile([export_all, nowarn_export_all]). -include_lib("common_test/include/ct.hrl"). +-import(util, [expect_publishes/2]). all() -> [ @@ -90,7 +91,7 @@ coerce_configuration_data(Config) -> {ok, _, _} = emqtt:subscribe(C, <<"TopicA">>, qos0), ok = emqtt:publish(C, <<"TopicA">>, <<"Payload">>), - expect_publishes(<<"TopicA">>, [<<"Payload">>]), + ok = expect_publishes(<<"TopicA">>, [<<"Payload">>]), ok = emqtt:disconnect(C). @@ -105,7 +106,7 @@ should_translate_amqp2mqtt_on_publish(Config) -> %% there's an active consumer {ok, _, _} = emqtt:subscribe(C, <<"TopicA/Device.Field">>, qos1), ok = emqtt:publish(C, <<"TopicA/Device.Field">>, #{}, <<"Payload">>, [{retain, true}]), - expect_publishes(<<"TopicA/Device/Field">>, [<<"Payload">>]), + ok = expect_publishes(<<"TopicA/Device/Field">>, [<<"Payload">>]), ok = emqtt:disconnect(C). %% ------------------------------------------------------------------- @@ -119,7 +120,7 @@ should_translate_amqp2mqtt_on_retention(Config) -> %% publish with retain = true before a consumer comes around ok = emqtt:publish(C, <<"TopicA/Device.Field">>, #{}, <<"Payload">>, [{retain, true}]), {ok, _, _} = emqtt:subscribe(C, <<"TopicA/Device.Field">>, qos1), - expect_publishes(<<"TopicA/Device/Field">>, [<<"Payload">>]), + ok = expect_publishes(<<"TopicA/Device/Field">>, [<<"Payload">>]), ok = emqtt:disconnect(C). %% ------------------------------------------------------------------- @@ -132,7 +133,7 @@ should_translate_amqp2mqtt_on_retention_search(Config) -> C = connect(P), ok = emqtt:publish(C, <<"TopicA/Device.Field">>, #{}, <<"Payload">>, [{retain, true}]), {ok, _, _} = emqtt:subscribe(C, <<"TopicA/Device/Field">>, qos1), - expect_publishes(<<"TopicA/Device/Field">>, [<<"Payload">>]), + ok = expect_publishes(<<"TopicA/Device/Field">>, [<<"Payload">>]), ok = emqtt:disconnect(C). does_not_retain(Config) -> @@ -156,14 +157,4 @@ connect(Port) -> {proto_ver, v4}, {ack_timeout, 1}]), {ok, _Properties} = emqtt:connect(C), - C. - -expect_publishes(_Topic, []) -> ok; -expect_publishes(Topic, [Payload|Rest]) -> - receive - {publish, #{topic := Topic, - payload := Payload}} -> - expect_publishes(Topic, Rest) - after 1500 -> - throw({publish_not_delivered, Payload}) - end. + C. \ No newline at end of file diff --git a/deps/rabbitmq_mqtt/test/util.erl b/deps/rabbitmq_mqtt/test/util.erl index 7ab8de96ade6..40b205146a4f 100644 --- a/deps/rabbitmq_mqtt/test/util.erl +++ b/deps/rabbitmq_mqtt/test/util.erl @@ -42,7 +42,7 @@ expect_publishes(Topic, [Payload|Rest]) -> payload := Payload}} -> expect_publishes(Topic, Rest) after 5000 -> - ct:fail({publish_not_received, Payload}) + {publish_not_received, Payload} end. sync_publish_result(Caller, Mref, Result) -> From 016451ee87296959cbc3042659321862bdb8027c Mon Sep 17 00:00:00 2001 From: David Ansari Date: Thu, 17 Nov 2022 11:48:56 +0000 Subject: [PATCH 047/118] Reset application env in MQTT flow tests so that they run independently of other tests --- deps/rabbitmq_mqtt/BUILD.bazel | 3 ++ deps/rabbitmq_mqtt/test/integration_SUITE.erl | 41 ++++++++++--------- 2 files changed, 25 insertions(+), 19 deletions(-) diff --git a/deps/rabbitmq_mqtt/BUILD.bazel b/deps/rabbitmq_mqtt/BUILD.bazel index ea7e7940ed36..02cc99940e75 100644 --- a/deps/rabbitmq_mqtt/BUILD.bazel +++ b/deps/rabbitmq_mqtt/BUILD.bazel @@ -186,6 +186,9 @@ rabbitmq_integration_suite( runtime_deps = [ "@emqtt//:erlang_app", ], + additional_beam = [ + ":util", + ], ) rabbitmq_suite( diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/integration_SUITE.erl index e50197ac1f96..a017f339315c 100644 --- a/deps/rabbitmq_mqtt/test/integration_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/integration_SUITE.erl @@ -13,6 +13,7 @@ -include_lib("rabbitmq_ct_helpers/include/rabbit_assert.hrl"). -import(rabbit_ct_broker_helpers, [rabbitmqctl_list/3, + rpc/5, rpc_all/4]). -import(rabbit_ct_helpers, [eventually/3, eventually/1]). @@ -38,11 +39,10 @@ groups() -> {cluster_size_3, [], [queue_down_qos1, consuming_classic_mirrored_queue_down, - consuming_classic_queue_down] ++ - common_tests() ++ - [flow_classic_mirrored_queue, + consuming_classic_queue_down, + flow_classic_mirrored_queue, flow_quorum_queue, - flow_stream] + flow_stream] ++ common_tests() } ]. @@ -212,18 +212,19 @@ publish_to_all_queue_types(Config, QoS) -> flow_classic_mirrored_queue(Config) -> QueueName = <<"flow">>, ok = rabbit_ct_broker_helpers:set_ha_policy(Config, 0, QueueName, <<"all">>), - flow(Config, [rabbit, credit_flow_default_credit, {2, 1}], <<"classic">>), + flow(Config, {rabbit, credit_flow_default_credit, {2, 1}}, <<"classic">>), ok = rabbit_ct_broker_helpers:clear_policy(Config, 0, QueueName). flow_quorum_queue(Config) -> - flow(Config, [rabbit, quorum_commands_soft_limit, 1], <<"quorum">>). + flow(Config, {rabbit, quorum_commands_soft_limit, 1}, <<"quorum">>). flow_stream(Config) -> - flow(Config, [rabbit, stream_messages_soft_limit, 1], <<"stream">>). + flow(Config, {rabbit, stream_messages_soft_limit, 1}, <<"stream">>). -flow(Config, Env, QueueType) - when is_list(Env), is_binary(QueueType) -> - Result = rpc_all(Config, application, set_env, Env), +flow(Config, {App, Par, Val}, QueueType) + when is_binary(QueueType) -> + {ok, DefaultVal} = rpc(Config, 0, application, get_env, [App, Par]), + Result = rpc_all(Config, application, set_env, [App, Par, Val]), ?assert(lists:all(fun(R) -> R =:= ok end, Result)), {Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0), @@ -253,7 +254,9 @@ flow(Config, Env, QueueType) ok = emqtt:disconnect(C), ?awaitMatch([], all_connection_pids(Config), 10_000, 1000), - ok = rabbit_ct_client_helpers:close_connection_and_channel(Conn, Ch). + ok = rabbit_ct_client_helpers:close_connection_and_channel(Conn, Ch), + Result = rpc_all(Config, application, set_env, [App, Par, DefaultVal]), + ok. events(Config) -> ok = rabbit_ct_broker_helpers:add_code_path_to_all_nodes(Config, event_recorder), @@ -451,19 +454,19 @@ consuming_classic_mirrored_queue_down(Config) -> ok = emqtt:disconnect(C2), ok = rabbit_ct_broker_helpers:start_node(Config, Server1), ?assertMatch([_Q], - rabbit_ct_broker_helpers:rpc(Config, Server1, rabbit_amqqueue, list, [])), + rpc(Config, Server1, rabbit_amqqueue, list, [])), %% "When a Client has determined that it has no further use for the session it should do a %% final connect with CleanSession set to 1 and then disconnect." {ok, C3} = emqtt:start_link([{clean_start, true} | Options]), {ok, _} = emqtt:connect(C3), ok = emqtt:disconnect(C3), ?assertEqual([], - rabbit_ct_broker_helpers:rpc(Config, Server1, rabbit_amqqueue, list, [])), + rpc(Config, Server1, rabbit_amqqueue, list, [])), ok = rabbit_ct_broker_helpers:clear_policy(Config, Server1, PolicyName). %% Consuming classic queue on a different node goes down. consuming_classic_queue_down(Config) -> - [Server1, Server2, _Server3] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + [Server1, _Server2, Server3] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), ClientId = Topic = atom_to_binary(?FUNCTION_NAME), %% Declare classic queue on Server1. @@ -472,9 +475,9 @@ consuming_classic_queue_down(Config) -> ok = emqtt:disconnect(C1), ProtoVer = v4, - %% Consume from Server2. + %% Consume from Server3. Options = [{host, "localhost"}, - {port, rabbit_ct_broker_helpers:get_node_config(Config, Server2, tcp_port_mqtt)}, + {port, rabbit_ct_broker_helpers:get_node_config(Config, Server3, tcp_port_mqtt)}, {clientid, ClientId}, {proto_ver, ProtoVer}], {ok, C2} = emqtt:start_link([{clean_start, false} | Options]), @@ -482,7 +485,7 @@ consuming_classic_queue_down(Config) -> %%TODO uncomment below 2 lines once consumers counter works for clean_sess = false % ?assertMatch(#{consumers := 1}, - % get_global_counters(Config, ProtoVer, Server2)), + % get_global_counters(Config, ProtoVer, Server3)), %% Let's stop the queue leader node. process_flag(trap_exit, true), @@ -491,7 +494,7 @@ consuming_classic_queue_down(Config) -> %% When the dedicated MQTT connection (non-mirrored classic) queue goes down, it is reasonable %% that the server closes the MQTT connection because the MQTT client cannot consume anymore. eventually(?_assertMatch(#{consumers := 0}, - get_global_counters(Config, ProtoVer, Server2))), + get_global_counters(Config, ProtoVer, Server3))), receive {'EXIT', C2, _} -> ok @@ -505,7 +508,7 @@ consuming_classic_queue_down(Config) -> {ok, _} = emqtt:connect(C3), ok = emqtt:disconnect(C3), ?assertEqual([], - rabbit_ct_broker_helpers:rpc(Config, Server1, rabbit_amqqueue, list, [])), + rpc(Config, Server1, rabbit_amqqueue, list, [])), ok. delete_create_queue(Config) -> From 80f8e0754fab4a3e00d5baa56b8e195cdaa2da26 Mon Sep 17 00:00:00 2001 From: Chunyi Lyu Date: Thu, 17 Nov 2022 18:24:00 +0000 Subject: [PATCH 048/118] Implement consumer global counter for clean sess false - remove has_subs from proc state; query datebase to check if a connection has subscription or not --- deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl | 1 - .../src/rabbit_mqtt_processor.erl | 79 +++++++++++-------- deps/rabbitmq_mqtt/test/integration_SUITE.erl | 14 +++- 3 files changed, 57 insertions(+), 37 deletions(-) diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl index 430e756de821..7b2a0fb60a6b 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl @@ -42,7 +42,6 @@ clean_sess :: boolean(), will_msg, exchange :: rabbit_exchange:name(), - has_subs = false :: boolean(), has_published = false :: boolean(), ssl_login_name, %% Retained messages handler. See rabbit_mqtt_retainer_sup diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 679024ddae34..6a0a5f1839bc 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -164,6 +164,7 @@ process_request(?SUBSCRIBE, #proc_state{send_fun = SendFun, retainer_pid = RPid} = PState0) -> rabbit_log_connection:debug("Received a SUBSCRIBE for topic(s) ~p", [Topics]), + HasSubsBefore = has_subs(PState0), {QosResponse, PState1} = lists:foldl(fun(_Topic, {[?SUBACK_FAILURE | _] = L, S}) -> %% Once a subscription failed, mark all following subscriptions @@ -185,9 +186,7 @@ process_request(?SUBSCRIBE, %% for the same queue case consume(Q, QoS, S1) of {ok, S2} -> - S3 = S2#proc_state{has_subs = true}, - maybe_increment_consumer(S3, S2), - {[QoS | L], S3}; + {[QoS | L], S2}; {error, _Reason} -> {[?SUBACK_FAILURE | L], S1} end; @@ -200,6 +199,7 @@ process_request(?SUBSCRIBE, {[?SUBACK_FAILURE | L], S0} end end, {[], PState0}, Topics), + maybe_increment_consumer(HasSubsBefore, PState1), SendFun( #mqtt_frame{fixed = #mqtt_frame_fixed{type = ?SUBACK}, variable = #mqtt_frame_suback{ @@ -223,6 +223,7 @@ process_request(?UNSUBSCRIBE, payload = undefined}, PState0 = #proc_state{send_fun = SendFun}) -> rabbit_log_connection:debug("Received an UNSUBSCRIBE for topic(s) ~p", [Topics]), + HasSubsBefore = has_subs(PState0), PState = lists:foldl( fun(#mqtt_topic{name = TopicName}, #proc_state{} = S0) -> case find_queue_name(TopicName, S0) of @@ -244,20 +245,8 @@ process_request(?UNSUBSCRIBE, variable = #mqtt_frame_suback{message_id = MessageId}}, PState), - PState3 = case rabbit_binding:list_for_destination(queue_name(?QOS_0, PState)) of - [] -> - case rabbit_binding:list_for_destination(queue_name(?QOS_1, PState)) of - [] -> - PState2 = #proc_state{has_subs = false}, - maybe_decrement_consumer(PState, PState2), - PState2; - _ -> - PState - end; - _ -> - PState - end, - {ok, PState3}; + maybe_decrement_consumer(HasSubsBefore, PState), + {ok, PState}; process_request(?PINGREQ, #mqtt_frame{}, PState = #proc_state{send_fun = SendFun}) -> rabbit_log_connection:debug("Received a PINGREQ"), @@ -462,7 +451,7 @@ start_keepalive(#mqtt_frame_connect{keep_alive = Seconds}, #proc_state{socket = Socket}) -> ok = rabbit_mqtt_keepalive:start(Seconds, Socket). -handle_clean_session(_, PState0 = #proc_state{clean_sess = false}) -> +handle_clean_session(_, PState0 = #proc_state{clean_sess = false, proto_ver = ProtoVer}) -> case get_queue(?QOS_1, PState0) of {error, not_found} -> %% Queue will be created later when client subscribes. @@ -470,6 +459,7 @@ handle_clean_session(_, PState0 = #proc_state{clean_sess = false}) -> {ok, Q} -> case consume(Q, ?QOS_1, PState0) of {ok, PState} -> + rabbit_global_counters:consumer_created(ProtoVer), {ok, _SessionPresent = true, PState}; {error, access_refused} -> {error, ?CONNACK_NOT_AUTHORIZED}; @@ -539,11 +529,22 @@ find_queue_name(TopicName, #proc_state{exchange = Exchange, end. lookup_binding(Exchange, QueueName, RoutingKey) -> - B= #binding{source = Exchange, + B = #binding{source = Exchange, destination = QueueName, key = RoutingKey}, lists:member(B, rabbit_binding:list_for_source_and_destination(Exchange, QueueName)). +has_subs(#proc_state{exchange = Exchange} = PState) -> + has_subs_between(Exchange, queue_name(?QOS_0, PState)) orelse + has_subs_between(Exchange, queue_name(?QOS_1, PState)). +has_subs_between(Exchange, QueueName) -> + case rabbit_binding:list_for_source_and_destination(Exchange, QueueName) of + [] -> + false; + _ -> + true + end. + hand_off_to_retainer(RetainerPid, Amqp2MqttFun, Topic0, #mqtt_msg{payload = <<"">>}) -> Topic1 = Amqp2MqttFun(Topic0), rabbit_mqtt_retainer:clear(RetainerPid, Topic1), @@ -1649,7 +1650,6 @@ ssl_login_name(Sock) -> format_status(#proc_state{queue_states = QState, proto_ver = ProtoVersion, - has_subs = HasSubs, unacked_client_pubs = UnackClientPubs, unacked_server_pubs = UnackSerPubs, packet_id = PackID, @@ -1667,7 +1667,6 @@ format_status(#proc_state{queue_states = QState, } = PState) -> #{queue_states => rabbit_queue_type:format_status(QState), proto_ver => ProtoVersion, - has_subs => HasSubs, unacked_client_pubs => UnackClientPubs, unacked_server_pubs => UnackSerPubs, packet_id => PackID, @@ -1706,21 +1705,35 @@ maybe_decrement_publisher(_) -> ok. %% multiple subscriptions from the same connection count as one consumer -maybe_increment_consumer(#proc_state{has_subs = true, proto_ver = ProtoVer}, - #proc_state{has_subs = false}) -> - rabbit_global_counters:consumer_created(ProtoVer); +maybe_increment_consumer(false, #proc_state{proto_ver = ProtoVer} = PState) -> + case has_subs(PState) of + true -> + rabbit_global_counters:consumer_created(ProtoVer); + false -> + ok + end; maybe_increment_consumer(_, _) -> ok. -maybe_decrement_consumer(#proc_state{has_subs = true, - proto_ver = ProtoVer}) -> - rabbit_global_counters:consumer_deleted(ProtoVer); -maybe_decrement_consumer(_) -> +%% when there were subscriptions but not anymore +maybe_decrement_consumer(true, #proc_state{proto_ver = ProtoVer} = PState) -> + case has_subs(PState) of + false -> + rabbit_global_counters:consumer_deleted(ProtoVer); + true -> + ok + end; +maybe_decrement_consumer(_, _) -> ok. -% when there were subscriptions but not anymore -maybe_decrement_consumer(#proc_state{has_subs = false, proto_ver = ProtoVer}, - #proc_state{has_subs = true}) -> - rabbit_global_counters:consumer_deleted(ProtoVer); -maybe_decrement_consumer(_, _) -> +%% used when connection is terminated +maybe_decrement_consumer(#proc_state{proto_ver = ProtoVer, + auth_state = #auth_state{vhost = _Vhost}} = PState) -> + case has_subs(PState) of + true -> + rabbit_global_counters:consumer_deleted(ProtoVer); + false -> + ok + end; +maybe_decrement_consumer(_) -> ok. diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/integration_SUITE.erl index a017f339315c..a10f9057cbc1 100644 --- a/deps/rabbitmq_mqtt/test/integration_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/integration_SUITE.erl @@ -483,9 +483,8 @@ consuming_classic_queue_down(Config) -> {ok, C2} = emqtt:start_link([{clean_start, false} | Options]), {ok, _} = emqtt:connect(C2), - %%TODO uncomment below 2 lines once consumers counter works for clean_sess = false - % ?assertMatch(#{consumers := 1}, - % get_global_counters(Config, ProtoVer, Server3)), + ?assertMatch(#{consumers := 1}, + get_global_counters(Config, ProtoVer, Server3)), %% Let's stop the queue leader node. process_flag(trap_exit, true), @@ -574,12 +573,21 @@ non_clean_sess_disconnect(Config) -> {C1, _} = connect(?FUNCTION_NAME, Config, [{clean_start, false}]), Topic = <<"test-topic1">>, {ok, _, [1]} = emqtt:subscribe(C1, Topic, qos1), + ?assertMatch(#{consumers := 1}, + get_global_counters(Config, v4)), + ok = emqtt:disconnect(C1), + ?assertMatch(#{consumers := 0}, + get_global_counters(Config, v4)), {C2, _} = connect(?FUNCTION_NAME, Config, [{clean_start, false}]), + ?assertMatch(#{consumers := 1}, + get_global_counters(Config, v4)), %% shouldn't receive message after unsubscribe {ok, _, _} = emqtt:unsubscribe(C2, Topic), + ?assertMatch(#{consumers := 0}, + get_global_counters(Config, v4)), Msg = <<"msg">>, {ok, _} = emqtt:publish(C2, Topic, Msg, qos1), {publish_not_received, Msg} = expect_publishes(Topic, [Msg]), From fb81a921165cae3435dfdeb53b4fb3c76d243e06 Mon Sep 17 00:00:00 2001 From: Chunyi Lyu Date: Thu, 17 Nov 2022 18:46:00 +0000 Subject: [PATCH 049/118] Add ws_connect helper to web mqtt tests --- deps/rabbitmq_web_mqtt/test/system_SUITE.erl | 115 +++++-------------- 1 file changed, 30 insertions(+), 85 deletions(-) diff --git a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl index 8c8cfe342fdb..8705945cc3ee 100644 --- a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl +++ b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl @@ -59,25 +59,12 @@ end_per_testcase(Testcase, Config) -> rabbit_ct_helpers:testcase_finished(Config, Testcase). connection(Config) -> - Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt), - {ok, C} = emqtt:start_link([{host, "127.0.0.1"}, - {username, "guest"}, - {password, "guest"}, - {ws_path, "/ws"}, - {port, Port}]), - {ok, _} = emqtt:ws_connect(C), + C = ws_connect(?FUNCTION_NAME, Config), ok = emqtt:disconnect(C). pubsub_shared_connection(Config) -> - Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt), - {ok, C} = emqtt:start_link([{host, "127.0.0.1"}, - {username, "guest"}, - {password, "guest"}, - {ws_path, "/ws"}, - {clientid, atom_to_binary(?FUNCTION_NAME)}, - {clean_start, true}, - {port, Port}]), - {ok, _} = emqtt:ws_connect(C), + C = ws_connect(?FUNCTION_NAME, Config), + Topic = <<"/topic/test-web-mqtt">>, {ok, _, [1]} = emqtt:subscribe(C, Topic, qos1), @@ -91,23 +78,8 @@ pubsub_shared_connection(Config) -> ok = emqtt:disconnect(C). pubsub_separate_connections(Config) -> - Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt), - {ok, Publisher} = emqtt:start_link([{host, "127.0.0.1"}, - {username, "guest"}, - {password, "guest"}, - {ws_path, "/ws"}, - {clientid, <<(atom_to_binary(?FUNCTION_NAME))/binary, "_publisher">>}, - {clean_start, true}, - {port, Port}]), - {ok, _} = emqtt:ws_connect(Publisher), - {ok, Consumer} = emqtt:start_link([{host, "127.0.0.1"}, - {username, "guest"}, - {password, "guest"}, - {ws_path, "/ws"}, - {clientid, <<(atom_to_binary(?FUNCTION_NAME))/binary, "_consumer">>}, - {clean_start, true}, - {port, Port}]), - {ok, _} = emqtt:ws_connect(Consumer), + Publisher = ws_connect(<<(atom_to_binary(?FUNCTION_NAME))/binary, "_publisher">>, Config), + Consumer = ws_connect(<<(atom_to_binary(?FUNCTION_NAME))/binary, "_consumer">>, Config), Topic = <<"/topic/test-web-mqtt">>, {ok, _, [1]} = emqtt:subscribe(Consumer, Topic, qos1), @@ -123,46 +95,21 @@ pubsub_separate_connections(Config) -> ok = emqtt:disconnect(Consumer). last_will_enabled(Config) -> - Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt), LastWillTopic = <<"/topic/web-mqtt-tests-ws1-last-will">>, LastWillMsg = <<"a last will and testament message">>, - {ok, Publisher} = emqtt:start_link([{host, "127.0.0.1"}, - {username, "guest"}, - {password, "guest"}, - {ws_path, "/ws"}, - {clientid, <<(atom_to_binary(?FUNCTION_NAME))/binary, "_publisher">>}, - {clean_start, true}, - {port, Port}, - {will_topic, LastWillTopic}, - {will_payload, LastWillMsg}, - {will_qos, 1} - ]), - {ok, _} = emqtt:ws_connect(Publisher), - {ok, Consumer} = emqtt:start_link([{host, "127.0.0.1"}, - {username, "guest"}, - {password, "guest"}, - {ws_path, "/ws"}, - {clientid, <<(atom_to_binary(?FUNCTION_NAME))/binary, "_consumer">>}, - {clean_start, true}, - {port, Port} - ]), - {ok, _} = emqtt:ws_connect(Consumer), + PubOpts = [{will_topic, LastWillTopic}, + {will_payload, LastWillMsg}, + {will_qos, 1}], + Publisher = ws_connect(<<(atom_to_binary(?FUNCTION_NAME))/binary, "_publisher">>, Config, PubOpts), + Consumer = ws_connect(<<(atom_to_binary(?FUNCTION_NAME))/binary, "_consumer">>, Config), {ok, _, [1]} = emqtt:subscribe(Consumer, LastWillTopic, qos1), ok = emqtt:disconnect(Publisher), ok = expect_publishes(Consumer, LastWillTopic, [LastWillMsg]), ok = emqtt:disconnect(Consumer). disconnect(Config) -> - Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt), - {ok, C} = emqtt:start_link([{host, "127.0.0.1"}, - {username, "guest"}, - {password, "guest"}, - {ws_path, "/ws"}, - {clientid, atom_to_binary(?FUNCTION_NAME)}, - {clean_start, true}, - {port, Port}]), + C = ws_connect(?FUNCTION_NAME, Config), process_flag(trap_exit, true), - {ok, _} = emqtt:ws_connect(C), eventually(?_assertEqual(1, num_mqtt_connections(Config, 0))), ok = emqtt:disconnect(C), receive @@ -177,16 +124,7 @@ disconnect(Config) -> keepalive(Config) -> KeepaliveSecs = 1, KeepaliveMs = timer:seconds(KeepaliveSecs), - Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt), - {ok, C} = emqtt:start_link([{keepalive, KeepaliveSecs}, - {host, "127.0.0.1"}, - {username, "guest"}, - {password, "guest"}, - {ws_path, "/ws"}, - {clientid, atom_to_binary(?FUNCTION_NAME)}, - {clean_start, true}, - {port, Port}]), - {ok, _} = emqtt:ws_connect(C), + C = ws_connect(?FUNCTION_NAME, Config, [{keepalive, KeepaliveSecs}]), %% Connection should stay up when client sends PING requests. timer:sleep(KeepaliveMs), @@ -210,17 +148,8 @@ keepalive(Config) -> ok = rpc(Config, 0, meck, unload, [Mod]). maintenance(Config) -> - Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt), - {ok, C} = emqtt:start_link([{host, "127.0.0.1"}, - {username, "guest"}, - {password, "guest"}, - {ws_path, "/ws"}, - {clientid, atom_to_binary(?FUNCTION_NAME)}, - {clean_start, true}, - {port, Port}]), - {ok, _} = emqtt:ws_connect(C), - unlink(C), - + C = ws_connect(?FUNCTION_NAME, Config), + true = unlink(C), eventually(?_assertEqual(1, num_mqtt_connections(Config, 0))), ok = rabbit_ct_broker_helpers:drain_node(Config, 0), eventually(?_assertEqual(0, num_mqtt_connections(Config, 0))), @@ -230,6 +159,22 @@ maintenance(Config) -> num_mqtt_connections(Config, Node) -> length(rpc(Config, Node, rabbit_mqtt, local_connection_pids, [])). +ws_connect(ClientId, Config) -> + ws_connect(ClientId, Config, []). +ws_connect(ClientId, Config, AdditionalOpts) -> + P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt), + Options = [{host, "localhost"}, + {username, "guest"}, + {password, "guest"}, + {ws_path, "/ws"}, + {port, P}, + {clientid, rabbit_data_coercion:to_binary(ClientId)}, + {proto_ver, v4} + ] ++ AdditionalOpts, + {ok, C} = emqtt:start_link(Options), + {ok, _Properties} = emqtt:ws_connect(C), + C. + expect_publishes(_ClientPid, _Topic, []) -> ok; expect_publishes(ClientPid, Topic, [Payload|Rest]) -> From e06d3e70694021bc9b340bf6a851cb1f78df546a Mon Sep 17 00:00:00 2001 From: David Ansari Date: Thu, 17 Nov 2022 20:13:27 +0000 Subject: [PATCH 050/118] Unblock queue when it gets deleted When a quorum queue or stream gets deleted while the MQTT connection process (or channel) is blocked by that deleted queue due to soft limit being exceeded, unblock that queue. In this commit, an unblock action is also returned with the eol. --- deps/rabbit/src/rabbit_channel.erl | 13 ++++----- deps/rabbit/src/rabbit_classic_queue.erl | 2 +- deps/rabbit/src/rabbit_fifo_client.erl | 6 ++--- deps/rabbit/src/rabbit_fifo_dlx_worker.erl | 5 ++-- deps/rabbit/src/rabbit_queue_type.erl | 10 +++---- deps/rabbit/src/rabbit_quorum_queue.erl | 2 +- deps/rabbit/src/rabbit_stream_queue.erl | 4 +-- .../src/rabbit_mqtt_processor.erl | 9 +++---- deps/rabbitmq_mqtt/test/integration_SUITE.erl | 27 ++++++++++++++----- 9 files changed, 47 insertions(+), 31 deletions(-) diff --git a/deps/rabbit/src/rabbit_channel.erl b/deps/rabbit/src/rabbit_channel.erl index f138b8b9f735..e276efa273b1 100644 --- a/deps/rabbit/src/rabbit_channel.erl +++ b/deps/rabbit/src/rabbit_channel.erl @@ -750,17 +750,18 @@ handle_cast({queue_event, QRef, Evt}, State1 = State0#ch{queue_states = QState1}, State = handle_queue_actions(Actions, State1), noreply_coalesce(State); - eol -> - State1 = handle_consuming_queue_down_or_eol(QRef, State0), + {eol, Actions} -> + State1 = handle_queue_actions(Actions, State0), + State2 = handle_consuming_queue_down_or_eol(QRef, State1), {ConfirmMXs, UC1} = - rabbit_confirms:remove_queue(QRef, State1#ch.unconfirmed), + rabbit_confirms:remove_queue(QRef, State2#ch.unconfirmed), %% Deleted queue is a special case. %% Do not nack the "rejected" messages. - State2 = record_confirms(ConfirmMXs, - State1#ch{unconfirmed = UC1}), + State3 = record_confirms(ConfirmMXs, + State2#ch{unconfirmed = UC1}), _ = erase_queue_stats(QRef), noreply_coalesce( - State2#ch{queue_states = rabbit_queue_type:remove(QRef, QueueStates0)}); + State3#ch{queue_states = rabbit_queue_type:remove(QRef, QueueStates0)}); {protocol_error, Type, Reason, ReasonArgs} -> rabbit_misc:protocol_error(Type, Reason, ReasonArgs) end. diff --git a/deps/rabbit/src/rabbit_classic_queue.erl b/deps/rabbit/src/rabbit_classic_queue.erl index 33a1c92bfadd..fd566df1d9ba 100644 --- a/deps/rabbit/src/rabbit_classic_queue.erl +++ b/deps/rabbit/src/rabbit_classic_queue.erl @@ -284,7 +284,7 @@ handle_event({down, Pid, QRef, Info}, #?STATE{monitored = Monitored, case rabbit_misc:is_abnormal_exit(Info) of false when Info =:= normal andalso Pid == MasterPid -> %% queue was deleted and masterpid is down - eol; + {eol, []}; false -> %% this assumes the mirror isn't part of the active set MsgSeqNos = maps:keys( diff --git a/deps/rabbit/src/rabbit_fifo_client.erl b/deps/rabbit/src/rabbit_fifo_client.erl index 2e916f7fb5b4..7b6c440be3cb 100644 --- a/deps/rabbit/src/rabbit_fifo_client.erl +++ b/deps/rabbit/src/rabbit_fifo_client.erl @@ -528,7 +528,7 @@ update_machine_state(Server, Conf) -> %% with them. -spec handle_ra_event(ra:server_id(), ra_server_proc:ra_event_body(), state()) -> {internal, Correlators :: [term()], actions(), state()} | - {rabbit_fifo:client_msg(), state()} | eol. + {rabbit_fifo:client_msg(), state()} | {eol, actions()}. handle_ra_event(From, {applied, Seqs}, #state{cfg = #cfg{queue_name = QRef, soft_limit = SftLmt @@ -613,8 +613,8 @@ handle_ra_event(_, timeout, #state{cfg = #cfg{servers = Servers}} = State0) -> State = resend_all_pending(State0#state{leader = Leader}), {ok, State, []} end; -handle_ra_event(_Leader, {machine, eol}, _State0) -> - eol. +handle_ra_event(_Leader, {machine, eol}, State) -> + {eol, [{unblock, cluster_name(State)}]}. %% @doc Attempts to enqueue a message using cast semantics. This provides no %% guarantees or retries if the message fails to achieve consensus or if the diff --git a/deps/rabbit/src/rabbit_fifo_dlx_worker.erl b/deps/rabbit/src/rabbit_fifo_dlx_worker.erl index ff61bbe5e5c1..3dbbfa79c5ad 100644 --- a/deps/rabbit/src/rabbit_fifo_dlx_worker.erl +++ b/deps/rabbit/src/rabbit_fifo_dlx_worker.erl @@ -151,8 +151,9 @@ handle_cast({queue_event, QRef, Evt}, State1 = State0#state{queue_type_state = QTypeState1}, State = handle_queue_actions(Actions, State1), {noreply, State}; - eol -> - remove_queue(QRef, State0); + {eol, Actions} -> + State = handle_queue_actions(Actions, State0), + remove_queue(QRef, State); {protocol_error, _Type, _Reason, _Args} -> {noreply, State0} end; diff --git a/deps/rabbit/src/rabbit_queue_type.erl b/deps/rabbit/src/rabbit_queue_type.erl index d61f8db41444..3865b736d4ae 100644 --- a/deps/rabbit/src/rabbit_queue_type.erl +++ b/deps/rabbit/src/rabbit_queue_type.erl @@ -191,7 +191,7 @@ %% this -callback handle_event(Event :: event(), queue_state()) -> - {ok, queue_state(), actions()} | {error, term()} | eol | + {ok, queue_state(), actions()} | {error, term()} | {eol, actions()} | {protocol_error, Type :: atom(), Reason :: string(), Args :: term()}. -callback deliver([{amqqueue:amqqueue(), queue_state()}], @@ -455,7 +455,7 @@ handle_down(Pid, QName, Info, State0) -> case handle_event(QName, {down, Pid, QName, Info}, State0) of {ok, State, Actions} -> {ok, State, Actions}; - eol -> + {eol, []} -> {eol, State0, QName}; Err -> Err @@ -463,7 +463,7 @@ handle_down(Pid, QName, Info, State0) -> %% messages sent from queues -spec handle_event(queue_name(), term(), state()) -> - {ok, state(), actions()} | eol | {error, term()} | + {ok, state(), actions()} | {eol, actions()} | {error, term()} | {protocol_error, Type :: atom(), Reason :: string(), Args :: term()}. handle_event(QRef, Evt, Ctxs) -> %% events can arrive after a queue state has been cleared up @@ -474,8 +474,8 @@ handle_event(QRef, Evt, Ctxs) -> case Mod:handle_event(Evt, State0) of {ok, State, Actions} -> {ok, set_ctx(QRef, Ctx#ctx{state = State}, Ctxs), Actions}; - Err -> - Err + Other -> + Other end; undefined -> {ok, Ctxs, []} diff --git a/deps/rabbit/src/rabbit_quorum_queue.erl b/deps/rabbit/src/rabbit_quorum_queue.erl index a61c64fdfe9f..97552a447062 100644 --- a/deps/rabbit/src/rabbit_quorum_queue.erl +++ b/deps/rabbit/src/rabbit_quorum_queue.erl @@ -152,7 +152,7 @@ update(Q, State) when ?amqqueue_is_quorum(Q) -> -spec handle_event({amqqueue:ra_server_id(), any()}, rabbit_fifo_client:state()) -> {ok, rabbit_fifo_client:state(), rabbit_queue_type:actions()} | - eol | + {eol, rabbit_queue_type:actions()} | {protocol_error, Type :: atom(), Reason :: string(), Args :: term()}. handle_event({From, Evt}, QState) -> rabbit_fifo_client:handle_ra_event(From, Evt, QState). diff --git a/deps/rabbit/src/rabbit_stream_queue.erl b/deps/rabbit/src/rabbit_stream_queue.erl index 70c4769d5918..36166c3e9ee4 100644 --- a/deps/rabbit/src/rabbit_stream_queue.erl +++ b/deps/rabbit/src/rabbit_stream_queue.erl @@ -469,8 +469,8 @@ handle_event({stream_local_member_change, Pid}, State = #stream_client{name = QN end, #{}, Readers0), {ok, State#stream_client{local_pid = Pid, readers = Readers1}, []}; -handle_event(eol, _State) -> - eol. +handle_event(eol, #stream_client{name = Name}) -> + {eol, [{unblock, Name}]}. is_recoverable(Q) -> Node = node(), diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 6a0a5f1839bc..015be0686612 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -1368,13 +1368,12 @@ handle_queue_event({queue_event, QName, Evt}, PState1 = PState0#proc_state{queue_states = QStates}, PState = handle_queue_actions(Actions, PState1), {ok, PState}; - eol -> - %%TODO handle consuming queue down - % State1 = handle_consuming_queue_down_or_eol(QRef, State0), + {eol, Actions} -> + PState1 = handle_queue_actions(Actions, PState0), {ConfirmMsgIds, U} = rabbit_mqtt_confirms:remove_queue(QName, U0), QStates = rabbit_queue_type:remove(QName, QStates0), - PState = PState0#proc_state{queue_states = QStates, - unacked_client_pubs = U}, + PState = PState1#proc_state{queue_states = QStates, + unacked_client_pubs = U}, send_puback(ConfirmMsgIds, PState), {ok, PState}; {protocol_error, _Type, _Reason, _ReasonArgs} = Error -> diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/integration_SUITE.erl index a10f9057cbc1..1cf5bcc50034 100644 --- a/deps/rabbitmq_mqtt/test/integration_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/integration_SUITE.erl @@ -244,7 +244,7 @@ flow(Config, {App, Par, Val}, QueueType) TestPid ! {self(), N0} end, [N]}) end, lists:seq(1, NumMsgs)), - ok = await_confirms(C, 1, NumMsgs), + ok = await_confirms_ordered(C, 1, NumMsgs), eventually(?_assertEqual( [[integer_to_binary(NumMsgs)]], rabbitmqctl_list(Config, 0, ["list_queues", "messages", "--no-table-headers"]) @@ -549,9 +549,12 @@ delete_create_queue(Config) -> %% Give queues some time to be fully deleted timer:sleep(2000), - %% We expect all confirms in the right order (because emqtt publishes with increasing packet ID). + %% We expect confirms for all messages. %% Confirm here does not mean that messages made it ever to the deleted queues. - ok = await_confirms(C, 1, NumMsgs), + %% It is valid for confirms to sporadically arrive out of order: This happens when the classic + %% queue is being deleted while the remaining messages are routed and confirmed to the 2nd and 3rd queues + %% before the monitor to the classic queue fires. + ok = await_confirms_unordered(C, NumMsgs), %% Recreate the same queues. DeclareQueues(), @@ -600,19 +603,31 @@ non_clean_sess_disconnect(Config) -> %% Internal helpers %% ------------------------------------------------------------------- -await_confirms(_, To, To) -> +await_confirms_ordered(_, To, To) -> ok; -await_confirms(From, N, To) -> +await_confirms_ordered(From, N, To) -> Expected = {From, N}, receive Expected -> - await_confirms(From, N + 1, To); + await_confirms_ordered(From, N + 1, To); Got -> ct:fail("Received unexpected message. Expected: ~p Got: ~p", [Expected, Got]) after 10_000 -> ct:fail("Did not receive expected message: ~p", [Expected]) end. +await_confirms_unordered(_, 0) -> + ok; +await_confirms_unordered(From, Left) -> + receive + {From, _N} -> + await_confirms_unordered(From, Left - 1); + Other -> + ct:fail("Received unexpected message: ~p", [Other]) + after 10_000 -> + ct:fail("~b confirms are missing", [Left]) + end. + connect(ClientId, Config) -> connect(ClientId, Config, []). From 61a33da83884e231196414959ad8279a5df19a22 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Fri, 18 Nov 2022 09:54:02 +0000 Subject: [PATCH 051/118] Make rabbit_fifo_dlx_worker tests less flaky Tests sporadically fail with: ``` === Ended at 2022-11-17 20:27:09 === Location: [{rabbit_fifo_dlx_integration_SUITE,assert_active_dlx_workers,938}, {test_server,ts_tc,1782}, {test_server,run_test_case_eval1,1291}, {test_server,run_test_case_eval,1223}] === === Reason: {assertMatch, [{module,rabbit_fifo_dlx_integration_SUITE}, {line,938}, {expression, "rabbit_ct_broker_helpers : rpc ( Config , Server , supervisor , count_children , [ rabbit_fifo_dlx_sup ] , 1000 )"}, {pattern,"[ _ , { active , N } , _ , _ ]"}, {value, [{specs,1}, {active,2}, {supervisors,0}, {workers,2}]}]} in function rabbit_fifo_dlx_integration_SUITE:assert_active_dlx_workers/3 (rabbit_fifo_dlx_integration_SUITE.erl, line 938) in call from test_server:ts_tc/3 (test_server.erl, line 1782) in call from test_server:run_test_case_eval1/6 (test_server.erl, line 1291) in call from test_server:run_test_case_eval/9 (test_server.erl, line 1223) ``` This commits attempts to remove that failure by using supervisor:which_children/1 because the docs for supervisor:count_children/1 say: "active - The count of all actively running child processes managed by this supervisor. For a simple_one_for_one supervisors, no check is done to ensure that each child process is still alive, although the result provided here is likely to be very accurate unless the supervisor is heavily overloaded." --- .../rabbit_fifo_dlx_integration_SUITE.erl | 29 ++++++++++--------- 1 file changed, 16 insertions(+), 13 deletions(-) diff --git a/deps/rabbit/test/rabbit_fifo_dlx_integration_SUITE.erl b/deps/rabbit/test/rabbit_fifo_dlx_integration_SUITE.erl index d9d8142eb0ca..5fc5a2400a00 100644 --- a/deps/rabbit/test/rabbit_fifo_dlx_integration_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_dlx_integration_SUITE.erl @@ -25,6 +25,8 @@ -import(rabbit_ct_helpers, [eventually/1, eventually/3, consistently/1]). +-import(rabbit_ct_broker_helpers, [rpc/5, + rpc/6]). -import(quorum_queue_SUITE, [publish/2, consume/3]). @@ -94,9 +96,8 @@ init_per_group(Group, Config, NodesCount) -> Config2 = rabbit_ct_helpers:run_steps(Config1, [fun merge_app_env/1 ] ++ rabbit_ct_broker_helpers:setup_steps()), - ok = rabbit_ct_broker_helpers:rpc( - Config2, 0, application, set_env, - [rabbit, channel_tick_interval, 100]), + ok = rpc(Config2, 0, application, set_env, + [rabbit, channel_tick_interval, 100]), case rabbit_ct_broker_helpers:enable_feature_flag(Config2, stream_queue) of ok -> Config2; Skip -> Skip @@ -147,6 +148,10 @@ end_per_testcase(Testcase, Config) -> delete_queue(Ch, ?config(target_queue_5, Config)), delete_queue(Ch, ?config(target_queue_6, Config)), #'exchange.delete_ok'{} = amqp_channel:call(Ch, #'exchange.delete'{exchange = ?config(dead_letter_exchange, Config)}), + + DlxWorkers = rabbit_ct_broker_helpers:rpc_all(Config, supervisor, which_children, [rabbit_fifo_dlx_sup]), + ?assert(lists:all(fun(L) -> L =:= [] end, DlxWorkers)), + Config1 = rabbit_ct_helpers:run_steps( Config, rabbit_ct_client_helpers:teardown_steps()), @@ -499,7 +504,7 @@ drop_head_falls_back_to_at_most_once(Config) -> consistently( ?_assertMatch( [_, {active, 0}, _, _], - rabbit_ct_broker_helpers:rpc(Config, Server, supervisor, count_children, [rabbit_fifo_dlx_sup]))). + rpc(Config, Server, supervisor, count_children, [rabbit_fifo_dlx_sup]))). %% Test that dynamically switching dead-letter-strategy works. switch_strategy(Config) -> @@ -973,12 +978,12 @@ single_dlx_worker(Config) -> assert_active_dlx_workers(0, Config, Follower0), ok = rabbit_ct_broker_helpers:start_node(Config, Server1), consistently( - ?_assertMatch( - [_, {active, 0}, _, _], - rabbit_ct_broker_helpers:rpc(Config, Server1, supervisor, count_children, [rabbit_fifo_dlx_sup], 1000))), + ?_assertEqual( + 0, + length(rpc(Config, Server1, supervisor, which_children, [rabbit_fifo_dlx_sup], 1000)))), - Pid = rabbit_ct_broker_helpers:rpc(Config, Leader0, erlang, whereis, [RaName]), - true = rabbit_ct_broker_helpers:rpc(Config, Leader0, erlang, exit, [Pid, kill]), + Pid = rpc(Config, Leader0, erlang, whereis, [RaName]), + true = rpc(Config, Leader0, erlang, exit, [Pid, kill]), {ok, _, {_, Leader1}} = ?awaitMatch({ok, _, _}, ra:members({RaName, Follower0}), 1000), @@ -989,9 +994,7 @@ single_dlx_worker(Config) -> assert_active_dlx_workers(1, Config, Leader1). assert_active_dlx_workers(N, Config, Server) -> - ?assertMatch( - [_, {active, N}, _, _], - rabbit_ct_broker_helpers:rpc(Config, Server, supervisor, count_children, [rabbit_fifo_dlx_sup], 1000)). + ?assertEqual(N, length(rpc(Config, Server, supervisor, which_children, [rabbit_fifo_dlx_sup], 2000))). declare_queue(Channel, Queue, Args) -> #'queue.declare_ok'{} = amqp_channel:call(Channel, #'queue.declare'{ @@ -1012,7 +1015,7 @@ delete_queue(Channel, Queue) -> #'queue.delete_ok'{message_count = 0} = amqp_channel:call(Channel, #'queue.delete'{queue = Queue}). get_global_counters(Config) -> - rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_global_counters, overview, []). + rpc(Config, 0, rabbit_global_counters, overview, []). %% Returns the delta of Metric between testcase start and now. counted(Metric, Config) -> From b2c87c59a0394029409988eec2ee71aa6804e263 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Fri, 18 Nov 2022 13:43:32 +0000 Subject: [PATCH 052/118] Minor reformatting and renaming --- .../src/rabbit_mqtt_processor.erl | 37 ++++++++++--------- deps/rabbitmq_mqtt/test/integration_SUITE.erl | 9 +++-- 2 files changed, 24 insertions(+), 22 deletions(-) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 015be0686612..444b828d412a 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -451,7 +451,8 @@ start_keepalive(#mqtt_frame_connect{keep_alive = Seconds}, #proc_state{socket = Socket}) -> ok = rabbit_mqtt_keepalive:start(Seconds, Socket). -handle_clean_session(_, PState0 = #proc_state{clean_sess = false, proto_ver = ProtoVer}) -> +handle_clean_session(_, PState0 = #proc_state{clean_sess = false, + proto_ver = ProtoVer}) -> case get_queue(?QOS_1, PState0) of {error, not_found} -> %% Queue will be created later when client subscribes. @@ -512,17 +513,17 @@ queue_name(QoS, #proc_state{client_id = ClientId, rabbit_misc:r(VHost, queue, QNameBin). find_queue_name(TopicName, #proc_state{exchange = Exchange, - mqtt2amqp_fun = Mqtt2AmqpFun} = PState) -> + mqtt2amqp_fun = Mqtt2AmqpFun} = PState) -> RoutingKey = Mqtt2AmqpFun(TopicName), - QName0 = queue_name(?QOS_0, PState), - case lookup_binding(Exchange, QName0, RoutingKey) of + QNameQoS0 = queue_name(?QOS_0, PState), + case lookup_binding(Exchange, QNameQoS0, RoutingKey) of true -> - {ok, QName0}; + {ok, QNameQoS0}; false -> - QName1 = queue_name(?QOS_1, PState), - case lookup_binding(Exchange, QName1, RoutingKey) of + QNameQoS1 = queue_name(?QOS_1, PState), + case lookup_binding(Exchange, QNameQoS1, RoutingKey) of true -> - {ok, QName1}; + {ok, QNameQoS1}; false -> {not_found, []} end @@ -530,14 +531,14 @@ find_queue_name(TopicName, #proc_state{exchange = Exchange, lookup_binding(Exchange, QueueName, RoutingKey) -> B = #binding{source = Exchange, - destination = QueueName, - key = RoutingKey}, + destination = QueueName, + key = RoutingKey}, lists:member(B, rabbit_binding:list_for_source_and_destination(Exchange, QueueName)). has_subs(#proc_state{exchange = Exchange} = PState) -> - has_subs_between(Exchange, queue_name(?QOS_0, PState)) orelse - has_subs_between(Exchange, queue_name(?QOS_1, PState)). -has_subs_between(Exchange, QueueName) -> + has_bindings_between(Exchange, queue_name(?QOS_0, PState)) orelse + has_bindings_between(Exchange, queue_name(?QOS_1, PState)). +has_bindings_between(Exchange, QueueName) -> case rabbit_binding:list_for_source_and_destination(Exchange, QueueName) of [] -> false; @@ -1704,7 +1705,8 @@ maybe_decrement_publisher(_) -> ok. %% multiple subscriptions from the same connection count as one consumer -maybe_increment_consumer(false, #proc_state{proto_ver = ProtoVer} = PState) -> +maybe_increment_consumer(_WasConsumer = false, + #proc_state{proto_ver = ProtoVer} = PState) -> case has_subs(PState) of true -> rabbit_global_counters:consumer_created(ProtoVer); @@ -1714,8 +1716,8 @@ maybe_increment_consumer(false, #proc_state{proto_ver = ProtoVer} = PState) -> maybe_increment_consumer(_, _) -> ok. -%% when there were subscriptions but not anymore -maybe_decrement_consumer(true, #proc_state{proto_ver = ProtoVer} = PState) -> +maybe_decrement_consumer(_WasConsumer = true, + #proc_state{proto_ver = ProtoVer} = PState) -> case has_subs(PState) of false -> rabbit_global_counters:consumer_deleted(ProtoVer); @@ -1725,9 +1727,8 @@ maybe_decrement_consumer(true, #proc_state{proto_ver = ProtoVer} = PState) -> maybe_decrement_consumer(_, _) -> ok. -%% used when connection is terminated maybe_decrement_consumer(#proc_state{proto_ver = ProtoVer, - auth_state = #auth_state{vhost = _Vhost}} = PState) -> + auth_state = #auth_state{vhost = _Vhost}} = PState) -> case has_subs(PState) of true -> rabbit_global_counters:consumer_deleted(ProtoVer); diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/integration_SUITE.erl index 1cf5bcc50034..7f81af654b99 100644 --- a/deps/rabbitmq_mqtt/test/integration_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/integration_SUITE.erl @@ -576,21 +576,22 @@ non_clean_sess_disconnect(Config) -> {C1, _} = connect(?FUNCTION_NAME, Config, [{clean_start, false}]), Topic = <<"test-topic1">>, {ok, _, [1]} = emqtt:subscribe(C1, Topic, qos1), + ProtoVer = v4, ?assertMatch(#{consumers := 1}, - get_global_counters(Config, v4)), + get_global_counters(Config, ProtoVer)), ok = emqtt:disconnect(C1), ?assertMatch(#{consumers := 0}, - get_global_counters(Config, v4)), + get_global_counters(Config, ProtoVer)), {C2, _} = connect(?FUNCTION_NAME, Config, [{clean_start, false}]), ?assertMatch(#{consumers := 1}, - get_global_counters(Config, v4)), + get_global_counters(Config, ProtoVer)), %% shouldn't receive message after unsubscribe {ok, _, _} = emqtt:unsubscribe(C2, Topic), ?assertMatch(#{consumers := 0}, - get_global_counters(Config, v4)), + get_global_counters(Config, ProtoVer)), Msg = <<"msg">>, {ok, _} = emqtt:publish(C2, Topic, Msg, qos1), {publish_not_received, Msg} = expect_publishes(Topic, [Msg]), From 0ba0a6e8f8ebd9f92148c3136806c53c765e3408 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Fri, 18 Nov 2022 15:42:21 +0000 Subject: [PATCH 053/118] Several small improvements 1. The mqtt_qos0 queue type uses now QName in the delivery. This makes the code simpler although it might be a bit less efficient because the tuple containing binaries is sent around and a hash is computed within rabbit_queue_type:module/2 2. Do not construct a new binary on every PUBACK. This is expensive with many PUBACKs per second. Instead, we store the QoS1 queue name in the process state (but only if the connection also consumes from that queue). 3. To make the code more readable, and less specialised, always handle queue actions when we call rabbit_queue_type:settle/5. This method only returns an action (delivery) when settling to the stream queue, which the MQTT plugin never does because an MQTT connection does not consume from a stream. It's not expensive at all to handle an empty list of queue actions. --- deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl | 2 + .../src/rabbit_mqtt_processor.erl | 47 +++++++++---------- .../src/rabbit_mqtt_qos0_queue.erl | 3 +- 3 files changed, 25 insertions(+), 27 deletions(-) diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl index 7b2a0fb60a6b..848a363e8008 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl @@ -42,6 +42,8 @@ clean_sess :: boolean(), will_msg, exchange :: rabbit_exchange:name(), + %% Set if client has at least one subscription with QoS 1. + queue_qos1 :: rabbit_amqqueue:name(), has_published = false :: boolean(), ssl_login_name, %% Retained messages handler. See rabbit_mqtt_retainer_sup diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 444b828d412a..3de6f1426e47 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -13,14 +13,6 @@ handle_ra_event/2, handle_down/2, handle_queue_event/2, soft_limit_exceeded/1, format_status/1]). -%%TODO Use single queue per MQTT subscriber connection? -%% * when publishing we store in x-mqtt-publish-qos header the publishing QoS -%% * routing key is present in the delivered message -%% * therefore, we can map routing key -> topic -> subscription -> subscription max QoS -%% Advantages: -%% * better scaling when single client creates subscriptions with different QoS levels -%% * no duplicates when single client creates overlapping subscriptions with different QoS levels - %% for testing purposes -export([get_vhost_username/1, get_vhost/3, get_vhost_from_user_mapping/2]). @@ -78,17 +70,15 @@ process_request(?PUBACK, #mqtt_frame{ variable = #mqtt_frame_publish{message_id = PacketId}}, #proc_state{unacked_server_pubs = U0, - queue_states = QStates0} = PState) -> + queue_states = QStates0, + queue_qos1 = QName} = PState) -> case maps:take(PacketId, U0) of {QMsgId, U} -> - %% TODO creating binary is expensive - QName = queue_name(?QOS_1, PState), case rabbit_queue_type:settle(QName, complete, ?CONSUMER_TAG, [QMsgId], QStates0) of - {ok, QStates, [] = _Actions} -> - % incr_queue_stats(QRef, MsgIds, State), - %%TODO handle actions - {ok, PState#proc_state{unacked_server_pubs = U, - queue_states = QStates }}; + {ok, QStates, Actions} -> + %%TODO rabbit_channel:incr_queue_stats/3 + {ok, handle_queue_actions(Actions, PState#proc_state{unacked_server_pubs = U, + queue_states = QStates})}; {protocol_error, _ErrorType, _Reason, _ReasonArgs} = Err -> {error, Err, PState} end; @@ -1032,7 +1022,8 @@ consume(Q, QoS, #proc_state{ fun(Q1) -> case rabbit_queue_type:consume(Q1, Spec, QStates0) of {ok, QStates} -> - PState = PState0#proc_state{queue_states = QStates}, + PState1 = PState0#proc_state{queue_states = QStates}, + PState = maybe_set_queue_qos1(QoS, PState1), {ok, PState}; {error, Reason} = Err -> rabbit_log:error("Failed to consume from ~s: ~p", @@ -1045,6 +1036,13 @@ consume(Q, QoS, #proc_state{ Err end. +%% To save memory, we only store the queue_qos1 value in process state if there is a QoS 1 subscription. +%% We store it in the process state such that we don't have to build the binary on every PUBACK we receive. +maybe_set_queue_qos1(?QOS_1, PState = #proc_state{queue_qos1 = undefined}) -> + PState#proc_state{queue_qos1 = queue_name(?QOS_1, PState)}; +maybe_set_queue_qos1(_, PState) -> + PState. + bind(QueueName, TopicName, PState) -> binding_action_with_checks({QueueName, TopicName, fun rabbit_binding:add/2}, PState). unbind(QueueName, TopicName, PState) -> @@ -1436,7 +1434,7 @@ deliver_to_client(Msgs, Ack, PState) -> deliver_one_to_client(Msg, Ack, S) end, PState, Msgs). -deliver_one_to_client(Msg = {QNameOrType, QPid, QMsgId, _Redelivered, +deliver_one_to_client(Msg = {QName, QPid, QMsgId, _Redelivered, #basic_message{content = #content{properties = #'P_basic'{headers = Headers}}}}, AckRequired, PState0) -> PublisherQoS = case rabbit_mqtt_util:table_lookup(Headers, <<"x-mqtt-publish-qos">>) of @@ -1454,13 +1452,13 @@ deliver_one_to_client(Msg = {QNameOrType, QPid, QMsgId, _Redelivered, end, QoS = effective_qos(PublisherQoS, SubscriberQoS), PState1 = maybe_publish_to_client(Msg, QoS, PState0), - PState = maybe_ack(AckRequired, QoS, QNameOrType, QMsgId, PState1), + PState = maybe_ack(AckRequired, QoS, QName, QMsgId, PState1), %%TODO GC % case GCThreshold of % undefined -> ok; % _ -> rabbit_basic:maybe_gc_large_msg(Content, GCThreshold) % end, - ok = maybe_notify_sent(QNameOrType, QPid, PState), + ok = maybe_notify_sent(QName, QPid, PState), PState. -spec effective_qos(qos(), qos()) -> qos(). @@ -1520,10 +1518,9 @@ increment_packet_id(Id) -> maybe_ack(_AckRequired = true, ?QOS_0, QName, QMsgId, PState = #proc_state{queue_states = QStates0}) -> case rabbit_queue_type:settle(QName, complete, ?CONSUMER_TAG, [QMsgId], QStates0) of - {ok, QStates, [] = _Actions} -> - % incr_queue_stats(QRef, MsgIds, State), - %%TODO handle actions - PState#proc_state{queue_states = QStates}; + {ok, QStates, Actions} -> + %%TODO rabbit_channel:incr_queue_stats/3 + handle_queue_actions(Actions, PState#proc_state{queue_states = QStates}); {protocol_error, _ErrorType, _Reason, _ReasonArgs} = Err -> %%TODO handle error throw(Err) @@ -1531,8 +1528,6 @@ maybe_ack(_AckRequired = true, ?QOS_0, QName, QMsgId, maybe_ack(_, _, _, _, PState) -> PState. -maybe_notify_sent(rabbit_mqtt_qos0_queue, _, _) -> - ok; maybe_notify_sent(QName, QPid, #proc_state{queue_states = QStates}) -> case rabbit_queue_type:module(QName, QStates) of {ok, rabbit_classic_queue} -> diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl index 7ae4ea77b1d8..39153a1d4897 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl @@ -95,8 +95,9 @@ delete(Q, _IfUnused, _IfEmpty, ActingUser) -> {[], rabbit_queue_type:actions()}. deliver([{Q, stateless}], Delivery = #delivery{message = BasicMessage}) -> Pid = amqqueue:get_pid(Q), + QName = amqqueue:get_name(Q), Msg = {queue_event, ?MODULE, - {?MODULE, Pid, _QMsgId = none, _Redelivered = false, BasicMessage}}, + {QName, Pid, _QMsgId = none, _Redelivered = false, BasicMessage}}, gen_server:cast(Pid, Msg), Actions = confirm(Delivery, Q), {[], Actions}. From 6e527fb940651327888fad6e278fba33cc236c29 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Mon, 21 Nov 2022 11:31:00 +0000 Subject: [PATCH 054/118] Replace existing subscription MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit "If a Server receives a SUBSCRIBE Packet containing a Topic Filter that is identical to an existing Subscription’s Topic Filter then it MUST completely replace that existing Subscription with a new Subscription. The Topic Filter in the new Subscription will be identical to that in the previous Subscription, although its maximum QoS value could be different. Any existing retained messages matching the Topic Filter MUST be re-sent, but the flow of publications MUST NOT be interrupted [MQTT-3.8.4-3]." --- .../src/rabbit_mqtt_processor.erl | 130 +++++++++++------- deps/rabbitmq_mqtt/test/integration_SUITE.erl | 46 +++++++ 2 files changed, 130 insertions(+), 46 deletions(-) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 3de6f1426e47..e921ff1fd424 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -154,7 +154,10 @@ process_request(?SUBSCRIBE, #proc_state{send_fun = SendFun, retainer_pid = RPid} = PState0) -> rabbit_log_connection:debug("Received a SUBSCRIBE for topic(s) ~p", [Topics]), - HasSubsBefore = has_subs(PState0), + TopicNamesQos0 = topic_names(?QOS_0, PState0), + TopicNamesQos1 = topic_names(?QOS_1, PState0), + HasSubsBefore = TopicNamesQos0 =/= [] orelse TopicNamesQos1 =/= [], + {QosResponse, PState1} = lists:foldl(fun(_Topic, {[?SUBACK_FAILURE | _] = L, S}) -> %% Once a subscription failed, mark all following subscriptions @@ -162,33 +165,37 @@ process_request(?SUBSCRIBE, %% to close the client connection anyways. {[?SUBACK_FAILURE | L], S}; (#mqtt_topic{name = TopicName, - qos = Qos0}, + qos = Qos0} = Topic, {L, S0}) -> QoS = supported_sub_qos(Qos0), - %%TODO check whether new subscription replaces an existing one - %% (i.e. same topic name and different QoS) - case ensure_queue(QoS, S0) of - {ok, Q} -> - QName = amqqueue:get_name(Q), - case bind(QName, TopicName, S0) of - {ok, _Output, S1} -> - %%TODO check what happens if we basic.consume multiple times - %% for the same queue - case consume(Q, QoS, S1) of - {ok, S2} -> - {[QoS | L], S2}; - {error, _Reason} -> - {[?SUBACK_FAILURE | L], S1} + case maybe_replace_old_sub(Topic, TopicNamesQos0, TopicNamesQos1, S0) of + {ok, S1} -> + case ensure_queue(QoS, S1) of + {ok, Q} -> + QName = amqqueue:get_name(Q), + case bind(QName, TopicName, S1) of + {ok, _Output, S2} -> + %%TODO check what happens if we basic.consume multiple times + %% for the same queue + case consume(Q, QoS, S2) of + {ok, S3} -> + {[QoS | L], S3}; + {error, _Reason} -> + {[?SUBACK_FAILURE | L], S2} + end; + {error, Reason, S2} -> + rabbit_log:error("Failed to bind ~s with topic ~s: ~p", + [rabbit_misc:rs(QName), TopicName, Reason]), + {[?SUBACK_FAILURE | L], S2} end; - {error, Reason, S} -> - rabbit_log:error("Failed to bind ~s with topic ~s: ~p", - [rabbit_misc:rs(QName), TopicName, Reason]), - {[?SUBACK_FAILURE | L], S} + {error, _} -> + {[?SUBACK_FAILURE | L], S1} end; - {error, _} -> - {[?SUBACK_FAILURE | L], S0} + {error, _Reason, S1} -> + {[?SUBACK_FAILURE | L], S1} end end, {[], PState0}, Topics), + maybe_increment_consumer(HasSubsBefore, PState1), SendFun( #mqtt_frame{fixed = #mqtt_frame_fixed{type = ?SUBACK}, @@ -215,21 +222,21 @@ process_request(?UNSUBSCRIBE, rabbit_log_connection:debug("Received an UNSUBSCRIBE for topic(s) ~p", [Topics]), HasSubsBefore = has_subs(PState0), PState = lists:foldl( - fun(#mqtt_topic{name = TopicName}, #proc_state{} = S0) -> - case find_queue_name(TopicName, S0) of - {ok, QName} -> - case unbind(QName, TopicName, S0) of - {ok, _, _} -> - PState0; - {error, Reason, State} -> - rabbit_log:error("Failed to unbind ~s with topic ~s: ~p", - [rabbit_misc:rs(QName), TopicName, Reason]), - State - end; - {not_found, _} -> - PState0 - end - end, PState0, Topics), + fun(#mqtt_topic{name = TopicName}, #proc_state{} = S0) -> + case find_queue_name(TopicName, S0) of + {ok, QName} -> + case unbind(QName, TopicName, S0) of + {ok, _, S} -> + S; + {error, Reason, S} -> + rabbit_log:error("Failed to unbind ~s with topic ~s: ~p", + [rabbit_misc:rs(QName), TopicName, Reason]), + S + end; + {not_found, _} -> + S0 + end + end, PState0, Topics), SendFun( #mqtt_frame{fixed = #mqtt_frame_fixed {type = ?UNSUBACK}, variable = #mqtt_frame_suback{message_id = MessageId}}, @@ -525,15 +532,46 @@ lookup_binding(Exchange, QueueName, RoutingKey) -> key = RoutingKey}, lists:member(B, rabbit_binding:list_for_source_and_destination(Exchange, QueueName)). -has_subs(#proc_state{exchange = Exchange} = PState) -> - has_bindings_between(Exchange, queue_name(?QOS_0, PState)) orelse - has_bindings_between(Exchange, queue_name(?QOS_1, PState)). -has_bindings_between(Exchange, QueueName) -> - case rabbit_binding:list_for_source_and_destination(Exchange, QueueName) of - [] -> - false; - _ -> - true +has_subs(PState) -> + topic_names(?QOS_0, PState) =/= [] orelse + topic_names(?QOS_1, PState) =/= []. + +topic_names(QoS, #proc_state{exchange = Exchange, + amqp2mqtt_fun = Amqp2MqttFun} = PState) -> + Bindings = rabbit_binding:list_for_source_and_destination(Exchange, queue_name(QoS, PState)), + lists:map(fun(B) -> Amqp2MqttFun(B#binding.key) end, Bindings). + +%% "If a Server receives a SUBSCRIBE Packet containing a Topic Filter that is identical +%% to an existing Subscription’s Topic Filter then it MUST completely replace that +%% existing Subscription with a new Subscription. The Topic Filter in the new Subscription +%% will be identical to that in the previous Subscription, although its maximum QoS value +%% could be different. [...]" [MQTT-3.8.4-3]. +%% +%% Therefore, if we receive a QoS 0 subscription for a topic that already has QoS 1, +%% we unbind QoS 1 (and vice versa). +maybe_replace_old_sub(#mqtt_topic{name = TopicName, qos = ?QOS_0}, + _, OldTopicNamesQos1, PState) -> + QName = queue_name(?QOS_1, PState), + maybe_unbind(TopicName, OldTopicNamesQos1, QName, PState); +maybe_replace_old_sub(#mqtt_topic{name = TopicName, qos = QoS}, + OldTopicNamesQos0, _, PState) + when QoS =:= ?QOS_1 orelse QoS =:= ?QOS_2 -> + QName = queue_name(?QOS_0, PState), + maybe_unbind(TopicName, OldTopicNamesQos0, QName, PState). + +maybe_unbind(TopicName, TopicNames, QName, PState0) -> + case lists:member(list_to_binary(TopicName), TopicNames) of + false -> + {ok, PState0}; + true -> + case unbind(QName, TopicName, PState0) of + {ok, _Output, PState} -> + {ok, PState}; + {error, Reason, _PState} = Err -> + rabbit_log:error("Failed to unbind ~s with topic ~s: ~p", + [rabbit_misc:rs(QName), TopicName, Reason]), + Err + end end. hand_off_to_retainer(RetainerPid, Amqp2MqttFun, Topic0, #mqtt_msg{payload = <<"">>}) -> diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/integration_SUITE.erl index 7f81af654b99..0f62adbac538 100644 --- a/deps/rabbitmq_mqtt/test/integration_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/integration_SUITE.erl @@ -55,6 +55,8 @@ common_tests() -> ,event_authentication_failure ,internal_event_handler ,non_clean_sess_disconnect + ,subscribe_same_topic_same_qos + ,subscribe_same_topic_different_qos ]. suite() -> @@ -600,6 +602,50 @@ non_clean_sess_disconnect(Config) -> {C3, _} = connect(?FUNCTION_NAME, Config, [{clean_start, true}]), ok = emqtt:disconnect(C3). +subscribe_same_topic_same_qos(Config) -> + {C, _} = connect(?FUNCTION_NAME, Config), + Topic = <<"a/b">>, + + {ok, _} = emqtt:publish(C, Topic, <<"retained">>, [{retain, true}, + {qos, 1}]), + %% Subscribe with QoS 0 + {ok, _, [0]} = emqtt:subscribe(C, Topic, qos0), + {ok, _} = emqtt:publish(C, Topic, <<"msg1">>, qos1), + %% Subscribe to same topic with same QoS + {ok, _, [0]} = emqtt:subscribe(C, Topic, qos0), + {ok, _} = emqtt:publish(C, Topic, <<"msg2">>, qos1), + + %% "Any existing retained messages matching the Topic Filter MUST be re-sent" [MQTT-3.8.4-3] + ok = expect_publishes(Topic, [<<"retained">>, <<"msg1">>, + <<"retained">>, <<"msg2">> + ]), + ok = emqtt:disconnect(C). + +subscribe_same_topic_different_qos(Config) -> + {C, _} = connect(?FUNCTION_NAME, Config, [{clean_start, false}]), + Topic = <<"b/c">>, + + {ok, _} = emqtt:publish(C, Topic, <<"retained">>, [{retain, true}, + {qos, 1}]), + %% Subscribe with QoS 0 + {ok, _, [0]} = emqtt:subscribe(C, Topic, qos0), + {ok, _} = emqtt:publish(C, Topic, <<"msg1">>, qos1), + %% Subscribe to same topic with QoS 1 + {ok, _, [1]} = emqtt:subscribe(C, Topic, qos1), + {ok, _} = emqtt:publish(C, Topic, <<"msg2">>, qos1), + %% Subscribe to same topic with QoS 0 again + {ok, _, [0]} = emqtt:subscribe(C, Topic, qos0), + {ok, _} = emqtt:publish(C, Topic, <<"msg3">>, qos1), + + %% "Any existing retained messages matching the Topic Filter MUST be re-sent" [MQTT-3.8.4-3] + ok = expect_publishes(Topic, [<<"retained">>, <<"msg1">>, + <<"retained">>, <<"msg2">>, + <<"retained">>, <<"msg3">>]), + + ok = emqtt:disconnect(C), + {C1, _} = connect(?FUNCTION_NAME, Config, [{clean_start, true}]), + ok = emqtt:disconnect(C1). + %% ------------------------------------------------------------------- %% Internal helpers %% ------------------------------------------------------------------- From c3779d9996a76be9d842594262bc44ba0cec0c4a Mon Sep 17 00:00:00 2001 From: Chunyi Lyu Date: Mon, 21 Nov 2022 15:25:34 +0000 Subject: [PATCH 055/118] Implement message consuming counters in mqtt --- deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl | 1 + deps/rabbitmq_mqtt/src/rabbit_mqtt.erl | 12 +++- .../src/rabbit_mqtt_processor.erl | 56 ++++++++++++++++--- deps/rabbitmq_mqtt/test/integration_SUITE.erl | 48 ++++++++++++---- deps/rabbitmq_mqtt/test/util.erl | 20 ++++--- .../src/rabbit_web_mqtt_handler.erl | 7 --- 6 files changed, 108 insertions(+), 36 deletions(-) diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl index 848a363e8008..31edcf655235 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl @@ -7,6 +7,7 @@ -define(CLIENT_ID_MAXLEN, 23). -define(PG_SCOPE, pg_scope_rabbitmq_mqtt_clientid). +-define(QOS0_QTYPE, rabbit_mqtt_qos0_queue). -include("rabbit_mqtt_types.hrl"). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl index 27823b9d5b75..39ecd0ab1997 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl @@ -20,8 +20,7 @@ local_connection_pids/0]). start(normal, []) -> - rabbit_global_counters:init([{protocol, ?MQTT_PROTO_V3}]), - rabbit_global_counters:init([{protocol, ?MQTT_PROTO_V4}]), + global_counters_init(), {ok, Listeners} = application:get_env(tcp_listeners), {ok, SslListeners} = application:get_env(ssl_listeners), case rabbit_mqtt_ff:track_client_id_in_ra() of @@ -88,3 +87,12 @@ local_connection_pids() -> pg:get_local_members(PgScope, Group) end, pg:which_groups(PgScope)) end. + +global_counters_init() -> + global_counters_init0(?MQTT_PROTO_V3), + global_counters_init0(?MQTT_PROTO_V4). +global_counters_init0(Proto) -> + rabbit_global_counters:init([{protocol, Proto}]), + rabbit_global_counters:init([{protocol, Proto}, {queue_type, ?QOS0_QTYPE}]), + rabbit_global_counters:init([{protocol, Proto}, {queue_type, rabbit_classic_queue}]), + rabbit_global_counters:init([{protocol, Proto}, {queue_type, rabbit_quorum_queue}]). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index e921ff1fd424..5b760f07efa0 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -77,6 +77,7 @@ process_request(?PUBACK, case rabbit_queue_type:settle(QName, complete, ?CONSUMER_TAG, [QMsgId], QStates0) of {ok, QStates, Actions} -> %%TODO rabbit_channel:incr_queue_stats/3 + record_message_manual_ack(QName, PState), {ok, handle_queue_actions(Actions, PState#proc_state{unacked_server_pubs = U, queue_states = QStates})}; {protocol_error, _ErrorType, _Reason, _ReasonArgs} = Err -> @@ -1024,7 +1025,7 @@ queue_args(_, _) -> end. queue_type(?QOS_0, true, _) -> - rabbit_mqtt_qos0_queue; + ?QOS0_QTYPE; queue_type(_, _, QArgs) -> rabbit_amqqueue:get_queue_type(QArgs). @@ -1040,7 +1041,7 @@ consume(Q, QoS, #proc_state{ case check_resource_access(User, QName, read, AuthzCtx) of ok -> case amqqueue:get_type(Q) of - rabbit_mqtt_qos0_queue -> + ?QOS0_QTYPE -> %% Messages get delivered directly to our process without %% explicitly calling rabbit_queue_type:consume/3. {ok, PState0}; @@ -1323,7 +1324,7 @@ maybe_delete_mqtt_qos0_queue(PState = #proc_state{clean_sess = true, {ok, Q} -> %% double check we delete the right queue case {amqqueue:get_type(Q), amqqueue:get_pid(Q)} of - {rabbit_mqtt_qos0_queue, Pid} + {?QOS0_QTYPE, Pid} when Pid =:= self() -> rabbit_queue_type:delete(Q, false, false, Username); _ -> @@ -1394,7 +1395,7 @@ handle_down({{'DOWN', QName}, _MRef, process, QPid, Reason}, {ok, PState} end. -handle_queue_event({queue_event, rabbit_mqtt_qos0_queue, Msg}, PState0) -> +handle_queue_event({queue_event, ?QOS0_QTYPE, Msg}, PState0) -> PState = deliver_one_to_client(Msg, false, PState0), {ok, PState}; handle_queue_event({queue_event, QName, Evt}, @@ -1489,7 +1490,7 @@ deliver_one_to_client(Msg = {QName, QPid, QMsgId, _Redelivered, ?QOS_0 end, QoS = effective_qos(PublisherQoS, SubscriberQoS), - PState1 = maybe_publish_to_client(Msg, QoS, PState0), + PState1 = maybe_publish_to_client(Msg, QoS, SubscriberQoS, PState0), PState = maybe_ack(AckRequired, QoS, QName, QMsgId, PState1), %%TODO GC % case GCThreshold of @@ -1506,15 +1507,15 @@ effective_qos(PublisherQoS, SubscriberQoS) -> %% [MQTT-3.8.4-8]." erlang:min(PublisherQoS, SubscriberQoS). -maybe_publish_to_client({_, _, _, _Redelivered = true, _}, ?QOS_0, PState) -> +maybe_publish_to_client({_, _, _, _Redelivered = true, _}, ?QOS_0, _, PState) -> %% Do not redeliver to MQTT subscriber who gets message at most once. PState; maybe_publish_to_client( - {_QName, _QPid, QMsgId, Redelivered, + {QName, _QPid, QMsgId, Redelivered, #basic_message{ routing_keys = [RoutingKey | _CcRoutes], content = #content{payload_fragments_rev = FragmentsRev}}}, - QoS, PState0 = #proc_state{amqp2mqtt_fun = Amqp2MqttFun, + QoS, SubscriberQos, PState0 = #proc_state{amqp2mqtt_fun = Amqp2MqttFun, send_fun = SendFun}) -> {PacketId, PState} = queue_message_id_to_packet_id(QMsgId, QoS, PState0), %%TODO support iolists when sending to client @@ -1536,6 +1537,7 @@ maybe_publish_to_client( topic_name = Amqp2MqttFun(RoutingKey)}, payload = Payload}, SendFun(Frame, PState), + record_message_delivered(QName, Redelivered, SubscriberQos, PState), PState. queue_message_id_to_packet_id(_, ?QOS_0, PState) -> @@ -1558,6 +1560,7 @@ maybe_ack(_AckRequired = true, ?QOS_0, QName, QMsgId, case rabbit_queue_type:settle(QName, complete, ?CONSUMER_TAG, [QMsgId], QStates0) of {ok, QStates, Actions} -> %%TODO rabbit_channel:incr_queue_stats/3 + record_message_auto_ack(QName, PState), handle_queue_actions(Actions, PState#proc_state{queue_states = QStates}); {protocol_error, _ErrorType, _Reason, _ReasonArgs} = Err -> %%TODO handle error @@ -1770,3 +1773,40 @@ maybe_decrement_consumer(#proc_state{proto_ver = ProtoVer, end; maybe_decrement_consumer(_) -> ok. + +record_message_delivered(QName, true, SubscriberQoS, #proc_state{proto_ver = ProtoVer, queue_states = QStates} = _PState) -> + case SubscriberQoS of + ?QOS_0 -> + rabbit_global_counters:messages_delivered(ProtoVer, ?QOS0_QTYPE, 1), + rabbit_global_counters:messages_redelivered(ProtoVer, ?QOS0_QTYPE, 1); + ?QOS_1 -> + case rabbit_queue_type:module(QName, QStates) of + {ok, QType} -> + rabbit_global_counters:messages_delivered(ProtoVer, QType, 1), + rabbit_global_counters:messages_redelivered(ProtoVer, QType, 1); + _ -> + noop + end + end; +record_message_delivered(QName, false, SubscriberQoS, #proc_state{proto_ver = ProtoVer, queue_states = QStates} = _PState) -> + case SubscriberQoS of + ?QOS_0 -> + rabbit_global_counters:messages_delivered(ProtoVer, ?QOS0_QTYPE, 1); + ?QOS_1 -> + case rabbit_queue_type:module(QName, QStates) of + {ok, QType} -> + rabbit_global_counters:messages_delivered(ProtoVer, QType, 1); + _ -> + noop + end + end. + +record_message_manual_ack(QName, #proc_state{proto_ver = ProtoVer, queue_states = QStates} = _PState) -> + {ok, QType} = rabbit_queue_type:module(QName, QStates), + rabbit_global_counters:messages_acknowledged(ProtoVer, QType, 1), + rabbit_global_counters:messages_delivered_consume_manual_ack(ProtoVer, QType, 1). + +record_message_auto_ack(QName, #proc_state{proto_ver = ProtoVer, queue_states = QStates} = _PState) -> + {ok, QType} = rabbit_queue_type:module(QName, QStates), + rabbit_global_counters:messages_acknowledged(ProtoVer, QType, 1), + rabbit_global_counters:messages_delivered_consume_auto_ack(ProtoVer, QType, 1). diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/integration_SUITE.erl index 0f62adbac538..bd9ed57b0193 100644 --- a/deps/rabbitmq_mqtt/test/integration_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/integration_SUITE.erl @@ -20,6 +20,7 @@ -import(util, [all_connection_pids/1, get_global_counters/2, get_global_counters/3, + get_global_counters/4, expect_publishes/2]). all() -> @@ -364,31 +365,56 @@ global_counters(Config, ProtoVer) -> Topic0 = <<"test-topic0">>, Topic1 = <<"test-topic1">>, - {ok, _, [1]} = emqtt:subscribe(C, Topic0, qos1), + Topic2 = <<"test-topic2">>, + {ok, _, [0]} = emqtt:subscribe(C, Topic0, qos0), {ok, _, [1]} = emqtt:subscribe(C, Topic1, qos1), - {ok, _} = emqtt:publish(C, Topic0, <<"testm">>, qos1), - {ok, _} = emqtt:publish(C, Topic1, <<"testm">>, qos1), + {ok, _, [1]} = emqtt:subscribe(C, Topic2, qos1), + ok = emqtt:publish(C, Topic0, <<"testm0">>, qos0), + ok = emqtt:publish(C, Topic1, <<"testm1">>, qos0), + {ok, _} = emqtt:publish(C, Topic2, <<"testm2">>, qos1), + ok = expect_publishes(Topic0, [<<"testm0">>]), + ok = expect_publishes(Topic1, [<<"testm1">>]), + ok = expect_publishes(Topic2, [<<"testm2">>]), ?assertEqual(#{publishers => 1, consumers => 1, - messages_confirmed_total => 2, - messages_received_confirm_total => 2, - messages_received_total => 2, - messages_routed_total => 2, + messages_confirmed_total => 1, + messages_received_confirm_total => 1, + messages_received_total => 3, + messages_routed_total => 3, messages_unroutable_dropped_total => 0, messages_unroutable_returned_total => 0}, get_global_counters(Config, ProtoVer)), + ?assertEqual(#{messages_delivered_total => 2, + messages_acknowledged_total => 2, + messages_delivered_consume_auto_ack_total => 1, + messages_delivered_consume_manual_ack_total => 1, + messages_delivered_get_auto_ack_total => 0, + messages_delivered_get_manual_ack_total => 0, + messages_get_empty_total => 0, + messages_redelivered_total => 0}, + get_global_counters(Config, ProtoVer, 0, [{queue_type, rabbit_classic_queue}])), + ?assertEqual(#{messages_delivered_total => 1, + messages_acknowledged_total => 0, + messages_delivered_consume_auto_ack_total => 0, + messages_delivered_consume_manual_ack_total => 0, + messages_delivered_get_auto_ack_total => 0, + messages_delivered_get_manual_ack_total => 0, + messages_get_empty_total => 0, + messages_redelivered_total => 0}, + get_global_counters(Config, ProtoVer, 0, [{queue_type, rabbit_mqtt_qos0_queue}])), + {ok, _, _} = emqtt:unsubscribe(C, Topic1), ?assertEqual(1, maps:get(consumers, get_global_counters(Config, ProtoVer))), ok = emqtt:disconnect(C), ?assertEqual(#{publishers => 0, consumers => 0, - messages_confirmed_total => 2, - messages_received_confirm_total => 2, - messages_received_total => 2, - messages_routed_total => 2, + messages_confirmed_total => 1, + messages_received_confirm_total => 1, + messages_received_total => 3, + messages_routed_total => 3, messages_unroutable_dropped_total => 0, messages_unroutable_returned_total => 0}, get_global_counters(Config, ProtoVer)). diff --git a/deps/rabbitmq_mqtt/test/util.erl b/deps/rabbitmq_mqtt/test/util.erl index 40b205146a4f..f31786bfd6b8 100644 --- a/deps/rabbitmq_mqtt/test/util.erl +++ b/deps/rabbitmq_mqtt/test/util.erl @@ -7,6 +7,7 @@ sync_publish_result/3, get_global_counters/2, get_global_counters/3, + get_global_counters/4, expect_publishes/2]). all_connection_pids(Config) -> @@ -49,12 +50,15 @@ sync_publish_result(Caller, Mref, Result) -> erlang:send(Caller, {Mref, Result}). get_global_counters(Config, ProtoVer) -> - get_global_counters(Config, ProtoVer, 0). - -get_global_counters(Config, v3, Node) -> - get_global_counters(Config, ?MQTT_PROTO_V3, Node); -get_global_counters(Config, v4, Node) -> - get_global_counters(Config, ?MQTT_PROTO_V4, Node); -get_global_counters(Config, Proto, Node) -> - maps:get([{protocol, Proto}], + get_global_counters(Config, ProtoVer, 0, []). + +get_global_counters(Config, ProtoVer, Node) -> + get_global_counters(Config, ProtoVer, Node, []). + +get_global_counters(Config, v3, Node, QType) -> + get_global_counters(Config, ?MQTT_PROTO_V3, Node, QType); +get_global_counters(Config, v4, Node, QType) -> + get_global_counters(Config, ?MQTT_PROTO_V4, Node, QType); +get_global_counters(Config, Proto, Node, QType) -> + maps:get([{protocol, Proto}] ++ QType, rabbit_ct_broker_helpers:rpc(Config, Node, rabbit_global_counters, overview, [])). diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index 234dc73e977e..fa5d8337fcf3 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -41,13 +41,6 @@ -define(CLOSE_PROTOCOL_ERROR, 1002). -define(CLOSE_INCONSISTENT_MSG_TYPE, 1007). -%%TODO call rabbit_networking:register_non_amqp_connection/1 so that we are notified -%% when need to force load the 'connection_created' event for the management plugin, see -%% https://github.com/rabbitmq/rabbitmq-management-agent/issues/58 -%% https://github.com/rabbitmq/rabbitmq-server/blob/90cc0e2abf944141feedaf3190d7b6d8b4741b11/deps/rabbitmq_stream/src/rabbit_stream_reader.erl#L536 -%% https://github.com/rabbitmq/rabbitmq-server/blob/90cc0e2abf944141feedaf3190d7b6d8b4741b11/deps/rabbitmq_stream/src/rabbit_stream_reader.erl#L189 -%% https://github.com/rabbitmq/rabbitmq-server/blob/7eb4084cf879fe6b1d26dd3c837bd180cb3a8546/deps/rabbitmq_management_agent/src/rabbit_mgmt_db_handler.erl#L72 - %% cowboy_sub_protcol upgrade(Req, Env, Handler, HandlerState) -> upgrade(Req, Env, Handler, HandlerState, #{}). From 573934259a05c1db8f6ad05dc8033173842d50a7 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Mon, 21 Nov 2022 17:26:40 +0000 Subject: [PATCH 056/118] Consume from queue once Each MQTT connection consumes from its queue at most once (unless when failing over). --- deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl | 2 +- deps/rabbitmq_mqtt/src/rabbit_mqtt.erl | 16 +-- .../src/rabbit_mqtt_processor.erl | 101 ++++++++++-------- deps/rabbitmq_mqtt/test/integration_SUITE.erl | 4 + 4 files changed, 72 insertions(+), 51 deletions(-) diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl index 31edcf655235..95cfa0df5503 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl @@ -7,7 +7,7 @@ -define(CLIENT_ID_MAXLEN, 23). -define(PG_SCOPE, pg_scope_rabbitmq_mqtt_clientid). --define(QOS0_QTYPE, rabbit_mqtt_qos0_queue). +-define(QUEUE_TYPE_QOS_0, rabbit_mqtt_qos0_queue). -include("rabbit_mqtt_types.hrl"). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl index 39ecd0ab1997..860b76ebaf24 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl @@ -89,10 +89,12 @@ local_connection_pids() -> end. global_counters_init() -> - global_counters_init0(?MQTT_PROTO_V3), - global_counters_init0(?MQTT_PROTO_V4). -global_counters_init0(Proto) -> - rabbit_global_counters:init([{protocol, Proto}]), - rabbit_global_counters:init([{protocol, Proto}, {queue_type, ?QOS0_QTYPE}]), - rabbit_global_counters:init([{protocol, Proto}, {queue_type, rabbit_classic_queue}]), - rabbit_global_counters:init([{protocol, Proto}, {queue_type, rabbit_quorum_queue}]). + global_counters_init(?MQTT_PROTO_V3), + global_counters_init(?MQTT_PROTO_V4). + +global_counters_init(ProtoVer) -> + Proto = {protocol, ProtoVer}, + rabbit_global_counters:init([Proto]), + rabbit_global_counters:init([Proto, {queue_type, ?QUEUE_TYPE_QOS_0}]), + rabbit_global_counters:init([Proto, {queue_type, rabbit_classic_queue}]), + rabbit_global_counters:init([Proto, {queue_type, rabbit_quorum_queue}]). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 5b760f07efa0..95ad24bde0dd 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -160,42 +160,46 @@ process_request(?SUBSCRIBE, HasSubsBefore = TopicNamesQos0 =/= [] orelse TopicNamesQos1 =/= [], {QosResponse, PState1} = - lists:foldl(fun(_Topic, {[?SUBACK_FAILURE | _] = L, S}) -> - %% Once a subscription failed, mark all following subscriptions - %% as failed instead of creating bindings because we are going - %% to close the client connection anyways. - {[?SUBACK_FAILURE | L], S}; - (#mqtt_topic{name = TopicName, - qos = Qos0} = Topic, - {L, S0}) -> - QoS = supported_sub_qos(Qos0), - case maybe_replace_old_sub(Topic, TopicNamesQos0, TopicNamesQos1, S0) of - {ok, S1} -> - case ensure_queue(QoS, S1) of - {ok, Q} -> - QName = amqqueue:get_name(Q), - case bind(QName, TopicName, S1) of - {ok, _Output, S2} -> - %%TODO check what happens if we basic.consume multiple times - %% for the same queue - case consume(Q, QoS, S2) of - {ok, S3} -> - {[QoS | L], S3}; - {error, _Reason} -> - {[?SUBACK_FAILURE | L], S2} - end; - {error, Reason, S2} -> - rabbit_log:error("Failed to bind ~s with topic ~s: ~p", - [rabbit_misc:rs(QName), TopicName, Reason]), - {[?SUBACK_FAILURE | L], S2} - end; - {error, _} -> - {[?SUBACK_FAILURE | L], S1} - end; - {error, _Reason, S1} -> - {[?SUBACK_FAILURE | L], S1} - end - end, {[], PState0}, Topics), + lists:foldl( + fun(_Topic, {[?SUBACK_FAILURE | _] = L, S}) -> + %% Once a subscription failed, mark all following subscriptions + %% as failed instead of creating bindings because we are going + %% to close the client connection anyway. + {[?SUBACK_FAILURE | L], S}; + (#mqtt_topic{name = TopicName, + qos = Qos0} = Topic, + {L, S0}) -> + QoS = supported_sub_qos(Qos0), + case maybe_replace_old_sub(Topic, TopicNamesQos0, TopicNamesQos1, S0) of + {ok, S1} -> + case ensure_queue(QoS, S1) of + {ok, Q} -> + QName = amqqueue:get_name(Q), + case bind(QName, TopicName, S1) of + {ok, _Output, S2} -> + case self_consumes(Q) of + false -> + case consume(Q, QoS, S2) of + {ok, S3} -> + {[QoS | L], S3}; + {error, _Reason} -> + {[?SUBACK_FAILURE | L], S2} + end; + true -> + {[QoS | L], S2} + end; + {error, Reason, S2} -> + rabbit_log:error("Failed to bind ~s with topic ~s: ~p", + [rabbit_misc:rs(QName), TopicName, Reason]), + {[?SUBACK_FAILURE | L], S2} + end; + {error, _} -> + {[?SUBACK_FAILURE | L], S1} + end; + {error, _Reason, S1} -> + {[?SUBACK_FAILURE | L], S1} + end + end, {[], PState0}, Topics), maybe_increment_consumer(HasSubsBefore, PState1), SendFun( @@ -445,6 +449,17 @@ return_connack(?CONNACK_ID_REJECTED, S) -> return_connack(?CONNACK_UNACCEPTABLE_PROTO_VER, S) -> {error, unsupported_protocol_version, S}. +-spec self_consumes(amqqueue:amqqueue()) -> boolean(). +self_consumes(Queue) -> + case amqqueue:get_type(Queue) of + ?QUEUE_TYPE_QOS_0 -> + false; + _ -> + lists:any(fun(Consumer) -> + element(1, Consumer) =:= self() + end, rabbit_amqqueue:consumers(Queue)) + end. + start_keepalive(#mqtt_frame_connect{keep_alive = Seconds}, #proc_state{socket = Socket}) -> ok = rabbit_mqtt_keepalive:start(Seconds, Socket). @@ -1025,7 +1040,7 @@ queue_args(_, _) -> end. queue_type(?QOS_0, true, _) -> - ?QOS0_QTYPE; + ?QUEUE_TYPE_QOS_0; queue_type(_, _, QArgs) -> rabbit_amqqueue:get_queue_type(QArgs). @@ -1041,7 +1056,7 @@ consume(Q, QoS, #proc_state{ case check_resource_access(User, QName, read, AuthzCtx) of ok -> case amqqueue:get_type(Q) of - ?QOS0_QTYPE -> + ?QUEUE_TYPE_QOS_0 -> %% Messages get delivered directly to our process without %% explicitly calling rabbit_queue_type:consume/3. {ok, PState0}; @@ -1324,7 +1339,7 @@ maybe_delete_mqtt_qos0_queue(PState = #proc_state{clean_sess = true, {ok, Q} -> %% double check we delete the right queue case {amqqueue:get_type(Q), amqqueue:get_pid(Q)} of - {?QOS0_QTYPE, Pid} + {?QUEUE_TYPE_QOS_0, Pid} when Pid =:= self() -> rabbit_queue_type:delete(Q, false, false, Username); _ -> @@ -1395,7 +1410,7 @@ handle_down({{'DOWN', QName}, _MRef, process, QPid, Reason}, {ok, PState} end. -handle_queue_event({queue_event, ?QOS0_QTYPE, Msg}, PState0) -> +handle_queue_event({queue_event, ?QUEUE_TYPE_QOS_0, Msg}, PState0) -> PState = deliver_one_to_client(Msg, false, PState0), {ok, PState}; handle_queue_event({queue_event, QName, Evt}, @@ -1777,8 +1792,8 @@ maybe_decrement_consumer(_) -> record_message_delivered(QName, true, SubscriberQoS, #proc_state{proto_ver = ProtoVer, queue_states = QStates} = _PState) -> case SubscriberQoS of ?QOS_0 -> - rabbit_global_counters:messages_delivered(ProtoVer, ?QOS0_QTYPE, 1), - rabbit_global_counters:messages_redelivered(ProtoVer, ?QOS0_QTYPE, 1); + rabbit_global_counters:messages_delivered(ProtoVer, ?QUEUE_TYPE_QOS_0, 1), + rabbit_global_counters:messages_redelivered(ProtoVer, ?QUEUE_TYPE_QOS_0, 1); ?QOS_1 -> case rabbit_queue_type:module(QName, QStates) of {ok, QType} -> @@ -1791,7 +1806,7 @@ record_message_delivered(QName, true, SubscriberQoS, #proc_state{proto_ver = Pro record_message_delivered(QName, false, SubscriberQoS, #proc_state{proto_ver = ProtoVer, queue_states = QStates} = _PState) -> case SubscriberQoS of ?QOS_0 -> - rabbit_global_counters:messages_delivered(ProtoVer, ?QOS0_QTYPE, 1); + rabbit_global_counters:messages_delivered(ProtoVer, ?QUEUE_TYPE_QOS_0, 1); ?QOS_1 -> case rabbit_queue_type:module(QName, QStates) of {ok, QType} -> diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/integration_SUITE.erl index bd9ed57b0193..9454dd465436 100644 --- a/deps/rabbitmq_mqtt/test/integration_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/integration_SUITE.erl @@ -668,6 +668,10 @@ subscribe_same_topic_different_qos(Config) -> <<"retained">>, <<"msg2">>, <<"retained">>, <<"msg3">>]), + %% There should be exactly one consumer for each queue: qos0 and qos1 + Consumers = rpc(Config, 0, rabbit_amqqueue, consumers_all, [<<"/">>]), + ?assertEqual(2, length(Consumers)), + ok = emqtt:disconnect(C), {C1, _} = connect(?FUNCTION_NAME, Config, [{clean_start, true}]), ok = emqtt:disconnect(C1). From 16fa12244e71ee957dbb95eca4db4c5fa291e49d Mon Sep 17 00:00:00 2001 From: David Ansari Date: Wed, 23 Nov 2022 16:44:42 +0000 Subject: [PATCH 057/118] Avoid exceptions in mixed version cluster 1. Avoid following exceptions in mixed version clusters when new MQTT connections are created: ``` {{exception,{undef,[{rabbit_mqtt_util,remove_duplicate_clientid_connections, [{<<"/">>, <<"publish_to_all_queue_types">>}, <0.1447.0>], []}]}}, [{erpc,execute_cast,3,[{file,"erpc.erl"},{line,621}]}]} ``` If feature flag delete_ra_cluster_mqtt_node is disabled, let's still populate pg with MQTT client IDs such that we don't have to migrate them from the Ra cluster to pg when we enable the feature flag. However, for actually closing duplicate MQTT client ID connections, if that feature flag is disabled, let's rely on the Ra cluster to take care of it. 2. Write a test ensuring the QoS responses are in the right order when a single SUBSCRIBE packet contains multiple subscriptions. --- deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl | 1 - deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl | 14 ++++++++++---- deps/rabbitmq_mqtt/test/integration_SUITE.erl | 9 +++++++++ 3 files changed, 19 insertions(+), 5 deletions(-) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 95ad24bde0dd..26cd3d6190bb 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -206,7 +206,6 @@ process_request(?SUBSCRIBE, #mqtt_frame{fixed = #mqtt_frame_fixed{type = ?SUBACK}, variable = #mqtt_frame_suback{ message_id = SubscribeMsgId, - %%TODO check correct order of QosResponse qos_table = QosResponse}}, PState1), case QosResponse of diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl index aecabc7d83ef..720a98ca246e 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl @@ -175,10 +175,16 @@ register_clientid(Vhost, ClientId) when is_binary(Vhost), is_binary(ClientId) -> PgGroup = {Vhost, ClientId}, ok = pg:join(persistent_term:get(?PG_SCOPE), PgGroup, self()), - ok = erpc:multicast([node() | nodes()], - ?MODULE, - remove_duplicate_clientid_connections, - [PgGroup, self()]). + case rabbit_mqtt_ff:track_client_id_in_ra() of + true -> + %% Ra node takes care of removing duplicate client ID connections. + ok; + false -> + ok = erpc:multicast([node() | nodes()], + ?MODULE, + remove_duplicate_clientid_connections, + [PgGroup, self()]) + end. -spec remove_duplicate_clientid_connections({rabbit_types:vhost(), binary()}, pid()) -> ok. remove_duplicate_clientid_connections(PgGroup, PidToKeep) -> diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/integration_SUITE.erl index 9454dd465436..d1e8a518c358 100644 --- a/deps/rabbitmq_mqtt/test/integration_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/integration_SUITE.erl @@ -58,6 +58,7 @@ common_tests() -> ,non_clean_sess_disconnect ,subscribe_same_topic_same_qos ,subscribe_same_topic_different_qos + ,subscribe_multiple ]. suite() -> @@ -676,6 +677,14 @@ subscribe_same_topic_different_qos(Config) -> {C1, _} = connect(?FUNCTION_NAME, Config, [{clean_start, true}]), ok = emqtt:disconnect(C1). +subscribe_multiple(Config) -> + {C, _} = connect(?FUNCTION_NAME, Config), + %% Subscribe to multiple topics at once + ?assertMatch({ok, _, [0, 1]}, + emqtt:subscribe(C, [{<<"topic0">>, qos0}, + {<<"topic1">>, qos1}])), + ok = emqtt:disconnect(C). + %% ------------------------------------------------------------------- %% Internal helpers %% ------------------------------------------------------------------- From 075bc066235ba915746d3201f76a9e4e24f988dd Mon Sep 17 00:00:00 2001 From: Chunyi Lyu Date: Thu, 24 Nov 2022 12:08:53 +0000 Subject: [PATCH 058/118] Handle messages_delivered_consume_*_ack counters at delivery - increment messages_delivered_consume_auto_ack if subscribe queue is type mqtt_qos0 queue or if publising QoS is 0 - increment messages_delivered_consume_manual_ack if both publising and subcribe are QoS 1 - increment messages_acknowledged at queue_type:settle() --- .../src/rabbit_mqtt_processor.erl | 51 +++++++++---------- deps/rabbitmq_mqtt/test/integration_SUITE.erl | 2 +- 2 files changed, 25 insertions(+), 28 deletions(-) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 26cd3d6190bb..2aa07891a523 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -77,7 +77,7 @@ process_request(?PUBACK, case rabbit_queue_type:settle(QName, complete, ?CONSUMER_TAG, [QMsgId], QStates0) of {ok, QStates, Actions} -> %%TODO rabbit_channel:incr_queue_stats/3 - record_message_manual_ack(QName, PState), + record_message_ack(QName, PState), {ok, handle_queue_actions(Actions, PState#proc_state{unacked_server_pubs = U, queue_states = QStates})}; {protocol_error, _ErrorType, _Reason, _ReasonArgs} = Err -> @@ -1551,7 +1551,7 @@ maybe_publish_to_client( topic_name = Amqp2MqttFun(RoutingKey)}, payload = Payload}, SendFun(Frame, PState), - record_message_delivered(QName, Redelivered, SubscriberQos, PState), + record_message_delivered(QName, Redelivered, SubscriberQos, QoS, PState), PState. queue_message_id_to_packet_id(_, ?QOS_0, PState) -> @@ -1574,7 +1574,7 @@ maybe_ack(_AckRequired = true, ?QOS_0, QName, QMsgId, case rabbit_queue_type:settle(QName, complete, ?CONSUMER_TAG, [QMsgId], QStates0) of {ok, QStates, Actions} -> %%TODO rabbit_channel:incr_queue_stats/3 - record_message_auto_ack(QName, PState), + record_message_ack(QName, PState), handle_queue_actions(Actions, PState#proc_state{queue_states = QStates}); {protocol_error, _ErrorType, _Reason, _ReasonArgs} = Err -> %%TODO handle error @@ -1788,39 +1788,36 @@ maybe_decrement_consumer(#proc_state{proto_ver = ProtoVer, maybe_decrement_consumer(_) -> ok. -record_message_delivered(QName, true, SubscriberQoS, #proc_state{proto_ver = ProtoVer, queue_states = QStates} = _PState) -> +record_message_delivered(QName, Redelivered, SubscriberQoS, QoS, #proc_state{proto_ver = ProtoVer, queue_states = QStates} = _PState) -> case SubscriberQoS of ?QOS_0 -> - rabbit_global_counters:messages_delivered(ProtoVer, ?QUEUE_TYPE_QOS_0, 1), - rabbit_global_counters:messages_redelivered(ProtoVer, ?QUEUE_TYPE_QOS_0, 1); + message_delivered(Redelivered, ProtoVer, ?QUEUE_TYPE_QOS_0), + rabbit_global_counters:messages_delivered_consume_auto_ack(ProtoVer, ?QUEUE_TYPE_QOS_0, 1); ?QOS_1 -> case rabbit_queue_type:module(QName, QStates) of {ok, QType} -> - rabbit_global_counters:messages_delivered(ProtoVer, QType, 1), - rabbit_global_counters:messages_redelivered(ProtoVer, QType, 1); - _ -> - noop - end - end; -record_message_delivered(QName, false, SubscriberQoS, #proc_state{proto_ver = ProtoVer, queue_states = QStates} = _PState) -> - case SubscriberQoS of - ?QOS_0 -> - rabbit_global_counters:messages_delivered(ProtoVer, ?QUEUE_TYPE_QOS_0, 1); - ?QOS_1 -> - case rabbit_queue_type:module(QName, QStates) of - {ok, QType} -> - rabbit_global_counters:messages_delivered(ProtoVer, QType, 1); + message_delivered(Redelivered, ProtoVer, QType), + message_delivered_auto_or_manual_ack(QoS, ProtoVer, QType); _ -> noop end end. -record_message_manual_ack(QName, #proc_state{proto_ver = ProtoVer, queue_states = QStates} = _PState) -> - {ok, QType} = rabbit_queue_type:module(QName, QStates), - rabbit_global_counters:messages_acknowledged(ProtoVer, QType, 1), +message_delivered(true, ProtoVer, QType) -> + rabbit_global_counters:messages_redelivered(ProtoVer, QType, 1), + rabbit_global_counters:messages_delivered(ProtoVer, QType, 1); +message_delivered(false, ProtoVer, QType) -> + rabbit_global_counters:messages_delivered(ProtoVer, QType, 1). + +message_delivered_auto_or_manual_ack(?QOS_0, ProtoVer, QType) -> + rabbit_global_counters:messages_delivered_consume_auto_ack(ProtoVer, QType, 1); +message_delivered_auto_or_manual_ack(?QOS_1, ProtoVer, QType) -> rabbit_global_counters:messages_delivered_consume_manual_ack(ProtoVer, QType, 1). -record_message_auto_ack(QName, #proc_state{proto_ver = ProtoVer, queue_states = QStates} = _PState) -> - {ok, QType} = rabbit_queue_type:module(QName, QStates), - rabbit_global_counters:messages_acknowledged(ProtoVer, QType, 1), - rabbit_global_counters:messages_delivered_consume_auto_ack(ProtoVer, QType, 1). +record_message_ack(QName, #proc_state{proto_ver = ProtoVer, queue_states = QStates} = _PState) -> + case rabbit_queue_type:module(QName, QStates) of + {ok, QType} -> + rabbit_global_counters:messages_acknowledged(ProtoVer, QType, 1); + _ -> + noop + end. \ No newline at end of file diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/integration_SUITE.erl index d1e8a518c358..edbae7d68da8 100644 --- a/deps/rabbitmq_mqtt/test/integration_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/integration_SUITE.erl @@ -398,7 +398,7 @@ global_counters(Config, ProtoVer) -> get_global_counters(Config, ProtoVer, 0, [{queue_type, rabbit_classic_queue}])), ?assertEqual(#{messages_delivered_total => 1, messages_acknowledged_total => 0, - messages_delivered_consume_auto_ack_total => 0, + messages_delivered_consume_auto_ack_total => 1, messages_delivered_consume_manual_ack_total => 0, messages_delivered_get_auto_ack_total => 0, messages_delivered_get_manual_ack_total => 0, From 65bc0c395bbd6541281e13bc3031380001f81ba6 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Thu, 24 Nov 2022 13:42:45 +0000 Subject: [PATCH 059/118] Fix global counters Prior to this commit messages_delivered for queue_type_qos0 is wrongfully incremented if clean session is false Also, delete duplicate code. --- .../src/rabbit_mqtt_processor.erl | 77 ++++++++++--------- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 1 - deps/rabbitmq_mqtt/test/integration_SUITE.erl | 26 ++++--- 3 files changed, 57 insertions(+), 47 deletions(-) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 2aa07891a523..fd1a9f9e79fb 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -77,7 +77,7 @@ process_request(?PUBACK, case rabbit_queue_type:settle(QName, complete, ?CONSUMER_TAG, [QMsgId], QStates0) of {ok, QStates, Actions} -> %%TODO rabbit_channel:incr_queue_stats/3 - record_message_ack(QName, PState), + message_acknowledged(QName, PState), {ok, handle_queue_actions(Actions, PState#proc_state{unacked_server_pubs = U, queue_states = QStates})}; {protocol_error, _ErrorType, _Reason, _ReasonArgs} = Err -> @@ -1505,7 +1505,7 @@ deliver_one_to_client(Msg = {QName, QPid, QMsgId, _Redelivered, end, QoS = effective_qos(PublisherQoS, SubscriberQoS), PState1 = maybe_publish_to_client(Msg, QoS, SubscriberQoS, PState0), - PState = maybe_ack(AckRequired, QoS, QName, QMsgId, PState1), + PState = maybe_auto_ack(AckRequired, QoS, QName, QMsgId, PState1), %%TODO GC % case GCThreshold of % undefined -> ok; @@ -1530,7 +1530,7 @@ maybe_publish_to_client( routing_keys = [RoutingKey | _CcRoutes], content = #content{payload_fragments_rev = FragmentsRev}}}, QoS, SubscriberQos, PState0 = #proc_state{amqp2mqtt_fun = Amqp2MqttFun, - send_fun = SendFun}) -> + send_fun = SendFun}) -> {PacketId, PState} = queue_message_id_to_packet_id(QMsgId, QoS, PState0), %%TODO support iolists when sending to client Payload = list_to_binary(lists:reverse(FragmentsRev)), @@ -1551,7 +1551,7 @@ maybe_publish_to_client( topic_name = Amqp2MqttFun(RoutingKey)}, payload = Payload}, SendFun(Frame, PState), - record_message_delivered(QName, Redelivered, SubscriberQos, QoS, PState), + message_delivered(QName, Redelivered, SubscriberQos, QoS, PState), PState. queue_message_id_to_packet_id(_, ?QOS_0, PState) -> @@ -1569,18 +1569,17 @@ increment_packet_id(Id) increment_packet_id(Id) -> Id + 1. -maybe_ack(_AckRequired = true, ?QOS_0, QName, QMsgId, - PState = #proc_state{queue_states = QStates0}) -> +maybe_auto_ack(_AckRequired = true, ?QOS_0, QName, QMsgId, + PState = #proc_state{queue_states = QStates0}) -> case rabbit_queue_type:settle(QName, complete, ?CONSUMER_TAG, [QMsgId], QStates0) of {ok, QStates, Actions} -> %%TODO rabbit_channel:incr_queue_stats/3 - record_message_ack(QName, PState), handle_queue_actions(Actions, PState#proc_state{queue_states = QStates}); {protocol_error, _ErrorType, _Reason, _ReasonArgs} = Err -> %%TODO handle error throw(Err) end; -maybe_ack(_, _, _, _, PState) -> +maybe_auto_ack(_, _, _, _, PState) -> PState. maybe_notify_sent(QName, QPid, #proc_state{queue_states = QStates}) -> @@ -1788,36 +1787,44 @@ maybe_decrement_consumer(#proc_state{proto_ver = ProtoVer, maybe_decrement_consumer(_) -> ok. -record_message_delivered(QName, Redelivered, SubscriberQoS, QoS, #proc_state{proto_ver = ProtoVer, queue_states = QStates} = _PState) -> - case SubscriberQoS of - ?QOS_0 -> - message_delivered(Redelivered, ProtoVer, ?QUEUE_TYPE_QOS_0), - rabbit_global_counters:messages_delivered_consume_auto_ack(ProtoVer, ?QUEUE_TYPE_QOS_0, 1); - ?QOS_1 -> - case rabbit_queue_type:module(QName, QStates) of - {ok, QType} -> - message_delivered(Redelivered, ProtoVer, QType), - message_delivered_auto_or_manual_ack(QoS, ProtoVer, QType); - _ -> - noop - end +message_acknowledged(QName, #proc_state{proto_ver = ProtoVer, + queue_states = QStates}) -> + case rabbit_queue_type:module(QName, QStates) of + {ok, QType} -> + rabbit_global_counters:messages_acknowledged(ProtoVer, QType, 1); + _ -> + ok end. -message_delivered(true, ProtoVer, QType) -> - rabbit_global_counters:messages_redelivered(ProtoVer, QType, 1), - rabbit_global_counters:messages_delivered(ProtoVer, QType, 1); -message_delivered(false, ProtoVer, QType) -> - rabbit_global_counters:messages_delivered(ProtoVer, QType, 1). +message_delivered(_, _Redelivered = false, _SubscriberQoS = ?QOS_0, _QoS = ?QOS_0, + #proc_state{clean_sess = true, + proto_ver = ProtoVer + }) -> + rabbit_global_counters:messages_delivered(ProtoVer, ?QUEUE_TYPE_QOS_0, 1), + %% Technically, the message is not acked to a queue at all. + %% However, from a user perspective it is still auto acked because: + %% "In automatic acknowledgement mode, a message is considered to be successfully + %% delivered immediately after it is sent." + rabbit_global_counters:messages_delivered_consume_auto_ack(ProtoVer, ?QUEUE_TYPE_QOS_0, 1); +message_delivered(QName, Redelivered, _, QoS, + #proc_state{proto_ver = ProtoVer, + queue_states = QStates + }) -> + case rabbit_queue_type:module(QName, QStates) of + {ok, QType} -> + rabbit_global_counters:messages_delivered(ProtoVer, QType, 1), + message_delivered_ack(QoS, ProtoVer, QType), + message_redelivered(Redelivered, ProtoVer, QType); + _ -> + ok + end. -message_delivered_auto_or_manual_ack(?QOS_0, ProtoVer, QType) -> +message_delivered_ack(?QOS_0, ProtoVer, QType) -> rabbit_global_counters:messages_delivered_consume_auto_ack(ProtoVer, QType, 1); -message_delivered_auto_or_manual_ack(?QOS_1, ProtoVer, QType) -> +message_delivered_ack(?QOS_1, ProtoVer, QType) -> rabbit_global_counters:messages_delivered_consume_manual_ack(ProtoVer, QType, 1). -record_message_ack(QName, #proc_state{proto_ver = ProtoVer, queue_states = QStates} = _PState) -> - case rabbit_queue_type:module(QName, QStates) of - {ok, QType} -> - rabbit_global_counters:messages_acknowledged(ProtoVer, QType, 1); - _ -> - noop - end. \ No newline at end of file +message_redelivered(true, ProtoVer, QType) -> + rabbit_global_counters:messages_redelivered(ProtoVer, QType, 1); +message_redelivered(_, _, _) -> + ok. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index 5ae1a99d790c..15701949106b 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -15,7 +15,6 @@ code_change/3, terminate/2, format_status/1]). %%TODO check where to best 'hibernate' when returning from callback -%%TODO use rabbit_global_counters for MQTT protocol -export([conserve_resources/3, close_connection/2]). diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/integration_SUITE.erl index edbae7d68da8..3bbeaff96789 100644 --- a/deps/rabbitmq_mqtt/test/integration_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/integration_SUITE.erl @@ -370,25 +370,29 @@ global_counters(Config, ProtoVer) -> {ok, _, [0]} = emqtt:subscribe(C, Topic0, qos0), {ok, _, [1]} = emqtt:subscribe(C, Topic1, qos1), {ok, _, [1]} = emqtt:subscribe(C, Topic2, qos1), + ok = emqtt:publish(C, Topic0, <<"testm0">>, qos0), ok = emqtt:publish(C, Topic1, <<"testm1">>, qos0), {ok, _} = emqtt:publish(C, Topic2, <<"testm2">>, qos1), + ok = emqtt:publish(C, <<"no/queue/bound">>, <<"msg-dropped">>, qos0), + {ok, _} = emqtt:publish(C, <<"no/queue/bound">>, <<"msg-returned">>, qos1), + ok = expect_publishes(Topic0, [<<"testm0">>]), ok = expect_publishes(Topic1, [<<"testm1">>]), ok = expect_publishes(Topic2, [<<"testm2">>]), ?assertEqual(#{publishers => 1, consumers => 1, - messages_confirmed_total => 1, - messages_received_confirm_total => 1, - messages_received_total => 3, + messages_confirmed_total => 2, + messages_received_confirm_total => 2, + messages_received_total => 5, messages_routed_total => 3, - messages_unroutable_dropped_total => 0, - messages_unroutable_returned_total => 0}, + messages_unroutable_dropped_total => 1, + messages_unroutable_returned_total => 1}, get_global_counters(Config, ProtoVer)), ?assertEqual(#{messages_delivered_total => 2, - messages_acknowledged_total => 2, + messages_acknowledged_total => 1, messages_delivered_consume_auto_ack_total => 1, messages_delivered_consume_manual_ack_total => 1, messages_delivered_get_auto_ack_total => 0, @@ -412,12 +416,12 @@ global_counters(Config, ProtoVer) -> ok = emqtt:disconnect(C), ?assertEqual(#{publishers => 0, consumers => 0, - messages_confirmed_total => 1, - messages_received_confirm_total => 1, - messages_received_total => 3, + messages_confirmed_total => 2, + messages_received_confirm_total => 2, + messages_received_total => 5, messages_routed_total => 3, - messages_unroutable_dropped_total => 0, - messages_unroutable_returned_total => 0}, + messages_unroutable_dropped_total => 1, + messages_unroutable_returned_total => 1}, get_global_counters(Config, ProtoVer)). queue_down_qos1(Config) -> From 14b3b93b25871d8dc2e40bbd6e84fa4a2f564909 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Thu, 24 Nov 2022 15:02:27 +0000 Subject: [PATCH 060/118] Make ff_SUITE less flaky --- deps/rabbitmq_mqtt/test/ff_SUITE.erl | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/deps/rabbitmq_mqtt/test/ff_SUITE.erl b/deps/rabbitmq_mqtt/test/ff_SUITE.erl index a101a4f548d8..8576a1c6f828 100644 --- a/deps/rabbitmq_mqtt/test/ff_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/ff_SUITE.erl @@ -76,8 +76,9 @@ enable_feature_flag(Config) -> ?assertEqual(ok, rabbit_ct_broker_helpers:enable_feature_flag(Config, ?FEATURE_FLAG)), %% Ra processes should be gone - ?assert(lists:all(fun(Pid) -> Pid =:= undefined end, - rabbit_ct_broker_helpers:rpc_all(Config, erlang, whereis, [mqtt_node]))), + rabbit_ct_helpers:eventually( + ?_assert(lists:all(fun(Pid) -> Pid =:= undefined end, + rabbit_ct_broker_helpers:rpc_all(Config, erlang, whereis, [mqtt_node])))), %% new client ID tracking works ?assertEqual(1, length(util:all_connection_pids(Config))), ?assert(erlang:is_process_alive(C)), From 76f4598d92831d5f1a13eeb3992b989e49727643 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Fri, 25 Nov 2022 11:39:43 +0000 Subject: [PATCH 061/118] Send last will if client did not DISCONNECT MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit "The Will Message MUST be published when the Network Connection is subsequently closed unless the Will Message has been deleted by the Server on receipt of a DISCONNECT Packet [MQTT-3.1.2-8]. Situations in which the Will Message is published include, but are not limited to: • An I/O error or network failure detected by the Server. • The Client fails to communicate within the Keep Alive time. • The Client closes the Network Connection without first sending a DISCONNECT Packet. • The Server closes the Network Connection because of a protocol error. " Prior to this commit, the will message was not sent in all scenarios where it should have been sent. In this commit, the will message is always sent unless the client sent a DISCONNECT packet to the server. We achieve this by sending the will message in the terminate callback. Note that the Reason passed into the terminate callback of rabbit_web_mqtt_handler is the atom 'stop' (that is, we cannot pass a custom reason here). Therefore, in order to know within the terminate callback whether the client sent a DISCONNECT packet, we have to modify the process state. Rather than including a new field into the process state record which requires 1 additional word per MQTT connection (i.e. expensive with millions of MQTT connection processes - we want to keep the process state small), we intead modify the state just before stopping the process to {SendWill, State}. --- .../src/rabbit_mqtt_processor.erl | 54 ++++++++++-------- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 47 ++++++--------- deps/rabbitmq_mqtt/test/reader_SUITE.erl | 57 ++++++++++++++----- .../src/rabbit_web_mqtt_handler.erl | 20 +++---- deps/rabbitmq_web_mqtt/test/system_SUITE.erl | 32 +++++++++-- 5 files changed, 123 insertions(+), 87 deletions(-) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index fd1a9f9e79fb..6095eee920af 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -8,8 +8,8 @@ -module(rabbit_mqtt_processor). -export([info/2, initial_state/2, initial_state/4, - process_frame/2, serialise/2, send_will/1, - terminate/2, handle_pre_hibernate/0, + process_frame/2, serialise/2, + terminate/3, handle_pre_hibernate/0, handle_ra_event/2, handle_down/2, handle_queue_event/2, soft_limit_exceeded/1, format_status/1]). @@ -259,7 +259,7 @@ process_request(?PINGREQ, #mqtt_frame{}, PState = #proc_state{send_fun = SendFun process_request(?DISCONNECT, #mqtt_frame{}, PState) -> rabbit_log_connection:debug("Received a DISCONNECT"), - {stop, PState}. + {stop, disconnect, PState}. process_connect(#mqtt_frame{ variable = #mqtt_frame_connect{ @@ -368,7 +368,7 @@ register_client(Frame = #mqtt_frame_connect{proto_ver = ProtoVersion}, ProtoHumanReadable = {'MQTT', human_readable_mqtt_version(ProtoVersion)}, PState#proc_state{ exchange = ExchangeName, - will_msg = make_will_msg(Frame), + will_msg = make_will_msg(Frame), retainer_pid = RetainerPid, register_state = RegisterState, proto_ver = protocol_integer_to_atom(ProtoVersion), @@ -1142,25 +1142,6 @@ binding_action( key = RoutingKey}, BindingFun(Binding, Username). -send_will(#proc_state{will_msg = undefined}) -> - ok; -send_will(PState = #proc_state{will_msg = WillMsg = #mqtt_msg{retain = Retain, - topic = Topic}, - retainer_pid = RPid, - amqp2mqtt_fun = Amqp2MqttFun}) -> - case check_topic_access(Topic, write, PState) of - ok -> - publish_to_queues(WillMsg, PState), - case Retain of - false -> - ok; - true -> - hand_off_to_retainer(RPid, Amqp2MqttFun, Topic, WillMsg) - end; - {error, access_refused = Reason} -> - rabbit_log:error("failed to send will message: ~p", [Reason]) - end. - publish_to_queues(undefined, PState) -> {ok, PState}; publish_to_queues( @@ -1297,7 +1278,8 @@ serialise_and_send_to_client(Frame, #proc_state{proto_ver = ProtoVer, socket = S serialise(Frame, #proc_state{proto_ver = ProtoVer}) -> rabbit_mqtt_frame:serialise(Frame, ProtoVer). -terminate(PState, ConnName) -> +terminate(SendWill, ConnName, PState) -> + maybe_send_will(SendWill, ConnName, PState), Infos = [{name, ConnName}, {node, node()}, {pid, self()}, @@ -1310,6 +1292,28 @@ terminate(PState, ConnName) -> maybe_decrement_publisher(PState), maybe_delete_mqtt_qos0_queue(PState). +maybe_send_will(true, ConnStr, + #proc_state{will_msg = WillMsg = #mqtt_msg{retain = Retain, + topic = Topic}, + retainer_pid = RPid, + amqp2mqtt_fun = Amqp2MqttFun} = PState) -> + rabbit_log_connection:debug("sending MQTT will message to topic ~s on connection ~s", + [Topic, ConnStr]), + case check_topic_access(Topic, write, PState) of + ok -> + publish_to_queues(WillMsg, PState), + case Retain of + false -> + ok; + true -> + hand_off_to_retainer(RPid, Amqp2MqttFun, Topic, WillMsg) + end; + {error, access_refused = Reason} -> + rabbit_log:error("failed to send will message: ~p", [Reason]) + end; +maybe_send_will(_, _, _) -> + ok. + additional_connection_closed_info( #proc_state{info = #info{proto_human = {ProtoName, ProtoVsn}}, auth_state = #auth_state{vhost = VHost, @@ -1777,7 +1781,7 @@ maybe_decrement_consumer(_, _) -> ok. maybe_decrement_consumer(#proc_state{proto_ver = ProtoVer, - auth_state = #auth_state{vhost = _Vhost}} = PState) -> + auth_state = #auth_state{}} = PState) -> case has_subs(PState) of true -> rabbit_global_counters:consumer_deleted(ProtoVer); diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index 15701949106b..338b70e1a01f 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -14,8 +14,6 @@ -export([init/1, handle_call/3, handle_cast/2, handle_info/2, code_change/3, terminate/2, format_status/1]). -%%TODO check where to best 'hibernate' when returning from callback - -export([conserve_resources/3, close_connection/2]). @@ -87,11 +85,10 @@ init(Ref) -> handle_call({info, InfoItems}, _From, State) -> Infos = lists:map( - fun(InfoItem) -> - {InfoItem, info_internal(InfoItem, State)} - end, - InfoItems), - {reply, Infos, State}; + fun(InfoItem) -> + {InfoItem, info_internal(InfoItem, State)} + end, InfoItems), + {reply, Infos, State, ?HIBERNATE_AFTER}; handle_call(Msg, From, State) -> {stop, {mqtt_unexpected_call, Msg, From}, State}. @@ -176,7 +173,7 @@ handle_info({keepalive, Req}, State = #state{keepalive = KState0, {noreply, State#state{keepalive = KState}, ?HIBERNATE_AFTER}; {error, timeout} -> rabbit_log_connection:error("closing MQTT connection ~p (keepalive timeout)", [ConnName]), - send_will_and_terminate({shutdown, keepalive_timeout}, State); + {stop, {shutdown, keepalive_timeout}, State}; {error, Reason} -> {stop, Reason, State} end; @@ -208,7 +205,7 @@ handle_info({{'DOWN', _QName}, _MRef, process, _Pid, _Reason} = Evt, {ok, PState} -> maybe_process_deferred_recv(control_throttle(pstate(State, PState))); {error, Reason} -> - send_will_and_terminate({shutdown, Reason}, State) + {stop, {shutdown, Reason, State}} end; handle_info({'DOWN', _MRef, process, QPid, _Reason}, State) -> @@ -218,12 +215,14 @@ handle_info({'DOWN', _MRef, process, QPid, _Reason}, State) -> handle_info(Msg, State) -> {stop, {mqtt_unexpected_msg, Msg}, State}. -terminate(Reason, State = #state{conn_name = ConnName, - keepalive = KState0, - proc_state = PState}) -> +terminate(Reason, State = #state{}) -> + terminate(Reason, {true, State}); +terminate(Reason, {SendWill, State = #state{conn_name = ConnName, + keepalive = KState0, + proc_state = PState}}) -> KState = rabbit_mqtt_keepalive:cancel_timer(KState0), maybe_emit_stats(State#state{keepalive = KState}), - rabbit_mqtt_processor:terminate(PState, ConnName), + rabbit_mqtt_processor:terminate(SendWill, ConnName, PState), log_terminate(Reason, State). log_terminate({network_error, {ssl_upgrade_error, closed}, ConnStr}, _State) -> @@ -332,8 +331,8 @@ process_received_bytes(Bytes, rabbit_log_connection:error("MQTT detected a framing error on connection ~ts: ~tp", [ConnStr, Error]), {stop, {shutdown, Error}, State}; - {stop, ProcState1} -> - {stop, normal, pstate(State, ProcState1)} + {stop, disconnect, ProcState1} -> + {stop, normal, {_SendWill = false, pstate(State, ProcState1)}} end; {error, {cannot_parse, Error, Stacktrace}} -> rabbit_log_connection:error("MQTT cannot parse a frame on connection '~ts', unparseable payload: ~tp, error: {~tp, ~tp} ", @@ -357,20 +356,6 @@ parse(Bytes, ParseState) -> {error, {cannot_parse, Reason, Stacktrace}} end. -%% TODO Send will message in all abnormal shutdowns? -%% => in terminate/2 depending on Reason -%% "The Will Message MUST be published when the Network Connection is subsequently -%% closed unless the Will Message has been deleted by the Server on receipt of a -%% DISCONNECT Packet [MQTT-3.1.2-8]." -send_will_and_terminate(State) -> - send_will_and_terminate({shutdown, conn_closed}, State). - -send_will_and_terminate(Reason, State = #state{conn_name = ConnStr, - proc_state = PState}) -> - rabbit_log_connection:debug("MQTT: about to send will message (if any) on connection ~p", [ConnStr]), - rabbit_mqtt_processor:send_will(PState), - {stop, Reason, State}. - network_error(closed, State = #state{conn_name = ConnStr, received_connect_frame = Connected}) -> @@ -380,13 +365,13 @@ network_error(closed, true -> rabbit_log_connection:info(Fmt, Args); false -> rabbit_log_connection:debug(Fmt, Args) end, - send_will_and_terminate(State); + {stop, {shutdown, conn_closed}, State}; network_error(Reason, State = #state{conn_name = ConnStr}) -> rabbit_log_connection:info("MQTT detected network error for ~p: ~p", [ConnStr, Reason]), - send_will_and_terminate(State). + {stop, {shutdown, conn_closed}, State}. run_socket(State = #state{ connection_state = blocked }) -> State; diff --git a/deps/rabbitmq_mqtt/test/reader_SUITE.erl b/deps/rabbitmq_mqtt/test/reader_SUITE.erl index 579373d644ae..fab95059d95a 100644 --- a/deps/rabbitmq_mqtt/test/reader_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/reader_SUITE.erl @@ -25,20 +25,22 @@ all() -> groups() -> [ - {non_parallel_tests, [], [ - block, - block_connack_timeout, - handle_invalid_frames, - login_timeout, - keepalive, - keepalive_turned_off, - stats, - clean_session_disconnect_client, - clean_session_kill_node, - quorum_clean_session_false, - quorum_clean_session_true, - classic_clean_session_true, - classic_clean_session_false + {non_parallel_tests, [], + [ + block, + block_connack_timeout, + handle_invalid_frames, + login_timeout, + keepalive, + keepalive_turned_off, + stats, + will, + clean_session_disconnect_client, + clean_session_kill_node, + quorum_clean_session_false, + quorum_clean_session_true, + classic_clean_session_true, + classic_clean_session_false ]} ]. @@ -369,5 +371,32 @@ classic_clean_session_true(Config) -> classic_clean_session_false(Config) -> validate_durable_queue_type(Config, <<"classicCleanSessionFalse">>, false, rabbit_classic_queue). +will(Config) -> + P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), + Topic = <<"will/topic">>, + Msg = <<"will msg">>, + {ok, Publisher} = emqtt:start_link([{port, P}, + {clientid, <<"will-publisher">>}, + {proto_ver, v4}, + {will_topic, Topic}, + {will_payload, Msg}, + {will_qos, qos0}, + {will_retain, false} + ]), + {ok, _} = emqtt:connect(Publisher), + [ServerPublisherPid] = all_connection_pids(Config), + + {ok, Subscriber} = emqtt:start_link([{port, P}, + {clientid, <<"will-subscriber">>}, + {proto_ver, v4}]), + {ok, _} = emqtt:connect(Subscriber), + {ok, _, _} = emqtt:subscribe(Subscriber, Topic, qos0), + + true = unlink(Publisher), + erlang:exit(ServerPublisherPid, test_will), + ok = expect_publishes(Topic, [Msg]), + + ok = emqtt:disconnect(Subscriber). + rpc(Config, M, F, A) -> rabbit_ct_broker_helpers:rpc(Config, 0, M, F, A). diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index fa5d8337fcf3..015a53dc1d93 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -210,15 +210,17 @@ websocket_info(Msg, State) -> {[], State, hibernate}. -spec terminate(any(), cowboy_req:req(), any()) -> ok. +terminate(Reason, Request, #state{} = State) -> + terminate(Reason, Request, {true, State}); terminate(_Reason, _Request, - #state{conn_name = ConnName, - proc_state = PState, - keepalive = KState} = State) -> + {SendWill, #state{conn_name = ConnName, + proc_state = PState, + keepalive = KState} = State}) -> rabbit_log_connection:info("closing Web MQTT connection ~p (~s)", [self(), ConnName]), maybe_emit_stats(State), rabbit_mqtt_keepalive:cancel_timer(KState), ok = file_handle_cache:release(), - stop_rabbit_mqtt_processor(PState, ConnName). + rabbit_mqtt_processor:terminate(SendWill, ConnName, PState). %% Internal. @@ -262,8 +264,8 @@ handle_data1(Data, State = #state{ parse_state = ParseState, stop(State, ?CLOSE_PROTOCOL_ERROR, Reason); {error, Error} -> stop_with_framing_error(State, Error, ConnStr); - {stop, _} -> - stop(State) + {stop, disconnect, ProcState1} -> + stop({_SendWill = false, State#state{proc_state = ProcState1}}) end; Other -> Other @@ -282,12 +284,6 @@ stop(State, CloseCode, Error0) -> Error = rabbit_data_coercion:to_binary(Error0), {[{close, CloseCode, Error}], State}. -stop_rabbit_mqtt_processor(undefined, _) -> - ok; -stop_rabbit_mqtt_processor(PState, ConnName) -> - rabbit_mqtt_processor:send_will(PState), - rabbit_mqtt_processor:terminate(PState, ConnName). - handle_credits(State0) -> %%TODO return hibernate? case control_throttle(State0) of diff --git a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl index 8705945cc3ee..896521be6294 100644 --- a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl +++ b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl @@ -25,7 +25,8 @@ groups() -> [connection , pubsub_shared_connection , pubsub_separate_connections - , last_will_enabled + , last_will_enabled_disconnect + , last_will_enabled_no_disconnect , disconnect , keepalive , maintenance @@ -94,7 +95,7 @@ pubsub_separate_connections(Config) -> ok = emqtt:disconnect(Publisher), ok = emqtt:disconnect(Consumer). -last_will_enabled(Config) -> +last_will_enabled_disconnect(Config) -> LastWillTopic = <<"/topic/web-mqtt-tests-ws1-last-will">>, LastWillMsg = <<"a last will and testament message">>, PubOpts = [{will_topic, LastWillTopic}, @@ -103,8 +104,29 @@ last_will_enabled(Config) -> Publisher = ws_connect(<<(atom_to_binary(?FUNCTION_NAME))/binary, "_publisher">>, Config, PubOpts), Consumer = ws_connect(<<(atom_to_binary(?FUNCTION_NAME))/binary, "_consumer">>, Config), {ok, _, [1]} = emqtt:subscribe(Consumer, LastWillTopic, qos1), + + %% Client sends DISCONNECT packet. Therefore, will message should not be sent. ok = emqtt:disconnect(Publisher), - ok = expect_publishes(Consumer, LastWillTopic, [LastWillMsg]), + ?assertEqual({publish_not_received, LastWillMsg}, + expect_publishes(Consumer, LastWillTopic, [LastWillMsg])), + + ok = emqtt:disconnect(Consumer). + +last_will_enabled_no_disconnect(Config) -> + LastWillTopic = <<"/topic/web-mqtt-tests-ws1-last-will">>, + LastWillMsg = <<"a last will and testament message">>, + PubOpts = [{will_topic, LastWillTopic}, + {will_payload, LastWillMsg}, + {will_qos, 1}], + _Publisher = ws_connect(<<(atom_to_binary(?FUNCTION_NAME))/binary, "_publisher">>, Config, PubOpts), + [ServerPublisherPid] = rpc(Config, 0, rabbit_mqtt, local_connection_pids, []), + Consumer = ws_connect(<<(atom_to_binary(?FUNCTION_NAME))/binary, "_consumer">>, Config), + {ok, _, [1]} = emqtt:subscribe(Consumer, LastWillTopic, qos1), + + %% Client does not send DISCONNECT packet. Therefore, will message should be sent. + erlang:exit(ServerPublisherPid, test_will), + ?assertEqual(ok, expect_publishes(Consumer, LastWillTopic, [LastWillMsg])), + ok = emqtt:disconnect(Consumer). disconnect(Config) -> @@ -183,6 +205,6 @@ expect_publishes(ClientPid, Topic, [Payload|Rest]) -> topic := Topic, payload := Payload}} -> expect_publishes(ClientPid, Topic, Rest) - after 5000 -> - throw({publish_not_received, Payload}) + after 1000 -> + {publish_not_received, Payload} end. From 6815ceb54ad8a3edb39ece4fc50776c66aa7d155 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Fri, 25 Nov 2022 12:09:58 +0000 Subject: [PATCH 062/118] Fix mixed version reader_SUITE will test --- deps/rabbitmq_mqtt/test/reader_SUITE.erl | 1 + 1 file changed, 1 insertion(+) diff --git a/deps/rabbitmq_mqtt/test/reader_SUITE.erl b/deps/rabbitmq_mqtt/test/reader_SUITE.erl index fab95059d95a..d8c01c31a256 100644 --- a/deps/rabbitmq_mqtt/test/reader_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/reader_SUITE.erl @@ -384,6 +384,7 @@ will(Config) -> {will_retain, false} ]), {ok, _} = emqtt:connect(Publisher), + timer:sleep(100), [ServerPublisherPid] = all_connection_pids(Config), {ok, Subscriber} = emqtt:start_link([{port, P}, From 7bc8208a1b6c35d94a5162cd6c43ffbf7baf6efc Mon Sep 17 00:00:00 2001 From: David Ansari Date: Mon, 28 Nov 2022 11:45:06 +0000 Subject: [PATCH 063/118] Remove local record definitions from header files Record #state{} is purely local to rabbit_mqtt_reader. Record #proc_state{} is purely local to rabbit_mqtt_processor. Therefore, move these record definitions to the defining module. This avoids unnecessarily exposing internal information. Now, that #proc_state{} is defined in rabbit_mqtt_processor, rename #proc_state to #state{}. --- deps/rabbitmq_mqtt/BUILD.bazel | 2 +- deps/rabbitmq_mqtt/include/mqtt_machine.hrl | 7 - deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl | 78 +- .../include/rabbit_mqtt_frame.hrl | 19 +- ...d_msg_store.hrl => rabbit_mqtt_retain.hrl} | 5 +- .../include/rabbit_mqtt_types.hrl | 9 - deps/rabbitmq_mqtt/src/rabbit_mqtt.erl | 5 +- .../src/rabbit_mqtt_confirms.erl | 2 +- deps/rabbitmq_mqtt/src/rabbit_mqtt_ff.erl | 2 +- deps/rabbitmq_mqtt/src/rabbit_mqtt_frame.erl | 1 + .../src/rabbit_mqtt_processor.erl | 907 +++++++++--------- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 18 +- .../src/rabbit_mqtt_retained_msg_store.erl | 15 +- .../rabbit_mqtt_retained_msg_store_dets.erl | 2 +- .../rabbit_mqtt_retained_msg_store_ets.erl | 2 +- .../src/rabbit_mqtt_retainer.erl | 2 +- deps/rabbitmq_mqtt/test/util.erl | 2 +- 17 files changed, 533 insertions(+), 545 deletions(-) rename deps/rabbitmq_mqtt/include/{rabbit_mqtt_retained_msg_store.hrl => rabbit_mqtt_retain.hrl} (66%) delete mode 100644 deps/rabbitmq_mqtt/include/rabbit_mqtt_types.hrl diff --git a/deps/rabbitmq_mqtt/BUILD.bazel b/deps/rabbitmq_mqtt/BUILD.bazel index 02cc99940e75..77183b212b74 100644 --- a/deps/rabbitmq_mqtt/BUILD.bazel +++ b/deps/rabbitmq_mqtt/BUILD.bazel @@ -100,7 +100,7 @@ rabbitmq_test_helper( "//deps/rabbit_common:erlang_app", ], hdrs = [ - "include/rabbit_mqtt_frame.hrl" + "include/rabbit_mqtt.hrl" ], ) diff --git a/deps/rabbitmq_mqtt/include/mqtt_machine.hrl b/deps/rabbitmq_mqtt/include/mqtt_machine.hrl index bddc2e53c7f6..61fc8f759359 100644 --- a/deps/rabbitmq_mqtt/include/mqtt_machine.hrl +++ b/deps/rabbitmq_mqtt/include/mqtt_machine.hrl @@ -5,13 +5,6 @@ %% Copyright (c) 2020-2023 VMware, Inc. or its affiliates. All rights reserved. %% -%%TODO decrease per connection memory overhead -%% since the Raft process memory can grow a few GBs with -%% millions of connections. -%% 1. Use binaries instead of string()s for the ConnectionId -%% 2. Use new Erlang 24 function erlang:monitor/3 with tag being the ConnectionId -%% so that we can get rid of pids fields because we won't lookup the ConnectionId -%% by PID anymore. -record(machine_state, { %% client ID to connection PID client_ids = #{}, diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl index 95cfa0df5503..4bfab130b743 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl @@ -5,82 +5,14 @@ %% Copyright (c) 2020-2023 VMware, Inc. or its affiliates. All rights reserved. %% --define(CLIENT_ID_MAXLEN, 23). -define(PG_SCOPE, pg_scope_rabbitmq_mqtt_clientid). --define(QUEUE_TYPE_QOS_0, rabbit_mqtt_qos0_queue). - --include("rabbit_mqtt_types.hrl"). - -%% reader state --record(state, - {socket, - proxy_socket, - conn_name, - await_recv, - deferred_recv, - received_connect_frame, - connection_state, - conserve, - parse_state, - proc_state, - stats_timer, - keepalive :: rabbit_mqtt_keepalive:state()}). -%% processor state --record(proc_state, - {socket, - proto_ver :: mqtt310 | mqtt311, - queue_states = rabbit_queue_type:init() :: rabbit_queue_type:state(), - %% Packet IDs published to queues but not yet confirmed. - unacked_client_pubs = rabbit_mqtt_confirms:init() :: rabbit_mqtt_confirms:state(), - %% Packet IDs published to MQTT subscribers but not yet acknowledged. - unacked_server_pubs = #{} :: #{packet_id() => QueueMsgId :: non_neg_integer()}, - %% Packet ID of next PUBLISH packet (with QoS > 0) sent from server to client. - %% (Not to be confused with packet IDs sent from client to server which can be the - %% same IDs because client and server assign IDs independently of each other.) - packet_id = 1 :: packet_id(), - client_id :: binary(), - clean_sess :: boolean(), - will_msg, - exchange :: rabbit_exchange:name(), - %% Set if client has at least one subscription with QoS 1. - queue_qos1 :: rabbit_amqqueue:name(), - has_published = false :: boolean(), - ssl_login_name, - %% Retained messages handler. See rabbit_mqtt_retainer_sup - %% and rabbit_mqtt_retainer. - retainer_pid, - auth_state, - peer_addr, - send_fun :: fun((Frame :: tuple(), proc_state()) -> term()), - %%TODO remove funs from state? - mqtt2amqp_fun, - amqp2mqtt_fun, - register_state, - conn_name, - info, - delivery_flow :: flow | noflow, - %% quorum queues and streams whose soft limit has been exceeded - soft_limit_exceeded = sets:new([{version, 2}]) :: sets:set() - }). - --type proc_state() :: #proc_state{}. - --record(auth_state, {username, - user, - vhost, - authz_ctx}). +-define(QUEUE_TYPE_QOS_0, rabbit_mqtt_qos0_queue). --record(info, {prefetch, - host, - port, - peer_host, - peer_port, - proto_human}). +-define(MQTT_GUIDE_URL, <<"https://rabbitmq.com/mqtt.html">>). -%% does not include vhost because vhost is used in the (D)ETS table name --record(retained_message, {topic, - mqtt_msg}). +-define(MQTT_PROTO_V3, mqtt310). +-define(MQTT_PROTO_V4, mqtt311). -define(INFO_ITEMS, [protocol, @@ -113,5 +45,3 @@ messages_unconfirmed, messages_unacknowledged ]). - --define(MQTT_GUIDE_URL, <<"https://rabbitmq.com/mqtt.html">>). diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt_frame.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt_frame.hrl index 5fe78c26e7ff..d5b0c6849289 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt_frame.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt_frame.hrl @@ -5,9 +5,9 @@ %% Copyright (c) 2020-2023 VMware, Inc. or its affiliates. All rights reserved. %% --define(PROTOCOL_NAMES, [{3, "MQIsdp"}, {4, "MQTT"}]). --define(MQTT_PROTO_V3, mqtt310). --define(MQTT_PROTO_V4, mqtt311). +-define(PROTOCOL_NAMES, + [{3, "MQIsdp"}, + {4, "MQTT"}]). %% frame types @@ -49,8 +49,9 @@ -define(SUBACK_FAILURE, 16#80). -type qos() :: ?QOS_0 | ?QOS_1 | ?QOS_2. -%%TODO remove message_id() --type message_id() :: any(). + +%% Packet identifier is a non zero two byte integer. +-type packet_id() :: 1..16#ffff. -record(mqtt_frame, {fixed, variable, @@ -77,16 +78,16 @@ return_code}). -record(mqtt_frame_publish, {topic_name, - message_id}). + message_id :: packet_id()}). -record(mqtt_topic, {name, qos}). --record(mqtt_frame_subscribe,{message_id, +-record(mqtt_frame_subscribe,{message_id :: packet_id(), topic_table :: nonempty_list(#mqtt_topic{}) }). --record(mqtt_frame_suback, {message_id, +-record(mqtt_frame_suback, {message_id :: packet_id(), qos_table = []}). -record(mqtt_frame_other, {other}). @@ -95,7 +96,7 @@ qos :: qos(), topic :: string(), dup :: boolean(), - message_id :: message_id(), + message_id :: packet_id(), payload :: binary()}). -type mqtt_msg() :: #mqtt_msg{}. diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt_retained_msg_store.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt_retain.hrl similarity index 66% rename from deps/rabbitmq_mqtt/include/rabbit_mqtt_retained_msg_store.hrl rename to deps/rabbitmq_mqtt/include/rabbit_mqtt_retain.hrl index 278336b3b73c..3cd38eb69a8a 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt_retained_msg_store.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt_retain.hrl @@ -3,4 +3,7 @@ %% file, You can obtain one at https://mozilla.org/MPL/2.0/. %% %% Copyright (c) 2020-2023 VMware, Inc. or its affiliates. All rights reserved. -%% + +%% does not include vhost because vhost is used in the (D)ETS table name +-record(retained_message, {topic, + mqtt_msg}). diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt_types.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt_types.hrl deleted file mode 100644 index 3233d7f69553..000000000000 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt_types.hrl +++ /dev/null @@ -1,9 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2020-2022 VMware, Inc. or its affiliates. All rights reserved. -%% - -%% Packet identifier is a non zero two byte integer. --type packet_id() :: 1..16#ffff. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl index 860b76ebaf24..84e76bb1f9be 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl @@ -7,11 +7,10 @@ -module(rabbit_mqtt). --include("rabbit_mqtt.hrl"). --include("rabbit_mqtt_frame.hrl"). - -behaviour(application). +-include("rabbit_mqtt.hrl"). + -export([start/2, stop/1]). -export([emit_connection_info_all/4, emit_connection_info_local/3, diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl index 0cfd899a8a4b..0f3b89908131 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl @@ -7,7 +7,7 @@ -module(rabbit_mqtt_confirms). --include("rabbit_mqtt_types.hrl"). +-include("rabbit_mqtt_frame.hrl"). -compile({no_auto_import, [size/1]}). -export([init/0, diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_ff.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_ff.erl index f8a10782fe82..56623b6c39ae 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_ff.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_ff.erl @@ -11,7 +11,7 @@ -rabbit_feature_flag( {delete_ra_cluster_mqtt_node, - #{desc => "Delete Ra cluster 'mqtt_node' because from now on MQTT client IDs are tracked locally", + #{desc => "Delete Ra cluster 'mqtt_node' since MQTT client IDs are tracked locally", stability => stable, callbacks => #{enable => {mqtt_node, delete}} }}). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_frame.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_frame.erl index fa8724afdb76..e1a0c3e4f2fa 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_frame.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_frame.erl @@ -11,6 +11,7 @@ -export([serialise/2]). -include("rabbit_mqtt_frame.hrl"). +-include("rabbit_mqtt.hrl"). -define(RESERVED, 0). -define(MAX_LEN, 16#fffffff). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 6095eee920af..1197c70257bb 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -16,6 +16,8 @@ %% for testing purposes -export([get_vhost_username/1, get_vhost/3, get_vhost_from_user_mapping/2]). +-export_type([state/0]). + -include_lib("rabbit_common/include/rabbit.hrl"). -include_lib("rabbit_common/include/rabbit_framing.hrl"). -include_lib("rabbit/include/amqqueue.hrl"). @@ -26,6 +28,57 @@ -define(MAX_PERMISSION_CACHE_SIZE, 12). -define(CONSUMER_TAG, mqtt). +-record(auth_state, {username, + user, + vhost, + authz_ctx}). + +-record(info, {prefetch, + host, + port, + peer_host, + peer_port, + proto_human}). + +-record(state, + {socket, + proto_ver :: mqtt310 | mqtt311, + queue_states = rabbit_queue_type:init() :: rabbit_queue_type:state(), + %% Packet IDs published to queues but not yet confirmed. + unacked_client_pubs = rabbit_mqtt_confirms:init() :: rabbit_mqtt_confirms:state(), + %% Packet IDs published to MQTT subscribers but not yet acknowledged. + unacked_server_pubs = #{} :: #{packet_id() => QueueMsgId :: non_neg_integer()}, + %% Packet ID of next PUBLISH packet (with QoS > 0) sent from server to client. + %% (Not to be confused with packet IDs sent from client to server which can be the + %% same IDs because client and server assign IDs independently of each other.) + packet_id = 1 :: packet_id(), + client_id :: binary(), + clean_sess :: boolean(), + will_msg, + exchange :: rabbit_exchange:name(), + %% Set if client has at least one subscription with QoS 1. + queue_qos1 :: rabbit_amqqueue:name(), + has_published = false :: boolean(), + ssl_login_name, + %% Retained messages handler. See rabbit_mqtt_retainer_sup + %% and rabbit_mqtt_retainer. + retainer_pid, + auth_state, + peer_addr, + send_fun :: fun((Frame :: tuple(), state()) -> term()), + %%TODO remove funs from state? + mqtt2amqp_fun, + amqp2mqtt_fun, + register_state, + conn_name, + info, + delivery_flow :: flow | noflow, + %% quorum queues and streams whose soft limit has been exceeded + soft_limit_exceeded = sets:new([{version, 2}]) :: sets:set() + }). + +-opaque state() :: #state{}. + initial_state(Socket, ConnectionName) -> {ok, {PeerAddr, _PeerPort}} = rabbit_net:peername(Socket), initial_state(Socket, @@ -36,66 +89,66 @@ initial_state(Socket, ConnectionName) -> initial_state(Socket, ConnectionName, SendFun, PeerAddr) -> {ok, {mqtt2amqp_fun, M2A}, {amqp2mqtt_fun, A2M}} = rabbit_mqtt_util:get_topic_translation_funs(), Flow = case rabbit_misc:get_env(rabbit, mirroring_flow_control, true) of - true -> flow; - false -> noflow + true -> flow; + false -> noflow end, - #proc_state{socket = Socket, - conn_name = ConnectionName, - ssl_login_name = ssl_login_name(Socket), - peer_addr = PeerAddr, - send_fun = SendFun, - mqtt2amqp_fun = M2A, - amqp2mqtt_fun = A2M, - delivery_flow = Flow}. + #state{socket = Socket, + conn_name = ConnectionName, + ssl_login_name = ssl_login_name(Socket), + peer_addr = PeerAddr, + send_fun = SendFun, + mqtt2amqp_fun = M2A, + amqp2mqtt_fun = A2M, + delivery_flow = Flow}. process_frame(#mqtt_frame{fixed = #mqtt_frame_fixed{type = Type}}, - PState = #proc_state{auth_state = undefined}) + State = #state{auth_state = undefined}) when Type =/= ?CONNECT -> - {error, connect_expected, PState}; -process_frame(Frame = #mqtt_frame{fixed = #mqtt_frame_fixed{type = Type}}, PState) -> - process_request(Type, Frame, PState). + {error, connect_expected, State}; +process_frame(Frame = #mqtt_frame{fixed = #mqtt_frame_fixed{type = Type}}, State) -> + process_request(Type, Frame, State). -process_request(?CONNECT, Frame, PState = #proc_state{socket = Socket}) -> +process_request(?CONNECT, Frame, State = #state{socket = Socket}) -> %% Check whether peer closed the connection. %% For example, this can happen when connection was blocked because of resource %% alarm and client therefore disconnected due to client side CONNACK timeout. case rabbit_net:socket_ends(Socket, inbound) of {error, Reason} -> - {error, {socket_ends, Reason}, PState}; + {error, {socket_ends, Reason}, State}; _ -> - process_connect(Frame, PState) + process_connect(Frame, State) end; process_request(?PUBACK, #mqtt_frame{ variable = #mqtt_frame_publish{message_id = PacketId}}, - #proc_state{unacked_server_pubs = U0, - queue_states = QStates0, - queue_qos1 = QName} = PState) -> + #state{unacked_server_pubs = U0, + queue_states = QStates0, + queue_qos1 = QName} = State) -> case maps:take(PacketId, U0) of {QMsgId, U} -> case rabbit_queue_type:settle(QName, complete, ?CONSUMER_TAG, [QMsgId], QStates0) of {ok, QStates, Actions} -> %%TODO rabbit_channel:incr_queue_stats/3 - message_acknowledged(QName, PState), - {ok, handle_queue_actions(Actions, PState#proc_state{unacked_server_pubs = U, - queue_states = QStates})}; + message_acknowledged(QName, State), + {ok, handle_queue_actions(Actions, State#state{unacked_server_pubs = U, + queue_states = QStates})}; {protocol_error, _ErrorType, _Reason, _ReasonArgs} = Err -> - {error, Err, PState} + {error, Err, State} end; error -> - {ok, PState} + {ok, State} end; process_request(?PUBLISH, Frame = #mqtt_frame{ fixed = Fixed = #mqtt_frame_fixed{qos = ?QOS_2}}, - PState) -> + State) -> % Downgrade QOS_2 to QOS_1 process_request(?PUBLISH, Frame#mqtt_frame{ fixed = Fixed#mqtt_frame_fixed{qos = ?QOS_1}}, - PState); + State); process_request(?PUBLISH, #mqtt_frame{ fixed = #mqtt_frame_fixed{qos = Qos, @@ -104,12 +157,12 @@ process_request(?PUBLISH, variable = #mqtt_frame_publish{topic_name = Topic, message_id = MessageId }, payload = Payload}, - PState0 = #proc_state{retainer_pid = RPid, - amqp2mqtt_fun = Amqp2MqttFun, - unacked_client_pubs = U, - proto_ver = ProtoVer}) -> + State0 = #state{retainer_pid = RPid, + amqp2mqtt_fun = Amqp2MqttFun, + unacked_client_pubs = U, + proto_ver = ProtoVer}) -> rabbit_global_counters:messages_received(ProtoVer, 1), - PState = maybe_increment_publisher(PState0), + State = maybe_increment_publisher(State0), Publish = fun() -> Msg = #mqtt_msg{retain = Retain, qos = Qos, @@ -117,7 +170,7 @@ process_request(?PUBLISH, dup = Dup, message_id = MessageId, payload = Payload}, - case publish_to_queues(Msg, PState) of + case publish_to_queues(Msg, State) of {ok, _} = Ok -> case Retain of false -> @@ -135,15 +188,15 @@ process_request(?PUBLISH, rabbit_global_counters:messages_received_confirm(ProtoVer, 1), case rabbit_mqtt_confirms:contains(MessageId, U) of false -> - publish_to_queues_with_checks(Topic, Publish, PState); + publish_to_queues_with_checks(Topic, Publish, State); true -> %% Client re-sent this PUBLISH packet. %% We already sent this message to target queues awaiting confirmations. %% Hence, we ignore this re-send. - {ok, PState} + {ok, State} end; _ -> - publish_to_queues_with_checks(Topic, Publish, PState) + publish_to_queues_with_checks(Topic, Publish, State) end; process_request(?SUBSCRIBE, @@ -152,14 +205,14 @@ process_request(?SUBSCRIBE, message_id = SubscribeMsgId, topic_table = Topics}, payload = undefined}, - #proc_state{send_fun = SendFun, - retainer_pid = RPid} = PState0) -> + #state{send_fun = SendFun, + retainer_pid = RPid} = State0) -> rabbit_log_connection:debug("Received a SUBSCRIBE for topic(s) ~p", [Topics]), - TopicNamesQos0 = topic_names(?QOS_0, PState0), - TopicNamesQos1 = topic_names(?QOS_1, PState0), + TopicNamesQos0 = topic_names(?QOS_0, State0), + TopicNamesQos1 = topic_names(?QOS_1, State0), HasSubsBefore = TopicNamesQos0 =/= [] orelse TopicNamesQos1 =/= [], - {QosResponse, PState1} = + {QosResponse, State1} = lists:foldl( fun(_Topic, {[?SUBACK_FAILURE | _] = L, S}) -> %% Once a subscription failed, mark all following subscriptions @@ -199,67 +252,67 @@ process_request(?SUBSCRIBE, {error, _Reason, S1} -> {[?SUBACK_FAILURE | L], S1} end - end, {[], PState0}, Topics), + end, {[], State0}, Topics), - maybe_increment_consumer(HasSubsBefore, PState1), + maybe_increment_consumer(HasSubsBefore, State1), SendFun( #mqtt_frame{fixed = #mqtt_frame_fixed{type = ?SUBACK}, variable = #mqtt_frame_suback{ message_id = SubscribeMsgId, qos_table = QosResponse}}, - PState1), + State1), case QosResponse of [?SUBACK_FAILURE | _] -> - {error, subscribe_error, PState1}; + {error, subscribe_error, State1}; _ -> - PState = lists:foldl(fun(Topic, S) -> - maybe_send_retained_message(RPid, Topic, S) - end, PState1, Topics), - {ok, PState} + State = lists:foldl(fun(Topic, S) -> + maybe_send_retained_message(RPid, Topic, S) + end, State1, Topics), + {ok, State} end; process_request(?UNSUBSCRIBE, #mqtt_frame{variable = #mqtt_frame_subscribe{message_id = MessageId, topic_table = Topics}, payload = undefined}, - PState0 = #proc_state{send_fun = SendFun}) -> + State0 = #state{send_fun = SendFun}) -> rabbit_log_connection:debug("Received an UNSUBSCRIBE for topic(s) ~p", [Topics]), - HasSubsBefore = has_subs(PState0), - PState = lists:foldl( - fun(#mqtt_topic{name = TopicName}, #proc_state{} = S0) -> - case find_queue_name(TopicName, S0) of - {ok, QName} -> - case unbind(QName, TopicName, S0) of - {ok, _, S} -> - S; - {error, Reason, S} -> - rabbit_log:error("Failed to unbind ~s with topic ~s: ~p", - [rabbit_misc:rs(QName), TopicName, Reason]), - S - end; - {not_found, _} -> - S0 - end - end, PState0, Topics), + HasSubsBefore = has_subs(State0), + State = lists:foldl( + fun(#mqtt_topic{name = TopicName}, #state{} = S0) -> + case find_queue_name(TopicName, S0) of + {ok, QName} -> + case unbind(QName, TopicName, S0) of + {ok, _, S} -> + S; + {error, Reason, S} -> + rabbit_log:error("Failed to unbind ~s with topic ~s: ~p", + [rabbit_misc:rs(QName), TopicName, Reason]), + S + end; + {not_found, _} -> + S0 + end + end, State0, Topics), SendFun( #mqtt_frame{fixed = #mqtt_frame_fixed {type = ?UNSUBACK}, variable = #mqtt_frame_suback{message_id = MessageId}}, - PState), + State), - maybe_decrement_consumer(HasSubsBefore, PState), - {ok, PState}; + maybe_decrement_consumer(HasSubsBefore, State), + {ok, State}; -process_request(?PINGREQ, #mqtt_frame{}, PState = #proc_state{send_fun = SendFun}) -> +process_request(?PINGREQ, #mqtt_frame{}, State = #state{send_fun = SendFun}) -> rabbit_log_connection:debug("Received a PINGREQ"), SendFun( #mqtt_frame{fixed = #mqtt_frame_fixed{type = ?PINGRESP}}, - PState), + State), rabbit_log_connection:debug("Sent a PINGRESP"), - {ok, PState}; + {ok, State}; -process_request(?DISCONNECT, #mqtt_frame{}, PState) -> +process_request(?DISCONNECT, #mqtt_frame{}, State) -> rabbit_log_connection:debug("Received a DISCONNECT"), - {stop, disconnect, PState}. + {stop, disconnect, State}. process_connect(#mqtt_frame{ variable = #mqtt_frame_connect{ @@ -268,11 +321,11 @@ process_connect(#mqtt_frame{ clean_sess = CleanSess, client_id = ClientId, keep_alive = Keepalive} = FrameConnect}, - PState0 = #proc_state{send_fun = SendFun}) -> + State0 = #state{send_fun = SendFun}) -> rabbit_log_connection:debug("Received a CONNECT, client ID: ~s, username: ~s, " "clean session: ~s, protocol version: ~p, keepalive: ~p", [ClientId, Username, CleanSess, ProtoVersion, Keepalive]), - {ReturnCode, SessionPresent, PState} = + {ReturnCode, SessionPresent, State} = case rabbit_misc:pipeline([fun check_protocol_version/1, fun check_client_id/1, fun check_credentials/2, @@ -281,18 +334,18 @@ process_connect(#mqtt_frame{ fun notify_connection_created/2, fun start_keepalive/2, fun handle_clean_session/2], - FrameConnect, PState0) of - {ok, SessionPresent0, PState1} -> - {?CONNACK_ACCEPT, SessionPresent0, PState1}; - {error, ReturnCode0, PState1} -> - {ReturnCode0, false, PState1} + FrameConnect, State0) of + {ok, SessionPresent0, State1} -> + {?CONNACK_ACCEPT, SessionPresent0, State1}; + {error, ReturnCode0, State1} -> + {ReturnCode0, false, State1} end, ResponseFrame = #mqtt_frame{fixed = #mqtt_frame_fixed{type = ?CONNACK}, variable = #mqtt_frame_connack{ session_present = SessionPresent, return_code = ReturnCode}}, - SendFun(ResponseFrame, PState), - return_connack(ReturnCode, PState). + SendFun(ResponseFrame, State), + return_connack(ReturnCode, State). client_id(<<>>) -> rabbit_mqtt_util:gen_client_id(); @@ -316,8 +369,8 @@ check_client_id(_) -> check_credentials(Frame = #mqtt_frame_connect{username = Username, password = Password}, - PState = #proc_state{ssl_login_name = SslLoginName, - peer_addr = PeerAddr}) -> + State = #state{ssl_login_name = SslLoginName, + peer_addr = PeerAddr}) -> Ip = list_to_binary(inet:ntoa(PeerAddr)), case creds(Username, Password, SslLoginName) of nocreds -> @@ -333,30 +386,30 @@ check_credentials(Frame = #mqtt_frame_connect{username = Username, rabbit_log_connection:error("MQTT login failed for user '~p': no password provided", [User]), {error, ?CONNACK_BAD_CREDENTIALS}; {UserBin, PassBin} -> - {ok, {UserBin, PassBin, Frame}, PState} + {ok, {UserBin, PassBin, Frame}, State} end. login({UserBin, PassBin, Frame = #mqtt_frame_connect{client_id = ClientId0, clean_sess = CleanSess}}, - PState0) -> + State0) -> ClientId = client_id(ClientId0), - case process_login(UserBin, PassBin, ClientId, PState0) of + case process_login(UserBin, PassBin, ClientId, State0) of already_connected -> {ok, already_connected}; - {ok, PState} -> - {ok, Frame, PState#proc_state{clean_sess = CleanSess, - client_id = ClientId}}; - {error, _Reason, _PState} = Err -> + {ok, State} -> + {ok, Frame, State#state{clean_sess = CleanSess, + client_id = ClientId}}; + {error, _Reason, _State} = Err -> Err end. -register_client(already_connected, _PState) -> +register_client(already_connected, _State) -> ok; register_client(Frame = #mqtt_frame_connect{proto_ver = ProtoVersion}, - PState = #proc_state{client_id = ClientId, - socket = Socket, - auth_state = #auth_state{vhost = VHost}}) -> + State = #state{client_id = ClientId, + socket = Socket, + auth_state = #auth_state{vhost = VHost}}) -> NewProcState = fun(RegisterState) -> rabbit_mqtt_util:register_clientid(VHost, ClientId), @@ -366,7 +419,7 @@ register_client(Frame = #mqtt_frame_connect{proto_ver = ProtoVersion}, ExchangeBin = rabbit_mqtt_util:env(exchange), ExchangeName = rabbit_misc:r(VHost, exchange, ExchangeBin), ProtoHumanReadable = {'MQTT', human_readable_mqtt_version(ProtoVersion)}, - PState#proc_state{ + State#state{ exchange = ExchangeName, will_msg = make_will_msg(Frame), retainer_pid = RetainerPid, @@ -399,15 +452,15 @@ register_client(Frame = #mqtt_frame_connect{proto_ver = ProtoVersion}, {ok, NewProcState(undefined)} end. -notify_connection_created(already_connected, _PState) -> +notify_connection_created(already_connected, _State) -> ok; notify_connection_created( _Frame, - #proc_state{socket = Sock, - conn_name = ConnName, - info = #info{proto_human = {ProtoName, ProtoVsn}}, - auth_state = #auth_state{vhost = VHost, - username = Username}} = PState) -> + #state{socket = Sock, + conn_name = ConnName, + info = #info{proto_human = {ProtoName, ProtoVsn}}, + auth_state = #auth_state{vhost = VHost, + username = Username}} = State) -> {ok, {PeerHost, PeerPort, Host, Port}} = rabbit_net:socket_ends(Sock, inbound), ConnectedAt = os:system_time(milli_seconds), Infos = [{host, Host}, @@ -426,7 +479,7 @@ notify_connection_created( rabbit_core_metrics:connection_created(self(), Infos), rabbit_event:notify(connection_created, Infos), rabbit_networking:register_non_amqp_connection(self()), - {ok, PState#proc_state{ + {ok, State#state{ %% We won't need conn_name anymore. Use less memmory by setting to undefined. conn_name = undefined}}. @@ -460,20 +513,20 @@ self_consumes(Queue) -> end. start_keepalive(#mqtt_frame_connect{keep_alive = Seconds}, - #proc_state{socket = Socket}) -> + #state{socket = Socket}) -> ok = rabbit_mqtt_keepalive:start(Seconds, Socket). -handle_clean_session(_, PState0 = #proc_state{clean_sess = false, - proto_ver = ProtoVer}) -> - case get_queue(?QOS_1, PState0) of +handle_clean_session(_, State0 = #state{clean_sess = false, + proto_ver = ProtoVer}) -> + case get_queue(?QOS_1, State0) of {error, not_found} -> %% Queue will be created later when client subscribes. - {ok, _SessionPresent = false, PState0}; + {ok, _SessionPresent = false, State0}; {ok, Q} -> - case consume(Q, ?QOS_1, PState0) of - {ok, PState} -> + case consume(Q, ?QOS_1, State0) of + {ok, State} -> rabbit_global_counters:consumer_created(ProtoVer), - {ok, _SessionPresent = true, PState}; + {ok, _SessionPresent = true, State}; {error, access_refused} -> {error, ?CONNACK_NOT_AUTHORIZED}; {error, _Reason} -> @@ -481,13 +534,13 @@ handle_clean_session(_, PState0 = #proc_state{clean_sess = false, {error, ?CONNACK_SERVER_UNAVAILABLE} end end; -handle_clean_session(_, PState = #proc_state{clean_sess = true, - auth_state = #auth_state{user = User, - username = Username, - authz_ctx = AuthzCtx}}) -> - case get_queue(?QOS_1, PState) of +handle_clean_session(_, State = #state{clean_sess = true, + auth_state = #auth_state{user = User, + username = Username, + authz_ctx = AuthzCtx}}) -> + case get_queue(?QOS_1, State) of {error, not_found} -> - {ok, _SessionPresent = false, PState}; + {ok, _SessionPresent = false, State}; {ok, Q0} -> QName = amqqueue:get_name(Q0), %% configure access to queue required for queue.delete @@ -507,32 +560,32 @@ handle_clean_session(_, PState = #proc_state{clean_sess = true, ({absent, _Q, _Reason}) -> ok end), - {ok, _SessionPresent = false, PState}; + {ok, _SessionPresent = false, State}; {error, access_refused} -> {error, ?CONNACK_NOT_AUTHORIZED} end end. --spec get_queue(qos(), proc_state()) -> +-spec get_queue(qos(), state()) -> {ok, amqqueue:amqqueue()} | {error, not_found}. -get_queue(QoS, PState) -> - QName = queue_name(QoS, PState), +get_queue(QoS, State) -> + QName = queue_name(QoS, State), rabbit_amqqueue:lookup(QName). -queue_name(QoS, #proc_state{client_id = ClientId, - auth_state = #auth_state{vhost = VHost}}) -> +queue_name(QoS, #state{client_id = ClientId, + auth_state = #auth_state{vhost = VHost}}) -> QNameBin = rabbit_mqtt_util:queue_name_bin(ClientId, QoS), rabbit_misc:r(VHost, queue, QNameBin). -find_queue_name(TopicName, #proc_state{exchange = Exchange, - mqtt2amqp_fun = Mqtt2AmqpFun} = PState) -> +find_queue_name(TopicName, #state{exchange = Exchange, + mqtt2amqp_fun = Mqtt2AmqpFun} = State) -> RoutingKey = Mqtt2AmqpFun(TopicName), - QNameQoS0 = queue_name(?QOS_0, PState), + QNameQoS0 = queue_name(?QOS_0, State), case lookup_binding(Exchange, QNameQoS0, RoutingKey) of true -> {ok, QNameQoS0}; false -> - QNameQoS1 = queue_name(?QOS_1, PState), + QNameQoS1 = queue_name(?QOS_1, State), case lookup_binding(Exchange, QNameQoS1, RoutingKey) of true -> {ok, QNameQoS1}; @@ -547,13 +600,13 @@ lookup_binding(Exchange, QueueName, RoutingKey) -> key = RoutingKey}, lists:member(B, rabbit_binding:list_for_source_and_destination(Exchange, QueueName)). -has_subs(PState) -> - topic_names(?QOS_0, PState) =/= [] orelse - topic_names(?QOS_1, PState) =/= []. +has_subs(State) -> + topic_names(?QOS_0, State) =/= [] orelse + topic_names(?QOS_1, State) =/= []. -topic_names(QoS, #proc_state{exchange = Exchange, - amqp2mqtt_fun = Amqp2MqttFun} = PState) -> - Bindings = rabbit_binding:list_for_source_and_destination(Exchange, queue_name(QoS, PState)), +topic_names(QoS, #state{exchange = Exchange, + amqp2mqtt_fun = Amqp2MqttFun} = State) -> + Bindings = rabbit_binding:list_for_source_and_destination(Exchange, queue_name(QoS, State)), lists:map(fun(B) -> Amqp2MqttFun(B#binding.key) end, Bindings). %% "If a Server receives a SUBSCRIBE Packet containing a Topic Filter that is identical @@ -565,24 +618,24 @@ topic_names(QoS, #proc_state{exchange = Exchange, %% Therefore, if we receive a QoS 0 subscription for a topic that already has QoS 1, %% we unbind QoS 1 (and vice versa). maybe_replace_old_sub(#mqtt_topic{name = TopicName, qos = ?QOS_0}, - _, OldTopicNamesQos1, PState) -> - QName = queue_name(?QOS_1, PState), - maybe_unbind(TopicName, OldTopicNamesQos1, QName, PState); + _, OldTopicNamesQos1, State) -> + QName = queue_name(?QOS_1, State), + maybe_unbind(TopicName, OldTopicNamesQos1, QName, State); maybe_replace_old_sub(#mqtt_topic{name = TopicName, qos = QoS}, - OldTopicNamesQos0, _, PState) + OldTopicNamesQos0, _, State) when QoS =:= ?QOS_1 orelse QoS =:= ?QOS_2 -> - QName = queue_name(?QOS_0, PState), - maybe_unbind(TopicName, OldTopicNamesQos0, QName, PState). + QName = queue_name(?QOS_0, State), + maybe_unbind(TopicName, OldTopicNamesQos0, QName, State). -maybe_unbind(TopicName, TopicNames, QName, PState0) -> +maybe_unbind(TopicName, TopicNames, QName, State0) -> case lists:member(list_to_binary(TopicName), TopicNames) of false -> - {ok, PState0}; + {ok, State0}; true -> - case unbind(QName, TopicName, PState0) of - {ok, _Output, PState} -> - {ok, PState}; - {error, Reason, _PState} = Err -> + case unbind(QName, TopicName, State0) of + {ok, _Output, State} -> + {ok, State}; + {error, Reason, _State} = Err -> rabbit_log:error("Failed to unbind ~s with topic ~s: ~p", [rabbit_misc:rs(QName), TopicName, Reason]), Err @@ -599,21 +652,21 @@ hand_off_to_retainer(RetainerPid, Amqp2MqttFun, Topic0, Msg) -> ok. maybe_send_retained_message(RPid, #mqtt_topic{name = Topic0, qos = SubscribeQos}, - #proc_state{amqp2mqtt_fun = Amqp2MqttFun, - packet_id = PacketId0, - send_fun = SendFun} = PState0) -> + #state{amqp2mqtt_fun = Amqp2MqttFun, + packet_id = PacketId0, + send_fun = SendFun} = State0) -> Topic1 = Amqp2MqttFun(Topic0), case rabbit_mqtt_retainer:fetch(RPid, Topic1) of undefined -> - PState0; + State0; Msg -> Qos = effective_qos(Msg#mqtt_msg.qos, SubscribeQos), - {PacketId, PState} = case Qos of - ?QOS_0 -> - {undefined, PState0}; - ?QOS_1 -> - {PacketId0, PState0#proc_state{packet_id = increment_packet_id(PacketId0)}} - end, + {PacketId, State} = case Qos of + ?QOS_0 -> + {undefined, State0}; + ?QOS_1 -> + {PacketId0, State0#state{packet_id = increment_packet_id(PacketId0)}} + end, SendFun( #mqtt_frame{fixed = #mqtt_frame_fixed{ type = ?PUBLISH, @@ -625,8 +678,8 @@ maybe_send_retained_message(RPid, #mqtt_topic{name = Topic0, qos = SubscribeQos} topic_name = Topic1 }, payload = Msg#mqtt_msg.payload}, - PState), - PState + State), + State end. make_will_msg(#mqtt_frame_connect{will_flag = false}) -> @@ -642,11 +695,11 @@ make_will_msg(#mqtt_frame_connect{will_retain = Retain, payload = Msg}. process_login(_UserBin, _PassBin, _ClientId, - #proc_state{peer_addr = Addr, - auth_state = #auth_state{username = Username, - user = User, - vhost = VHost - }}) + #state{peer_addr = Addr, + auth_state = #auth_state{username = Username, + user = User, + vhost = VHost + }}) when Username =/= undefined, User =/= undefined, VHost =/= underfined -> UsernameStr = rabbit_data_coercion:to_list(Username), VHostStr = rabbit_data_coercion:to_list(VHost), @@ -655,10 +708,10 @@ process_login(_UserBin, _PassBin, _ClientId, [UsernameStr, VHostStr]), already_connected; process_login(UserBin, PassBin, ClientId, - #proc_state{socket = Sock, - ssl_login_name = SslLoginName, - peer_addr = Addr, - auth_state = undefined} = PState0) -> + #state{socket = Sock, + ssl_login_name = SslLoginName, + peer_addr = Addr, + auth_state = undefined} = State0) -> {ok, {_PeerHost, _PeerPort, _Host, Port}} = rabbit_net:socket_ends(Sock, inbound), {VHostPickedUsing, {VHost, UsernameBin}} = get_vhost(UserBin, SslLoginName, Port), rabbit_log_connection:debug( @@ -678,11 +731,11 @@ process_login(UserBin, PassBin, ClientId, fun check_vhost_access/2, fun check_user_loopback/2 ], - Input, PState0) of - {ok, _Output, PState} -> + Input, State0) of + {ok, _Output, State} -> rabbit_core_metrics:auth_attempt_succeeded(RemoteIpAddressBin, UsernameBin, mqtt), - {ok, PState}; - {error, _Reason, _PState} = Err -> + {ok, State}; + {error, _Reason, _State} = Err -> rabbit_core_metrics:auth_attempt_failed(RemoteIpAddressBin, UsernameBin, mqtt), Err end. @@ -730,7 +783,7 @@ check_user_login(#{vhost := VHost, username_bin := UsernameBin, pass_bin := PassBin, client_id := ClientId - } = In, PState) -> + } = In, State) -> AuthProps = case PassBin of none -> %% SSL user name provided. @@ -744,19 +797,19 @@ check_user_login(#{vhost := VHost, case rabbit_access_control:check_user_login( UsernameBin, AuthProps) of {ok, User = #user{username = Username}} -> - notify_auth_result(user_authentication_success, Username, PState), - {ok, maps:put(user, User, In), PState}; + notify_auth_result(user_authentication_success, Username, State), + {ok, maps:put(user, User, In), State}; {refused, Username, Msg, Args} -> rabbit_log_connection:error( "Error on MQTT connection ~p~n" "access refused for user '~s' in vhost '~s' " ++ Msg, [self(), Username, VHost] ++ Args), - notify_auth_result(user_authentication_failure, Username, PState), + notify_auth_result(user_authentication_failure, Username, State), {error, ?CONNACK_BAD_CREDENTIALS} end. -notify_auth_result(AuthResult, Username, #proc_state{conn_name = ConnName}) -> +notify_auth_result(AuthResult, Username, #state{conn_name = ConnName}) -> rabbit_event:notify( AuthResult, [ @@ -786,7 +839,7 @@ check_vhost_access(#{vhost := VHost, client_id := ClientId, user := User = #user{username = Username} } = In, - #proc_state{peer_addr = PeerAddr} = PState) -> + #state{peer_addr = PeerAddr} = State) -> AuthzCtx = #{<<"client_id">> => ClientId}, try rabbit_access_control:check_vhost_access( User, @@ -794,7 +847,7 @@ check_vhost_access(#{vhost := VHost, {ip, PeerAddr}, AuthzCtx) of ok -> - {ok, maps:put(authz_ctx, AuthzCtx, In), PState} + {ok, maps:put(authz_ctx, AuthzCtx, In), State} catch exit:#amqp_error{name = not_allowed} -> rabbit_log_connection:error( "Error on MQTT connection ~p~n" @@ -808,14 +861,14 @@ check_user_loopback(#{vhost := VHost, user := User, authz_ctx := AuthzCtx }, - #proc_state{peer_addr = PeerAddr} = PState) -> + #state{peer_addr = PeerAddr} = State) -> case rabbit_access_control:check_user_loopback(UsernameBin, PeerAddr) of ok -> AuthState = #auth_state{user = User, username = UsernameBin, vhost = VHost, authz_ctx = AuthzCtx}, - {ok, PState#proc_state{auth_state = AuthState}}; + {ok, State#state{auth_state = AuthState}}; not_allowed -> rabbit_log_connection:warning( "MQTT login failed: user '~s' can only connect via localhost", @@ -836,8 +889,8 @@ get_vhost_no_ssl(UserBin, Port) -> {vhost_in_username_or_default, get_vhost_username(UserBin)}; false -> PortVirtualHostMapping = rabbit_runtime_parameters:value_global( - mqtt_port_to_vhost_mapping - ), + mqtt_port_to_vhost_mapping + ), case get_vhost_from_port_mapping(Port, PortVirtualHostMapping) of undefined -> {default_vhost, {rabbit_mqtt_util:env(vhost), UserBin}}; @@ -848,13 +901,13 @@ get_vhost_no_ssl(UserBin, Port) -> get_vhost_ssl(UserBin, SslLoginName, Port) -> UserVirtualHostMapping = rabbit_runtime_parameters:value_global( - mqtt_default_vhosts - ), + mqtt_default_vhosts + ), case get_vhost_from_user_mapping(SslLoginName, UserVirtualHostMapping) of undefined -> PortVirtualHostMapping = rabbit_runtime_parameters:value_global( - mqtt_port_to_vhost_mapping - ), + mqtt_port_to_vhost_mapping + ), case get_vhost_from_port_mapping(Port, PortVirtualHostMapping) of undefined -> {vhost_in_username_or_default, get_vhost_username(UserBin)}; @@ -904,11 +957,11 @@ get_vhost_from_port_mapping(_Port, not_found) -> get_vhost_from_port_mapping(Port, Mapping) -> M = rabbit_data_coercion:to_proplist(Mapping), Res = case rabbit_misc:pget(rabbit_data_coercion:to_binary(Port), M) of - undefined -> - undefined; - VHost -> - VHost - end, + undefined -> + undefined; + VHost -> + VHost + end, Res. human_readable_vhost_lookup_strategy(vhost_in_username_or_default) -> @@ -920,7 +973,7 @@ human_readable_vhost_lookup_strategy(cert_to_vhost_mapping) -> human_readable_vhost_lookup_strategy(default_vhost) -> "plugin configuration or default"; human_readable_vhost_lookup_strategy(Val) -> - atom_to_list(Val). + atom_to_list(Val). creds(User, Pass, SSLLoginName) -> DefaultUser = rabbit_mqtt_util:env(default_user), @@ -928,17 +981,17 @@ creds(User, Pass, SSLLoginName) -> {ok, Anon} = application:get_env(?APP, allow_anonymous), {ok, TLSAuth} = application:get_env(?APP, ssl_cert_login), HaveDefaultCreds = Anon =:= true andalso - is_binary(DefaultUser) andalso - is_binary(DefaultPass), + is_binary(DefaultUser) andalso + is_binary(DefaultPass), CredentialsProvided = User =/= undefined orelse - Pass =/= undefined, + Pass =/= undefined, CorrectCredentials = is_list(User) andalso - is_list(Pass), + is_list(Pass), SSLLoginProvided = TLSAuth =:= true andalso - SSLLoginName =/= none, + SSLLoginName =/= none, case {CredentialsProvided, CorrectCredentials, SSLLoginProvided, HaveDefaultCreds} of %% Username and password take priority @@ -962,15 +1015,15 @@ delivery_mode(?QOS_0) -> 1; delivery_mode(?QOS_1) -> 2; delivery_mode(?QOS_2) -> 2. -ensure_queue(QoS, #proc_state{ +ensure_queue(QoS, #state{ client_id = ClientId, clean_sess = CleanSess, auth_state = #auth_state{ vhost = VHost, user = User = #user{username = Username}, authz_ctx = AuthzCtx} - } = PState) -> - case get_queue(QoS, PState) of + } = State) -> + case get_queue(QoS, State) of {ok, Q} -> {ok, Q}; {error, not_found} -> @@ -1043,13 +1096,13 @@ queue_type(?QOS_0, true, _) -> queue_type(_, _, QArgs) -> rabbit_amqqueue:get_queue_type(QArgs). -consume(Q, QoS, #proc_state{ +consume(Q, QoS, #state{ queue_states = QStates0, auth_state = #auth_state{ user = User = #user{username = Username}, authz_ctx = AuthzCtx}, info = #info{prefetch = Prefetch} - } = PState0) -> + } = State0) -> QName = amqqueue:get_name(Q), %% read access to queue required for basic.consume case check_resource_access(User, QName, read, AuthzCtx) of @@ -1058,7 +1111,7 @@ consume(Q, QoS, #proc_state{ ?QUEUE_TYPE_QOS_0 -> %% Messages get delivered directly to our process without %% explicitly calling rabbit_queue_type:consume/3. - {ok, PState0}; + {ok, State0}; _ -> Spec = #{no_ack => QoS =:= ?QOS_0, channel_pid => self(), @@ -1075,9 +1128,9 @@ consume(Q, QoS, #proc_state{ fun(Q1) -> case rabbit_queue_type:consume(Q1, Spec, QStates0) of {ok, QStates} -> - PState1 = PState0#proc_state{queue_states = QStates}, - PState = maybe_set_queue_qos1(QoS, PState1), - {ok, PState}; + State1 = State0#state{queue_states = QStates}, + State = maybe_set_queue_qos1(QoS, State1), + {ok, State}; {error, Reason} = Err -> rabbit_log:error("Failed to consume from ~s: ~p", [rabbit_misc:rs(QName), Reason]), @@ -1091,17 +1144,17 @@ consume(Q, QoS, #proc_state{ %% To save memory, we only store the queue_qos1 value in process state if there is a QoS 1 subscription. %% We store it in the process state such that we don't have to build the binary on every PUBACK we receive. -maybe_set_queue_qos1(?QOS_1, PState = #proc_state{queue_qos1 = undefined}) -> - PState#proc_state{queue_qos1 = queue_name(?QOS_1, PState)}; -maybe_set_queue_qos1(_, PState) -> - PState. +maybe_set_queue_qos1(?QOS_1, State = #state{queue_qos1 = undefined}) -> + State#state{queue_qos1 = queue_name(?QOS_1, State)}; +maybe_set_queue_qos1(_, State) -> + State. -bind(QueueName, TopicName, PState) -> - binding_action_with_checks({QueueName, TopicName, fun rabbit_binding:add/2}, PState). -unbind(QueueName, TopicName, PState) -> - binding_action_with_checks({QueueName, TopicName, fun rabbit_binding:remove/2}, PState). +bind(QueueName, TopicName, State) -> + binding_action_with_checks({QueueName, TopicName, fun rabbit_binding:add/2}, State). +unbind(QueueName, TopicName, State) -> + binding_action_with_checks({QueueName, TopicName, fun rabbit_binding:remove/2}, State). -binding_action_with_checks(Input, PState) -> +binding_action_with_checks(Input, State) -> %% Same permission checks required for both binding and unbinding %% queue to / from topic exchange. rabbit_misc:pipeline( @@ -1109,50 +1162,50 @@ binding_action_with_checks(Input, PState) -> fun check_exchange_read_access/2, fun check_topic_access/2, fun binding_action/2], - Input, PState). + Input, State). check_queue_write_access( {QueueName, _, _}, - #proc_state{auth_state = #auth_state{ - user = User, - authz_ctx = AuthzCtx}}) -> + #state{auth_state = #auth_state{ + user = User, + authz_ctx = AuthzCtx}}) -> %% write access to queue required for queue.(un)bind check_resource_access(User, QueueName, write, AuthzCtx). check_exchange_read_access( - _, #proc_state{exchange = ExchangeName, - auth_state = #auth_state{ - user = User, - authz_ctx = AuthzCtx}}) -> + _, #state{exchange = ExchangeName, + auth_state = #auth_state{ + user = User, + authz_ctx = AuthzCtx}}) -> %% read access to exchange required for queue.(un)bind check_resource_access(User, ExchangeName, read, AuthzCtx). -check_topic_access({_, TopicName, _}, PState) -> - check_topic_access(TopicName, read, PState). +check_topic_access({_, TopicName, _}, State) -> + check_topic_access(TopicName, read, State). binding_action( {QueueName, TopicName, BindingFun}, - #proc_state{exchange = ExchangeName, - auth_state = #auth_state{ - user = #user{username = Username}}, - mqtt2amqp_fun = Mqtt2AmqpFun}) -> + #state{exchange = ExchangeName, + auth_state = #auth_state{ + user = #user{username = Username}}, + mqtt2amqp_fun = Mqtt2AmqpFun}) -> RoutingKey = Mqtt2AmqpFun(TopicName), Binding = #binding{source = ExchangeName, destination = QueueName, key = RoutingKey}, BindingFun(Binding, Username). -publish_to_queues(undefined, PState) -> - {ok, PState}; +publish_to_queues(undefined, State) -> + {ok, State}; publish_to_queues( #mqtt_msg{qos = Qos, topic = Topic, dup = Dup, message_id = MessageId, payload = Payload}, - #proc_state{exchange = ExchangeName, - mqtt2amqp_fun = Mqtt2AmqpFun, - delivery_flow = Flow} = PState) -> + #state{exchange = ExchangeName, + mqtt2amqp_fun = Mqtt2AmqpFun, + delivery_flow = Flow} = State) -> RoutingKey = Mqtt2AmqpFun(Topic), Confirm = Qos > ?QOS_0, Headers = [{<<"x-mqtt-publish-qos">>, byte, Qos}, @@ -1186,16 +1239,16 @@ publish_to_queues( case rabbit_exchange:lookup(ExchangeName) of {ok, Exchange} -> QNames = rabbit_exchange:route(Exchange, Delivery), - deliver_to_queues(Delivery, QNames, PState); + deliver_to_queues(Delivery, QNames, State); {error, not_found} -> rabbit_log:error("~s not found", [rabbit_misc:rs(ExchangeName)]), - {error, exchange_not_found, PState} + {error, exchange_not_found, State} end. deliver_to_queues(Delivery, RoutedToQNames, - PState0 = #proc_state{queue_states = QStates0, - proto_ver = ProtoVer}) -> + State0 = #state{queue_states = QStates0, + proto_ver = ProtoVer}) -> %% TODO only lookup fields that are needed using ets:select / match? %% TODO Use ETS continuations to be more space efficient Qs0 = rabbit_amqqueue:lookup(RoutedToQNames), @@ -1203,66 +1256,66 @@ deliver_to_queues(Delivery, case rabbit_queue_type:deliver(Qs, Delivery, QStates0) of {ok, QStates, Actions} -> rabbit_global_counters:messages_routed(ProtoVer, length(Qs)), - PState = process_routing_confirm(Delivery, Qs, - PState0#proc_state{queue_states = QStates}), + State = process_routing_confirm(Delivery, Qs, + State0#state{queue_states = QStates}), %% Actions must be processed after registering confirms as actions may %% contain rejections of publishes. - {ok, handle_queue_actions(Actions, PState)}; + {ok, handle_queue_actions(Actions, State)}; {error, Reason} -> rabbit_log:error("Failed to deliver message with packet_id=~p to queues: ~p", [Delivery#delivery.msg_seq_no, Reason]), - {error, Reason, PState0} + {error, Reason, State0} end. process_routing_confirm(#delivery{confirm = false}, - [], PState = #proc_state{proto_ver = ProtoVer}) -> + [], State = #state{proto_ver = ProtoVer}) -> rabbit_global_counters:messages_unroutable_dropped(ProtoVer, 1), - PState; + State; process_routing_confirm(#delivery{confirm = true, msg_seq_no = undefined}, - [], PState = #proc_state{proto_ver = ProtoVer}) -> + [], State = #state{proto_ver = ProtoVer}) -> %% unroutable will message with QoS > 0 rabbit_global_counters:messages_unroutable_dropped(ProtoVer, 1), - PState; + State; process_routing_confirm(#delivery{confirm = true, msg_seq_no = MsgId}, - [], PState = #proc_state{proto_ver = ProtoVer}) -> + [], State = #state{proto_ver = ProtoVer}) -> rabbit_global_counters:messages_unroutable_returned(ProtoVer, 1), %% MQTT 5 spec: %% If the Server knows that there are no matching subscribers, it MAY use %% Reason Code 0x10 (No matching subscribers) instead of 0x00 (Success). - send_puback(MsgId, PState), - PState; -process_routing_confirm(#delivery{confirm = false}, _, PState) -> - PState; + send_puback(MsgId, State), + State; +process_routing_confirm(#delivery{confirm = false}, _, State) -> + State; process_routing_confirm(#delivery{confirm = true, - msg_seq_no = undefined}, [_|_], PState) -> + msg_seq_no = undefined}, [_|_], State) -> %% routable will message with QoS > 0 - PState; + State; process_routing_confirm(#delivery{confirm = true, msg_seq_no = MsgId}, - Qs, PState = #proc_state{unacked_client_pubs = U0}) -> + Qs, State = #state{unacked_client_pubs = U0}) -> QNames = lists:map(fun amqqueue:get_name/1, Qs), U = rabbit_mqtt_confirms:insert(MsgId, QNames, U0), - PState#proc_state{unacked_client_pubs = U}. + State#state{unacked_client_pubs = U}. -send_puback(MsgIds0, PState) +send_puback(MsgIds0, State) when is_list(MsgIds0) -> %% Classic queues confirm messages unordered. %% Let's sort them here assuming most MQTT clients send with an increasing packet identifier. MsgIds = lists:usort(MsgIds0), lists:foreach(fun(Id) -> - send_puback(Id, PState) + send_puback(Id, State) end, MsgIds); -send_puback(MsgId, PState = #proc_state{send_fun = SendFun, - proto_ver = ProtoVer}) -> +send_puback(MsgId, State = #state{send_fun = SendFun, + proto_ver = ProtoVer}) -> rabbit_global_counters:messages_confirmed(ProtoVer, 1), SendFun( #mqtt_frame{fixed = #mqtt_frame_fixed{type = ?PUBACK}, variable = #mqtt_frame_publish{message_id = MsgId}}, - PState). + State). -serialise_and_send_to_client(Frame, #proc_state{proto_ver = ProtoVer, socket = Sock}) -> +serialise_and_send_to_client(Frame, #state{proto_ver = ProtoVer, socket = Sock}) -> %%TODO Test sending large frames at high speed: %% Will we need garbage collection as done in rabbit_writer:maybe_gc_large_msg/1? %%TODO batch to fill up MTU if there are messages in the Erlang mailbox? @@ -1275,33 +1328,33 @@ serialise_and_send_to_client(Frame, #proc_state{proto_ver = ProtoVer, socket = S [Sock, Error, Frame]) end. -serialise(Frame, #proc_state{proto_ver = ProtoVer}) -> +serialise(Frame, #state{proto_ver = ProtoVer}) -> rabbit_mqtt_frame:serialise(Frame, ProtoVer). -terminate(SendWill, ConnName, PState) -> - maybe_send_will(SendWill, ConnName, PState), +terminate(SendWill, ConnName, State) -> + maybe_send_will(SendWill, ConnName, State), Infos = [{name, ConnName}, {node, node()}, {pid, self()}, {disconnected_at, os:system_time(milli_seconds)} - ] ++ additional_connection_closed_info(PState), + ] ++ additional_connection_closed_info(State), rabbit_event:notify(connection_closed, Infos), rabbit_networking:unregister_non_amqp_connection(self()), - maybe_unregister_client(PState), - maybe_decrement_consumer(PState), - maybe_decrement_publisher(PState), - maybe_delete_mqtt_qos0_queue(PState). + maybe_unregister_client(State), + maybe_decrement_consumer(State), + maybe_decrement_publisher(State), + maybe_delete_mqtt_qos0_queue(State). maybe_send_will(true, ConnStr, - #proc_state{will_msg = WillMsg = #mqtt_msg{retain = Retain, - topic = Topic}, - retainer_pid = RPid, - amqp2mqtt_fun = Amqp2MqttFun} = PState) -> + #state{will_msg = WillMsg = #mqtt_msg{retain = Retain, + topic = Topic}, + retainer_pid = RPid, + amqp2mqtt_fun = Amqp2MqttFun} = State) -> rabbit_log_connection:debug("sending MQTT will message to topic ~s on connection ~s", [Topic, ConnStr]), - case check_topic_access(Topic, write, PState) of + case check_topic_access(Topic, write, State) of ok -> - publish_to_queues(WillMsg, PState), + publish_to_queues(WillMsg, State), case Retain of false -> ok; @@ -1315,16 +1368,16 @@ maybe_send_will(_, _, _) -> ok. additional_connection_closed_info( - #proc_state{info = #info{proto_human = {ProtoName, ProtoVsn}}, - auth_state = #auth_state{vhost = VHost, - username = Username}}) -> + #state{info = #info{proto_human = {ProtoName, ProtoVsn}}, + auth_state = #auth_state{vhost = VHost, + username = Username}}) -> [{protocol, {ProtoName, binary_to_list(ProtoVsn)}}, {vhost, VHost}, {user, Username}]; additional_connection_closed_info(_) -> []. -maybe_unregister_client(#proc_state{client_id = ClientId}) +maybe_unregister_client(#state{client_id = ClientId}) when ClientId =/= undefined -> case rabbit_mqtt_ff:track_client_id_in_ra() of true -> @@ -1336,9 +1389,9 @@ maybe_unregister_client(#proc_state{client_id = ClientId}) maybe_unregister_client(_) -> ok. -maybe_delete_mqtt_qos0_queue(PState = #proc_state{clean_sess = true, - auth_state = #auth_state{username = Username}}) -> - case get_queue(?QOS_0, PState) of +maybe_delete_mqtt_qos0_queue(State = #state{clean_sess = true, + auth_state = #auth_state{username = Username}}) -> + case get_queue(?QOS_0, State) of {ok, Q} -> %% double check we delete the right queue case {amqqueue:get_type(Q), amqqueue:get_pid(Q)} of @@ -1360,91 +1413,91 @@ handle_pre_hibernate() -> ok. handle_ra_event({applied, [{Corr, ok}]}, - PState = #proc_state{register_state = {pending, Corr}}) -> + State = #state{register_state = {pending, Corr}}) -> %% success case - command was applied transition into registered state - PState#proc_state{register_state = registered}; + State#state{register_state = registered}; handle_ra_event({not_leader, Leader, Corr}, - PState = #proc_state{register_state = {pending, Corr}, - client_id = ClientId}) -> + State = #state{register_state = {pending, Corr}, + client_id = ClientId}) -> case rabbit_mqtt_ff:track_client_id_in_ra() of true -> %% retry command against actual leader {ok, NewCorr} = rabbit_mqtt_collector:register(Leader, ClientId, self()), - PState#proc_state{register_state = {pending, NewCorr}}; + State#state{register_state = {pending, NewCorr}}; false -> - PState + State end; handle_ra_event(register_timeout, - PState = #proc_state{register_state = {pending, _Corr}, - client_id = ClientId}) -> + State = #state{register_state = {pending, _Corr}, + client_id = ClientId}) -> case rabbit_mqtt_ff:track_client_id_in_ra() of true -> {ok, NewCorr} = rabbit_mqtt_collector:register(ClientId, self()), - PState#proc_state{register_state = {pending, NewCorr}}; + State#state{register_state = {pending, NewCorr}}; false -> - PState + State end; -handle_ra_event(register_timeout, PState) -> - PState; -handle_ra_event(Evt, PState) -> +handle_ra_event(register_timeout, State) -> + State; +handle_ra_event(Evt, State) -> %% log these? rabbit_log:debug("unhandled ra_event: ~w ", [Evt]), - PState. + State. handle_down({{'DOWN', QName}, _MRef, process, QPid, Reason}, - PState0 = #proc_state{queue_states = QStates0, - unacked_client_pubs = U0}) -> + State0 = #state{queue_states = QStates0, + unacked_client_pubs = U0}) -> credit_flow:peer_down(QPid), case rabbit_queue_type:handle_down(QPid, QName, Reason, QStates0) of {ok, QStates1, Actions} -> - PState1 = PState0#proc_state{queue_states = QStates1}, - try handle_queue_actions(Actions, PState1) of - PState -> - {ok, PState} + State1 = State0#state{queue_states = QStates1}, + try handle_queue_actions(Actions, State1) of + State -> + {ok, State} catch throw:consuming_queue_down -> {error, consuming_queue_down} end; {eol, QStates1, QRef} -> {ConfirmMsgIds, U} = rabbit_mqtt_confirms:remove_queue(QRef, U0), QStates = rabbit_queue_type:remove(QRef, QStates1), - PState = PState0#proc_state{queue_states = QStates, - unacked_client_pubs = U}, - send_puback(ConfirmMsgIds, PState), - {ok, PState} + State = State0#state{queue_states = QStates, + unacked_client_pubs = U}, + send_puback(ConfirmMsgIds, State), + {ok, State} end. -handle_queue_event({queue_event, ?QUEUE_TYPE_QOS_0, Msg}, PState0) -> - PState = deliver_one_to_client(Msg, false, PState0), - {ok, PState}; +handle_queue_event({queue_event, ?QUEUE_TYPE_QOS_0, Msg}, State0) -> + State = deliver_one_to_client(Msg, false, State0), + {ok, State}; handle_queue_event({queue_event, QName, Evt}, - PState0 = #proc_state{queue_states = QStates0, - unacked_client_pubs = U0}) -> + State0 = #state{queue_states = QStates0, + unacked_client_pubs = U0}) -> case rabbit_queue_type:handle_event(QName, Evt, QStates0) of {ok, QStates, Actions} -> - PState1 = PState0#proc_state{queue_states = QStates}, - PState = handle_queue_actions(Actions, PState1), - {ok, PState}; + State1 = State0#state{queue_states = QStates}, + State = handle_queue_actions(Actions, State1), + {ok, State}; {eol, Actions} -> - PState1 = handle_queue_actions(Actions, PState0), + State1 = handle_queue_actions(Actions, State0), {ConfirmMsgIds, U} = rabbit_mqtt_confirms:remove_queue(QName, U0), QStates = rabbit_queue_type:remove(QName, QStates0), - PState = PState1#proc_state{queue_states = QStates, - unacked_client_pubs = U}, - send_puback(ConfirmMsgIds, PState), - {ok, PState}; + State = State1#state{queue_states = QStates, + unacked_client_pubs = U}, + send_puback(ConfirmMsgIds, State), + {ok, State}; {protocol_error, _Type, _Reason, _ReasonArgs} = Error -> - {error, Error, PState0} + {error, Error, State0} end. -handle_queue_actions(Actions, #proc_state{} = PState0) -> +handle_queue_actions(Actions, #state{} = State0) -> lists:foldl( fun ({deliver, ?CONSUMER_TAG, Ack, Msgs}, S) -> deliver_to_client(Msgs, Ack, S); - ({settled, QName, MsgIds}, S = #proc_state{unacked_client_pubs = U0}) -> + ({settled, QName, MsgIds}, S = #state{unacked_client_pubs = U0}) -> {ConfirmMsgIds, U} = rabbit_mqtt_confirms:confirm(MsgIds, QName, U0), send_puback(ConfirmMsgIds, S), - S#proc_state{unacked_client_pubs = U}; - ({rejected, _QName, MsgIds}, S = #proc_state{unacked_client_pubs = U0}) -> + S#state{unacked_client_pubs = U}; + ({rejected, _QName, MsgIds}, S = #state{unacked_client_pubs = U0}) -> %% Negative acks are supported in MQTT v5 only. %% Therefore, in MQTT v3 and v4 we ignore rejected messages. U = lists:foldl( @@ -1454,28 +1507,28 @@ handle_queue_actions(Actions, #proc_state{} = PState0) -> {error, not_found} -> Acc0 end end, U0, MsgIds), - S#proc_state{unacked_client_pubs = U}; - ({block, QName}, S = #proc_state{soft_limit_exceeded = SLE}) -> - S#proc_state{soft_limit_exceeded = sets:add_element(QName, SLE)}; - ({unblock, QName}, S = #proc_state{soft_limit_exceeded = SLE}) -> - S#proc_state{soft_limit_exceeded = sets:del_element(QName, SLE)}; + S#state{unacked_client_pubs = U}; + ({block, QName}, S = #state{soft_limit_exceeded = SLE}) -> + S#state{soft_limit_exceeded = sets:add_element(QName, SLE)}; + ({unblock, QName}, S = #state{soft_limit_exceeded = SLE}) -> + S#state{soft_limit_exceeded = sets:del_element(QName, SLE)}; ({queue_down, QName}, S) -> handle_queue_down(QName, S) - end, PState0, Actions). + end, State0, Actions). -handle_queue_down(QName, PState0 = #proc_state{client_id = ClientId}) -> +handle_queue_down(QName, State0 = #state{client_id = ClientId}) -> %% Classic queue is down. case rabbit_amqqueue:lookup(QName) of {ok, Q} -> case rabbit_mqtt_util:qos_from_queue_name(QName, ClientId) of no_consuming_queue -> - PState0; + State0; QoS -> %% Consuming classic queue is down. %% Let's try to re-consume: HA failover for classic mirrored queues. - case consume(Q, QoS, PState0) of - {ok, PState} -> - PState; + case consume(Q, QoS, State0) of + {ok, State} -> + State; {error, _Reason} -> rabbit_log:info("Terminating MQTT connection because consuming ~s is down.", [rabbit_misc:rs(QName)]), @@ -1483,17 +1536,17 @@ handle_queue_down(QName, PState0 = #proc_state{client_id = ClientId}) -> end end; {error, not_found} -> - PState0 + State0 end. -deliver_to_client(Msgs, Ack, PState) -> +deliver_to_client(Msgs, Ack, State) -> lists:foldl(fun(Msg, S) -> deliver_one_to_client(Msg, Ack, S) - end, PState, Msgs). + end, State, Msgs). deliver_one_to_client(Msg = {QName, QPid, QMsgId, _Redelivered, #basic_message{content = #content{properties = #'P_basic'{headers = Headers}}}}, - AckRequired, PState0) -> + AckRequired, State0) -> PublisherQoS = case rabbit_mqtt_util:table_lookup(Headers, <<"x-mqtt-publish-qos">>) of {byte, QoS0} -> QoS0; @@ -1508,15 +1561,15 @@ deliver_one_to_client(Msg = {QName, QPid, QMsgId, _Redelivered, ?QOS_0 end, QoS = effective_qos(PublisherQoS, SubscriberQoS), - PState1 = maybe_publish_to_client(Msg, QoS, SubscriberQoS, PState0), - PState = maybe_auto_ack(AckRequired, QoS, QName, QMsgId, PState1), + State1 = maybe_publish_to_client(Msg, QoS, SubscriberQoS, State0), + State = maybe_auto_ack(AckRequired, QoS, QName, QMsgId, State1), %%TODO GC % case GCThreshold of % undefined -> ok; % _ -> rabbit_basic:maybe_gc_large_msg(Content, GCThreshold) % end, - ok = maybe_notify_sent(QName, QPid, PState), - PState. + ok = maybe_notify_sent(QName, QPid, State), + State. -spec effective_qos(qos(), qos()) -> qos(). effective_qos(PublisherQoS, SubscriberQoS) -> @@ -1525,17 +1578,17 @@ effective_qos(PublisherQoS, SubscriberQoS) -> %% [MQTT-3.8.4-8]." erlang:min(PublisherQoS, SubscriberQoS). -maybe_publish_to_client({_, _, _, _Redelivered = true, _}, ?QOS_0, _, PState) -> +maybe_publish_to_client({_, _, _, _Redelivered = true, _}, ?QOS_0, _, State) -> %% Do not redeliver to MQTT subscriber who gets message at most once. - PState; + State; maybe_publish_to_client( {QName, _QPid, QMsgId, Redelivered, #basic_message{ routing_keys = [RoutingKey | _CcRoutes], content = #content{payload_fragments_rev = FragmentsRev}}}, - QoS, SubscriberQos, PState0 = #proc_state{amqp2mqtt_fun = Amqp2MqttFun, - send_fun = SendFun}) -> - {PacketId, PState} = queue_message_id_to_packet_id(QMsgId, QoS, PState0), + QoS, SubscriberQos, State0 = #state{amqp2mqtt_fun = Amqp2MqttFun, + send_fun = SendFun}) -> + {PacketId, State} = queue_message_id_to_packet_id(QMsgId, QoS, State0), %%TODO support iolists when sending to client Payload = list_to_binary(lists:reverse(FragmentsRev)), Frame = @@ -1554,17 +1607,17 @@ maybe_publish_to_client( message_id = PacketId, topic_name = Amqp2MqttFun(RoutingKey)}, payload = Payload}, - SendFun(Frame, PState), - message_delivered(QName, Redelivered, SubscriberQos, QoS, PState), - PState. + SendFun(Frame, State), + message_delivered(QName, Redelivered, SubscriberQos, QoS, State), + State. -queue_message_id_to_packet_id(_, ?QOS_0, PState) -> +queue_message_id_to_packet_id(_, ?QOS_0, State) -> %% "A PUBLISH packet MUST NOT contain a Packet Identifier if its QoS value is set to 0 [MQTT-2.2.1-2]." - {undefined, PState}; -queue_message_id_to_packet_id(QMsgId, ?QOS_1, #proc_state{packet_id = PktId, - unacked_server_pubs = U} = PState) -> - {PktId, PState#proc_state{packet_id = increment_packet_id(PktId), - unacked_server_pubs = maps:put(PktId, QMsgId, U)}}. + {undefined, State}; +queue_message_id_to_packet_id(QMsgId, ?QOS_1, #state{packet_id = PktId, + unacked_server_pubs = U} = State) -> + {PktId, State#state{packet_id = increment_packet_id(PktId), + unacked_server_pubs = maps:put(PktId, QMsgId, U)}}. -spec increment_packet_id(packet_id()) -> packet_id(). increment_packet_id(Id) @@ -1574,19 +1627,19 @@ increment_packet_id(Id) -> Id + 1. maybe_auto_ack(_AckRequired = true, ?QOS_0, QName, QMsgId, - PState = #proc_state{queue_states = QStates0}) -> + State = #state{queue_states = QStates0}) -> case rabbit_queue_type:settle(QName, complete, ?CONSUMER_TAG, [QMsgId], QStates0) of {ok, QStates, Actions} -> %%TODO rabbit_channel:incr_queue_stats/3 - handle_queue_actions(Actions, PState#proc_state{queue_states = QStates}); + handle_queue_actions(Actions, State#state{queue_states = QStates}); {protocol_error, _ErrorType, _Reason, _ReasonArgs} = Err -> %%TODO handle error throw(Err) end; -maybe_auto_ack(_, _, _, _, PState) -> - PState. +maybe_auto_ack(_, _, _, _, State) -> + State. -maybe_notify_sent(QName, QPid, #proc_state{queue_states = QStates}) -> +maybe_notify_sent(QName, QPid, #state{queue_states = QStates}) -> case rabbit_queue_type:module(QName, QStates) of {ok, rabbit_classic_queue} -> rabbit_amqqueue:notify_sent(QPid, self()); @@ -1596,19 +1649,19 @@ maybe_notify_sent(QName, QPid, #proc_state{queue_states = QStates}) -> publish_to_queues_with_checks( TopicName, PublishFun, - #proc_state{exchange = Exchange, - auth_state = #auth_state{user = User, - authz_ctx = AuthzCtx}} = PState) -> + #state{exchange = Exchange, + auth_state = #auth_state{user = User, + authz_ctx = AuthzCtx}} = State) -> case check_resource_access(User, Exchange, write, AuthzCtx) of ok -> - case check_topic_access(TopicName, write, PState) of + case check_topic_access(TopicName, write, State) of ok -> PublishFun(); {error, access_refused} -> - {error, unauthorized, PState} + {error, unauthorized, State} end; {error, access_refused} -> - {error, unauthorized, PState} + {error, unauthorized, State} end. check_resource_access(User, Resource, Perm, Context) -> @@ -1633,7 +1686,7 @@ check_resource_access(User, Resource, Perm, Context) -> end. check_topic_access(TopicName, Access, - #proc_state{ + #state{ auth_state = #auth_state{user = User = #user{username = Username}, vhost = VHost, authz_ctx = AuthzCtx}, @@ -1667,24 +1720,24 @@ check_topic_access(TopicName, Access, end end. -info(protocol, #proc_state{info = #info{proto_human = Val}}) -> Val; -info(host, #proc_state{info = #info{host = Val}}) -> Val; -info(port, #proc_state{info = #info{port = Val}}) -> Val; -info(peer_host, #proc_state{info = #info{peer_host = Val}}) -> Val; -info(peer_port, #proc_state{info = #info{peer_port = Val}}) -> Val; -info(ssl_login_name, #proc_state{ssl_login_name = Val}) -> Val; -info(client_id, #proc_state{client_id = Val}) -> +info(protocol, #state{info = #info{proto_human = Val}}) -> Val; +info(host, #state{info = #info{host = Val}}) -> Val; +info(port, #state{info = #info{port = Val}}) -> Val; +info(peer_host, #state{info = #info{peer_host = Val}}) -> Val; +info(peer_port, #state{info = #info{peer_port = Val}}) -> Val; +info(ssl_login_name, #state{ssl_login_name = Val}) -> Val; +info(client_id, #state{client_id = Val}) -> rabbit_data_coercion:to_binary(Val); -info(vhost, #proc_state{auth_state = #auth_state{vhost = Val}}) -> Val; -info(user, #proc_state{auth_state = #auth_state{username = Val}}) -> Val; -info(clean_sess, #proc_state{clean_sess = Val}) -> Val; -info(will_msg, #proc_state{will_msg = Val}) -> Val; -info(retainer_pid, #proc_state{retainer_pid = Val}) -> Val; -info(exchange, #proc_state{exchange = #resource{name = Val}}) -> Val; -info(prefetch, #proc_state{info = #info{prefetch = Val}}) -> Val; -info(messages_unconfirmed, #proc_state{unacked_client_pubs = Val}) -> +info(vhost, #state{auth_state = #auth_state{vhost = Val}}) -> Val; +info(user, #state{auth_state = #auth_state{username = Val}}) -> Val; +info(clean_sess, #state{clean_sess = Val}) -> Val; +info(will_msg, #state{will_msg = Val}) -> Val; +info(retainer_pid, #state{retainer_pid = Val}) -> Val; +info(exchange, #state{exchange = #resource{name = Val}}) -> Val; +info(prefetch, #state{info = #info{prefetch = Val}}) -> Val; +info(messages_unconfirmed, #state{unacked_client_pubs = Val}) -> rabbit_mqtt_confirms:size(Val); -info(messages_unacknowledged, #proc_state{unacked_server_pubs = Val}) -> +info(messages_unacknowledged, #state{unacked_server_pubs = Val}) -> maps:size(Val); info(Other, _) -> throw({bad_argument, Other}). @@ -1701,23 +1754,23 @@ ssl_login_name(Sock) -> nossl -> none end. -format_status(#proc_state{queue_states = QState, - proto_ver = ProtoVersion, - unacked_client_pubs = UnackClientPubs, - unacked_server_pubs = UnackSerPubs, - packet_id = PackID, - client_id = ClientID, - clean_sess = CleanSess, - will_msg = WillMsg, - exchange = Exchange, - ssl_login_name = SSLLoginName, - retainer_pid = RetainerPid, - auth_state = AuthState, - peer_addr = PeerAddr, - register_state = RegisterState, - conn_name = ConnName, - info = Info - } = PState) -> +format_status(#state{queue_states = QState, + proto_ver = ProtoVersion, + unacked_client_pubs = UnackClientPubs, + unacked_server_pubs = UnackSerPubs, + packet_id = PackID, + client_id = ClientID, + clean_sess = CleanSess, + will_msg = WillMsg, + exchange = Exchange, + ssl_login_name = SSLLoginName, + retainer_pid = RetainerPid, + auth_state = AuthState, + peer_addr = PeerAddr, + register_state = RegisterState, + conn_name = ConnName, + info = Info + } = State) -> #{queue_states => rabbit_queue_type:format_status(QState), proto_ver => ProtoVersion, unacked_client_pubs => UnackClientPubs, @@ -1734,9 +1787,9 @@ format_status(#proc_state{queue_states = QState, register_state => RegisterState, conn_name => ConnName, info => Info, - soft_limit_exceeded => soft_limit_exceeded(PState)}. + soft_limit_exceeded => soft_limit_exceeded(State)}. -soft_limit_exceeded(#proc_state{soft_limit_exceeded = SLE}) -> +soft_limit_exceeded(#state{soft_limit_exceeded = SLE}) -> not sets:is_empty(SLE). protocol_integer_to_atom(3) -> @@ -1744,23 +1797,23 @@ protocol_integer_to_atom(3) -> protocol_integer_to_atom(4) -> ?MQTT_PROTO_V4. -maybe_increment_publisher(PState = #proc_state{has_published = false, - proto_ver = ProtoVer}) -> +maybe_increment_publisher(State = #state{has_published = false, + proto_ver = ProtoVer}) -> rabbit_global_counters:publisher_created(ProtoVer), - PState#proc_state{has_published = true}; -maybe_increment_publisher(PState) -> - PState. + State#state{has_published = true}; +maybe_increment_publisher(State) -> + State. -maybe_decrement_publisher(#proc_state{has_published = true, - proto_ver = ProtoVer}) -> +maybe_decrement_publisher(#state{has_published = true, + proto_ver = ProtoVer}) -> rabbit_global_counters:publisher_deleted(ProtoVer); maybe_decrement_publisher(_) -> ok. %% multiple subscriptions from the same connection count as one consumer maybe_increment_consumer(_WasConsumer = false, - #proc_state{proto_ver = ProtoVer} = PState) -> - case has_subs(PState) of + #state{proto_ver = ProtoVer} = State) -> + case has_subs(State) of true -> rabbit_global_counters:consumer_created(ProtoVer); false -> @@ -1770,8 +1823,8 @@ maybe_increment_consumer(_, _) -> ok. maybe_decrement_consumer(_WasConsumer = true, - #proc_state{proto_ver = ProtoVer} = PState) -> - case has_subs(PState) of + #state{proto_ver = ProtoVer} = State) -> + case has_subs(State) of false -> rabbit_global_counters:consumer_deleted(ProtoVer); true -> @@ -1780,9 +1833,9 @@ maybe_decrement_consumer(_WasConsumer = true, maybe_decrement_consumer(_, _) -> ok. -maybe_decrement_consumer(#proc_state{proto_ver = ProtoVer, - auth_state = #auth_state{}} = PState) -> - case has_subs(PState) of +maybe_decrement_consumer(#state{proto_ver = ProtoVer, + auth_state = #auth_state{}} = State) -> + case has_subs(State) of true -> rabbit_global_counters:consumer_deleted(ProtoVer); false -> @@ -1791,8 +1844,8 @@ maybe_decrement_consumer(#proc_state{proto_ver = ProtoVer, maybe_decrement_consumer(_) -> ok. -message_acknowledged(QName, #proc_state{proto_ver = ProtoVer, - queue_states = QStates}) -> +message_acknowledged(QName, #state{proto_ver = ProtoVer, + queue_states = QStates}) -> case rabbit_queue_type:module(QName, QStates) of {ok, QType} -> rabbit_global_counters:messages_acknowledged(ProtoVer, QType, 1); @@ -1801,9 +1854,9 @@ message_acknowledged(QName, #proc_state{proto_ver = ProtoVer, end. message_delivered(_, _Redelivered = false, _SubscriberQoS = ?QOS_0, _QoS = ?QOS_0, - #proc_state{clean_sess = true, - proto_ver = ProtoVer - }) -> + #state{clean_sess = true, + proto_ver = ProtoVer + }) -> rabbit_global_counters:messages_delivered(ProtoVer, ?QUEUE_TYPE_QOS_0, 1), %% Technically, the message is not acked to a queue at all. %% However, from a user perspective it is still auto acked because: @@ -1811,9 +1864,9 @@ message_delivered(_, _Redelivered = false, _SubscriberQoS = ?QOS_0, _QoS = ?QOS_ %% delivered immediately after it is sent." rabbit_global_counters:messages_delivered_consume_auto_ack(ProtoVer, ?QUEUE_TYPE_QOS_0, 1); message_delivered(QName, Redelivered, _, QoS, - #proc_state{proto_ver = ProtoVer, - queue_states = QStates - }) -> + #state{proto_ver = ProtoVer, + queue_states = QStates + }) -> case rabbit_queue_type:module(QName, QStates) of {ok, QType} -> rabbit_global_counters:messages_delivered(ProtoVer, QType, 1), diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index 338b70e1a01f..b7e82ba9567e 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -25,6 +25,20 @@ -define(OTHER_METRICS, [recv_cnt, send_cnt, send_pend, garbage_collection, state]). -define(HIBERNATE_AFTER, 1000). +-record(state, + {socket, + proxy_socket, + conn_name, + await_recv, + deferred_recv, + received_connect_frame, + connection_state, + conserve, + parse_state, + proc_state, + stats_timer, + keepalive :: rabbit_mqtt_keepalive:state()}). + %%---------------------------------------------------------------------------- start_link(Ref, _Transport, []) -> @@ -344,7 +358,9 @@ process_received_bytes(Bytes, {stop, {shutdown, Error}, State} end. -pstate(State = #state {}, PState = #proc_state{}) -> +-spec pstate(#state{}, rabbit_mqtt_processor:state()) + -> #state{}. +pstate(State = #state {}, PState) -> State #state{ proc_state = PState }. %%---------------------------------------------------------------------------- diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store.erl index cbe5752884b5..5c016eb87741 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store.erl @@ -22,13 +22,14 @@ behaviour_info(_Other) -> table_name_for(VHost) -> rabbit_mqtt_util:vhost_name_to_table_name(VHost). -%% TODO -%% Support retained messages in RabbitMQ cluster: -%% * SUBSCRIBE on a different node than PUBLISH with retain -%% * replicate retained message for data safety +%% TODO Support retained messages in RabbitMQ cluster, for +%% 1. support PUBLISH with retain on a different node than SUBSCRIBE +%% 2. replicate retained message for data safety %% -%% Possible solutions: +%% Possible solution for 1. +%% * retained message store backend does RPCs to peer nodes to lookup and delete +%% +%% Possible solutions for 2. %% * rabbitmq_mqtt_retained_msg_store_mnesia %% * rabbitmq_mqtt_retained_msg_store_khepri -%% * rabbitmq_mqtt_retained_msg_store_ra (implementing our own ra machine) or -%% * use existing mqtt_machine +%% * rabbitmq_mqtt_retained_msg_store_ra (implementing our own ra machine) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_dets.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_dets.erl index f449b1e8e45e..9900129bd634 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_dets.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_dets.erl @@ -8,7 +8,7 @@ -module(rabbit_mqtt_retained_msg_store_dets). -behaviour(rabbit_mqtt_retained_msg_store). --include("rabbit_mqtt.hrl"). +-include("rabbit_mqtt_retain.hrl"). -export([new/2, recover/2, insert/3, lookup/2, delete/2, terminate/1]). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_ets.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_ets.erl index b427750cc178..93e8305123c0 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_ets.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_ets.erl @@ -8,7 +8,7 @@ -module(rabbit_mqtt_retained_msg_store_ets). -behaviour(rabbit_mqtt_retained_msg_store). --include("rabbit_mqtt.hrl"). +-include("rabbit_mqtt_retain.hrl"). -export([new/2, recover/2, insert/3, lookup/2, delete/2, terminate/1]). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer.erl index baae37ea6b80..3e903b7af0b8 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer.erl @@ -8,7 +8,7 @@ -module(rabbit_mqtt_retainer). -behaviour(gen_server2). --include("rabbit_mqtt.hrl"). +-include("rabbit_mqtt_retain.hrl"). -include("rabbit_mqtt_frame.hrl"). -export([init/1, handle_call/3, handle_cast/2, handle_info/2, diff --git a/deps/rabbitmq_mqtt/test/util.erl b/deps/rabbitmq_mqtt/test/util.erl index f31786bfd6b8..73283d90e618 100644 --- a/deps/rabbitmq_mqtt/test/util.erl +++ b/deps/rabbitmq_mqtt/test/util.erl @@ -1,6 +1,6 @@ -module(util). --include("rabbit_mqtt_frame.hrl"). +-include("rabbit_mqtt.hrl"). -export([all_connection_pids/1, publish_qos1_timeout/4, From 1493cbe13d7288c926f0df84f0ddb091e63673e0 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Mon, 28 Nov 2022 12:21:57 +0000 Subject: [PATCH 064/118] Rename message_id to packet_id MQTT spec only talks about "Packet Identifier", but never about "Message Identitier". RabbitMQ has message identifiers (for example the classic queue store uses message identifiers to uniquely identify internal messages). So, let's not confuse these two terms and be specific. --- .../include/rabbit_mqtt_frame.hrl | 8 +- deps/rabbitmq_mqtt/src/rabbit_mqtt_frame.erl | 30 ++++---- .../src/rabbit_mqtt_processor.erl | 74 +++++++++---------- 3 files changed, 56 insertions(+), 56 deletions(-) diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt_frame.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt_frame.hrl index d5b0c6849289..70af22e25dc7 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt_frame.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt_frame.hrl @@ -78,16 +78,16 @@ return_code}). -record(mqtt_frame_publish, {topic_name, - message_id :: packet_id()}). + packet_id :: packet_id()}). -record(mqtt_topic, {name, qos}). --record(mqtt_frame_subscribe,{message_id :: packet_id(), +-record(mqtt_frame_subscribe,{packet_id :: packet_id(), topic_table :: nonempty_list(#mqtt_topic{}) }). --record(mqtt_frame_suback, {message_id :: packet_id(), +-record(mqtt_frame_suback, {packet_id :: packet_id(), qos_table = []}). -record(mqtt_frame_other, {other}). @@ -96,7 +96,7 @@ qos :: qos(), topic :: string(), dup :: boolean(), - message_id :: packet_id(), + packet_id :: packet_id(), payload :: binary()}). -type mqtt_msg() :: #mqtt_msg{}. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_frame.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_frame.erl index e1a0c3e4f2fa..accf9d9fcf08 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_frame.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_frame.erl @@ -84,23 +84,23 @@ parse_frame(Bin, #mqtt_frame_fixed{ type = Type, end; {?PUBLISH, <>} -> {TopicName, Rest1} = parse_utf(FrameBin), - {MessageId, Payload} = case Qos of + {PacketId, Payload} = case Qos of 0 -> {undefined, Rest1}; _ -> <> = Rest1, {M, R} end, wrap(Fixed, #mqtt_frame_publish { topic_name = TopicName, - message_id = MessageId }, + packet_id = PacketId }, Payload, Rest); {?PUBACK, <>} -> - <> = FrameBin, - wrap(Fixed, #mqtt_frame_publish { message_id = MessageId }, Rest); + <> = FrameBin, + wrap(Fixed, #mqtt_frame_publish { packet_id = PacketId }, Rest); {Subs, <>} when Subs =:= ?SUBSCRIBE orelse Subs =:= ?UNSUBSCRIBE -> 1 = Qos, - <> = FrameBin, + <> = FrameBin, Topics = parse_topics(Subs, Rest1, []), - wrap(Fixed, #mqtt_frame_subscribe { message_id = MessageId, + wrap(Fixed, #mqtt_frame_subscribe { packet_id = PacketId, topic_table = Topics }, Rest); {Minimal, Rest} when Minimal =:= ?DISCONNECT orelse Minimal =:= ?PINGREQ -> @@ -166,11 +166,11 @@ serialise_variable(#mqtt_frame_fixed { type = ?CONNACK } = Fixed, serialise_fixed(Fixed, VariableBin, PayloadBin); serialise_variable(#mqtt_frame_fixed { type = SubAck } = Fixed, - #mqtt_frame_suback { message_id = MessageId, + #mqtt_frame_suback { packet_id = PacketId, qos_table = Qos }, <<>> = _PayloadBin, Vsn) when SubAck =:= ?SUBACK orelse SubAck =:= ?UNSUBACK -> - VariableBin = <>, + VariableBin = <>, QosBin = case Vsn of ?MQTT_PROTO_V3 -> << <> || Q <- Qos >>; @@ -183,20 +183,20 @@ serialise_variable(#mqtt_frame_fixed { type = SubAck } = Fixed, serialise_variable(#mqtt_frame_fixed { type = ?PUBLISH, qos = Qos } = Fixed, #mqtt_frame_publish { topic_name = TopicName, - message_id = MessageId }, + packet_id = PacketId }, PayloadBin, _Vsn) -> TopicBin = serialise_utf(TopicName), - MessageIdBin = case Qos of + PacketIdBin = case Qos of 0 -> <<>>; - 1 -> <> + 1 -> <> end, - serialise_fixed(Fixed, <>, PayloadBin); + serialise_fixed(Fixed, <>, PayloadBin); serialise_variable(#mqtt_frame_fixed { type = ?PUBACK } = Fixed, - #mqtt_frame_publish { message_id = MessageId }, + #mqtt_frame_publish { packet_id = PacketId }, PayloadBin, _Vsn) -> - MessageIdBin = <>, - serialise_fixed(Fixed, MessageIdBin, PayloadBin); + PacketIdBin = <>, + serialise_fixed(Fixed, PacketIdBin, PayloadBin); serialise_variable(#mqtt_frame_fixed {} = Fixed, undefined, diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 1197c70257bb..dc4040fa914d 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -121,7 +121,7 @@ process_request(?CONNECT, Frame, State = #state{socket = Socket}) -> process_request(?PUBACK, #mqtt_frame{ - variable = #mqtt_frame_publish{message_id = PacketId}}, + variable = #mqtt_frame_publish{packet_id = PacketId}}, #state{unacked_server_pubs = U0, queue_states = QStates0, queue_qos1 = QName} = State) -> @@ -155,7 +155,7 @@ process_request(?PUBLISH, retain = Retain, dup = Dup }, variable = #mqtt_frame_publish{topic_name = Topic, - message_id = MessageId }, + packet_id = PacketId }, payload = Payload}, State0 = #state{retainer_pid = RPid, amqp2mqtt_fun = Amqp2MqttFun, @@ -168,7 +168,7 @@ process_request(?PUBLISH, qos = Qos, topic = Topic, dup = Dup, - message_id = MessageId, + packet_id = PacketId, payload = Payload}, case publish_to_queues(Msg, State) of {ok, _} = Ok -> @@ -186,7 +186,7 @@ process_request(?PUBLISH, case Qos of N when N > ?QOS_0 -> rabbit_global_counters:messages_received_confirm(ProtoVer, 1), - case rabbit_mqtt_confirms:contains(MessageId, U) of + case rabbit_mqtt_confirms:contains(PacketId, U) of false -> publish_to_queues_with_checks(Topic, Publish, State); true -> @@ -202,7 +202,7 @@ process_request(?PUBLISH, process_request(?SUBSCRIBE, #mqtt_frame{ variable = #mqtt_frame_subscribe{ - message_id = SubscribeMsgId, + packet_id = SubscribePktId, topic_table = Topics}, payload = undefined}, #state{send_fun = SendFun, @@ -258,7 +258,7 @@ process_request(?SUBSCRIBE, SendFun( #mqtt_frame{fixed = #mqtt_frame_fixed{type = ?SUBACK}, variable = #mqtt_frame_suback{ - message_id = SubscribeMsgId, + packet_id = SubscribePktId, qos_table = QosResponse}}, State1), case QosResponse of @@ -272,7 +272,7 @@ process_request(?SUBSCRIBE, end; process_request(?UNSUBSCRIBE, - #mqtt_frame{variable = #mqtt_frame_subscribe{message_id = MessageId, + #mqtt_frame{variable = #mqtt_frame_subscribe{packet_id = PacketId, topic_table = Topics}, payload = undefined}, State0 = #state{send_fun = SendFun}) -> @@ -296,7 +296,7 @@ process_request(?UNSUBSCRIBE, end, State0, Topics), SendFun( #mqtt_frame{fixed = #mqtt_frame_fixed {type = ?UNSUBACK}, - variable = #mqtt_frame_suback{message_id = MessageId}}, + variable = #mqtt_frame_suback{packet_id = PacketId}}, State), maybe_decrement_consumer(HasSubsBefore, State), @@ -674,7 +674,7 @@ maybe_send_retained_message(RPid, #mqtt_topic{name = Topic0, qos = SubscribeQos} dup = false, retain = Msg#mqtt_msg.retain }, variable = #mqtt_frame_publish{ - message_id = PacketId, + packet_id = PacketId, topic_name = Topic1 }, payload = Msg#mqtt_msg.payload}, @@ -1201,7 +1201,7 @@ publish_to_queues( #mqtt_msg{qos = Qos, topic = Topic, dup = Dup, - message_id = MessageId, + packet_id = PacketId, payload = Payload}, #state{exchange = ExchangeName, mqtt2amqp_fun = Mqtt2AmqpFun, @@ -1233,7 +1233,7 @@ publish_to_queues( confirm = Confirm, sender = self(), message = BasicMessage, - msg_seq_no = MessageId, + msg_seq_no = PacketId, flow = Flow }, case rabbit_exchange:lookup(ExchangeName) of @@ -1278,13 +1278,13 @@ process_routing_confirm(#delivery{confirm = true, rabbit_global_counters:messages_unroutable_dropped(ProtoVer, 1), State; process_routing_confirm(#delivery{confirm = true, - msg_seq_no = MsgId}, + msg_seq_no = PktId}, [], State = #state{proto_ver = ProtoVer}) -> rabbit_global_counters:messages_unroutable_returned(ProtoVer, 1), %% MQTT 5 spec: %% If the Server knows that there are no matching subscribers, it MAY use %% Reason Code 0x10 (No matching subscribers) instead of 0x00 (Success). - send_puback(MsgId, State), + send_puback(PktId, State), State; process_routing_confirm(#delivery{confirm = false}, _, State) -> State; @@ -1293,26 +1293,26 @@ process_routing_confirm(#delivery{confirm = true, %% routable will message with QoS > 0 State; process_routing_confirm(#delivery{confirm = true, - msg_seq_no = MsgId}, + msg_seq_no = PktId}, Qs, State = #state{unacked_client_pubs = U0}) -> QNames = lists:map(fun amqqueue:get_name/1, Qs), - U = rabbit_mqtt_confirms:insert(MsgId, QNames, U0), + U = rabbit_mqtt_confirms:insert(PktId, QNames, U0), State#state{unacked_client_pubs = U}. -send_puback(MsgIds0, State) - when is_list(MsgIds0) -> +send_puback(PktIds0, State) + when is_list(PktIds0) -> %% Classic queues confirm messages unordered. %% Let's sort them here assuming most MQTT clients send with an increasing packet identifier. - MsgIds = lists:usort(MsgIds0), + PktIds = lists:usort(PktIds0), lists:foreach(fun(Id) -> send_puback(Id, State) - end, MsgIds); -send_puback(MsgId, State = #state{send_fun = SendFun, + end, PktIds); +send_puback(PktId, State = #state{send_fun = SendFun, proto_ver = ProtoVer}) -> rabbit_global_counters:messages_confirmed(ProtoVer, 1), SendFun( #mqtt_frame{fixed = #mqtt_frame_fixed{type = ?PUBACK}, - variable = #mqtt_frame_publish{message_id = MsgId}}, + variable = #mqtt_frame_publish{packet_id = PktId}}, State). serialise_and_send_to_client(Frame, #state{proto_ver = ProtoVer, socket = Sock}) -> @@ -1458,11 +1458,11 @@ handle_down({{'DOWN', QName}, _MRef, process, QPid, Reason}, {error, consuming_queue_down} end; {eol, QStates1, QRef} -> - {ConfirmMsgIds, U} = rabbit_mqtt_confirms:remove_queue(QRef, U0), + {ConfirmPktIds, U} = rabbit_mqtt_confirms:remove_queue(QRef, U0), QStates = rabbit_queue_type:remove(QRef, QStates1), State = State0#state{queue_states = QStates, unacked_client_pubs = U}, - send_puback(ConfirmMsgIds, State), + send_puback(ConfirmPktIds, State), {ok, State} end. @@ -1479,11 +1479,11 @@ handle_queue_event({queue_event, QName, Evt}, {ok, State}; {eol, Actions} -> State1 = handle_queue_actions(Actions, State0), - {ConfirmMsgIds, U} = rabbit_mqtt_confirms:remove_queue(QName, U0), + {ConfirmPktIds, U} = rabbit_mqtt_confirms:remove_queue(QName, U0), QStates = rabbit_queue_type:remove(QName, QStates0), State = State1#state{queue_states = QStates, unacked_client_pubs = U}, - send_puback(ConfirmMsgIds, State), + send_puback(ConfirmPktIds, State), {ok, State}; {protocol_error, _Type, _Reason, _ReasonArgs} = Error -> {error, Error, State0} @@ -1493,20 +1493,20 @@ handle_queue_actions(Actions, #state{} = State0) -> lists:foldl( fun ({deliver, ?CONSUMER_TAG, Ack, Msgs}, S) -> deliver_to_client(Msgs, Ack, S); - ({settled, QName, MsgIds}, S = #state{unacked_client_pubs = U0}) -> - {ConfirmMsgIds, U} = rabbit_mqtt_confirms:confirm(MsgIds, QName, U0), - send_puback(ConfirmMsgIds, S), + ({settled, QName, PktIds}, S = #state{unacked_client_pubs = U0}) -> + {ConfirmPktIds, U} = rabbit_mqtt_confirms:confirm(PktIds, QName, U0), + send_puback(ConfirmPktIds, S), S#state{unacked_client_pubs = U}; - ({rejected, _QName, MsgIds}, S = #state{unacked_client_pubs = U0}) -> + ({rejected, _QName, PktIds}, S = #state{unacked_client_pubs = U0}) -> %% Negative acks are supported in MQTT v5 only. %% Therefore, in MQTT v3 and v4 we ignore rejected messages. U = lists:foldl( - fun(MsgId, Acc0) -> - case rabbit_mqtt_confirms:reject(MsgId, Acc0) of + fun(PktId, Acc0) -> + case rabbit_mqtt_confirms:reject(PktId, Acc0) of {ok, Acc} -> Acc; {error, not_found} -> Acc0 end - end, U0, MsgIds), + end, U0, PktIds), S#state{unacked_client_pubs = U}; ({block, QName}, S = #state{soft_limit_exceeded = SLE}) -> S#state{soft_limit_exceeded = sets:add_element(QName, SLE)}; @@ -1588,7 +1588,7 @@ maybe_publish_to_client( content = #content{payload_fragments_rev = FragmentsRev}}}, QoS, SubscriberQos, State0 = #state{amqp2mqtt_fun = Amqp2MqttFun, send_fun = SendFun}) -> - {PacketId, State} = queue_message_id_to_packet_id(QMsgId, QoS, State0), + {PacketId, State} = queue_packet_id_to_packet_id(QMsgId, QoS, State0), %%TODO support iolists when sending to client Payload = list_to_binary(lists:reverse(FragmentsRev)), Frame = @@ -1604,18 +1604,18 @@ maybe_publish_to_client( %% Therefore, we do not consider header value <<"x-mqtt-dup">> here. dup = Redelivered}, variable = #mqtt_frame_publish{ - message_id = PacketId, + packet_id = PacketId, topic_name = Amqp2MqttFun(RoutingKey)}, payload = Payload}, SendFun(Frame, State), message_delivered(QName, Redelivered, SubscriberQos, QoS, State), State. -queue_message_id_to_packet_id(_, ?QOS_0, State) -> +queue_packet_id_to_packet_id(_, ?QOS_0, State) -> %% "A PUBLISH packet MUST NOT contain a Packet Identifier if its QoS value is set to 0 [MQTT-2.2.1-2]." {undefined, State}; -queue_message_id_to_packet_id(QMsgId, ?QOS_1, #state{packet_id = PktId, - unacked_server_pubs = U} = State) -> +queue_packet_id_to_packet_id(QMsgId, ?QOS_1, #state{packet_id = PktId, + unacked_server_pubs = U} = State) -> {PktId, State#state{packet_id = increment_packet_id(PktId), unacked_server_pubs = maps:put(PktId, QMsgId, U)}}. From 6605a16830188b9266e424ab6df1c36465ba86ff Mon Sep 17 00:00:00 2001 From: David Ansari Date: Mon, 28 Nov 2022 16:58:16 +0000 Subject: [PATCH 065/118] Fix rabbitmq_web_mqtt:system_SUITE-mixed --- deps/rabbitmq_web_mqtt/test/system_SUITE.erl | 1 + 1 file changed, 1 insertion(+) diff --git a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl index 896521be6294..e393758d92fd 100644 --- a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl +++ b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl @@ -119,6 +119,7 @@ last_will_enabled_no_disconnect(Config) -> {will_payload, LastWillMsg}, {will_qos, 1}], _Publisher = ws_connect(<<(atom_to_binary(?FUNCTION_NAME))/binary, "_publisher">>, Config, PubOpts), + timer:sleep(100), [ServerPublisherPid] = rpc(Config, 0, rabbit_mqtt, local_connection_pids, []), Consumer = ws_connect(<<(atom_to_binary(?FUNCTION_NAME))/binary, "_consumer">>, Config), {ok, _, [1]} = emqtt:subscribe(Consumer, LastWillTopic, qos1), From 30a9ea521ee057e94899f7d3474213dceb893cb3 Mon Sep 17 00:00:00 2001 From: Chunyi Lyu Date: Tue, 29 Nov 2022 11:41:50 +0000 Subject: [PATCH 066/118] Use connect helper func in more mqtt tests - reduce code duplication - connect helper does not unlink the connection process by default --- deps/rabbitmq_mqtt/BUILD.bazel | 3 + deps/rabbitmq_mqtt/test/command_SUITE.erl | 20 +---- deps/rabbitmq_mqtt/test/integration_SUITE.erl | 59 +++++--------- deps/rabbitmq_mqtt/test/reader_SUITE.erl | 80 ++++--------------- deps/rabbitmq_mqtt/test/retainer_SUITE.erl | 30 ++----- deps/rabbitmq_mqtt/test/util.erl | 18 ++++- 6 files changed, 69 insertions(+), 141 deletions(-) diff --git a/deps/rabbitmq_mqtt/BUILD.bazel b/deps/rabbitmq_mqtt/BUILD.bazel index 77183b212b74..51a61885c395 100644 --- a/deps/rabbitmq_mqtt/BUILD.bazel +++ b/deps/rabbitmq_mqtt/BUILD.bazel @@ -142,6 +142,9 @@ rabbitmq_integration_suite( runtime_deps = [ "@emqtt//:erlang_app", ], + additional_beam = [ + ":util", + ], ) rabbitmq_integration_suite( diff --git a/deps/rabbitmq_mqtt/test/command_SUITE.erl b/deps/rabbitmq_mqtt/test/command_SUITE.erl index 78ef887888ad..cae8df7e4233 100644 --- a/deps/rabbitmq_mqtt/test/command_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/command_SUITE.erl @@ -11,6 +11,7 @@ -include_lib("eunit/include/eunit.hrl"). -include_lib("amqp_client/include/amqp_client.hrl"). -include("rabbit_mqtt.hrl"). +-import(util, [connect/3]). -define(COMMAND, 'Elixir.RabbitMQ.CLI.Ctl.Commands.ListMqttConnectionsCommand'). @@ -80,27 +81,14 @@ run(Config) -> %% No connections [] = 'Elixir.Enum':to_list(?COMMAND:run([], Opts)), - P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), - {ok, C1} = emqtt:start_link([{host, "localhost"}, - {port, P}, - {clientid, <<"simpleClient">>}, - {proto_ver, v4}, - {ack_timeout, 1}]), - {ok, _} = emqtt:connect(C1), + C1 = connect(<<"simpleClient">>, Config, [{ack_timeout, 1}]), + timer:sleep(100), [[{client_id, <<"simpleClient">>}]] = 'Elixir.Enum':to_list(?COMMAND:run([<<"client_id">>], Opts)), - - {ok, C2} = emqtt:start_link([{host, "localhost"}, - {port, P}, - {clientid, <<"simpleClient1">>}, - {proto_ver, v4}, - {username, <<"guest">>}, - {password, <<"guest">>}, - {ack_timeout, 1}]), - {ok, _} = emqtt:connect(C2), + C2 = connect(<<"simpleClient1">>, Config, [{ack_timeout, 1}]), timer:sleep(200), [[{client_id, <<"simpleClient">>}, {user, <<"guest">>}], diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/integration_SUITE.erl index 3bbeaff96789..a62224241945 100644 --- a/deps/rabbitmq_mqtt/test/integration_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/integration_SUITE.erl @@ -21,7 +21,9 @@ get_global_counters/2, get_global_counters/3, get_global_counters/4, - expect_publishes/2]). + expect_publishes/2, + connect/2, + connect/3]). all() -> [ @@ -128,7 +130,7 @@ quorum_queue_rejects(Config) -> declare_queue(Ch, Name, [{<<"x-queue-type">>, longstr, <<"quorum">>}]), bind(Ch, Name, Name), - {C, _} = connect(Name, Config, [{retry_interval, 1}]), + C = connect(Name, Config, [{retry_interval, 1}]), {ok, _} = emqtt:publish(C, Name, <<"m1">>, qos1), {ok, _} = emqtt:publish(C, Name, <<"m2">>, qos1), %% We expect m3 to be rejected and dropped. @@ -176,7 +178,7 @@ publish_to_all_queue_types(Config, QoS) -> bind(Ch, SQ, Topic), NumMsgs = 2000, - {C, _} = connect(?FUNCTION_NAME, Config, [{retry_interval, 2}]), + C = connect(?FUNCTION_NAME, Config, [{retry_interval, 2}]), lists:foreach(fun(N) -> case emqtt:publish(C, Topic, integer_to_binary(N), QoS) of ok -> @@ -237,7 +239,7 @@ flow(Config, {App, Par, Val}, QueueType) bind(Ch, QueueName, Topic), NumMsgs = 1000, - {C, _} = connect(?FUNCTION_NAME, Config, [{retry_interval, 600}, + C = connect(?FUNCTION_NAME, Config, [{retry_interval, 600}, {max_inflight, NumMsgs}]), TestPid = self(), lists:foreach( @@ -268,7 +270,7 @@ events(Config) -> ok = gen_event:add_handler({rabbit_event, Server}, event_recorder, []), ClientId = atom_to_binary(?FUNCTION_NAME), - {C, _} = connect(ClientId, Config), + C = connect(ClientId, Config), [E0, E1] = get_events(Server), assert_event_type(user_authentication_success, E0), @@ -357,12 +359,7 @@ global_counters_v4(Config) -> global_counters(Config, v4). global_counters(Config, ProtoVer) -> - Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), - {ok, C} = emqtt:start_link([{host, "localhost"}, - {port, Port}, - {proto_ver, ProtoVer}, - {clientid, atom_to_binary(?FUNCTION_NAME)}]), - {ok, _Properties} = emqtt:connect(C), + C = connect(?FUNCTION_NAME, Config, [{proto_ver, ProtoVer}]), Topic0 = <<"test-topic0">>, Topic1 = <<"test-topic1">>, @@ -432,7 +429,8 @@ queue_down_qos1(Config) -> ok = rabbit_ct_client_helpers:close_connection_and_channel(Conn1, Ch1), ok = rabbit_ct_broker_helpers:stop_node(Config, 1), - {C, _} = connect(?FUNCTION_NAME, Config, [{retry_interval, 2}]), + C = connect(?FUNCTION_NAME, Config, [{retry_interval, 2}]), + %% classic queue is down, therefore message is rejected ?assertEqual(puback_timeout, util:publish_qos1_timeout(C, Topic, <<"msg">>, 500)), @@ -460,7 +458,7 @@ consuming_classic_mirrored_queue_down(Config) -> {<<"queue-master-locator">>, <<"client-local">>}]), %% Declare queue leader on Server1. - {C1, _} = connect(?FUNCTION_NAME, Config, [{clean_start, false}]), + C1 = connect(?FUNCTION_NAME, Config, [{clean_start, false}]), {ok, _, _} = emqtt:subscribe(C1, Topic, qos1), ok = emqtt:disconnect(C1), @@ -503,7 +501,7 @@ consuming_classic_queue_down(Config) -> ClientId = Topic = atom_to_binary(?FUNCTION_NAME), %% Declare classic queue on Server1. - {C1, _} = connect(?FUNCTION_NAME, Config, [{clean_start, false}]), + C1 = connect(?FUNCTION_NAME, Config, [{clean_start, false}]), {ok, _, _} = emqtt:subscribe(C1, Topic, qos1), ok = emqtt:disconnect(C1), @@ -561,7 +559,7 @@ delete_create_queue(Config) -> DeclareQueues(), %% some large retry_interval to avoid re-sending - {C, _} = connect(?FUNCTION_NAME, Config, [{retry_interval, 300}]), + C = connect(?FUNCTION_NAME, Config, [{retry_interval, 300}]), NumMsgs = 50, TestPid = self(), spawn( @@ -606,7 +604,7 @@ delete_create_queue(Config) -> ok = emqtt:disconnect(C). non_clean_sess_disconnect(Config) -> - {C1, _} = connect(?FUNCTION_NAME, Config, [{clean_start, false}]), + C1 = connect(?FUNCTION_NAME, Config, [{clean_start, false}]), Topic = <<"test-topic1">>, {ok, _, [1]} = emqtt:subscribe(C1, Topic, qos1), ProtoVer = v4, @@ -617,7 +615,7 @@ non_clean_sess_disconnect(Config) -> ?assertMatch(#{consumers := 0}, get_global_counters(Config, ProtoVer)), - {C2, _} = connect(?FUNCTION_NAME, Config, [{clean_start, false}]), + C2 = connect(?FUNCTION_NAME, Config, [{clean_start, false}]), ?assertMatch(#{consumers := 1}, get_global_counters(Config, ProtoVer)), @@ -628,13 +626,14 @@ non_clean_sess_disconnect(Config) -> Msg = <<"msg">>, {ok, _} = emqtt:publish(C2, Topic, Msg, qos1), {publish_not_received, Msg} = expect_publishes(Topic, [Msg]), + ok = emqtt:disconnect(C2), %% connect with clean sess true to clean up - {C3, _} = connect(?FUNCTION_NAME, Config, [{clean_start, true}]), + C3 = connect(?FUNCTION_NAME, Config, [{clean_start, true}]), ok = emqtt:disconnect(C3). subscribe_same_topic_same_qos(Config) -> - {C, _} = connect(?FUNCTION_NAME, Config), + C = connect(?FUNCTION_NAME, Config), Topic = <<"a/b">>, {ok, _} = emqtt:publish(C, Topic, <<"retained">>, [{retain, true}, @@ -653,7 +652,7 @@ subscribe_same_topic_same_qos(Config) -> ok = emqtt:disconnect(C). subscribe_same_topic_different_qos(Config) -> - {C, _} = connect(?FUNCTION_NAME, Config, [{clean_start, false}]), + C = connect(?FUNCTION_NAME, Config, [{clean_start, false}]), Topic = <<"b/c">>, {ok, _} = emqtt:publish(C, Topic, <<"retained">>, [{retain, true}, @@ -678,11 +677,11 @@ subscribe_same_topic_different_qos(Config) -> ?assertEqual(2, length(Consumers)), ok = emqtt:disconnect(C), - {C1, _} = connect(?FUNCTION_NAME, Config, [{clean_start, true}]), + C1 = connect(?FUNCTION_NAME, Config, [{clean_start, true}]), ok = emqtt:disconnect(C1). subscribe_multiple(Config) -> - {C, _} = connect(?FUNCTION_NAME, Config), + C = connect(?FUNCTION_NAME, Config), %% Subscribe to multiple topics at once ?assertMatch({ok, _, [0, 1]}, emqtt:subscribe(C, [{<<"topic0">>, qos0}, @@ -718,22 +717,6 @@ await_confirms_unordered(From, Left) -> ct:fail("~b confirms are missing", [Left]) end. -connect(ClientId, Config) -> - connect(ClientId, Config, []). - -connect(ClientId, Config, AdditionalOpts) -> - P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), - Options = [{host, "localhost"}, - {port, P}, - {clientid, rabbit_data_coercion:to_binary(ClientId)}, - {proto_ver, v4} - ] ++ AdditionalOpts, - {ok, C} = emqtt:start_link(Options), - {ok, _Properties} = emqtt:connect(C), - true = unlink(C), - MRef = monitor(process, C), - {C, MRef}. - declare_queue(Ch, QueueName, Args) when is_pid(Ch), is_binary(QueueName), is_list(Args) -> #'queue.declare_ok'{} = amqp_channel:call( diff --git a/deps/rabbitmq_mqtt/test/reader_SUITE.erl b/deps/rabbitmq_mqtt/test/reader_SUITE.erl index d8c01c31a256..f230173e004a 100644 --- a/deps/rabbitmq_mqtt/test/reader_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/reader_SUITE.erl @@ -16,7 +16,9 @@ eventually/3]). -import(util, [all_connection_pids/1, publish_qos1_timeout/4, - expect_publishes/2]). + expect_publishes/2, + connect/2, + connect/3]). all() -> [ @@ -93,12 +95,7 @@ end_per_testcase(Testcase, Config) -> %% ------------------------------------------------------------------- block(Config) -> - P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), - {ok, C} = emqtt:start_link([{host, "localhost"}, - {port, P}, - {clientid, atom_to_binary(?FUNCTION_NAME)}, - {proto_ver, v4}]), - {ok, _Properties} = emqtt:connect(C), + C = connect(?FUNCTION_NAME, Config), %% Only here to ensure the connection is really up {ok, _, _} = emqtt:subscribe(C, <<"TopicA">>), @@ -195,15 +192,9 @@ login_timeout(Config) -> keepalive(Config) -> KeepaliveSecs = 1, KeepaliveMs = timer:seconds(KeepaliveSecs), - P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), ProtoVer = v4, - {ok, C} = emqtt:start_link([{keepalive, KeepaliveSecs}, - {host, "localhost"}, - {port, P}, - {clientid, atom_to_binary(?FUNCTION_NAME)}, - {proto_ver, ProtoVer} - ]), - {ok, _Properties} = emqtt:connect(C), + C = connect(?FUNCTION_NAME, Config, [{keepalive, KeepaliveSecs}, + {proto_ver, ProtoVer}]), ok = emqtt:publish(C, <<"ignored">>, <<"msg">>), %% Connection should stay up when client sends PING requests. @@ -229,14 +220,7 @@ keepalive(Config) -> keepalive_turned_off(Config) -> %% "A Keep Alive value of zero (0) has the effect of turning off the keep alive mechanism." KeepaliveSecs = 0, - P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), - {ok, C} = emqtt:start_link([{keepalive, KeepaliveSecs}, - {host, "localhost"}, - {port, P}, - {clientid, atom_to_binary(?FUNCTION_NAME)}, - {proto_ver, v4} - ]), - {ok, _Properties} = emqtt:connect(C), + C = connect(?FUNCTION_NAME, Config, [{keepalive, KeepaliveSecs}]), ok = emqtt:publish(C, <<"TopicB">>, <<"Payload">>), %% Mock the server socket to not have received any bytes. @@ -252,12 +236,7 @@ keepalive_turned_off(Config) -> ok = emqtt:disconnect(C). stats(Config) -> - P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), - {ok, C} = emqtt:start_link([{host, "localhost"}, - {port, P}, - {clientid, atom_to_binary(?FUNCTION_NAME)}, - {proto_ver, v4}]), - {ok, _Properties} = emqtt:connect(C), + C = connect(?FUNCTION_NAME, Config), %% Wait for stats being emitted (every 100ms) timer:sleep(300), %% Retrieve the connection Pid @@ -283,14 +262,8 @@ get_env() -> rabbit_mqtt_util:env(durable_queue_type). validate_durable_queue_type(Config, ClientName, CleanSession, ExpectedQueueType) -> - P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), - {ok, C} = emqtt:start_link([{host, "localhost"}, - {port, P}, - {clean_start, CleanSession}, - {clientid, ClientName}, - {proto_ver, v4}]), - {ok, _Properties} = emqtt:connect(C), + C = connect(ClientName, Config, [{clean_start, CleanSession}]), {ok, _, _} = emqtt:subscribe(C, <<"TopicB">>, qos1), ok = emqtt:publish(C, <<"TopicB">>, <<"Payload">>), ok = expect_publishes(<<"TopicB">>, [<<"Payload">>]), @@ -302,13 +275,7 @@ validate_durable_queue_type(Config, ClientName, CleanSession, ExpectedQueueType) ok = emqtt:disconnect(C). clean_session_disconnect_client(Config) -> - P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), - {ok, C} = emqtt:start_link([{clean_start, true}, - {host, "localhost"}, - {port, P}, - {clientid, atom_to_binary(?FUNCTION_NAME)}, - {proto_ver, v4}]), - {ok, _Properties} = emqtt:connect(C), + C = connect(?FUNCTION_NAME, Config), {ok, _, _} = emqtt:subscribe(C, <<"topic0">>, qos0), L0 = rpc(Config, rabbit_amqqueue, list_by_type, [rabbit_mqtt_qos0_queue]), @@ -325,13 +292,7 @@ clean_session_disconnect_client(Config) -> ?assertEqual(0, length(L)). clean_session_kill_node(Config) -> - P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), - {ok, C} = emqtt:start_link([{clean_start, true}, - {host, "localhost"}, - {port, P}, - {clientid, atom_to_binary(?FUNCTION_NAME)}, - {proto_ver, v4}]), - {ok, _Properties} = emqtt:connect(C), + C = connect(?FUNCTION_NAME, Config), {ok, _, _} = emqtt:subscribe(C, <<"topic0">>, qos0), L0 = rpc(Config, rabbit_amqqueue, list_by_type, [rabbit_mqtt_qos0_queue]), @@ -372,25 +333,16 @@ classic_clean_session_false(Config) -> validate_durable_queue_type(Config, <<"classicCleanSessionFalse">>, false, rabbit_classic_queue). will(Config) -> - P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), Topic = <<"will/topic">>, Msg = <<"will msg">>, - {ok, Publisher} = emqtt:start_link([{port, P}, - {clientid, <<"will-publisher">>}, - {proto_ver, v4}, - {will_topic, Topic}, - {will_payload, Msg}, - {will_qos, qos0}, - {will_retain, false} - ]), - {ok, _} = emqtt:connect(Publisher), + Publisher = connect(<<"will-publisher">>, Config, [{will_topic, Topic}, + {will_payload, Msg}, + {will_qos, qos0}, + {will_retain, false}]), timer:sleep(100), [ServerPublisherPid] = all_connection_pids(Config), - {ok, Subscriber} = emqtt:start_link([{port, P}, - {clientid, <<"will-subscriber">>}, - {proto_ver, v4}]), - {ok, _} = emqtt:connect(Subscriber), + Subscriber = connect(<<"will-subscriber">>, Config), {ok, _, _} = emqtt:subscribe(Subscriber, Topic, qos0), true = unlink(Publisher), diff --git a/deps/rabbitmq_mqtt/test/retainer_SUITE.erl b/deps/rabbitmq_mqtt/test/retainer_SUITE.erl index 2c0afd20c04d..5a329fe6b702 100644 --- a/deps/rabbitmq_mqtt/test/retainer_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/retainer_SUITE.erl @@ -8,7 +8,8 @@ -compile([export_all, nowarn_export_all]). -include_lib("common_test/include/ct.hrl"). --import(util, [expect_publishes/2]). +-import(util, [expect_publishes/2, + connect/3]). all() -> [ @@ -86,8 +87,7 @@ end_per_testcase(Testcase, Config) -> %% ------------------------------------------------------------------- coerce_configuration_data(Config) -> - P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), - C = connect(P), + C = connect(<<"simpleClientRetainer">>, Config, [{ack_timeout, 1}]), {ok, _, _} = emqtt:subscribe(C, <<"TopicA">>, qos0), ok = emqtt:publish(C, <<"TopicA">>, <<"Payload">>), @@ -101,8 +101,7 @@ coerce_configuration_data(Config) -> %% sent messages for the translated topic (TopicA/Device/Field) %% ------------------------------------------------------------------- should_translate_amqp2mqtt_on_publish(Config) -> - P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), - C = connect(P), + C = connect(<<"simpleClientRetainer">>, Config, [{ack_timeout, 1}]), %% there's an active consumer {ok, _, _} = emqtt:subscribe(C, <<"TopicA/Device.Field">>, qos1), ok = emqtt:publish(C, <<"TopicA/Device.Field">>, #{}, <<"Payload">>, [{retain, true}]), @@ -115,8 +114,7 @@ should_translate_amqp2mqtt_on_publish(Config) -> %% sent the retained message for the translated topic (TopicA/Device/Field) %% ------------------------------------------------------------------- should_translate_amqp2mqtt_on_retention(Config) -> - P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), - C = connect(P), + C = connect(<<"simpleClientRetainer">>, Config, [{ack_timeout, 1}]), %% publish with retain = true before a consumer comes around ok = emqtt:publish(C, <<"TopicA/Device.Field">>, #{}, <<"Payload">>, [{retain, true}]), {ok, _, _} = emqtt:subscribe(C, <<"TopicA/Device.Field">>, qos1), @@ -129,16 +127,14 @@ should_translate_amqp2mqtt_on_retention(Config) -> %% sent retained message for the translated topic (TopicA/Device/Field) %% ------------------------------------------------------------------- should_translate_amqp2mqtt_on_retention_search(Config) -> - P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), - C = connect(P), + C = connect(<<"simpleClientRetainer">>, Config, [{ack_timeout, 1}]), ok = emqtt:publish(C, <<"TopicA/Device.Field">>, #{}, <<"Payload">>, [{retain, true}]), {ok, _, _} = emqtt:subscribe(C, <<"TopicA/Device/Field">>, qos1), ok = expect_publishes(<<"TopicA/Device/Field">>, [<<"Payload">>]), ok = emqtt:disconnect(C). does_not_retain(Config) -> - P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), - C = connect(P), + C = connect(<<"simpleClientRetainer">>, Config, [{ack_timeout, 1}]), ok = emqtt:publish(C, <<"TopicA/Device.Field">>, #{}, <<"Payload">>, [{retain, true}]), {ok, _, _} = emqtt:subscribe(C, <<"TopicA/Device.Field">>, qos1), receive @@ -147,14 +143,4 @@ does_not_retain(Config) -> after 1000 -> ok end, - ok = emqtt:disconnect(C). - -connect(Port) -> - {ok, C} = emqtt:start_link( - [{host, "localhost"}, - {port, Port}, - {clientid, <<"simpleClientRetainer">>}, - {proto_ver, v4}, - {ack_timeout, 1}]), - {ok, _Properties} = emqtt:connect(C), - C. \ No newline at end of file + ok = emqtt:disconnect(C). \ No newline at end of file diff --git a/deps/rabbitmq_mqtt/test/util.erl b/deps/rabbitmq_mqtt/test/util.erl index 73283d90e618..71fe7f9bc362 100644 --- a/deps/rabbitmq_mqtt/test/util.erl +++ b/deps/rabbitmq_mqtt/test/util.erl @@ -8,7 +8,9 @@ get_global_counters/2, get_global_counters/3, get_global_counters/4, - expect_publishes/2]). + expect_publishes/2, + connect/2, + connect/3]). all_connection_pids(Config) -> Nodes = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), @@ -62,3 +64,17 @@ get_global_counters(Config, v4, Node, QType) -> get_global_counters(Config, Proto, Node, QType) -> maps:get([{protocol, Proto}] ++ QType, rabbit_ct_broker_helpers:rpc(Config, Node, rabbit_global_counters, overview, [])). + +connect(ClientId, Config) -> + connect(ClientId, Config, []). + +connect(ClientId, Config, AdditionalOpts) -> + P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), + Options = [{host, "localhost"}, + {port, P}, + {clientid, rabbit_data_coercion:to_binary(ClientId)}, + {proto_ver, v4} + ] ++ AdditionalOpts, + {ok, C} = emqtt:start_link(Options), + {ok, _Properties} = emqtt:connect(C), + C. \ No newline at end of file From f842ffd250f71094735cf69472af5749e584f906 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Tue, 29 Nov 2022 18:13:50 +0000 Subject: [PATCH 067/118] Add feature flag rabbit_mqtt_qos0_queue Routing to a queue of type rabbit_mqtt_qos0_queue hosted on a remote node requires knowledge of that queue type on the local node. --- deps/rabbit/src/rabbit_amqqueue_process.erl | 3 +- ...l.Commands.DecommissionMqttNodeCommand.erl | 1 - deps/rabbitmq_mqtt/src/rabbit_mqtt_ff.erl | 8 + .../src/rabbit_mqtt_processor.erl | 201 ++++++++++-------- .../src/rabbit_mqtt_qos0_queue.erl | 10 +- deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl | 1 - deps/rabbitmq_mqtt/test/ff_SUITE.erl | 90 ++++++-- deps/rabbitmq_mqtt/test/integration_SUITE.erl | 123 ++++++++--- deps/rabbitmq_mqtt/test/java_SUITE_data/mvnw | 1 - deps/rabbitmq_mqtt/test/reader_SUITE.erl | 35 +-- 10 files changed, 310 insertions(+), 163 deletions(-) diff --git a/deps/rabbit/src/rabbit_amqqueue_process.erl b/deps/rabbit/src/rabbit_amqqueue_process.erl index 08cacb762fdd..c4835a5238a0 100644 --- a/deps/rabbit/src/rabbit_amqqueue_process.erl +++ b/deps/rabbit/src/rabbit_amqqueue_process.erl @@ -121,6 +121,7 @@ -define(CREATION_EVENT_KEYS, [name, + type, durable, auto_delete, arguments, @@ -129,7 +130,7 @@ user_who_performed_action ]). --define(INFO_KEYS, [pid | ?CREATION_EVENT_KEYS ++ ?STATISTICS_KEYS -- [name]]). +-define(INFO_KEYS, [pid | ?CREATION_EVENT_KEYS ++ ?STATISTICS_KEYS -- [name, type]]). %%---------------------------------------------------------------------------- diff --git a/deps/rabbitmq_mqtt/src/Elixir.RabbitMQ.CLI.Ctl.Commands.DecommissionMqttNodeCommand.erl b/deps/rabbitmq_mqtt/src/Elixir.RabbitMQ.CLI.Ctl.Commands.DecommissionMqttNodeCommand.erl index cbe5ad8bb541..fa8e09341c6b 100644 --- a/deps/rabbitmq_mqtt/src/Elixir.RabbitMQ.CLI.Ctl.Commands.DecommissionMqttNodeCommand.erl +++ b/deps/rabbitmq_mqtt/src/Elixir.RabbitMQ.CLI.Ctl.Commands.DecommissionMqttNodeCommand.erl @@ -58,7 +58,6 @@ run([Node], #{node := NodeName, " from the cluster", [Node]))}; Result -> %% 'ok' or 'timeout' - %% TODO: Ra will timeout if the node is not a cluster member - should this be fixed?? Result end. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_ff.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_ff.erl index 56623b6c39ae..892fe8fec7a0 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_ff.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_ff.erl @@ -7,8 +7,16 @@ -module(rabbit_mqtt_ff). +-include("rabbit_mqtt.hrl"). + -export([track_client_id_in_ra/0]). +-rabbit_feature_flag( + {?QUEUE_TYPE_QOS_0, + #{desc => "Support pseudo queue type for MQTT QoS 0 omitting a queue process", + stability => stable + }}). + -rabbit_feature_flag( {delete_ra_cluster_mqtt_node, #{desc => "Delete Ra cluster 'mqtt_node' since MQTT client IDs are tracked locally", diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index dc4040fa914d..e572bea2c0f3 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -519,7 +519,7 @@ start_keepalive(#mqtt_frame_connect{keep_alive = Seconds}, handle_clean_session(_, State0 = #state{clean_sess = false, proto_ver = ProtoVer}) -> case get_queue(?QOS_1, State0) of - {error, not_found} -> + {error, _} -> %% Queue will be created later when client subscribes. {ok, _SessionPresent = false, State0}; {ok, Q} -> @@ -539,27 +539,14 @@ handle_clean_session(_, State = #state{clean_sess = true, username = Username, authz_ctx = AuthzCtx}}) -> case get_queue(?QOS_1, State) of - {error, not_found} -> + {error, _} -> {ok, _SessionPresent = false, State}; {ok, Q0} -> QName = amqqueue:get_name(Q0), %% configure access to queue required for queue.delete case check_resource_access(User, QName, configure, AuthzCtx) of ok -> - rabbit_amqqueue:with( - QName, - fun (Q) -> - rabbit_queue_type:delete(Q, false, false, Username) - end, - fun (not_found) -> - ok; - ({absent, Q, crashed}) -> - rabbit_classic_queue:delete_crashed(Q, Username); - ({absent, Q, stopped}) -> - rabbit_classic_queue:delete_crashed(Q, Username); - ({absent, _Q, _Reason}) -> - ok - end), + delete_queue(QName, Username), {ok, _SessionPresent = false, State}; {error, access_refused} -> {error, ?CONNACK_NOT_AUTHORIZED} @@ -567,10 +554,24 @@ handle_clean_session(_, State = #state{clean_sess = true, end. -spec get_queue(qos(), state()) -> - {ok, amqqueue:amqqueue()} | {error, not_found}. + {ok, amqqueue:amqqueue()} | + {error, not_found | {resource_locked, amqqueue:amqqueue()}}. get_queue(QoS, State) -> QName = queue_name(QoS, State), - rabbit_amqqueue:lookup(QName). + case rabbit_amqqueue:lookup(QName) of + {ok, Q} = Ok -> + try rabbit_amqqueue:check_exclusive_access(Q, self()) of + ok -> + Ok + catch + exit:#amqp_error{name = resource_locked} -> + %% This can happen when same client ID re-connects + %% while its old connection is not yet closed. + {error, {resource_locked, Q}} + end; + {error, not_found} = Err -> + Err + end. queue_name(QoS, #state{client_id = ClientId, auth_state = #auth_state{vhost = VHost}}) -> @@ -1015,55 +1016,65 @@ delivery_mode(?QOS_0) -> 1; delivery_mode(?QOS_1) -> 2; delivery_mode(?QOS_2) -> 2. -ensure_queue(QoS, #state{ +ensure_queue(QoS, State = #state{auth_state = #auth_state{user = #user{username = Username}}}) -> + case get_queue(QoS, State) of + {ok, Q} -> + {ok, Q}; + {error, {resource_locked, Q}} -> + QName = amqqueue:get_name(Q), + rabbit_log:debug("MQTT deleting exclusive ~s owned by ~p", + [rabbit_misc:rs(QName), + ?amqqueue_v2_field_exclusive_owner(Q)]), + delete_queue(QName, Username), + create_queue(QoS, State); + {error, not_found} -> + create_queue(QoS, State) + end. + +create_queue(QoS, #state{ client_id = ClientId, clean_sess = CleanSess, auth_state = #auth_state{ vhost = VHost, user = User = #user{username = Username}, authz_ctx = AuthzCtx} - } = State) -> - case get_queue(QoS, State) of - {ok, Q} -> - {ok, Q}; - {error, not_found} -> - QNameBin = rabbit_mqtt_util:queue_name_bin(ClientId, QoS), - QName = rabbit_misc:r(VHost, queue, QNameBin), - %% configure access to queue required for queue.declare - case check_resource_access(User, QName, configure, AuthzCtx) of - ok -> - case rabbit_vhost_limit:is_over_queue_limit(VHost) of - false -> - rabbit_core_metrics:queue_declared(QName), - QArgs = queue_args(QoS, CleanSess), - Q0 = amqqueue:new(QName, - self(), - _Durable = true, - _AutoDelete = false, - queue_owner(QoS, CleanSess), - QArgs, - VHost, - #{user => Username}, - queue_type(QoS, CleanSess, QArgs) - ), - case rabbit_queue_type:declare(Q0, node()) of - {new, Q} when ?is_amqqueue(Q) -> - rabbit_core_metrics:queue_created(QName), - {ok, Q}; - Other -> - rabbit_log:error("Failed to declare ~s: ~p", - [rabbit_misc:rs(QName), Other]), - {error, queue_declare} - end; - {true, Limit} -> - rabbit_log:error("cannot declare ~s because " - "queue limit ~p in vhost '~s' is reached", - [rabbit_misc:rs(QName), Limit, VHost]), - {error, access_refused} + }) -> + QNameBin = rabbit_mqtt_util:queue_name_bin(ClientId, QoS), + QName = rabbit_misc:r(VHost, queue, QNameBin), + %% configure access to queue required for queue.declare + case check_resource_access(User, QName, configure, AuthzCtx) of + ok -> + case rabbit_vhost_limit:is_over_queue_limit(VHost) of + false -> + rabbit_core_metrics:queue_declared(QName), + QArgs = queue_args(QoS, CleanSess), + Q0 = amqqueue:new(QName, + self(), + _Durable = true, + _AutoDelete = false, + queue_owner(QoS, CleanSess), + QArgs, + VHost, + #{user => Username}, + queue_type(QoS, CleanSess, QArgs) + ), + case rabbit_queue_type:declare(Q0, node()) of + {new, Q} when ?is_amqqueue(Q) -> + rabbit_core_metrics:queue_created(QName), + {ok, Q}; + Other -> + rabbit_log:error("Failed to declare ~s: ~p", + [rabbit_misc:rs(QName), Other]), + {error, queue_declare} end; - {error, access_refused} = E -> - E - end + {true, Limit} -> + rabbit_log:error("cannot declare ~s because " + "queue limit ~p in vhost '~s' is reached", + [rabbit_misc:rs(QName), Limit, VHost]), + {error, access_refused} + end; + {error, access_refused} = E -> + E end. queue_owner(QoS, CleanSess) @@ -1091,8 +1102,13 @@ queue_args(_, _) -> Args end. -queue_type(?QOS_0, true, _) -> - ?QUEUE_TYPE_QOS_0; +queue_type(?QOS_0, true, QArgs) -> + case rabbit_feature_flags:is_enabled(?QUEUE_TYPE_QOS_0) of + true -> + ?QUEUE_TYPE_QOS_0; + false -> + rabbit_amqqueue:get_queue_type(QArgs) + end; queue_type(_, _, QArgs) -> rabbit_amqqueue:get_queue_type(QArgs). @@ -1401,12 +1417,28 @@ maybe_delete_mqtt_qos0_queue(State = #state{clean_sess = true, _ -> ok end; - {error, not_found} -> + _ -> ok end; maybe_delete_mqtt_qos0_queue(_) -> ok. +delete_queue(QName, Username) -> + rabbit_amqqueue:with( + QName, + fun (Q) -> + rabbit_queue_type:delete(Q, false, false, Username) + end, + fun (not_found) -> + ok; + ({absent, Q, crashed}) -> + rabbit_classic_queue:delete_crashed(Q, Username); + ({absent, Q, stopped}) -> + rabbit_classic_queue:delete_crashed(Q, Username); + ({absent, _Q, _Reason}) -> + ok + end). + handle_pre_hibernate() -> erase(permission_cache), erase(topic_permission_cache), @@ -1544,7 +1576,7 @@ deliver_to_client(Msgs, Ack, State) -> deliver_one_to_client(Msg, Ack, S) end, State, Msgs). -deliver_one_to_client(Msg = {QName, QPid, QMsgId, _Redelivered, +deliver_one_to_client(Msg = {QNameOrType, QPid, QMsgId, _Redelivered, #basic_message{content = #content{properties = #'P_basic'{headers = Headers}}}}, AckRequired, State0) -> PublisherQoS = case rabbit_mqtt_util:table_lookup(Headers, <<"x-mqtt-publish-qos">>) of @@ -1561,14 +1593,14 @@ deliver_one_to_client(Msg = {QName, QPid, QMsgId, _Redelivered, ?QOS_0 end, QoS = effective_qos(PublisherQoS, SubscriberQoS), - State1 = maybe_publish_to_client(Msg, QoS, SubscriberQoS, State0), - State = maybe_auto_ack(AckRequired, QoS, QName, QMsgId, State1), + State1 = maybe_publish_to_client(Msg, QoS, State0), + State = maybe_auto_ack(AckRequired, QoS, QNameOrType, QMsgId, State1), %%TODO GC % case GCThreshold of % undefined -> ok; % _ -> rabbit_basic:maybe_gc_large_msg(Content, GCThreshold) % end, - ok = maybe_notify_sent(QName, QPid, State), + ok = maybe_notify_sent(QNameOrType, QPid, State), State. -spec effective_qos(qos(), qos()) -> qos(). @@ -1578,16 +1610,16 @@ effective_qos(PublisherQoS, SubscriberQoS) -> %% [MQTT-3.8.4-8]." erlang:min(PublisherQoS, SubscriberQoS). -maybe_publish_to_client({_, _, _, _Redelivered = true, _}, ?QOS_0, _, State) -> +maybe_publish_to_client({_, _, _, _Redelivered = true, _}, ?QOS_0, State) -> %% Do not redeliver to MQTT subscriber who gets message at most once. State; maybe_publish_to_client( - {QName, _QPid, QMsgId, Redelivered, + {QNameOrType, _QPid, QMsgId, Redelivered, #basic_message{ routing_keys = [RoutingKey | _CcRoutes], content = #content{payload_fragments_rev = FragmentsRev}}}, - QoS, SubscriberQos, State0 = #state{amqp2mqtt_fun = Amqp2MqttFun, - send_fun = SendFun}) -> + QoS, State0 = #state{amqp2mqtt_fun = Amqp2MqttFun, + send_fun = SendFun}) -> {PacketId, State} = queue_packet_id_to_packet_id(QMsgId, QoS, State0), %%TODO support iolists when sending to client Payload = list_to_binary(lists:reverse(FragmentsRev)), @@ -1608,7 +1640,7 @@ maybe_publish_to_client( topic_name = Amqp2MqttFun(RoutingKey)}, payload = Payload}, SendFun(Frame, State), - message_delivered(QName, Redelivered, SubscriberQos, QoS, State), + message_delivered(QNameOrType, Redelivered, QoS, State), State. queue_packet_id_to_packet_id(_, ?QOS_0, State) -> @@ -1639,6 +1671,8 @@ maybe_auto_ack(_AckRequired = true, ?QOS_0, QName, QMsgId, maybe_auto_ack(_, _, _, _, State) -> State. +maybe_notify_sent(?QUEUE_TYPE_QOS_0, _, _) -> + ok; maybe_notify_sent(QName, QPid, #state{queue_states = QStates}) -> case rabbit_queue_type:module(QName, QStates) of {ok, rabbit_classic_queue} -> @@ -1679,9 +1713,11 @@ check_resource_access(User, Resource, Perm, Context) -> CacheTail = lists:sublist(Cache, ?MAX_PERMISSION_CACHE_SIZE-1), put(permission_cache, [V | CacheTail]), ok - catch exit:{amqp_error, access_refused, Msg, _AmqpMethod} -> - rabbit_log:error("MQTT resource access refused: ~s", [Msg]), - {error, access_refused} + catch + exit:#amqp_error{name = access_refused, + explanation = Msg} -> + rabbit_log:error("MQTT resource access refused: ~s", [Msg]), + {error, access_refused} end end. @@ -1714,7 +1750,8 @@ check_topic_access(TopicName, Access, put(topic_permission_cache, [Key | CacheTail]), ok catch - exit:{amqp_error, access_refused, Msg, _AmqpMethod} -> + exit:#amqp_error{name = access_refused, + explanation = Msg} -> rabbit_log:error("MQTT topic access refused: ~s", [Msg]), {error, access_refused} end @@ -1853,20 +1890,16 @@ message_acknowledged(QName, #state{proto_ver = ProtoVer, ok end. -message_delivered(_, _Redelivered = false, _SubscriberQoS = ?QOS_0, _QoS = ?QOS_0, - #state{clean_sess = true, - proto_ver = ProtoVer - }) -> +message_delivered(?QUEUE_TYPE_QOS_0, false, ?QOS_0, #state{proto_ver = ProtoVer}) -> rabbit_global_counters:messages_delivered(ProtoVer, ?QUEUE_TYPE_QOS_0, 1), %% Technically, the message is not acked to a queue at all. %% However, from a user perspective it is still auto acked because: %% "In automatic acknowledgement mode, a message is considered to be successfully %% delivered immediately after it is sent." rabbit_global_counters:messages_delivered_consume_auto_ack(ProtoVer, ?QUEUE_TYPE_QOS_0, 1); -message_delivered(QName, Redelivered, _, QoS, - #state{proto_ver = ProtoVer, - queue_states = QStates - }) -> +message_delivered(QName, Redelivered, QoS, #state{proto_ver = ProtoVer, + queue_states = QStates + }) -> case rabbit_queue_type:module(QName, QStates) of {ok, QType} -> rabbit_global_counters:messages_delivered(ProtoVer, QType, 1), diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl index 39153a1d4897..332f55e9945f 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl @@ -16,13 +16,6 @@ %% %% All messages are delivered at most once. -%% TODO -% Write test for mixed version cluster where the new node declares new -% queue type and old node routes to new queue, but does not know -% the new queuey type module yet. This might either require a feature -% flag or dropping the MQTT at-most-once message during the rolling -% update from and old to a new node. - -module(rabbit_mqtt_qos0_queue). -behaviour(rabbit_queue_type). @@ -95,9 +88,8 @@ delete(Q, _IfUnused, _IfEmpty, ActingUser) -> {[], rabbit_queue_type:actions()}. deliver([{Q, stateless}], Delivery = #delivery{message = BasicMessage}) -> Pid = amqqueue:get_pid(Q), - QName = amqqueue:get_name(Q), Msg = {queue_event, ?MODULE, - {QName, Pid, _QMsgId = none, _Redelivered = false, BasicMessage}}, + {?MODULE, Pid, _QMsgId = none, _Redelivered = false, BasicMessage}}, gen_server:cast(Pid, Msg), Actions = confirm(Delivery, Q), {[], Actions}. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl index 720a98ca246e..a1fe946e5b44 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl @@ -141,7 +141,6 @@ env(Key) -> undefined -> undefined end. -%% TODO: move to rabbit_common coerce_env_value(default_pass, Val) -> rabbit_data_coercion:to_binary(Val); coerce_env_value(default_user, Val) -> rabbit_data_coercion:to_binary(Val); coerce_env_value(exchange, Val) -> rabbit_data_coercion:to_binary(Val); diff --git a/deps/rabbitmq_mqtt/test/ff_SUITE.erl b/deps/rabbitmq_mqtt/test/ff_SUITE.erl index 8576a1c6f828..bb98108fb4e0 100644 --- a/deps/rabbitmq_mqtt/test/ff_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/ff_SUITE.erl @@ -6,14 +6,18 @@ -module(ff_SUITE). -%% Test suite for the feature flag delete_ra_cluster_mqtt_node - -compile([export_all, nowarn_export_all]). -include_lib("common_test/include/ct.hrl"). -include_lib("eunit/include/eunit.hrl"). --define(FEATURE_FLAG, delete_ra_cluster_mqtt_node). +-import(rabbit_ct_broker_helpers, [rpc/5]). +-import(rabbit_ct_helpers, [eventually/1]). +-import(util, [expect_publishes/2, + get_global_counters/4 + ]). + +-define(PROTO_VER, v4). all() -> [ @@ -22,12 +26,13 @@ all() -> groups() -> [ - {cluster_size_3, [], [enable_feature_flag]} + {cluster_size_3, [], [delete_ra_cluster_mqtt_node, + rabbit_mqtt_qos0_queue]} ]. suite() -> [ - {timetrap, {minutes, 5}} + {timetrap, {minutes, 2}} ]. init_per_suite(Config) -> @@ -40,31 +45,31 @@ end_per_suite(Config) -> init_per_group(Group = cluster_size_3, Config0) -> Config1 = rabbit_ct_helpers:set_config(Config0, [{rmq_nodes_count, 3}, {rmq_nodename_suffix, Group}]), - Config2 = rabbit_ct_helpers:merge_app_env( - Config1, {rabbit, [{forced_feature_flags_on_init, []}]}), - Config = rabbit_ct_helpers:run_steps(Config2, - rabbit_ct_broker_helpers:setup_steps() ++ - rabbit_ct_client_helpers:setup_steps()), - case rabbit_ct_broker_helpers:is_feature_flag_supported(Config, ?FEATURE_FLAG) of - true -> - Config; - false -> - end_per_group(Group, Config), - {skip, io_lib:format("feature flag ~s is unsupported", [?FEATURE_FLAG])} - end. + Config = rabbit_ct_helpers:merge_app_env( + Config1, {rabbit, [{forced_feature_flags_on_init, []}]}), + rabbit_ct_helpers:run_steps(Config, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps()). end_per_group(_Group, Config) -> rabbit_ct_helpers:run_steps(Config, rabbit_ct_client_helpers:teardown_steps() ++ rabbit_ct_broker_helpers:teardown_steps()). -init_per_testcase(_TestCase, Config) -> - Config. +init_per_testcase(TestCase, Config) -> + case rabbit_ct_broker_helpers:is_feature_flag_supported(Config, TestCase) of + true -> + Config; + false -> + {skip, io_lib:format("feature flag ~s is unsupported", + [TestCase])} + end. end_per_testcase(_TestCase, Config) -> Config. -enable_feature_flag(Config) -> +delete_ra_cluster_mqtt_node(Config) -> + FeatureFlag = ?FUNCTION_NAME, C = connect_to_node(Config, 1, <<"my-client">>), timer:sleep(500), %% old client ID tracking works @@ -73,7 +78,8 @@ enable_feature_flag(Config) -> ?assert(lists:all(fun erlang:is_pid/1, rabbit_ct_broker_helpers:rpc_all(Config, erlang, whereis, [mqtt_node]))), - ?assertEqual(ok, rabbit_ct_broker_helpers:enable_feature_flag(Config, ?FEATURE_FLAG)), + ?assertEqual(ok, + rabbit_ct_broker_helpers:enable_feature_flag(Config, FeatureFlag)), %% Ra processes should be gone rabbit_ct_helpers:eventually( @@ -84,12 +90,52 @@ enable_feature_flag(Config) -> ?assert(erlang:is_process_alive(C)), ok = emqtt:disconnect(C). +rabbit_mqtt_qos0_queue(Config) -> + FeatureFlag = ?FUNCTION_NAME, + Msg = Topic = ClientId = atom_to_binary(?FUNCTION_NAME), + + C1 = connect_to_node(Config, 0, ClientId), + {ok, _, [0]} = emqtt:subscribe(C1, Topic, qos0), + ok = emqtt:publish(C1, Topic, Msg, qos0), + ok = expect_publishes(Topic, [Msg]), + ?assertEqual(1, + length(rpc(Config, 0, rabbit_amqqueue, list_by_type, [rabbit_classic_queue]))), + + ?assertEqual(ok, + rabbit_ct_broker_helpers:enable_feature_flag(Config, FeatureFlag)), + + %% Queue type does not chanage for existing connection. + ?assertEqual(1, + length(rpc(Config, 0, rabbit_amqqueue, list_by_type, [rabbit_classic_queue]))), + ok = emqtt:publish(C1, Topic, Msg, qos0), + ok = expect_publishes(Topic, [Msg]), + ?assertMatch(#{messages_delivered_total := 2, + messages_delivered_consume_auto_ack_total := 2}, + get_global_counters(Config, ?PROTO_VER, 0, [{queue_type, rabbit_classic_queue}])), + + %% Reconnecting with the same client ID will terminate the old connection. + true = unlink(C1), + C2 = connect_to_node(Config, 0, ClientId), + {ok, _, [0]} = emqtt:subscribe(C2, Topic, qos0), + %% This time, we get the new queue type. + eventually( + ?_assertEqual(0, + length(rpc(Config, 0, rabbit_amqqueue, list_by_type, [rabbit_classic_queue])))), + ?assertEqual(1, + length(rpc(Config, 0, rabbit_amqqueue, list_by_type, [FeatureFlag]))), + ok = emqtt:publish(C2, Topic, Msg, qos0), + ok = expect_publishes(Topic, [Msg]), + ?assertMatch(#{messages_delivered_total := 1, + messages_delivered_consume_auto_ack_total := 1}, + get_global_counters(Config, ?PROTO_VER, 0, [{queue_type, FeatureFlag}])), + ok = emqtt:disconnect(C2). + connect_to_node(Config, Node, ClientID) -> Port = rabbit_ct_broker_helpers:get_node_config(Config, Node, tcp_port_mqtt), {ok, C} = emqtt:start_link([{host, "localhost"}, {port, Port}, {clientid, ClientID}, - {proto_ver, v4}, + {proto_ver, ?PROTO_VER}, {connect_timeout, 1}, {ack_timeout, 1}]), {ok, _Properties} = emqtt:connect(C), diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/integration_SUITE.erl index a62224241945..ddf14d8155d1 100644 --- a/deps/rabbitmq_mqtt/test/integration_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/integration_SUITE.erl @@ -45,7 +45,8 @@ groups() -> consuming_classic_queue_down, flow_classic_mirrored_queue, flow_quorum_queue, - flow_stream] ++ common_tests() + flow_stream, + rabbit_mqtt_qos0_queue] ++ common_tests() } ]. @@ -288,15 +289,30 @@ events(Config) -> {ok, _, _} = emqtt:subscribe(C, <<"TopicA">>, qos0), - [E2, E3] = get_events(Server), - assert_event_type(queue_created, E2), QueueNameBin = <<"mqtt-subscription-", ClientId/binary, "qos0">>, QueueName = {resource, <<"/">>, queue, QueueNameBin}, + [E2, E3 | E4] = get_events(Server), + QueueType = case rabbit_ct_helpers:is_mixed_versions(Config) of + false -> + ?assertEqual([], E4), + rabbit_mqtt_qos0_queue; + true -> + %% Feature flag rabbit_mqtt_qos0_queue is disabled. + [ConsumerCreated] = E4, + assert_event_type(consumer_created, ConsumerCreated), + assert_event_prop([{queue, QueueName}, + {ack_required, false}, + {exclusive, false}, + {arguments, []}], + ConsumerCreated), + classic + end, + assert_event_type(queue_created, E2), assert_event_prop([{name, QueueName}, {durable, true}, {auto_delete, false}, {exclusive, true}, - {type, rabbit_mqtt_qos0_queue}, + {type, QueueType}, {arguments, []}], E2), assert_event_type(binding_created, E3), @@ -310,16 +326,25 @@ events(Config) -> {ok, _, _} = emqtt:unsubscribe(C, <<"TopicA">>), - [E4] = get_events(Server), - assert_event_type(binding_deleted, E4), + [E5] = get_events(Server), + assert_event_type(binding_deleted, E5), ok = emqtt:disconnect(C), - [E5, E6] = get_events(Server), - assert_event_type(connection_closed, E5), - assert_event_prop(ExpectedConnectionProps, E5), - assert_event_type(queue_deleted, E6), - assert_event_prop({name, QueueName}, E6), + [E6, E7 | E8] = get_events(Server), + assert_event_type(connection_closed, E6), + assert_event_prop(ExpectedConnectionProps, E6), + case rabbit_ct_helpers:is_mixed_versions(Config) of + false -> + assert_event_type(queue_deleted, E7), + assert_event_prop({name, QueueName}, E7); + true -> + assert_event_type(consumer_deleted, E7), + assert_event_prop({queue, QueueName}, E7), + [QueueDeleted] = E8, + assert_event_type(queue_deleted, QueueDeleted), + assert_event_prop({name, QueueName}, QueueDeleted) + end, ok = gen_event:delete_handler({rabbit_event, Server}, event_recorder, []). @@ -388,24 +413,38 @@ global_counters(Config, ProtoVer) -> messages_unroutable_returned_total => 1}, get_global_counters(Config, ProtoVer)), - ?assertEqual(#{messages_delivered_total => 2, - messages_acknowledged_total => 1, - messages_delivered_consume_auto_ack_total => 1, - messages_delivered_consume_manual_ack_total => 1, - messages_delivered_get_auto_ack_total => 0, - messages_delivered_get_manual_ack_total => 0, - messages_get_empty_total => 0, - messages_redelivered_total => 0}, - get_global_counters(Config, ProtoVer, 0, [{queue_type, rabbit_classic_queue}])), - ?assertEqual(#{messages_delivered_total => 1, - messages_acknowledged_total => 0, - messages_delivered_consume_auto_ack_total => 1, - messages_delivered_consume_manual_ack_total => 0, - messages_delivered_get_auto_ack_total => 0, - messages_delivered_get_manual_ack_total => 0, - messages_get_empty_total => 0, - messages_redelivered_total => 0}, - get_global_counters(Config, ProtoVer, 0, [{queue_type, rabbit_mqtt_qos0_queue}])), + case rabbit_ct_helpers:is_mixed_versions(Config) of + false -> + ?assertEqual(#{messages_delivered_total => 2, + messages_acknowledged_total => 1, + messages_delivered_consume_auto_ack_total => 1, + messages_delivered_consume_manual_ack_total => 1, + messages_delivered_get_auto_ack_total => 0, + messages_delivered_get_manual_ack_total => 0, + messages_get_empty_total => 0, + messages_redelivered_total => 0}, + get_global_counters(Config, ProtoVer, 0, [{queue_type, rabbit_classic_queue}])), + ?assertEqual(#{messages_delivered_total => 1, + messages_acknowledged_total => 0, + messages_delivered_consume_auto_ack_total => 1, + messages_delivered_consume_manual_ack_total => 0, + messages_delivered_get_auto_ack_total => 0, + messages_delivered_get_manual_ack_total => 0, + messages_get_empty_total => 0, + messages_redelivered_total => 0}, + get_global_counters(Config, ProtoVer, 0, [{queue_type, rabbit_mqtt_qos0_queue}])); + true -> + %% Feature flag rabbit_mqtt_qos0_queue is disabled. + ?assertEqual(#{messages_delivered_total => 3, + messages_acknowledged_total => 1, + messages_delivered_consume_auto_ack_total => 2, + messages_delivered_consume_manual_ack_total => 1, + messages_delivered_get_auto_ack_total => 0, + messages_delivered_get_manual_ack_total => 0, + messages_get_empty_total => 0, + messages_redelivered_total => 0}, + get_global_counters(Config, ProtoVer, 0, [{queue_type, rabbit_classic_queue}])) + end, {ok, _, _} = emqtt:unsubscribe(C, Topic1), ?assertEqual(1, maps:get(consumers, get_global_counters(Config, ProtoVer))), @@ -688,6 +727,30 @@ subscribe_multiple(Config) -> {<<"topic1">>, qos1}])), ok = emqtt:disconnect(C). +%% This test is mostly interesting in mixed version mode where feature flag +%% rabbit_mqtt_qos0_queue is disabled and therefore a classic queue gets created. +rabbit_mqtt_qos0_queue(Config) -> + Topic = atom_to_binary(?FUNCTION_NAME), + + %% Place MQTT subscriber process on new node in mixed version. + Sub = connect(<<"subscriber">>, Config), + {ok, _, [0]} = emqtt:subscribe(Sub, Topic, qos0), + + %% Place MQTT publisher process on old node in mixed version. + {ok, Pub} = emqtt:start_link( + [{port, rabbit_ct_broker_helpers:get_node_config(Config, 1, tcp_port_mqtt)}, + {clientid, <<"publisher">>}, + {proto_ver, v4} + ]), + {ok, _Properties} = emqtt:connect(Pub), + + Msg = <<"msg">>, + ok = emqtt:publish(Pub, Topic, Msg, qos0), + ok = expect_publishes(Topic, [Msg]), + + ok = emqtt:disconnect(Sub), + ok = emqtt:disconnect(Pub). + %% ------------------------------------------------------------------- %% Internal helpers %% ------------------------------------------------------------------- @@ -745,7 +808,7 @@ bind(Ch, QueueName, Topic) routing_key = Topic}). get_events(Node) -> - timer:sleep(100), %% events are sent and processed asynchronously + timer:sleep(300), %% events are sent and processed asynchronously Result = gen_event:call({rabbit_event, Node}, event_recorder, take_state), ?assert(is_list(Result)), Result. diff --git a/deps/rabbitmq_mqtt/test/java_SUITE_data/mvnw b/deps/rabbitmq_mqtt/test/java_SUITE_data/mvnw index 8b9da3b8b600..88203bbfeccf 100755 --- a/deps/rabbitmq_mqtt/test/java_SUITE_data/mvnw +++ b/deps/rabbitmq_mqtt/test/java_SUITE_data/mvnw @@ -114,7 +114,6 @@ if $mingw ; then M2_HOME="`(cd "$M2_HOME"; pwd)`" [ -n "$JAVA_HOME" ] && JAVA_HOME="`(cd "$JAVA_HOME"; pwd)`" - # TODO classpath? fi if [ -z "$JAVA_HOME" ]; then diff --git a/deps/rabbitmq_mqtt/test/reader_SUITE.erl b/deps/rabbitmq_mqtt/test/reader_SUITE.erl index f230173e004a..695819828743 100644 --- a/deps/rabbitmq_mqtt/test/reader_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/reader_SUITE.erl @@ -276,14 +276,18 @@ validate_durable_queue_type(Config, ClientName, CleanSession, ExpectedQueueType) clean_session_disconnect_client(Config) -> C = connect(?FUNCTION_NAME, Config), - {ok, _, _} = emqtt:subscribe(C, <<"topic0">>, qos0), - L0 = rpc(Config, rabbit_amqqueue, list_by_type, [rabbit_mqtt_qos0_queue]), - ?assertEqual(1, length(L0)), - {ok, _, _} = emqtt:subscribe(C, <<"topic1">>, qos1), - L1 = rpc(Config, rabbit_amqqueue, list_by_type, [rabbit_classic_queue]), - ?assertEqual(1, length(L1)), + QsQos0 = rpc(Config, rabbit_amqqueue, list_by_type, [rabbit_mqtt_qos0_queue]), + QsClassic = rpc(Config, rabbit_amqqueue, list_by_type, [rabbit_classic_queue]), + case rabbit_ct_helpers:is_mixed_versions(Config) of + false -> + ?assertEqual(1, length(QsQos0)), + ?assertEqual(1, length(QsClassic)); + true -> + ?assertEqual(0, length(QsQos0)), + ?assertEqual(2, length(QsClassic)) + end, ok = emqtt:disconnect(C), %% After terminating a clean session, we expect any session state to be cleaned up on the server. @@ -293,15 +297,18 @@ clean_session_disconnect_client(Config) -> clean_session_kill_node(Config) -> C = connect(?FUNCTION_NAME, Config), - {ok, _, _} = emqtt:subscribe(C, <<"topic0">>, qos0), - L0 = rpc(Config, rabbit_amqqueue, list_by_type, [rabbit_mqtt_qos0_queue]), - ?assertEqual(1, length(L0)), - {ok, _, _} = emqtt:subscribe(C, <<"topic1">>, qos1), - L1 = rpc(Config, rabbit_amqqueue, list_by_type, [rabbit_classic_queue]), - ?assertEqual(1, length(L1)), - + QsQos0 = rpc(Config, rabbit_amqqueue, list_by_type, [rabbit_mqtt_qos0_queue]), + QsClassic = rpc(Config, rabbit_amqqueue, list_by_type, [rabbit_classic_queue]), + case rabbit_ct_helpers:is_mixed_versions(Config) of + false -> + ?assertEqual(1, length(QsQos0)), + ?assertEqual(1, length(QsClassic)); + true -> + ?assertEqual(0, length(QsQos0)), + ?assertEqual(2, length(QsClassic)) + end, ?assertEqual(2, rpc(Config, ets, info, [rabbit_durable_queue, size])), process_flag(trap_exit, true), @@ -352,4 +359,4 @@ will(Config) -> ok = emqtt:disconnect(Subscriber). rpc(Config, M, F, A) -> - rabbit_ct_broker_helpers:rpc(Config, 0, M, F, A). + rpc(Config, 0, M, F, A). From 46e8a65d96ebb6ebb1cb2a45f677e3d92381fada Mon Sep 17 00:00:00 2001 From: Chunyi Lyu Date: Wed, 30 Nov 2022 16:11:51 +0000 Subject: [PATCH 068/118] Check if state.stats_timer is undefined to avoid crashing - if #state.stats_timer is undefined, rabbit_event:if_enabled crashes - remove compression related TODO from web_mqtt. It's a intentional default behavior set in: https://github.com/rabbitmq/rabbitmq-web-mqtt/pull/35 --- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 2 +- deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index b7e82ba9567e..d541ab77c3be 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -423,7 +423,7 @@ maybe_process_deferred_recv(State = #state{ deferred_recv = Data, socket = Sock handle_info({tcp, Sock, Data}, State#state{ deferred_recv = undefined }). -maybe_emit_stats(undefined) -> +maybe_emit_stats(#state{stats_timer = undefined}) -> ok; maybe_emit_stats(State) -> rabbit_event:if_enabled(State, #state.stats_timer, diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index 015a53dc1d93..0616cf4da90b 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -67,7 +67,6 @@ init(Req, Opts) -> {PeerAddr, _PeerPort} = maps:get(peer, Req), SockInfo = maps:get(proxy_header, Req, undefined), WsOpts0 = proplists:get_value(ws_opts, Opts, #{}), - %%TODO is compress needed? WsOpts = maps:merge(#{compress => true}, WsOpts0), Req2 = case cowboy_req:header(<<"sec-websocket-protocol">>, Req) of undefined -> Req; @@ -317,7 +316,8 @@ send_reply(Frame, PState) -> ensure_stats_timer(State) -> rabbit_event:ensure_stats_timer(State, #state.stats_timer, emit_stats). -%% TODO if #state.stats_timer is undefined, rabbit_event:if_enabled crashes +maybe_emit_stats(#state{stats_timer = undefined}) -> + ok; maybe_emit_stats(State) -> rabbit_event:if_enabled(State, #state.stats_timer, fun() -> emit_stats(State) end). From 7782142020c41eeb613fe2b262bdee8f981b702c Mon Sep 17 00:00:00 2001 From: David Ansari Date: Thu, 1 Dec 2022 14:45:17 +0000 Subject: [PATCH 069/118] Reduce memory usage of reader processes Reduce memory usage of (Web) MQTT connection process and STOMP reader process by storing the connection name as binary instead of string. Previously: 82 = erts_debug:size("192.168.2.104:52497 -> 192.168.2.175:1883"). The binary <<"192.168.2.104:52497 -> 192.168.2.175:1883">> requires 8 words. So, for 1 million MQTT connections, this commit should save (82 - 8) words * 1,000,000 = 592 MB of memory. --- .../src/rabbit_mqtt_processor.erl | 2 +- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 83 ++++++++++--------- .../src/rabbit_stomp_reader.erl | 45 +++++----- .../src/rabbit_web_mqtt_handler.erl | 25 +++--- 4 files changed, 79 insertions(+), 76 deletions(-) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index e572bea2c0f3..0c2323784f05 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -66,7 +66,7 @@ auth_state, peer_addr, send_fun :: fun((Frame :: tuple(), state()) -> term()), - %%TODO remove funs from state? + %%TODO remove funs from state? 11 words each of the funs. mqtt2amqp_fun, amqp2mqtt_fun, register_state, diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index d541ab77c3be..0200add9a303 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -67,18 +67,19 @@ init(Ref) -> RealSocket = rabbit_net:unwrap_socket(Sock), case rabbit_net:connection_string(Sock, inbound) of {ok, ConnStr} -> - rabbit_log_connection:debug("MQTT accepting TCP connection ~tp (~ts)", [self(), ConnStr]), + ConnName = rabbit_data_coercion:to_binary(ConnStr), + rabbit_log_connection:debug("MQTT accepting TCP connection ~tp (~ts)", [self(), ConnName]), rabbit_alarm:register( self(), {?MODULE, conserve_resources, []}), LoginTimeout = application:get_env(rabbitmq_mqtt, login_timeout, 10_000), erlang:send_after(LoginTimeout, self(), login_timeout), - ProcessorState = rabbit_mqtt_processor:initial_state(RealSocket, ConnStr), + ProcessorState = rabbit_mqtt_processor:initial_state(RealSocket, ConnName), gen_server:enter_loop(?MODULE, [], rabbit_event:init_stats_timer( control_throttle( #state{socket = RealSocket, proxy_socket = rabbit_net:maybe_get_proxy_socket(Sock), - conn_name = ConnStr, + conn_name = ConnName, await_recv = false, connection_state = running, received_connect_frame = false, @@ -194,13 +195,13 @@ handle_info({keepalive, Req}, State = #state{keepalive = KState0, handle_info(login_timeout, State = #state{received_connect_frame = true}) -> {noreply, State, ?HIBERNATE_AFTER}; -handle_info(login_timeout, State = #state{conn_name = ConnStr}) -> +handle_info(login_timeout, State = #state{conn_name = ConnName}) -> %% The connection is also closed if the CONNECT frame happens to %% be already in the `deferred_recv' buffer. This can happen while %% the connection is blocked because of a resource alarm. However %% we don't know what is in the buffer, it can be arbitrary bytes, %% and we don't want to skip closing the connection in that case. - rabbit_log_connection:error("closing MQTT connection ~tp (login timeout)", [ConnStr]), + rabbit_log_connection:error("closing MQTT connection ~tp (login timeout)", [ConnName]), {stop, {shutdown, login_timeout}, State}; handle_info(emit_stats, State) -> @@ -239,33 +240,33 @@ terminate(Reason, {SendWill, State = #state{conn_name = ConnName, rabbit_mqtt_processor:terminate(SendWill, ConnName, PState), log_terminate(Reason, State). -log_terminate({network_error, {ssl_upgrade_error, closed}, ConnStr}, _State) -> +log_terminate({network_error, {ssl_upgrade_error, closed}, ConnName}, _State) -> rabbit_log_connection:error("MQTT detected TLS upgrade error on ~s: connection closed", - [ConnStr]); + [ConnName]); log_terminate({network_error, {ssl_upgrade_error, - {tls_alert, "handshake failure"}}, ConnStr}, _State) -> - log_tls_alert(handshake_failure, ConnStr); + {tls_alert, "handshake failure"}}, ConnName}, _State) -> + log_tls_alert(handshake_failure, ConnName); log_terminate({network_error, {ssl_upgrade_error, - {tls_alert, "unknown ca"}}, ConnStr}, _State) -> - log_tls_alert(unknown_ca, ConnStr); + {tls_alert, "unknown ca"}}, ConnName}, _State) -> + log_tls_alert(unknown_ca, ConnName); log_terminate({network_error, {ssl_upgrade_error, - {tls_alert, {Err, _}}}, ConnStr}, _State) -> - log_tls_alert(Err, ConnStr); + {tls_alert, {Err, _}}}, ConnName}, _State) -> + log_tls_alert(Err, ConnName); log_terminate({network_error, {ssl_upgrade_error, - {tls_alert, Alert}}, ConnStr}, _State) -> - log_tls_alert(Alert, ConnStr); -log_terminate({network_error, {ssl_upgrade_error, Reason}, ConnStr}, _State) -> + {tls_alert, Alert}}, ConnName}, _State) -> + log_tls_alert(Alert, ConnName); +log_terminate({network_error, {ssl_upgrade_error, Reason}, ConnName}, _State) -> rabbit_log_connection:error("MQTT detected TLS upgrade error on ~s: ~p", - [ConnStr, Reason]); + [ConnName, Reason]); -log_terminate({network_error, Reason, ConnStr}, _State) -> +log_terminate({network_error, Reason, ConnName}, _State) -> rabbit_log_connection:error("MQTT detected network error on ~s: ~p", - [ConnStr, Reason]); + [ConnName, Reason]); log_terminate({network_error, Reason}, _State) -> rabbit_log_connection:error("MQTT detected network error: ~p", [Reason]); @@ -282,28 +283,28 @@ code_change(_OldVsn, State, _Extra) -> %%---------------------------------------------------------------------------- -log_tls_alert(handshake_failure, ConnStr) -> +log_tls_alert(handshake_failure, ConnName) -> rabbit_log_connection:error("MQTT detected TLS upgrade error on ~ts: handshake failure", - [ConnStr]); -log_tls_alert(unknown_ca, ConnStr) -> + [ConnName]); +log_tls_alert(unknown_ca, ConnName) -> rabbit_log_connection:error("MQTT detected TLS certificate verification error on ~ts: alert 'unknown CA'", - [ConnStr]); -log_tls_alert(Alert, ConnStr) -> + [ConnName]); +log_tls_alert(Alert, ConnName) -> rabbit_log_connection:error("MQTT detected TLS upgrade error on ~ts: alert ~ts", - [ConnStr, Alert]). + [ConnName, Alert]). process_received_bytes(<<>>, State = #state{received_connect_frame = false, proc_state = PState, - conn_name = ConnStr}) -> + conn_name = ConnName}) -> rabbit_log_connection:info("Accepted MQTT connection ~p (~s, client id: ~s)", - [self(), ConnStr, rabbit_mqtt_processor:info(client_id, PState)]), + [self(), ConnName, rabbit_mqtt_processor:info(client_id, PState)]), {noreply, ensure_stats_timer(State#state{received_connect_frame = true}), ?HIBERNATE_AFTER}; process_received_bytes(<<>>, State) -> {noreply, ensure_stats_timer(State), ?HIBERNATE_AFTER}; process_received_bytes(Bytes, State = #state{ parse_state = ParseState, proc_state = ProcState, - conn_name = ConnStr }) -> + conn_name = ConnName }) -> case parse(Bytes, ParseState) of {more, ParseState1} -> {noreply, @@ -319,42 +320,42 @@ process_received_bytes(Bytes, proc_state = ProcState1}); %% PUBLISH and more {error, unauthorized = Reason, ProcState1} -> - rabbit_log_connection:error("MQTT connection ~ts is closing due to an authorization failure", [ConnStr]), + rabbit_log_connection:error("MQTT connection ~ts is closing due to an authorization failure", [ConnName]), {stop, {shutdown, Reason}, pstate(State, ProcState1)}; %% CONNECT frames only {error, unauthenticated = Reason, ProcState1} -> - rabbit_log_connection:error("MQTT connection ~ts is closing due to an authentication failure", [ConnStr]), + rabbit_log_connection:error("MQTT connection ~ts is closing due to an authentication failure", [ConnName]), {stop, {shutdown, Reason}, pstate(State, ProcState1)}; %% CONNECT frames only {error, invalid_client_id = Reason, ProcState1} -> - rabbit_log_connection:error("MQTT cannot accept connection ~ts: client uses an invalid ID", [ConnStr]), + rabbit_log_connection:error("MQTT cannot accept connection ~ts: client uses an invalid ID", [ConnName]), {stop, {shutdown, Reason}, pstate(State, ProcState1)}; %% CONNECT frames only {error, unsupported_protocol_version = Reason, ProcState1} -> - rabbit_log_connection:error("MQTT cannot accept connection ~ts: incompatible protocol version", [ConnStr]), + rabbit_log_connection:error("MQTT cannot accept connection ~ts: incompatible protocol version", [ConnName]), {stop, {shutdown, Reason}, pstate(State, ProcState1)}; {error, unavailable = Reason, ProcState1} -> rabbit_log_connection:error("MQTT cannot accept connection ~ts due to an internal error or unavailable component", - [ConnStr]), + [ConnName]), {stop, {shutdown, Reason}, pstate(State, ProcState1)}; {error, Reason, ProcState1} -> rabbit_log_connection:error("MQTT protocol error on connection ~ts: ~tp", - [ConnStr, Reason]), + [ConnName, Reason]), {stop, {shutdown, Reason}, pstate(State, ProcState1)}; {error, Error} -> rabbit_log_connection:error("MQTT detected a framing error on connection ~ts: ~tp", - [ConnStr, Error]), + [ConnName, Error]), {stop, {shutdown, Error}, State}; {stop, disconnect, ProcState1} -> {stop, normal, {_SendWill = false, pstate(State, ProcState1)}} end; {error, {cannot_parse, Error, Stacktrace}} -> rabbit_log_connection:error("MQTT cannot parse a frame on connection '~ts', unparseable payload: ~tp, error: {~tp, ~tp} ", - [ConnStr, Bytes, Error, Stacktrace]), + [ConnName, Bytes, Error, Stacktrace]), {stop, {shutdown, Error}, State}; {error, Error} -> rabbit_log_connection:error("MQTT detected a framing error on connection ~ts: ~tp", - [ConnStr, Error]), + [ConnName, Error]), {stop, {shutdown, Error}, State} end. @@ -373,10 +374,10 @@ parse(Bytes, ParseState) -> end. network_error(closed, - State = #state{conn_name = ConnStr, + State = #state{conn_name = ConnName, received_connect_frame = Connected}) -> Fmt = "MQTT connection ~p will terminate because peer closed TCP connection", - Args = [ConnStr], + Args = [ConnName], case Connected of true -> rabbit_log_connection:info(Fmt, Args); false -> rabbit_log_connection:debug(Fmt, Args) @@ -384,9 +385,9 @@ network_error(closed, {stop, {shutdown, conn_closed}, State}; network_error(Reason, - State = #state{conn_name = ConnStr}) -> + State = #state{conn_name = ConnName}) -> rabbit_log_connection:info("MQTT detected network error for ~p: ~p", - [ConnStr, Reason]), + [ConnName, Reason]), {stop, {shutdown, conn_closed}, State}. run_socket(State = #state{ connection_state = blocked }) -> diff --git a/deps/rabbitmq_stomp/src/rabbit_stomp_reader.erl b/deps/rabbitmq_stomp/src/rabbit_stomp_reader.erl index 5a57366fcfc8..1c6b6fa23b54 100644 --- a/deps/rabbitmq_stomp/src/rabbit_stomp_reader.erl +++ b/deps/rabbitmq_stomp/src/rabbit_stomp_reader.erl @@ -57,12 +57,13 @@ init([SupHelperPid, Ref, Configuration]) -> case rabbit_net:connection_string(Sock, inbound) of {ok, ConnStr} -> + ConnName = rabbit_data_coercion:to_binary(ConnStr), ProcInitArgs = processor_args(Configuration, Sock), ProcState = rabbit_stomp_processor:initial_state(Configuration, ProcInitArgs), rabbit_log_connection:info("accepting STOMP connection ~tp (~ts)", - [self(), ConnStr]), + [self(), ConnName]), ParseState = rabbit_stomp_frame:initial_state(), _ = register_resource_alarm(), @@ -74,7 +75,7 @@ init([SupHelperPid, Ref, Configuration]) -> rabbit_event:init_stats_timer( run_socket(control_throttle( #reader_state{socket = RealSocket, - conn_name = ConnStr, + conn_name = ConnName, parse_state = ParseState, processor_state = ProcState, heartbeat_sup = SupHelperPid, @@ -301,34 +302,34 @@ code_change(_OldVsn, State, _Extra) -> {ok, State}. -log_reason({network_error, {ssl_upgrade_error, closed}, ConnStr}, _State) -> +log_reason({network_error, {ssl_upgrade_error, closed}, ConnName}, _State) -> rabbit_log_connection:error("STOMP detected TLS upgrade error on ~ts: connection closed", - [ConnStr]); + [ConnName]); log_reason({network_error, {ssl_upgrade_error, - {tls_alert, "handshake failure"}}, ConnStr}, _State) -> - log_tls_alert(handshake_failure, ConnStr); + {tls_alert, "handshake failure"}}, ConnName}, _State) -> + log_tls_alert(handshake_failure, ConnName); log_reason({network_error, {ssl_upgrade_error, - {tls_alert, "unknown ca"}}, ConnStr}, _State) -> - log_tls_alert(unknown_ca, ConnStr); + {tls_alert, "unknown ca"}}, ConnName}, _State) -> + log_tls_alert(unknown_ca, ConnName); log_reason({network_error, {ssl_upgrade_error, - {tls_alert, {Err, _}}}, ConnStr}, _State) -> - log_tls_alert(Err, ConnStr); + {tls_alert, {Err, _}}}, ConnName}, _State) -> + log_tls_alert(Err, ConnName); log_reason({network_error, {ssl_upgrade_error, - {tls_alert, Alert}}, ConnStr}, _State) -> - log_tls_alert(Alert, ConnStr); -log_reason({network_error, {ssl_upgrade_error, Reason}, ConnStr}, _State) -> + {tls_alert, Alert}}, ConnName}, _State) -> + log_tls_alert(Alert, ConnName); +log_reason({network_error, {ssl_upgrade_error, Reason}, ConnName}, _State) -> rabbit_log_connection:error("STOMP detected TLS upgrade error on ~ts: ~tp", - [ConnStr, Reason]); + [ConnName, Reason]); -log_reason({network_error, Reason, ConnStr}, _State) -> +log_reason({network_error, Reason, ConnName}, _State) -> rabbit_log_connection:error("STOMP detected network error on ~ts: ~tp", - [ConnStr, Reason]); + [ConnName, Reason]); log_reason({network_error, Reason}, _State) -> rabbit_log_connection:error("STOMP detected network error: ~tp", [Reason]); @@ -355,15 +356,15 @@ log_reason(Reason, #reader_state{processor_state = ProcState}) -> rabbit_log_connection:warning("STOMP connection ~ts terminated" " with reason ~tp, closing it", [AdapterName, Reason]). -log_tls_alert(handshake_failure, ConnStr) -> +log_tls_alert(handshake_failure, ConnName) -> rabbit_log_connection:error("STOMP detected TLS upgrade error on ~ts: handshake failure", - [ConnStr]); -log_tls_alert(unknown_ca, ConnStr) -> + [ConnName]); +log_tls_alert(unknown_ca, ConnName) -> rabbit_log_connection:error("STOMP detected TLS certificate verification error on ~ts: alert 'unknown CA'", - [ConnStr]); -log_tls_alert(Alert, ConnStr) -> + [ConnName]); +log_tls_alert(Alert, ConnName) -> rabbit_log_connection:error("STOMP detected TLS upgrade error on ~ts: alert ~ts", - [ConnStr, Alert]). + [ConnName, Alert]). %%---------------------------------------------------------------------------- diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index 0616cf4da90b..3c9769ce318d 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -92,14 +92,15 @@ websocket_init(State0 = #state{socket = Sock, peername = PeerAddr}) -> ok = file_handle_cache:obtain(), case rabbit_net:connection_string(Sock, inbound) of {ok, ConnStr} -> + ConnName = rabbit_data_coercion:to_binary(ConnStr), State = State0#state{ - conn_name = ConnStr, + conn_name = ConnName, socket = Sock }, - rabbit_log_connection:info("Accepting Web MQTT connection ~p (~s)", [self(), ConnStr]), + rabbit_log_connection:info("Accepting Web MQTT connection ~p (~s)", [self(), ConnName]), RealSocket = rabbit_net:unwrap_socket(Sock), ProcessorState = rabbit_mqtt_processor:initial_state(RealSocket, - ConnStr, + ConnName, fun send_reply/2, PeerAddr), process_flag(trap_exit, true), @@ -223,28 +224,28 @@ terminate(_Reason, _Request, %% Internal. -handle_data(Data, State0 = #state{conn_name = ConnStr}) -> +handle_data(Data, State0 = #state{conn_name = ConnName}) -> case handle_data1(Data, State0) of {ok, State1 = #state{state = blocked}, hibernate} -> {[{active, false}], State1, hibernate}; {error, Error} -> - stop_with_framing_error(State0, Error, ConnStr); + stop_with_framing_error(State0, Error, ConnName); Other -> Other end. handle_data1(<<>>, State0 = #state{received_connect_frame = false, proc_state = PState, - conn_name = ConnStr}) -> + conn_name = ConnName}) -> rabbit_log_connection:info("Accepted web MQTT connection ~p (~s, client id: ~s)", - [self(), ConnStr, rabbit_mqtt_processor:info(client_id, PState)]), + [self(), ConnName, rabbit_mqtt_processor:info(client_id, PState)]), State = State0#state{received_connect_frame = true}, {ok, ensure_stats_timer(control_throttle(State)), hibernate}; handle_data1(<<>>, State) -> {ok, ensure_stats_timer(control_throttle(State)), hibernate}; handle_data1(Data, State = #state{ parse_state = ParseState, proc_state = ProcState, - conn_name = ConnStr }) -> + conn_name = ConnName }) -> case rabbit_mqtt_frame:parse(Data, ParseState) of {more, ParseState1} -> {ok, ensure_stats_timer(control_throttle( @@ -259,10 +260,10 @@ handle_data1(Data, State = #state{ parse_state = ParseState, proc_state = ProcState1}); {error, Reason, _} -> rabbit_log_connection:info("MQTT protocol error ~tp for connection ~tp", - [Reason, ConnStr]), + [Reason, ConnName]), stop(State, ?CLOSE_PROTOCOL_ERROR, Reason); {error, Error} -> - stop_with_framing_error(State, Error, ConnStr); + stop_with_framing_error(State, Error, ConnName); {stop, disconnect, ProcState1} -> stop({_SendWill = false, State#state{proc_state = ProcState1}}) end; @@ -270,10 +271,10 @@ handle_data1(Data, State = #state{ parse_state = ParseState, Other end. -stop_with_framing_error(State, Error0, ConnStr) -> +stop_with_framing_error(State, Error0, ConnName) -> Error1 = rabbit_misc:format("~tp", [Error0]), rabbit_log_connection:error("MQTT detected framing error '~ts' for connection ~tp", - [Error1, ConnStr]), + [Error1, ConnName]), stop(State, ?CLOSE_INCONSISTENT_MSG_TYPE, Error1). stop(State) -> From 1180429c962e9f5821776bfd54d62b1db8b0e8b7 Mon Sep 17 00:00:00 2001 From: Chunyi Lyu Date: Thu, 1 Dec 2022 17:20:53 +0000 Subject: [PATCH 070/118] Terminate connection if mqtt not found in web sup protocol - per MQTT spec --- deps/rabbitmq_web_mqtt/BUILD.bazel | 1 + .../src/rabbit_web_mqtt_handler.erl | 41 +++++++++++-------- .../test/proxy_protocol_SUITE.erl | 2 +- deps/rabbitmq_web_mqtt/test/system_SUITE.erl | 20 ++++++++- 4 files changed, 46 insertions(+), 18 deletions(-) diff --git a/deps/rabbitmq_web_mqtt/BUILD.bazel b/deps/rabbitmq_web_mqtt/BUILD.bazel index c6fcea47136f..472e0018bac4 100644 --- a/deps/rabbitmq_web_mqtt/BUILD.bazel +++ b/deps/rabbitmq_web_mqtt/BUILD.bazel @@ -90,6 +90,7 @@ rabbitmq_integration_suite( runtime_deps = [ "@emqtt//:erlang_app", "@gun//:erlang_app", + "@cowlib//:erlang_app", ], ) diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index 3c9769ce318d..9a789e8fd0de 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -68,22 +68,25 @@ init(Req, Opts) -> SockInfo = maps:get(proxy_header, Req, undefined), WsOpts0 = proplists:get_value(ws_opts, Opts, #{}), WsOpts = maps:merge(#{compress => true}, WsOpts0), - Req2 = case cowboy_req:header(<<"sec-websocket-protocol">>, Req) of - undefined -> Req; - %%TODO check whether client offers mqtt: - %% MQTT spec: - %% "The Client MUST include “mqtt” in the list of WebSocket Sub Protocols it offers" - SecWsProtocol -> - cowboy_req:set_resp_header(<<"sec-websocket-protocol">>, SecWsProtocol, Req) - end, - {?MODULE, Req2, #state{ - parse_state = rabbit_mqtt_frame:initial_state(), - state = running, - conserve_resources = false, - socket = SockInfo, - peername = PeerAddr, - received_connect_frame = false - }, WsOpts}. + case cowboy_req:parse_header(<<"sec-websocket-protocol">>, Req) of + undefined -> + no_supported_sub_protocol(undefined, Req); + Protocol -> + case lists:member(<<"mqtt">>, Protocol) of + false -> + no_supported_sub_protocol(Protocol, Req); + true -> + {?MODULE, cowboy_req:set_resp_header(<<"sec-websocket-protocol">>, <<"mqtt">>, Req), + #state{ + parse_state = rabbit_mqtt_frame:initial_state(), + state = running, + conserve_resources = false, + socket = SockInfo, + peername = PeerAddr, + received_connect_frame = false + }, WsOpts} + end + end. -spec websocket_init(State) -> {cowboy_websocket:commands(), State} | @@ -210,6 +213,8 @@ websocket_info(Msg, State) -> {[], State, hibernate}. -spec terminate(any(), cowboy_req:req(), any()) -> ok. +terminate(_Reason, _Req, #state{state = undefined}) -> + ok; terminate(Reason, Request, #state{} = State) -> terminate(Reason, Request, {true, State}); terminate(_Reason, _Request, @@ -224,6 +229,10 @@ terminate(_Reason, _Request, %% Internal. +no_supported_sub_protocol(Protocol, Req) -> + rabbit_log_connection:error("Web MQTT: mqtt not found in client supported protocol, protocol: ~tp", [Protocol]), + {ok, cowboy_req:reply(400, Req), #state{}}. + handle_data(Data, State0 = #state{conn_name = ConnName}) -> case handle_data1(Data, State0) of {ok, State1 = #state{state = blocked}, hibernate} -> diff --git a/deps/rabbitmq_web_mqtt/test/proxy_protocol_SUITE.erl b/deps/rabbitmq_web_mqtt/test/proxy_protocol_SUITE.erl index 1750eba3cce6..7410879bd8e7 100644 --- a/deps/rabbitmq_web_mqtt/test/proxy_protocol_SUITE.erl +++ b/deps/rabbitmq_web_mqtt/test/proxy_protocol_SUITE.erl @@ -82,7 +82,7 @@ proxy_protocol(Config) -> Protocol = ?config(protocol, Config), WS = rfc6455_client:new(Protocol ++ "://127.0.0.1:" ++ PortStr ++ "/ws", self(), - undefined, [], "PROXY TCP4 192.168.1.1 192.168.1.2 80 81\r\n"), + undefined, ["mqtt"], "PROXY TCP4 192.168.1.1 192.168.1.2 80 81\r\n"), {ok, _} = rfc6455_client:open(WS), rfc6455_client:send_binary(WS, rabbit_ws_test_util:mqtt_3_1_1_connect_frame()), {binary, _P} = rfc6455_client:recv(WS), diff --git a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl index e393758d92fd..d6941505cd15 100644 --- a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl +++ b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl @@ -30,13 +30,16 @@ groups() -> , disconnect , keepalive , maintenance + , client_no_supported_protocol + , client_not_support_mqtt ]} ]. init_per_suite(Config) -> rabbit_ct_helpers:log_environment(), Config1 = rabbit_ct_helpers:set_config(Config, [ - {rmq_nodename_suffix, ?MODULE} + {rmq_nodename_suffix, ?MODULE}, + {protocol, "ws"} ]), rabbit_ct_helpers:run_setup_steps(Config1, rabbit_ct_broker_helpers:setup_steps() ++ @@ -178,6 +181,21 @@ maintenance(Config) -> eventually(?_assertEqual(0, num_mqtt_connections(Config, 0))), ok = rabbit_ct_broker_helpers:revive_node(Config, 0). +client_no_supported_protocol(Config) -> + client_protocol_test(Config, []). + +client_not_support_mqtt(Config) -> + client_protocol_test(Config, ["not-mqtt-protocol"]). + +client_protocol_test(Config, Protocol) -> + PortStr = rabbit_ws_test_util:get_web_mqtt_port_str(Config), + WS = rfc6455_client:new("ws://localhost:" ++ PortStr ++ "/ws", self(), undefined, Protocol), + {_, [{http_response, Res}]} = rfc6455_client:open(WS), + {'HTTP/1.1', 400, <<"Bad Request">>, _} = cow_http:parse_status_line(rabbit_data_coercion:to_binary(Res)), + rfc6455_client:send_binary(WS, rabbit_ws_test_util:mqtt_3_1_1_connect_frame()), + {close, _P} = rfc6455_client:recv(WS), + ok. + %% Web mqtt connections are tracked together with mqtt connections num_mqtt_connections(Config, Node) -> length(rpc(Config, Node, rabbit_mqtt, local_connection_pids, [])). From 86de0a15573b102187345cc1251857d45211cd04 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Fri, 2 Dec 2022 15:01:58 +0000 Subject: [PATCH 071/118] Reduce memory usage of MQTT connection process by removing the two fields referencing a function: mqtt2amqp_fun and amqp2mqtt_fun Each field required 1 words + 11 words for the function reference. Therefore, for 1 million MQTT connections this commit saves: (1+11) * 2 * 1,000,000 words = 192 MB of memory In addition, the code is now simpler to understand. There is no change in behaviour except for the sparkplug environment variable being read upon application start. We put the compiled regexes into persistent term because they are the same for all MQTT connections. --- deps/rabbitmq_mqtt/src/rabbit_mqtt.erl | 11 +- .../src/rabbit_mqtt_processor.erl | 61 ++++------ deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl | 114 +++++++++--------- deps/rabbitmq_mqtt/test/util_SUITE.erl | 15 ++- deps/rabbitmq_web_mqtt/Makefile | 2 + 5 files changed, 99 insertions(+), 104 deletions(-) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl index 84e76bb1f9be..e8861de33340 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl @@ -19,7 +19,8 @@ local_connection_pids/0]). start(normal, []) -> - global_counters_init(), + init_global_counters(), + rabbit_mqtt_util:init_sparkplug(), {ok, Listeners} = application:get_env(tcp_listeners), {ok, SslListeners} = application:get_env(ssl_listeners), case rabbit_mqtt_ff:track_client_id_in_ra() of @@ -87,11 +88,11 @@ local_connection_pids() -> end, pg:which_groups(PgScope)) end. -global_counters_init() -> - global_counters_init(?MQTT_PROTO_V3), - global_counters_init(?MQTT_PROTO_V4). +init_global_counters() -> + init_global_counters(?MQTT_PROTO_V3), + init_global_counters(?MQTT_PROTO_V4). -global_counters_init(ProtoVer) -> +init_global_counters(ProtoVer) -> Proto = {protocol, ProtoVer}, rabbit_global_counters:init([Proto]), rabbit_global_counters:init([Proto, {queue_type, ?QUEUE_TYPE_QOS_0}]), diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 0c2323784f05..741cdf35fef1 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -18,6 +18,9 @@ -export_type([state/0]). +-import(rabbit_mqtt_util, [mqtt_to_amqp/1, + amqp_to_mqtt/1]). + -include_lib("rabbit_common/include/rabbit.hrl"). -include_lib("rabbit_common/include/rabbit_framing.hrl"). -include_lib("rabbit/include/amqqueue.hrl"). @@ -66,9 +69,6 @@ auth_state, peer_addr, send_fun :: fun((Frame :: tuple(), state()) -> term()), - %%TODO remove funs from state? 11 words each of the funs. - mqtt2amqp_fun, - amqp2mqtt_fun, register_state, conn_name, info, @@ -87,7 +87,6 @@ initial_state(Socket, ConnectionName) -> PeerAddr). initial_state(Socket, ConnectionName, SendFun, PeerAddr) -> - {ok, {mqtt2amqp_fun, M2A}, {amqp2mqtt_fun, A2M}} = rabbit_mqtt_util:get_topic_translation_funs(), Flow = case rabbit_misc:get_env(rabbit, mirroring_flow_control, true) of true -> flow; false -> noflow @@ -97,8 +96,6 @@ initial_state(Socket, ConnectionName, SendFun, PeerAddr) -> ssl_login_name = ssl_login_name(Socket), peer_addr = PeerAddr, send_fun = SendFun, - mqtt2amqp_fun = M2A, - amqp2mqtt_fun = A2M, delivery_flow = Flow}. process_frame(#mqtt_frame{fixed = #mqtt_frame_fixed{type = Type}}, @@ -158,7 +155,6 @@ process_request(?PUBLISH, packet_id = PacketId }, payload = Payload}, State0 = #state{retainer_pid = RPid, - amqp2mqtt_fun = Amqp2MqttFun, unacked_client_pubs = U, proto_ver = ProtoVer}) -> rabbit_global_counters:messages_received(ProtoVer, 1), @@ -176,7 +172,7 @@ process_request(?PUBLISH, false -> ok; true -> - hand_off_to_retainer(RPid, Amqp2MqttFun, Topic, Msg) + hand_off_to_retainer(RPid, Topic, Msg) end, Ok; Error -> @@ -578,9 +574,8 @@ queue_name(QoS, #state{client_id = ClientId, QNameBin = rabbit_mqtt_util:queue_name_bin(ClientId, QoS), rabbit_misc:r(VHost, queue, QNameBin). -find_queue_name(TopicName, #state{exchange = Exchange, - mqtt2amqp_fun = Mqtt2AmqpFun} = State) -> - RoutingKey = Mqtt2AmqpFun(TopicName), +find_queue_name(TopicName, #state{exchange = Exchange} = State) -> + RoutingKey = mqtt_to_amqp(TopicName), QNameQoS0 = queue_name(?QOS_0, State), case lookup_binding(Exchange, QNameQoS0, RoutingKey) of true -> @@ -605,10 +600,9 @@ has_subs(State) -> topic_names(?QOS_0, State) =/= [] orelse topic_names(?QOS_1, State) =/= []. -topic_names(QoS, #state{exchange = Exchange, - amqp2mqtt_fun = Amqp2MqttFun} = State) -> +topic_names(QoS, #state{exchange = Exchange} = State) -> Bindings = rabbit_binding:list_for_source_and_destination(Exchange, queue_name(QoS, State)), - lists:map(fun(B) -> Amqp2MqttFun(B#binding.key) end, Bindings). + lists:map(fun(B) -> amqp_to_mqtt(B#binding.key) end, Bindings). %% "If a Server receives a SUBSCRIBE Packet containing a Topic Filter that is identical %% to an existing Subscription’s Topic Filter then it MUST completely replace that @@ -643,20 +637,19 @@ maybe_unbind(TopicName, TopicNames, QName, State0) -> end end. -hand_off_to_retainer(RetainerPid, Amqp2MqttFun, Topic0, #mqtt_msg{payload = <<"">>}) -> - Topic1 = Amqp2MqttFun(Topic0), +hand_off_to_retainer(RetainerPid, Topic0, #mqtt_msg{payload = <<"">>}) -> + Topic1 = amqp_to_mqtt(Topic0), rabbit_mqtt_retainer:clear(RetainerPid, Topic1), ok; -hand_off_to_retainer(RetainerPid, Amqp2MqttFun, Topic0, Msg) -> - Topic1 = Amqp2MqttFun(Topic0), +hand_off_to_retainer(RetainerPid, Topic0, Msg) -> + Topic1 = amqp_to_mqtt(Topic0), rabbit_mqtt_retainer:retain(RetainerPid, Topic1, Msg), ok. maybe_send_retained_message(RPid, #mqtt_topic{name = Topic0, qos = SubscribeQos}, - #state{amqp2mqtt_fun = Amqp2MqttFun, - packet_id = PacketId0, + #state{packet_id = PacketId0, send_fun = SendFun} = State0) -> - Topic1 = Amqp2MqttFun(Topic0), + Topic1 = amqp_to_mqtt(Topic0), case rabbit_mqtt_retainer:fetch(RPid, Topic1) of undefined -> State0; @@ -1203,9 +1196,9 @@ binding_action( {QueueName, TopicName, BindingFun}, #state{exchange = ExchangeName, auth_state = #auth_state{ - user = #user{username = Username}}, - mqtt2amqp_fun = Mqtt2AmqpFun}) -> - RoutingKey = Mqtt2AmqpFun(TopicName), + user = #user{username = Username}} + }) -> + RoutingKey = mqtt_to_amqp(TopicName), Binding = #binding{source = ExchangeName, destination = QueueName, key = RoutingKey}, @@ -1220,9 +1213,8 @@ publish_to_queues( packet_id = PacketId, payload = Payload}, #state{exchange = ExchangeName, - mqtt2amqp_fun = Mqtt2AmqpFun, delivery_flow = Flow} = State) -> - RoutingKey = Mqtt2AmqpFun(Topic), + RoutingKey = mqtt_to_amqp(Topic), Confirm = Qos > ?QOS_0, Headers = [{<<"x-mqtt-publish-qos">>, byte, Qos}, {<<"x-mqtt-dup">>, bool, Dup}], @@ -1364,8 +1356,8 @@ terminate(SendWill, ConnName, State) -> maybe_send_will(true, ConnStr, #state{will_msg = WillMsg = #mqtt_msg{retain = Retain, topic = Topic}, - retainer_pid = RPid, - amqp2mqtt_fun = Amqp2MqttFun} = State) -> + retainer_pid = RPid + } = State) -> rabbit_log_connection:debug("sending MQTT will message to topic ~s on connection ~s", [Topic, ConnStr]), case check_topic_access(Topic, write, State) of @@ -1375,7 +1367,7 @@ maybe_send_will(true, ConnStr, false -> ok; true -> - hand_off_to_retainer(RPid, Amqp2MqttFun, Topic, WillMsg) + hand_off_to_retainer(RPid, Topic, WillMsg) end; {error, access_refused = Reason} -> rabbit_log:error("failed to send will message: ~p", [Reason]) @@ -1618,8 +1610,7 @@ maybe_publish_to_client( #basic_message{ routing_keys = [RoutingKey | _CcRoutes], content = #content{payload_fragments_rev = FragmentsRev}}}, - QoS, State0 = #state{amqp2mqtt_fun = Amqp2MqttFun, - send_fun = SendFun}) -> + QoS, State0 = #state{send_fun = SendFun}) -> {PacketId, State} = queue_packet_id_to_packet_id(QMsgId, QoS, State0), %%TODO support iolists when sending to client Payload = list_to_binary(lists:reverse(FragmentsRev)), @@ -1637,7 +1628,7 @@ maybe_publish_to_client( dup = Redelivered}, variable = #mqtt_frame_publish{ packet_id = PacketId, - topic_name = Amqp2MqttFun(RoutingKey)}, + topic_name = amqp_to_mqtt(RoutingKey)}, payload = Payload}, SendFun(Frame, State), message_delivered(QNameOrType, Redelivered, QoS, State), @@ -1727,8 +1718,8 @@ check_topic_access(TopicName, Access, vhost = VHost, authz_ctx = AuthzCtx}, exchange = #resource{name = ExchangeBin}, - client_id = ClientId, - mqtt2amqp_fun = Mqtt2AmqpFun}) -> + client_id = ClientId + }) -> Cache = case get(topic_permission_cache) of undefined -> []; Other -> Other @@ -1741,7 +1732,7 @@ check_topic_access(TopicName, Access, Resource = #resource{virtual_host = VHost, kind = topic, name = ExchangeBin}, - RoutingKey = Mqtt2AmqpFun(TopicName), + RoutingKey = mqtt_to_amqp(TopicName), Context = #{routing_key => RoutingKey, variable_map => AuthzCtx}, try rabbit_access_control:check_topic_access(User, Resource, Access, Context) of diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl index a1fe946e5b44..ba6f7aab703f 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl @@ -19,13 +19,18 @@ path_for/2, path_for/3, vhost_name_to_table_name/1, - get_topic_translation_funs/0, register_clientid/2, - remove_duplicate_clientid_connections/2 + remove_duplicate_clientid_connections/2, + init_sparkplug/0, + mqtt_to_amqp/1, + amqp_to_mqtt/1 ]). -define(MAX_TOPIC_TRANSLATION_CACHE_SIZE, 12). +-define(SPARKPLUG_MP_MQTT_TO_AMQP, sparkplug_mp_mqtt_to_amqp). +-define(SPARKPLUG_MP_AMQP_TO_MQTT, sparkplug_mp_amqp_to_mqtt). + -spec queue_name_bin(binary(), qos()) -> binary(). queue_name_bin(ClientId, QoS) -> @@ -56,14 +61,53 @@ qos_from_queue_name(#resource{name = Name}, ClientId) -> queue_name_prefix(ClientId) -> <<"mqtt-subscription-", ClientId/binary, "qos">>. -cached(CacheName, Fun, Arg) -> - Cache = - case get(CacheName) of - undefined -> - []; - Other -> - Other +init_sparkplug() -> + case env(sparkplug) of + true -> + {ok, M2A_SpRe} = re:compile("^sp[AB]v\\d+\\.\\d+/"), + {ok, A2M_SpRe} = re:compile("^sp[AB]v\\d+___\\d+\\."), + ok = persistent_term:put(?SPARKPLUG_MP_MQTT_TO_AMQP, M2A_SpRe), + ok = persistent_term:put(?SPARKPLUG_MP_AMQP_TO_MQTT, A2M_SpRe); + _ -> + ok + end. + +mqtt_to_amqp(Topic) -> + T = case persistent_term:get(?SPARKPLUG_MP_MQTT_TO_AMQP, no_sparkplug) of + no_sparkplug -> + Topic; + M2A_SpRe -> + case re:run(Topic, M2A_SpRe) of + nomatch -> + Topic; + {match, _} -> + string:replace(Topic, ".", "___", leading) + end end, + cached(mta_cache, fun to_amqp/1, T). + +amqp_to_mqtt(Topic) -> + T = cached(atm_cache, fun to_mqtt/1, Topic), + case persistent_term:get(?SPARKPLUG_MP_AMQP_TO_MQTT, no_sparkplug) of + no_sparkplug -> + T; + A2M_SpRe -> + case re:run(Topic, A2M_SpRe) of + nomatch -> + T; + {match, _} -> + T1 = string:replace(T, "___", ".", leading), + erlang:iolist_to_binary(T1) + end + end. + +cached(CacheName, Fun, Arg) -> + Cache = case get(CacheName) of + undefined -> + []; + Other -> + Other + end, case lists:keyfind(Arg, 1, Cache) of {_, V} -> V; @@ -74,6 +118,11 @@ cached(CacheName, Fun, Arg) -> V end. +%% amqp mqtt descr +%% * + match one topic level +%% # # match multiple topic levels +%% . / topic level separator + to_amqp(T0) -> T1 = string:replace(T0, "/", ".", all), T2 = string:replace(T1, "+", "*", all), @@ -84,53 +133,6 @@ to_mqtt(T0) -> T2 = string:replace(T1, ".", "/", all), erlang:iolist_to_binary(T2). -%% amqp mqtt descr -%% * + match one topic level -%% # # match multiple topic levels -%% . / topic level separator -get_topic_translation_funs() -> - SparkplugB = env(sparkplug), - ToAmqpFun = fun(Topic) -> - cached(mta_cache, fun to_amqp/1, Topic) - end, - ToMqttFun = fun(Topic) -> - cached(atm_cache, fun to_mqtt/1, Topic) - end, - {M2AFun, A2MFun} = case SparkplugB of - true -> - {ok, M2A_SpRe} = re:compile("^sp[AB]v\\d+\\.\\d+/"), - {ok, A2M_SpRe} = re:compile("^sp[AB]v\\d+___\\d+\\."), - M2A = fun(T0) -> - case re:run(T0, M2A_SpRe) of - nomatch -> - ToAmqpFun(T0); - {match, _} -> - T1 = string:replace(T0, ".", "___", leading), - ToAmqpFun(T1) - end - end, - A2M = fun(T0) -> - case re:run(T0, A2M_SpRe) of - nomatch -> - ToMqttFun(T0); - {match, _} -> - T1 = ToMqttFun(T0), - T2 = string:replace(T1, "___", ".", leading), - erlang:iolist_to_binary(T2) - end - end, - {M2A, A2M}; - _ -> - M2A = fun(T) -> - ToAmqpFun(T) - end, - A2M = fun(T) -> - ToMqttFun(T) - end, - {M2A, A2M} - end, - {ok, {mqtt2amqp_fun, M2AFun}, {amqp2mqtt_fun, A2MFun}}. - -spec gen_client_id() -> binary(). gen_client_id() -> rabbit_misc:base64url(rabbit_guid:gen_secure()). diff --git a/deps/rabbitmq_mqtt/test/util_SUITE.erl b/deps/rabbitmq_mqtt/test/util_SUITE.erl index 5f5c4327b151..f9d7bc4bd122 100644 --- a/deps/rabbitmq_mqtt/test/util_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/util_SUITE.erl @@ -55,26 +55,25 @@ coerce_default_pass(_) -> mqtt_amqp_topic_translation(_) -> ok = application:set_env(rabbitmq_mqtt, sparkplug, true), - {ok, {mqtt2amqp_fun, Mqtt2AmqpFun}, {amqp2mqtt_fun, Amqp2MqttFun}} = - rabbit_mqtt_util:get_topic_translation_funs(), + ok = rabbit_mqtt_util:init_sparkplug(), T0 = "/foo/bar/+/baz", T0_As_Amqp = <<".foo.bar.*.baz">>, T0_As_Mqtt = <<"/foo/bar/+/baz">>, - ?assertEqual(T0_As_Amqp, Mqtt2AmqpFun(T0)), - ?assertEqual(T0_As_Mqtt, Amqp2MqttFun(T0_As_Amqp)), + ?assertEqual(T0_As_Amqp, rabbit_mqtt_util:mqtt_to_amqp(T0)), + ?assertEqual(T0_As_Mqtt, rabbit_mqtt_util:amqp_to_mqtt(T0_As_Amqp)), T1 = "spAv1.0/foo/bar/+/baz", T1_As_Amqp = <<"spAv1___0.foo.bar.*.baz">>, T1_As_Mqtt = <<"spAv1.0/foo/bar/+/baz">>, - ?assertEqual(T1_As_Amqp, Mqtt2AmqpFun(T1)), - ?assertEqual(T1_As_Mqtt, Amqp2MqttFun(T1_As_Amqp)), + ?assertEqual(T1_As_Amqp, rabbit_mqtt_util:mqtt_to_amqp(T1)), + ?assertEqual(T1_As_Mqtt, rabbit_mqtt_util:amqp_to_mqtt(T1_As_Amqp)), T2 = "spBv2.90/foo/bar/+/baz", T2_As_Amqp = <<"spBv2___90.foo.bar.*.baz">>, T2_As_Mqtt = <<"spBv2.90/foo/bar/+/baz">>, - ?assertEqual(T2_As_Amqp, Mqtt2AmqpFun(T2)), - ?assertEqual(T2_As_Mqtt, Amqp2MqttFun(T2_As_Amqp)), + ?assertEqual(T2_As_Amqp, rabbit_mqtt_util:mqtt_to_amqp(T2)), + ?assertEqual(T2_As_Mqtt, rabbit_mqtt_util:amqp_to_mqtt(T2_As_Amqp)), ok = application:unset_env(rabbitmq_mqtt, sparkplug), ok. diff --git a/deps/rabbitmq_web_mqtt/Makefile b/deps/rabbitmq_web_mqtt/Makefile index 27d82360fb72..95ef34e7663a 100644 --- a/deps/rabbitmq_web_mqtt/Makefile +++ b/deps/rabbitmq_web_mqtt/Makefile @@ -24,6 +24,8 @@ TEST_DEPS = emqtt rabbitmq_ct_helpers rabbitmq_ct_client_helpers # See rabbitmq-components.mk. BUILD_DEPS += ranch +dep_emqtt = git https://github.com/emqx/emqtt.git 1.7.0-rc.2 + DEP_EARLY_PLUGINS = rabbit_common/mk/rabbitmq-early-plugin.mk DEP_PLUGINS = rabbit_common/mk/rabbitmq-plugin.mk From 3980c2859608ac379ba52e008c539908bfffd1f7 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Sat, 3 Dec 2022 18:53:15 +0000 Subject: [PATCH 072/118] Allow higher load on Mnesia by default Prior to this commit, when connecting or disconnecting many thousands of MQTT subscribers, RabbitMQ printed many times: ``` [warning] <0.241.0> Mnesia('rabbit@mqtt-rabbit-1-server-0.mqtt-rabbit-1-nodes.default'): ** WARNING ** Mnesia is overloaded: {dump_log,write_threshold} ``` Each MQTT subscription causes queues and bindings to be written into Mnesia. In order to allow for higher Mnesia load, the user can configure ``` [ {mnesia,[ {dump_log_write_threshold, 10000} ]} ]. ``` in advanced.config or set this value via ``` RABBITMQ_SERVER_ADDITIONAL_ERL_ARGS="-mnesia dump_log_write_threshold 10000" ``` The Mnesia default for dump_log_write_threshold is 1,000. The Mnesia default for dump_log_time_threshold is 180,000 ms. It is reasonable to increase the default for dump_log_write_threshold from 1,000 to 5,000 and in return decrease the default dump_log_time_threshold from 3 minutes to 1.5 minutes. This way, users can achieve higher MQTT scalability by default. This setting cannot be changed at Mnesia runtime, it needs to be set before Mnesia gets started. Since the rabbitmq_mqtt plugin can be enabled dynamically after Mnesia started, this setting must therefore apply globally to RabbitMQ. Users can continue to set their own defaults via advanced.config or RABBITMQ_SERVER_ADDITIONAL_ERL_ARGS. They continue to be respected as shown by the new test suite included in this commit. --- .../src/rabbit_prelaunch_conf.erl | 9 +- deps/rabbit/test/quorum_queue_SUITE.erl | 2 - .../src/rabbit_ct_broker_helpers.erl | 3 +- deps/rabbitmq_mqtt/BUILD.bazel | 6 ++ deps/rabbitmq_mqtt/test/config_SUITE.erl | 94 +++++++++++++++++++ 5 files changed, 109 insertions(+), 5 deletions(-) create mode 100644 deps/rabbitmq_mqtt/test/config_SUITE.erl diff --git a/deps/rabbit/apps/rabbitmq_prelaunch/src/rabbit_prelaunch_conf.erl b/deps/rabbit/apps/rabbitmq_prelaunch/src/rabbit_prelaunch_conf.erl index b6156ff5c27c..f67a0c1bfd94 100644 --- a/deps/rabbit/apps/rabbitmq_prelaunch/src/rabbit_prelaunch_conf.erl +++ b/deps/rabbit/apps/rabbitmq_prelaunch/src/rabbit_prelaunch_conf.erl @@ -122,8 +122,13 @@ set_default_config() -> {schedule_ms_limit, 0}, {heap_word_limit, 0}, {busy_port, false}, - {busy_dist_port, true}]} - | OsirisConfig + {busy_dist_port, true}]}, + {mnesia, + [ + {dump_log_write_threshold, 5000}, + {dump_log_time_threshold, 90000} + ]} + | OsirisConfig ], apply_erlang_term_based_config(Config). diff --git a/deps/rabbit/test/quorum_queue_SUITE.erl b/deps/rabbit/test/quorum_queue_SUITE.erl index 485ab9926d12..15f15102a68f 100644 --- a/deps/rabbit/test/quorum_queue_SUITE.erl +++ b/deps/rabbit/test/quorum_queue_SUITE.erl @@ -171,8 +171,6 @@ init_per_suite(Config0) -> rabbit_ct_helpers:log_environment(), Config1 = rabbit_ct_helpers:merge_app_env( Config0, {rabbit, [{quorum_tick_interval, 1000}]}), - rabbit_ct_helpers:merge_app_env( - Config1, {aten, [{poll_interval, 1000}]}), rabbit_ct_helpers:run_setup_steps(Config1, []). end_per_suite(Config) -> diff --git a/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl b/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl index 142a048099eb..b38aaab20a4b 100644 --- a/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl +++ b/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl @@ -639,6 +639,7 @@ do_start_rabbitmq_node(Config, NodeConfig, I) -> InitialNodename = ?config(initial_nodename, NodeConfig), DistPort = ?config(tcp_port_erlang_dist, NodeConfig), ConfigFile = ?config(erlang_node_config_filename, NodeConfig), + AdditionalErlArgs = rabbit_ct_helpers:get_config(Config, additional_erl_args, []), %% Use inet_proxy_dist to handle distribution. This is used by the %% partitions testsuite. DistMod = rabbit_ct_helpers:get_config(Config, erlang_dist_module), @@ -735,7 +736,7 @@ do_start_rabbitmq_node(Config, NodeConfig, I) -> {"RABBITMQ_DIST_PORT=~b", [DistPort]}, {"RABBITMQ_CONFIG_FILE=~ts", [ConfigFile]}, {"RABBITMQ_SERVER_START_ARGS=~ts", [StartArgs1]}, - "RABBITMQ_SERVER_ADDITIONAL_ERL_ARGS=+S 2 +sbwt very_short +A 24", + {"RABBITMQ_SERVER_ADDITIONAL_ERL_ARGS=+S 2 +sbwt very_short +A 24 ~ts", [AdditionalErlArgs]}, "RABBITMQ_LOG=debug", "RMQCTL_WAIT_TIMEOUT=180", {"TEST_TMPDIR=~ts", [PrivDir]} diff --git a/deps/rabbitmq_mqtt/BUILD.bazel b/deps/rabbitmq_mqtt/BUILD.bazel index 51a61885c395..8636e92d40a6 100644 --- a/deps/rabbitmq_mqtt/BUILD.bazel +++ b/deps/rabbitmq_mqtt/BUILD.bazel @@ -151,6 +151,12 @@ rabbitmq_integration_suite( name = "config_schema_SUITE", ) + +rabbitmq_integration_suite( + name = "config_SUITE", + size = "small", +) + rabbitmq_integration_suite( name = "java_SUITE", flaky = True, diff --git a/deps/rabbitmq_mqtt/test/config_SUITE.erl b/deps/rabbitmq_mqtt/test/config_SUITE.erl new file mode 100644 index 000000000000..edcfc4934190 --- /dev/null +++ b/deps/rabbitmq_mqtt/test/config_SUITE.erl @@ -0,0 +1,94 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2007-2022 VMware, Inc. or its affiliates. All rights reserved. + +-module(config_SUITE). +-compile([export_all, + nowarn_export_all]). + +-include_lib("eunit/include/eunit.hrl"). + +-import(rabbit_ct_broker_helpers, [rpc/5]). + +all() -> + [ + {group, mnesia} + ]. + +groups() -> + [ + {mnesia, [shuffle], + [ + rabbitmq_default, + environment_set, + flag_set + ]} + ]. + +suite() -> + [{timetrap, {seconds, 30}}]. + +%% ------------------------------------------------------------------- +%% Testsuite setup/teardown. +%% ------------------------------------------------------------------- + +init_per_suite(Config) -> + rabbit_ct_helpers:log_environment(), + rabbit_ct_helpers:run_setup_steps(Config). + +end_per_suite(Config) -> + rabbit_ct_helpers:run_teardown_steps(Config). + +init_per_testcase(rabbitmq_default = Test, Config) -> + init_per_testcase0(Test, Config); +init_per_testcase(environment_set = Test, Config0) -> + Config = rabbit_ct_helpers:merge_app_env( + Config0, {mnesia, [{dump_log_write_threshold, 25000}, + {dump_log_time_threshold, 60000}]}), + init_per_testcase0(Test, Config); +init_per_testcase(flag_set = Test, Config0) -> + Config = [{additional_erl_args, "-mnesia dump_log_write_threshold 15000"} | Config0], + init_per_testcase0(Test, Config). + +init_per_testcase0(Testcase, Config0) -> + Config1 = rabbit_ct_helpers:set_config(Config0, {rmq_nodename_suffix, Testcase}), + Config = rabbit_ct_helpers:run_steps( + Config1, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps()), + rabbit_ct_helpers:testcase_started(Config, Testcase). + +end_per_testcase(Testcase, Config0) -> + Config = rabbit_ct_helpers:testcase_finished(Config0, Testcase), + rabbit_ct_helpers:run_teardown_steps( + Config, + rabbit_ct_client_helpers:teardown_steps() ++ + rabbit_ct_broker_helpers:teardown_steps()). + +%% ------------------------------------------------------------------- +%% Testsuite cases +%% ------------------------------------------------------------------- + +%% The MQTT plugin expects Mnesia dump_log_write_threshold to be increased +%% from 1000 (Mnesia default) to 5000 (RabbitMQ default). +rabbitmq_default(Config) -> + ?assertEqual(5_000, + rpc(Config, 0, mnesia, system_info, [dump_log_write_threshold])), + ?assertEqual(90_000, + rpc(Config, 0, mnesia, system_info, [dump_log_time_threshold])). + +%% User configured setting in advanced.config should be respected. +environment_set(Config) -> + ?assertEqual(25_000, + rpc(Config, 0, mnesia, system_info, [dump_log_write_threshold])), + ?assertEqual(60_000, + rpc(Config, 0, mnesia, system_info, [dump_log_time_threshold])). + +%% User configured setting in RABBITMQ_SERVER_ADDITIONAL_ERL_ARGS should be respected. +flag_set(Config) -> + ?assertEqual(15_000, + rpc(Config, 0, mnesia, system_info, [dump_log_write_threshold])), + ?assertEqual(90_000, + rpc(Config, 0, mnesia, system_info, [dump_log_time_threshold])). From 15636fdb905c758fd8deca0a736528deaa7ac740 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Sun, 4 Dec 2022 14:39:22 +0000 Subject: [PATCH 073/118] Rename frame to packet The MQTT protocol specs define the term "MQTT Control Packet". The MQTT specs never talk about "frame". Let's reflect this naming in the source code since things get confusing otherwise: Packets belong to MQTT. Frames belong to AMQP 0.9.1 or web sockets. --- deps/rabbit_common/src/rabbit_writer.erl | 2 +- ..._mqtt_frame.hrl => rabbit_mqtt_packet.hrl} | 92 ++++----- .../src/rabbit_mqtt_confirms.erl | 2 +- ..._mqtt_frame.erl => rabbit_mqtt_packet.erl} | 66 +++---- .../src/rabbit_mqtt_processor.erl | 180 +++++++++--------- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 40 ++-- .../src/rabbit_mqtt_retainer.erl | 2 +- deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl | 2 +- .../test/proxy_protocol_SUITE.erl | 6 +- deps/rabbitmq_mqtt/test/reader_SUITE.erl | 8 +- .../src/rabbit_web_mqtt_handler.erl | 24 +-- .../test/proxy_protocol_SUITE.erl | 2 +- .../test/src/rabbit_ws_test_util.erl | 4 +- deps/rabbitmq_web_mqtt/test/system_SUITE.erl | 2 +- 14 files changed, 212 insertions(+), 220 deletions(-) rename deps/rabbitmq_mqtt/include/{rabbit_mqtt_frame.hrl => rabbit_mqtt_packet.hrl} (50%) rename deps/rabbitmq_mqtt/src/{rabbit_mqtt_frame.erl => rabbit_mqtt_packet.erl} (79%) diff --git a/deps/rabbit_common/src/rabbit_writer.erl b/deps/rabbit_common/src/rabbit_writer.erl index 6d4610cd0978..a284d203b1b5 100644 --- a/deps/rabbit_common/src/rabbit_writer.erl +++ b/deps/rabbit_common/src/rabbit_writer.erl @@ -64,7 +64,7 @@ -define(HIBERNATE_AFTER, 5000). %% 1GB --define(DEFAULT_GC_THRESHOLD, 1000000000). +-define(DEFAULT_GC_THRESHOLD, 1_000_000_000). %%--------------------------------------------------------------------------- diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt_frame.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt_packet.hrl similarity index 50% rename from deps/rabbitmq_mqtt/include/rabbit_mqtt_frame.hrl rename to deps/rabbitmq_mqtt/include/rabbit_mqtt_packet.hrl index 70af22e25dc7..db5d8fe9a5d6 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt_frame.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt_packet.hrl @@ -9,7 +9,7 @@ [{3, "MQIsdp"}, {4, "MQTT"}]). -%% frame types +%% packet types -define(CONNECT, 1). -define(CONNACK, 2). @@ -53,50 +53,50 @@ %% Packet identifier is a non zero two byte integer. -type packet_id() :: 1..16#ffff. --record(mqtt_frame, {fixed, - variable, - payload}). - --record(mqtt_frame_fixed, {type = 0, - dup = 0, - qos = 0, - retain = 0}). - --record(mqtt_frame_connect, {proto_ver, - will_retain, - will_qos, - will_flag, - clean_sess, - keep_alive, - client_id, - will_topic, - will_msg, - username, - password}). - --record(mqtt_frame_connack, {session_present, - return_code}). - --record(mqtt_frame_publish, {topic_name, - packet_id :: packet_id()}). - --record(mqtt_topic, {name, - qos}). - --record(mqtt_frame_subscribe,{packet_id :: packet_id(), - topic_table :: nonempty_list(#mqtt_topic{}) - }). - --record(mqtt_frame_suback, {packet_id :: packet_id(), - qos_table = []}). - --record(mqtt_frame_other, {other}). - --record(mqtt_msg, {retain :: boolean(), - qos :: qos(), - topic :: string(), - dup :: boolean(), - packet_id :: packet_id(), - payload :: binary()}). +-record(mqtt_packet, {fixed, + variable, + payload}). + +-record(mqtt_packet_fixed, {type = 0, + dup = 0, + qos = 0, + retain = 0}). + +-record(mqtt_packet_connect, {proto_ver, + will_retain, + will_qos, + will_flag, + clean_sess, + keep_alive, + client_id, + will_topic, + will_msg, + username, + password}). + +-record(mqtt_packet_connack, {session_present, + return_code}). + +-record(mqtt_packet_publish, {topic_name, + packet_id :: packet_id()}). + +-record(mqtt_topic, {name, + qos}). + +-record(mqtt_packet_subscribe,{packet_id :: packet_id(), + topic_table :: nonempty_list(#mqtt_topic{}) + }). + +-record(mqtt_packet_suback, {packet_id :: packet_id(), + qos_table = []}). + +-record(mqtt_packet_other, {other}). + +-record(mqtt_msg, {retain :: boolean(), + qos :: qos(), + topic :: string(), + dup :: boolean(), + packet_id :: packet_id(), + payload :: binary()}). -type mqtt_msg() :: #mqtt_msg{}. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl index 0f3b89908131..36f2f12a1e39 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl @@ -7,7 +7,7 @@ -module(rabbit_mqtt_confirms). --include("rabbit_mqtt_frame.hrl"). +-include("rabbit_mqtt_packet.hrl"). -compile({no_auto_import, [size/1]}). -export([init/0, diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_frame.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_packet.erl similarity index 79% rename from deps/rabbitmq_mqtt/src/rabbit_mqtt_frame.erl rename to deps/rabbitmq_mqtt/src/rabbit_mqtt_packet.erl index accf9d9fcf08..113374b3d166 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_frame.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_packet.erl @@ -5,12 +5,12 @@ %% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved. %% --module(rabbit_mqtt_frame). +-module(rabbit_mqtt_packet). -export([parse/2, initial_state/0]). -export([serialise/2]). --include("rabbit_mqtt_frame.hrl"). +-include("rabbit_mqtt_packet.hrl"). -include("rabbit_mqtt.hrl"). -define(RESERVED, 0). @@ -23,7 +23,7 @@ initial_state() -> none. parse(<<>>, none) -> {more, fun(Bin) -> parse(Bin, none) end}; parse(<>, none) -> - parse_remaining_len(Rest, #mqtt_frame_fixed{ type = MessageType, + parse_remaining_len(Rest, #mqtt_packet_fixed{ type = MessageType, dup = bool(Dup), qos = QoS, retain = bool(Retain) }); @@ -36,19 +36,19 @@ parse_remaining_len(Rest, Fixed) -> parse_remaining_len(_Bin, _Fixed, _Multiplier, Length) when Length > ?MAX_LEN -> - {error, invalid_mqtt_frame_len}; + {error, invalid_mqtt_packet_len}; parse_remaining_len(<<>>, Fixed, Multiplier, Length) -> {more, fun(Bin) -> parse_remaining_len(Bin, Fixed, Multiplier, Length) end}; parse_remaining_len(<<1:1, Len:7, Rest/binary>>, Fixed, Multiplier, Value) -> parse_remaining_len(Rest, Fixed, Multiplier * ?HIGHBIT, Value + Len * Multiplier); parse_remaining_len(<<0:1, Len:7, Rest/binary>>, Fixed, Multiplier, Value) -> - parse_frame(Rest, Fixed, Value + Len * Multiplier). + parse_packet(Rest, Fixed, Value + Len * Multiplier). -parse_frame(Bin, #mqtt_frame_fixed{ type = Type, +parse_packet(Bin, #mqtt_packet_fixed{ type = Type, qos = Qos } = Fixed, Length) -> case {Type, Bin} of - {?CONNECT, <>} -> - {ProtoName, Rest1} = parse_utf(FrameBin), + {?CONNECT, <>} -> + {ProtoName, Rest1} = parse_utf(PacketBin), <> = Rest1, < wrap(Fixed, - #mqtt_frame_connect{ + #mqtt_packet_connect{ proto_ver = ProtoVersion, will_retain = bool(WillRetain), will_qos = WillQos, @@ -82,25 +82,25 @@ parse_frame(Bin, #mqtt_frame_fixed{ type = Type, false -> {error, protocol_header_corrupt} end; - {?PUBLISH, <>} -> - {TopicName, Rest1} = parse_utf(FrameBin), + {?PUBLISH, <>} -> + {TopicName, Rest1} = parse_utf(PacketBin), {PacketId, Payload} = case Qos of 0 -> {undefined, Rest1}; _ -> <> = Rest1, {M, R} end, - wrap(Fixed, #mqtt_frame_publish { topic_name = TopicName, + wrap(Fixed, #mqtt_packet_publish { topic_name = TopicName, packet_id = PacketId }, Payload, Rest); - {?PUBACK, <>} -> - <> = FrameBin, - wrap(Fixed, #mqtt_frame_publish { packet_id = PacketId }, Rest); - {Subs, <>} + {?PUBACK, <>} -> + <> = PacketBin, + wrap(Fixed, #mqtt_packet_publish { packet_id = PacketId }, Rest); + {Subs, <>} when Subs =:= ?SUBSCRIBE orelse Subs =:= ?UNSUBSCRIBE -> 1 = Qos, - <> = FrameBin, + <> = PacketBin, Topics = parse_topics(Subs, Rest1, []), - wrap(Fixed, #mqtt_frame_subscribe { packet_id = PacketId, + wrap(Fixed, #mqtt_packet_subscribe { packet_id = PacketId, topic_table = Topics }, Rest); {Minimal, Rest} when Minimal =:= ?DISCONNECT orelse Minimal =:= ?PINGREQ -> @@ -108,7 +108,7 @@ parse_frame(Bin, #mqtt_frame_fixed{ type = Type, wrap(Fixed, Rest); {_, TooShortBin} -> {more, fun(BinMore) -> - parse_frame(<>, + parse_packet(<>, Fixed, Length) end} end. @@ -123,11 +123,11 @@ parse_topics(?UNSUBSCRIBE = Sub, Bin, Topics) -> parse_topics(Sub, Rest, [#mqtt_topic { name = Name } | Topics]). wrap(Fixed, Variable, Payload, Rest) -> - {ok, #mqtt_frame { variable = Variable, fixed = Fixed, payload = Payload }, Rest}. + {ok, #mqtt_packet { variable = Variable, fixed = Fixed, payload = Payload }, Rest}. wrap(Fixed, Variable, Rest) -> - {ok, #mqtt_frame { variable = Variable, fixed = Fixed }, Rest}. + {ok, #mqtt_packet { variable = Variable, fixed = Fixed }, Rest}. wrap(Fixed, Rest) -> - {ok, #mqtt_frame { fixed = Fixed }, Rest}. + {ok, #mqtt_packet { fixed = Fixed }, Rest}. parse_utf(Bin, 0) -> {undefined, Bin}; @@ -150,7 +150,7 @@ bool(1) -> true. %% serialisation -serialise(#mqtt_frame{fixed = Fixed, +serialise(#mqtt_packet{fixed = Fixed, variable = Variable, payload = Payload}, Vsn) -> serialise_variable(Fixed, Variable, serialise_payload(Payload), Vsn). @@ -158,15 +158,15 @@ serialise(#mqtt_frame{fixed = Fixed, serialise_payload(undefined) -> <<>>; serialise_payload(B) when is_binary(B) -> B. -serialise_variable(#mqtt_frame_fixed { type = ?CONNACK } = Fixed, - #mqtt_frame_connack { session_present = SessionPresent, +serialise_variable(#mqtt_packet_fixed { type = ?CONNACK } = Fixed, + #mqtt_packet_connack { session_present = SessionPresent, return_code = ReturnCode }, <<>> = PayloadBin, _Vsn) -> VariableBin = <>, serialise_fixed(Fixed, VariableBin, PayloadBin); -serialise_variable(#mqtt_frame_fixed { type = SubAck } = Fixed, - #mqtt_frame_suback { packet_id = PacketId, +serialise_variable(#mqtt_packet_fixed { type = SubAck } = Fixed, + #mqtt_packet_suback { packet_id = PacketId, qos_table = Qos }, <<>> = _PayloadBin, Vsn) when SubAck =:= ?SUBACK orelse SubAck =:= ?UNSUBACK -> @@ -180,9 +180,9 @@ serialise_variable(#mqtt_frame_fixed { type = SubAck } = Fixed, end, serialise_fixed(Fixed, VariableBin, QosBin); -serialise_variable(#mqtt_frame_fixed { type = ?PUBLISH, +serialise_variable(#mqtt_packet_fixed { type = ?PUBLISH, qos = Qos } = Fixed, - #mqtt_frame_publish { topic_name = TopicName, + #mqtt_packet_publish { topic_name = TopicName, packet_id = PacketId }, PayloadBin, _Vsn) -> TopicBin = serialise_utf(TopicName), @@ -192,18 +192,18 @@ serialise_variable(#mqtt_frame_fixed { type = ?PUBLISH, end, serialise_fixed(Fixed, <>, PayloadBin); -serialise_variable(#mqtt_frame_fixed { type = ?PUBACK } = Fixed, - #mqtt_frame_publish { packet_id = PacketId }, +serialise_variable(#mqtt_packet_fixed { type = ?PUBACK } = Fixed, + #mqtt_packet_publish { packet_id = PacketId }, PayloadBin, _Vsn) -> PacketIdBin = <>, serialise_fixed(Fixed, PacketIdBin, PayloadBin); -serialise_variable(#mqtt_frame_fixed {} = Fixed, +serialise_variable(#mqtt_packet_fixed {} = Fixed, undefined, <<>> = _PayloadBin, _Vsn) -> serialise_fixed(Fixed, <<>>, <<>>). -serialise_fixed(#mqtt_frame_fixed{ type = Type, +serialise_fixed(#mqtt_packet_fixed{ type = Type, dup = Dup, qos = Qos, retain = Retain }, VariableBin, PayloadBin) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 741cdf35fef1..41604cb663d4 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -8,7 +8,7 @@ -module(rabbit_mqtt_processor). -export([info/2, initial_state/2, initial_state/4, - process_frame/2, serialise/2, + process_packet/2, serialise/2, terminate/3, handle_pre_hibernate/0, handle_ra_event/2, handle_down/2, handle_queue_event/2, soft_limit_exceeded/1, format_status/1]). @@ -25,7 +25,7 @@ -include_lib("rabbit_common/include/rabbit_framing.hrl"). -include_lib("rabbit/include/amqqueue.hrl"). -include("rabbit_mqtt.hrl"). --include("rabbit_mqtt_frame.hrl"). +-include("rabbit_mqtt_packet.hrl"). -define(APP, rabbitmq_mqtt). -define(MAX_PERMISSION_CACHE_SIZE, 12). @@ -68,7 +68,7 @@ retainer_pid, auth_state, peer_addr, - send_fun :: fun((Frame :: tuple(), state()) -> term()), + send_fun :: fun((Packet :: tuple(), state()) -> term()), register_state, conn_name, info, @@ -98,14 +98,14 @@ initial_state(Socket, ConnectionName, SendFun, PeerAddr) -> send_fun = SendFun, delivery_flow = Flow}. -process_frame(#mqtt_frame{fixed = #mqtt_frame_fixed{type = Type}}, +process_packet(#mqtt_packet{fixed = #mqtt_packet_fixed{type = Type}}, State = #state{auth_state = undefined}) when Type =/= ?CONNECT -> {error, connect_expected, State}; -process_frame(Frame = #mqtt_frame{fixed = #mqtt_frame_fixed{type = Type}}, State) -> - process_request(Type, Frame, State). +process_packet(Packet = #mqtt_packet{fixed = #mqtt_packet_fixed{type = Type}}, State) -> + process_request(Type, Packet, State). -process_request(?CONNECT, Frame, State = #state{socket = Socket}) -> +process_request(?CONNECT, Packet, State = #state{socket = Socket}) -> %% Check whether peer closed the connection. %% For example, this can happen when connection was blocked because of resource %% alarm and client therefore disconnected due to client side CONNACK timeout. @@ -113,12 +113,12 @@ process_request(?CONNECT, Frame, State = #state{socket = Socket}) -> {error, Reason} -> {error, {socket_ends, Reason}, State}; _ -> - process_connect(Frame, State) + process_connect(Packet, State) end; process_request(?PUBACK, - #mqtt_frame{ - variable = #mqtt_frame_publish{packet_id = PacketId}}, + #mqtt_packet{ + variable = #mqtt_packet_publish{packet_id = PacketId}}, #state{unacked_server_pubs = U0, queue_states = QStates0, queue_qos1 = QName} = State) -> @@ -138,21 +138,21 @@ process_request(?PUBACK, end; process_request(?PUBLISH, - Frame = #mqtt_frame{ - fixed = Fixed = #mqtt_frame_fixed{qos = ?QOS_2}}, + Packet = #mqtt_packet{ + fixed = Fixed = #mqtt_packet_fixed{qos = ?QOS_2}}, State) -> % Downgrade QOS_2 to QOS_1 process_request(?PUBLISH, - Frame#mqtt_frame{ - fixed = Fixed#mqtt_frame_fixed{qos = ?QOS_1}}, + Packet#mqtt_packet{ + fixed = Fixed#mqtt_packet_fixed{qos = ?QOS_1}}, State); process_request(?PUBLISH, - #mqtt_frame{ - fixed = #mqtt_frame_fixed{qos = Qos, - retain = Retain, - dup = Dup }, - variable = #mqtt_frame_publish{topic_name = Topic, - packet_id = PacketId }, + #mqtt_packet{ + fixed = #mqtt_packet_fixed{qos = Qos, + retain = Retain, + dup = Dup }, + variable = #mqtt_packet_publish{topic_name = Topic, + packet_id = PacketId }, payload = Payload}, State0 = #state{retainer_pid = RPid, unacked_client_pubs = U, @@ -164,7 +164,7 @@ process_request(?PUBLISH, qos = Qos, topic = Topic, dup = Dup, - packet_id = PacketId, + packet_id = PacketId, payload = Payload}, case publish_to_queues(Msg, State) of {ok, _} = Ok -> @@ -196,8 +196,8 @@ process_request(?PUBLISH, end; process_request(?SUBSCRIBE, - #mqtt_frame{ - variable = #mqtt_frame_subscribe{ + #mqtt_packet{ + variable = #mqtt_packet_subscribe{ packet_id = SubscribePktId, topic_table = Topics}, payload = undefined}, @@ -252,10 +252,10 @@ process_request(?SUBSCRIBE, maybe_increment_consumer(HasSubsBefore, State1), SendFun( - #mqtt_frame{fixed = #mqtt_frame_fixed{type = ?SUBACK}, - variable = #mqtt_frame_suback{ - packet_id = SubscribePktId, - qos_table = QosResponse}}, + #mqtt_packet{fixed = #mqtt_packet_fixed{type = ?SUBACK}, + variable = #mqtt_packet_suback{ + packet_id = SubscribePktId, + qos_table = QosResponse}}, State1), case QosResponse of [?SUBACK_FAILURE | _] -> @@ -268,8 +268,8 @@ process_request(?SUBSCRIBE, end; process_request(?UNSUBSCRIBE, - #mqtt_frame{variable = #mqtt_frame_subscribe{packet_id = PacketId, - topic_table = Topics}, + #mqtt_packet{variable = #mqtt_packet_subscribe{packet_id = PacketId, + topic_table = Topics}, payload = undefined}, State0 = #state{send_fun = SendFun}) -> rabbit_log_connection:debug("Received an UNSUBSCRIBE for topic(s) ~p", [Topics]), @@ -291,32 +291,32 @@ process_request(?UNSUBSCRIBE, end end, State0, Topics), SendFun( - #mqtt_frame{fixed = #mqtt_frame_fixed {type = ?UNSUBACK}, - variable = #mqtt_frame_suback{packet_id = PacketId}}, + #mqtt_packet{fixed = #mqtt_packet_fixed {type = ?UNSUBACK}, + variable = #mqtt_packet_suback{packet_id = PacketId}}, State), maybe_decrement_consumer(HasSubsBefore, State), {ok, State}; -process_request(?PINGREQ, #mqtt_frame{}, State = #state{send_fun = SendFun}) -> +process_request(?PINGREQ, #mqtt_packet{}, State = #state{send_fun = SendFun}) -> rabbit_log_connection:debug("Received a PINGREQ"), SendFun( - #mqtt_frame{fixed = #mqtt_frame_fixed{type = ?PINGRESP}}, + #mqtt_packet{fixed = #mqtt_packet_fixed{type = ?PINGRESP}}, State), rabbit_log_connection:debug("Sent a PINGRESP"), {ok, State}; -process_request(?DISCONNECT, #mqtt_frame{}, State) -> +process_request(?DISCONNECT, #mqtt_packet{}, State) -> rabbit_log_connection:debug("Received a DISCONNECT"), {stop, disconnect, State}. -process_connect(#mqtt_frame{ - variable = #mqtt_frame_connect{ +process_connect(#mqtt_packet{ + variable = #mqtt_packet_connect{ username = Username, proto_ver = ProtoVersion, clean_sess = CleanSess, client_id = ClientId, - keep_alive = Keepalive} = FrameConnect}, + keep_alive = Keepalive} = PacketConnect}, State0 = #state{send_fun = SendFun}) -> rabbit_log_connection:debug("Received a CONNECT, client ID: ~s, username: ~s, " "clean session: ~s, protocol version: ~p, keepalive: ~p", @@ -330,17 +330,17 @@ process_connect(#mqtt_frame{ fun notify_connection_created/2, fun start_keepalive/2, fun handle_clean_session/2], - FrameConnect, State0) of + PacketConnect, State0) of {ok, SessionPresent0, State1} -> {?CONNACK_ACCEPT, SessionPresent0, State1}; {error, ReturnCode0, State1} -> {ReturnCode0, false, State1} end, - ResponseFrame = #mqtt_frame{fixed = #mqtt_frame_fixed{type = ?CONNACK}, - variable = #mqtt_frame_connack{ - session_present = SessionPresent, - return_code = ReturnCode}}, - SendFun(ResponseFrame, State), + ResponsePacket = #mqtt_packet{fixed = #mqtt_packet_fixed{type = ?CONNACK}, + variable = #mqtt_packet_connack{ + session_present = SessionPresent, + return_code = ReturnCode}}, + SendFun(ResponsePacket, State), return_connack(ReturnCode, State). client_id(<<>>) -> @@ -349,7 +349,7 @@ client_id(ClientId) when is_binary(ClientId) -> ClientId. -check_protocol_version(#mqtt_frame_connect{proto_ver = ProtoVersion}) -> +check_protocol_version(#mqtt_packet_connect{proto_ver = ProtoVersion}) -> case lists:member(ProtoVersion, proplists:get_keys(?PROTOCOL_NAMES)) of true -> ok; @@ -357,14 +357,14 @@ check_protocol_version(#mqtt_frame_connect{proto_ver = ProtoVersion}) -> {error, ?CONNACK_UNACCEPTABLE_PROTO_VER} end. -check_client_id(#mqtt_frame_connect{clean_sess = false, - client_id = []}) -> +check_client_id(#mqtt_packet_connect{clean_sess = false, + client_id = []}) -> {error, ?CONNACK_ID_REJECTED}; check_client_id(_) -> ok. -check_credentials(Frame = #mqtt_frame_connect{username = Username, - password = Password}, +check_credentials(Packet = #mqtt_packet_connect{username = Username, + password = Password}, State = #state{ssl_login_name = SslLoginName, peer_addr = PeerAddr}) -> Ip = list_to_binary(inet:ntoa(PeerAddr)), @@ -382,11 +382,11 @@ check_credentials(Frame = #mqtt_frame_connect{username = Username, rabbit_log_connection:error("MQTT login failed for user '~p': no password provided", [User]), {error, ?CONNACK_BAD_CREDENTIALS}; {UserBin, PassBin} -> - {ok, {UserBin, PassBin, Frame}, State} + {ok, {UserBin, PassBin, Packet}, State} end. login({UserBin, PassBin, - Frame = #mqtt_frame_connect{client_id = ClientId0, + Packet = #mqtt_packet_connect{client_id = ClientId0, clean_sess = CleanSess}}, State0) -> ClientId = client_id(ClientId0), @@ -394,7 +394,7 @@ login({UserBin, PassBin, already_connected -> {ok, already_connected}; {ok, State} -> - {ok, Frame, State#state{clean_sess = CleanSess, + {ok, Packet, State#state{clean_sess = CleanSess, client_id = ClientId}}; {error, _Reason, _State} = Err -> Err @@ -402,7 +402,7 @@ login({UserBin, PassBin, register_client(already_connected, _State) -> ok; -register_client(Frame = #mqtt_frame_connect{proto_ver = ProtoVersion}, +register_client(Packet = #mqtt_packet_connect{proto_ver = ProtoVersion}, State = #state{client_id = ClientId, socket = Socket, auth_state = #auth_state{vhost = VHost}}) -> @@ -417,7 +417,7 @@ register_client(Frame = #mqtt_frame_connect{proto_ver = ProtoVersion}, ProtoHumanReadable = {'MQTT', human_readable_mqtt_version(ProtoVersion)}, State#state{ exchange = ExchangeName, - will_msg = make_will_msg(Frame), + will_msg = make_will_msg(Packet), retainer_pid = RetainerPid, register_state = RegisterState, proto_ver = protocol_integer_to_atom(ProtoVersion), @@ -451,7 +451,7 @@ register_client(Frame = #mqtt_frame_connect{proto_ver = ProtoVersion}, notify_connection_created(already_connected, _State) -> ok; notify_connection_created( - _Frame, + _Packet, #state{socket = Sock, conn_name = ConnName, info = #info{proto_human = {ProtoName, ProtoVsn}}, @@ -508,7 +508,7 @@ self_consumes(Queue) -> end, rabbit_amqqueue:consumers(Queue)) end. -start_keepalive(#mqtt_frame_connect{keep_alive = Seconds}, +start_keepalive(#mqtt_packet_connect{keep_alive = Seconds}, #state{socket = Socket}) -> ok = rabbit_mqtt_keepalive:start(Seconds, Socket). @@ -662,26 +662,26 @@ maybe_send_retained_message(RPid, #mqtt_topic{name = Topic0, qos = SubscribeQos} {PacketId0, State0#state{packet_id = increment_packet_id(PacketId0)}} end, SendFun( - #mqtt_frame{fixed = #mqtt_frame_fixed{ - type = ?PUBLISH, - qos = Qos, - dup = false, - retain = Msg#mqtt_msg.retain - }, variable = #mqtt_frame_publish{ - packet_id = PacketId, - topic_name = Topic1 - }, + #mqtt_packet{fixed = #mqtt_packet_fixed{ + type = ?PUBLISH, + qos = Qos, + dup = false, + retain = Msg#mqtt_msg.retain + }, variable = #mqtt_packet_publish{ + packet_id = PacketId, + topic_name = Topic1 + }, payload = Msg#mqtt_msg.payload}, State), State end. -make_will_msg(#mqtt_frame_connect{will_flag = false}) -> +make_will_msg(#mqtt_packet_connect{will_flag = false}) -> undefined; -make_will_msg(#mqtt_frame_connect{will_retain = Retain, - will_qos = Qos, - will_topic = Topic, - will_msg = Msg}) -> +make_will_msg(#mqtt_packet_connect{will_retain = Retain, + will_qos = Qos, + will_topic = Topic, + will_msg = Msg}) -> #mqtt_msg{retain = Retain, qos = Qos, topic = Topic, @@ -1210,7 +1210,7 @@ publish_to_queues( #mqtt_msg{qos = Qos, topic = Topic, dup = Dup, - packet_id = PacketId, + packet_id = PacketId, payload = Payload}, #state{exchange = ExchangeName, delivery_flow = Flow} = State) -> @@ -1319,25 +1319,22 @@ send_puback(PktId, State = #state{send_fun = SendFun, proto_ver = ProtoVer}) -> rabbit_global_counters:messages_confirmed(ProtoVer, 1), SendFun( - #mqtt_frame{fixed = #mqtt_frame_fixed{type = ?PUBACK}, - variable = #mqtt_frame_publish{packet_id = PktId}}, + #mqtt_packet{fixed = #mqtt_packet_fixed{type = ?PUBACK}, + variable = #mqtt_packet_publish{packet_id = PktId}}, State). -serialise_and_send_to_client(Frame, #state{proto_ver = ProtoVer, socket = Sock}) -> - %%TODO Test sending large frames at high speed: - %% Will we need garbage collection as done in rabbit_writer:maybe_gc_large_msg/1? - %%TODO batch to fill up MTU if there are messages in the Erlang mailbox? - %% Check rabbit_writer:maybe_flush/1 - Data = rabbit_mqtt_frame:serialise(Frame, ProtoVer), +serialise_and_send_to_client(Packet, #state{proto_ver = ProtoVer, socket = Sock}) -> + Data = rabbit_mqtt_packet:serialise(Packet, ProtoVer), try rabbit_net:port_command(Sock, Data) - catch _:Error -> - rabbit_log_connection:error("MQTT: a socket write failed, the socket might already be closed"), - rabbit_log_connection:debug("Failed to write to socket ~p, error: ~p, frame: ~p", - [Sock, Error, Frame]) + catch error:Error -> + rabbit_log_connection:error( + "MQTT: a socket write failed: ~p", [Error]), + rabbit_log_connection:debug( + "Failed to write to socket ~p, error: ~p, packet: ~p", [Sock, Error, Packet]) end. -serialise(Frame, #state{proto_ver = ProtoVer}) -> - rabbit_mqtt_frame:serialise(Frame, ProtoVer). +serialise(Packet, #state{proto_ver = ProtoVer}) -> + rabbit_mqtt_packet:serialise(Packet, ProtoVer). terminate(SendWill, ConnName, State) -> maybe_send_will(SendWill, ConnName, State), @@ -1587,11 +1584,6 @@ deliver_one_to_client(Msg = {QNameOrType, QPid, QMsgId, _Redelivered, QoS = effective_qos(PublisherQoS, SubscriberQoS), State1 = maybe_publish_to_client(Msg, QoS, State0), State = maybe_auto_ack(AckRequired, QoS, QNameOrType, QMsgId, State1), - %%TODO GC - % case GCThreshold of - % undefined -> ok; - % _ -> rabbit_basic:maybe_gc_large_msg(Content, GCThreshold) - % end, ok = maybe_notify_sent(QNameOrType, QPid, State), State. @@ -1614,9 +1606,9 @@ maybe_publish_to_client( {PacketId, State} = queue_packet_id_to_packet_id(QMsgId, QoS, State0), %%TODO support iolists when sending to client Payload = list_to_binary(lists:reverse(FragmentsRev)), - Frame = - #mqtt_frame{ - fixed = #mqtt_frame_fixed{ + Packet = + #mqtt_packet{ + fixed = #mqtt_packet_fixed{ type = ?PUBLISH, qos = QoS, %% "The value of the DUP flag from an incoming PUBLISH packet is not @@ -1626,11 +1618,11 @@ maybe_publish_to_client( %% the outgoing PUBLISH packet is a retransmission [MQTT-3.3.1-3]." %% Therefore, we do not consider header value <<"x-mqtt-dup">> here. dup = Redelivered}, - variable = #mqtt_frame_publish{ + variable = #mqtt_packet_publish{ packet_id = PacketId, topic_name = amqp_to_mqtt(RoutingKey)}, payload = Payload}, - SendFun(Frame, State), + SendFun(Packet, State), message_delivered(QNameOrType, Redelivered, QoS, State), State. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index 0200add9a303..b750ab61e37e 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -31,7 +31,7 @@ conn_name, await_recv, deferred_recv, - received_connect_frame, + received_connect_packet, connection_state, conserve, parse_state, @@ -82,9 +82,9 @@ init(Ref) -> conn_name = ConnName, await_recv = false, connection_state = running, - received_connect_frame = false, + received_connect_packet = false, conserve = false, - parse_state = rabbit_mqtt_frame:initial_state(), + parse_state = rabbit_mqtt_packet:initial_state(), proc_state = ProcessorState }), #state.stats_timer) ); {network_error, Reason} -> @@ -193,10 +193,10 @@ handle_info({keepalive, Req}, State = #state{keepalive = KState0, {stop, Reason, State} end; -handle_info(login_timeout, State = #state{received_connect_frame = true}) -> +handle_info(login_timeout, State = #state{received_connect_packet = true}) -> {noreply, State, ?HIBERNATE_AFTER}; handle_info(login_timeout, State = #state{conn_name = ConnName}) -> - %% The connection is also closed if the CONNECT frame happens to + %% The connection is also closed if the CONNECT packet happens to %% be already in the `deferred_recv' buffer. This can happen while %% the connection is blocked because of a resource alarm. However %% we don't know what is in the buffer, it can be arbitrary bytes, @@ -293,12 +293,12 @@ log_tls_alert(Alert, ConnName) -> rabbit_log_connection:error("MQTT detected TLS upgrade error on ~ts: alert ~ts", [ConnName, Alert]). -process_received_bytes(<<>>, State = #state{received_connect_frame = false, +process_received_bytes(<<>>, State = #state{received_connect_packet = false, proc_state = PState, conn_name = ConnName}) -> rabbit_log_connection:info("Accepted MQTT connection ~p (~s, client id: ~s)", [self(), ConnName, rabbit_mqtt_processor:info(client_id, PState)]), - {noreply, ensure_stats_timer(State#state{received_connect_frame = true}), ?HIBERNATE_AFTER}; + {noreply, ensure_stats_timer(State#state{received_connect_packet = true}), ?HIBERNATE_AFTER}; process_received_bytes(<<>>, State) -> {noreply, ensure_stats_timer(State), ?HIBERNATE_AFTER}; process_received_bytes(Bytes, @@ -310,10 +310,10 @@ process_received_bytes(Bytes, {noreply, ensure_stats_timer( State #state{ parse_state = ParseState1 }), ?HIBERNATE_AFTER}; - {ok, Frame, Rest} -> - case rabbit_mqtt_processor:process_frame(Frame, ProcState) of + {ok, Packet, Rest} -> + case rabbit_mqtt_processor:process_packet(Packet, ProcState) of {ok, ProcState1} -> - PS = rabbit_mqtt_frame:initial_state(), + PS = rabbit_mqtt_packet:initial_state(), process_received_bytes( Rest, State #state{parse_state = PS, @@ -322,15 +322,15 @@ process_received_bytes(Bytes, {error, unauthorized = Reason, ProcState1} -> rabbit_log_connection:error("MQTT connection ~ts is closing due to an authorization failure", [ConnName]), {stop, {shutdown, Reason}, pstate(State, ProcState1)}; - %% CONNECT frames only + %% CONNECT packets only {error, unauthenticated = Reason, ProcState1} -> rabbit_log_connection:error("MQTT connection ~ts is closing due to an authentication failure", [ConnName]), {stop, {shutdown, Reason}, pstate(State, ProcState1)}; - %% CONNECT frames only + %% CONNECT packets only {error, invalid_client_id = Reason, ProcState1} -> rabbit_log_connection:error("MQTT cannot accept connection ~ts: client uses an invalid ID", [ConnName]), {stop, {shutdown, Reason}, pstate(State, ProcState1)}; - %% CONNECT frames only + %% CONNECT packets only {error, unsupported_protocol_version = Reason, ProcState1} -> rabbit_log_connection:error("MQTT cannot accept connection ~ts: incompatible protocol version", [ConnName]), {stop, {shutdown, Reason}, pstate(State, ProcState1)}; @@ -350,7 +350,7 @@ process_received_bytes(Bytes, {stop, normal, {_SendWill = false, pstate(State, ProcState1)}} end; {error, {cannot_parse, Error, Stacktrace}} -> - rabbit_log_connection:error("MQTT cannot parse a frame on connection '~ts', unparseable payload: ~tp, error: {~tp, ~tp} ", + rabbit_log_connection:error("MQTT cannot parse a packet on connection '~ts', unparseable payload: ~tp, error: {~tp, ~tp} ", [ConnName, Bytes, Error, Stacktrace]), {stop, {shutdown, Error}, State}; {error, Error} -> @@ -367,7 +367,7 @@ pstate(State = #state {}, PState) -> %%---------------------------------------------------------------------------- parse(Bytes, ParseState) -> try - rabbit_mqtt_frame:parse(Bytes, ParseState) + rabbit_mqtt_packet:parse(Bytes, ParseState) catch _:Reason:Stacktrace -> {error, {cannot_parse, Reason, Stacktrace}} @@ -375,7 +375,7 @@ parse(Bytes, ParseState) -> network_error(closed, State = #state{conn_name = ConnName, - received_connect_frame = Connected}) -> + received_connect_packet = Connected}) -> Fmt = "MQTT connection ~p will terminate because peer closed TCP connection", Args = [ConnName], case Connected of @@ -430,7 +430,7 @@ maybe_emit_stats(State) -> rabbit_event:if_enabled(State, #state.stats_timer, fun() -> emit_stats(State) end). -emit_stats(State=#state{received_connect_frame = false}) -> +emit_stats(State=#state{received_connect_packet = false}) -> %% Avoid emitting stats on terminate when the connection has not yet been %% established, as this causes orphan entries on the stats database State1 = rabbit_event:reset_stats_timer(State, #state.stats_timer), @@ -469,7 +469,7 @@ info_internal(reductions, _State) -> Reductions; info_internal(conn_name, #state{conn_name = Val}) -> rabbit_data_coercion:to_binary(Val); -info_internal(connection_state, #state{received_connect_frame = false}) -> +info_internal(connection_state, #state{received_connect_packet = false}) -> starting; info_internal(connection_state, #state{connection_state = Val}) -> Val; @@ -511,7 +511,7 @@ format_state(#state{proc_state = PState, conn_name = ConnName, await_recv = AwaitRecv, deferred_recv = DeferredRecv, - received_connect_frame = ReceivedConnectFrame, + received_connect_packet = ReceivedConnectPacket, connection_state = ConnectionState, conserve = Conserve, stats_timer = StatsTimer, @@ -523,7 +523,7 @@ format_state(#state{proc_state = PState, conn_name => ConnName, await_recv => AwaitRecv, deferred_recv => DeferredRecv, - received_connect_frame => ReceivedConnectFrame, + received_connect_packet => ReceivedConnectPacket, connection_state => ConnectionState, conserve => Conserve, stats_timer => StatsTimer, diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer.erl index 3e903b7af0b8..906087338724 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer.erl @@ -9,7 +9,7 @@ -behaviour(gen_server2). -include("rabbit_mqtt_retain.hrl"). --include("rabbit_mqtt_frame.hrl"). +-include("rabbit_mqtt_packet.hrl"). -export([init/1, handle_call/3, handle_cast/2, handle_info/2, terminate/2, code_change/3, start_link/2]). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl index ba6f7aab703f..11769b4e6371 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl @@ -9,7 +9,7 @@ -include_lib("rabbit_common/include/resource.hrl"). -include("rabbit_mqtt.hrl"). --include("rabbit_mqtt_frame.hrl"). +-include("rabbit_mqtt_packet.hrl"). -export([queue_name_bin/2, qos_from_queue_name/2, diff --git a/deps/rabbitmq_mqtt/test/proxy_protocol_SUITE.erl b/deps/rabbitmq_mqtt/test/proxy_protocol_SUITE.erl index 87b6faead203..df3ee52dd6e1 100644 --- a/deps/rabbitmq_mqtt/test/proxy_protocol_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/proxy_protocol_SUITE.erl @@ -64,7 +64,7 @@ proxy_protocol(Config) -> {ok, Socket} = gen_tcp:connect({127,0,0,1}, Port, [binary, {active, false}, {packet, raw}]), ok = inet:send(Socket, "PROXY TCP4 192.168.1.1 192.168.1.2 80 81\r\n"), - ok = inet:send(Socket, mqtt_3_1_1_connect_frame()), + ok = inet:send(Socket, mqtt_3_1_1_connect_packet()), {ok, _Packet} = gen_tcp:recv(Socket, 0, ?TIMEOUT), ConnectionName = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, connection_name, []), match = re:run(ConnectionName, <<"^192.168.1.1:80 -> 192.168.1.2:81$">>, [{capture, none}]), @@ -78,7 +78,7 @@ proxy_protocol_tls(Config) -> [binary, {active, false}, {packet, raw}]), ok = inet:send(Socket, "PROXY TCP4 192.168.1.1 192.168.1.2 80 81\r\n"), {ok, SslSocket} = ssl:connect(Socket, [], ?TIMEOUT), - ok = ssl:send(SslSocket, mqtt_3_1_1_connect_frame()), + ok = ssl:send(SslSocket, mqtt_3_1_1_connect_packet()), {ok, _Packet} = ssl:recv(SslSocket, 0, ?TIMEOUT), ConnectionName = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, connection_name, []), @@ -95,7 +95,7 @@ connection_name() -> merge_app_env(MqttConfig, Config) -> rabbit_ct_helpers:merge_app_env(Config, MqttConfig). -mqtt_3_1_1_connect_frame() -> +mqtt_3_1_1_connect_packet() -> <<16, 24, 0, diff --git a/deps/rabbitmq_mqtt/test/reader_SUITE.erl b/deps/rabbitmq_mqtt/test/reader_SUITE.erl index 695819828743..c3956deb8ac8 100644 --- a/deps/rabbitmq_mqtt/test/reader_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/reader_SUITE.erl @@ -31,7 +31,7 @@ groups() -> [ block, block_connack_timeout, - handle_invalid_frames, + handle_invalid_packets, login_timeout, keepalive, keepalive_turned_off, @@ -129,7 +129,7 @@ block_connack_timeout(Config) -> %% Let connection block. timer:sleep(100), - %% We can still connect via TCP, but CONNECT frame will not be processed on the server. + %% We can still connect via TCP, but CONNECT packet will not be processed on the server. {ok, Client} = emqtt:start_link([{host, "localhost"}, {port, P}, {clientid, atom_to_binary(?FUNCTION_NAME)}, @@ -151,7 +151,7 @@ block_connack_timeout(Config) -> {connected, MqttReader} = rpc(Config, erlang, port_info, [NewPort, connected]), MqttReaderMRef = monitor(process, MqttReader), - %% Unblock connection. CONNECT frame will be processed on the server. + %% Unblock connection. CONNECT packet will be processed on the server. rpc(Config, vm_memory_monitor, set_vm_memory_high_watermark, [0.4]), receive @@ -166,7 +166,7 @@ block_connack_timeout(Config) -> ?assertEqual([], all_connection_pids(Config)), ok. -handle_invalid_frames(Config) -> +handle_invalid_packets(Config) -> N = rpc(Config, ets, info, [connection_metrics, size]), P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), {ok, C} = gen_tcp:connect("localhost", P, []), diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index 9a789e8fd0de..a03e9093e17c 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -32,7 +32,7 @@ socket, peername, stats_timer, - received_connect_frame, + received_connect_packet, keepalive :: rabbit_mqtt_keepalive:state() }). @@ -78,12 +78,12 @@ init(Req, Opts) -> true -> {?MODULE, cowboy_req:set_resp_header(<<"sec-websocket-protocol">>, <<"mqtt">>, Req), #state{ - parse_state = rabbit_mqtt_frame:initial_state(), + parse_state = rabbit_mqtt_packet:initial_state(), state = running, conserve_resources = false, socket = SockInfo, peername = PeerAddr, - received_connect_frame = false + received_connect_packet = false }, WsOpts} end end. @@ -243,26 +243,26 @@ handle_data(Data, State0 = #state{conn_name = ConnName}) -> Other end. -handle_data1(<<>>, State0 = #state{received_connect_frame = false, +handle_data1(<<>>, State0 = #state{received_connect_packet = false, proc_state = PState, conn_name = ConnName}) -> rabbit_log_connection:info("Accepted web MQTT connection ~p (~s, client id: ~s)", [self(), ConnName, rabbit_mqtt_processor:info(client_id, PState)]), - State = State0#state{received_connect_frame = true}, + State = State0#state{received_connect_packet = true}, {ok, ensure_stats_timer(control_throttle(State)), hibernate}; handle_data1(<<>>, State) -> {ok, ensure_stats_timer(control_throttle(State)), hibernate}; handle_data1(Data, State = #state{ parse_state = ParseState, proc_state = ProcState, conn_name = ConnName }) -> - case rabbit_mqtt_frame:parse(Data, ParseState) of + case rabbit_mqtt_packet:parse(Data, ParseState) of {more, ParseState1} -> {ok, ensure_stats_timer(control_throttle( State #state{ parse_state = ParseState1 })), hibernate}; - {ok, Frame, Rest} -> - case rabbit_mqtt_processor:process_frame(Frame, ProcState) of + {ok, Packet, Rest} -> + case rabbit_mqtt_processor:process_packet(Packet, ProcState) of {ok, ProcState1} -> - PS = rabbit_mqtt_frame:initial_state(), + PS = rabbit_mqtt_packet:initial_state(), handle_data1( Rest, State#state{parse_state = PS, @@ -320,8 +320,8 @@ control_throttle(State = #state{state = CS, State end. -send_reply(Frame, PState) -> - self() ! {reply, rabbit_mqtt_processor:serialise(Frame, PState)}. +send_reply(Packet, PState) -> + self() ! {reply, rabbit_mqtt_processor:serialise(Packet, PState)}. ensure_stats_timer(State) -> rabbit_event:ensure_stats_timer(State, #state.stats_timer, emit_stats). @@ -332,7 +332,7 @@ maybe_emit_stats(State) -> rabbit_event:if_enabled(State, #state.stats_timer, fun() -> emit_stats(State) end). -emit_stats(State=#state{received_connect_frame = false}) -> +emit_stats(State=#state{received_connect_packet = false}) -> %% Avoid emitting stats on terminate when the connection has not yet been %% established, as this causes orphan entries on the stats database State1 = rabbit_event:reset_stats_timer(State, #state.stats_timer), diff --git a/deps/rabbitmq_web_mqtt/test/proxy_protocol_SUITE.erl b/deps/rabbitmq_web_mqtt/test/proxy_protocol_SUITE.erl index 7410879bd8e7..95cd3e5c444e 100644 --- a/deps/rabbitmq_web_mqtt/test/proxy_protocol_SUITE.erl +++ b/deps/rabbitmq_web_mqtt/test/proxy_protocol_SUITE.erl @@ -84,7 +84,7 @@ proxy_protocol(Config) -> WS = rfc6455_client:new(Protocol ++ "://127.0.0.1:" ++ PortStr ++ "/ws", self(), undefined, ["mqtt"], "PROXY TCP4 192.168.1.1 192.168.1.2 80 81\r\n"), {ok, _} = rfc6455_client:open(WS), - rfc6455_client:send_binary(WS, rabbit_ws_test_util:mqtt_3_1_1_connect_frame()), + rfc6455_client:send_binary(WS, rabbit_ws_test_util:mqtt_3_1_1_connect_packet()), {binary, _P} = rfc6455_client:recv(WS), ConnectionName = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, connection_name, []), diff --git a/deps/rabbitmq_web_mqtt/test/src/rabbit_ws_test_util.erl b/deps/rabbitmq_web_mqtt/test/src/rabbit_ws_test_util.erl index 6aa74f13b847..b89787781201 100644 --- a/deps/rabbitmq_web_mqtt/test/src/rabbit_ws_test_util.erl +++ b/deps/rabbitmq_web_mqtt/test/src/rabbit_ws_test_util.erl @@ -8,7 +8,7 @@ -module(rabbit_ws_test_util). -export([update_app_env/3, get_web_mqtt_port_str/1, - mqtt_3_1_1_connect_frame/0]). + mqtt_3_1_1_connect_packet/0]). update_app_env(Config, Key, Val) -> ok = rabbit_ct_broker_helpers:rpc(Config, 0, @@ -30,7 +30,7 @@ get_web_mqtt_port_str(Config) -> end, integer_to_list(Port). -mqtt_3_1_1_connect_frame() -> +mqtt_3_1_1_connect_packet() -> <<16, 24, 0, diff --git a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl index d6941505cd15..bb8ec77a156d 100644 --- a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl +++ b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl @@ -192,7 +192,7 @@ client_protocol_test(Config, Protocol) -> WS = rfc6455_client:new("ws://localhost:" ++ PortStr ++ "/ws", self(), undefined, Protocol), {_, [{http_response, Res}]} = rfc6455_client:open(WS), {'HTTP/1.1', 400, <<"Bad Request">>, _} = cow_http:parse_status_line(rabbit_data_coercion:to_binary(Res)), - rfc6455_client:send_binary(WS, rabbit_ws_test_util:mqtt_3_1_1_connect_frame()), + rfc6455_client:send_binary(WS, rabbit_ws_test_util:mqtt_3_1_1_connect_packet()), {close, _P} = rfc6455_client:recv(WS), ok. From 61f6ca7b66ac8e62f1ec3562302045b0dc4e2e2f Mon Sep 17 00:00:00 2001 From: David Ansari Date: Sun, 4 Dec 2022 16:29:07 +0000 Subject: [PATCH 074/118] Support iodata() when sending message to MQTT client When the MQTT connection receives an AMQP 0.9.1 message, it will contain a list of payload fragments. This commit avoids the expensive operation of turning that list into a binary. All I/O methods accept iodata(): * erlang:port_command/2 * ssl:send/2 * In Web MQTT, cowboy websockets accept iodata(): https://github.com/ninenines/cowlib/blob/0d04cfffa3bd49b10ccc783c381a261c15b7e7a3/src/cow_ws.erl#L58 --- .../include/rabbit_mqtt_packet.hrl | 14 ++--- deps/rabbitmq_mqtt/src/rabbit_mqtt_packet.erl | 51 +++++++++-------- .../src/rabbit_mqtt_processor.erl | 4 +- deps/rabbitmq_mqtt/test/integration_SUITE.erl | 57 ++++++++++++++----- 4 files changed, 80 insertions(+), 46 deletions(-) diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt_packet.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt_packet.hrl index db5d8fe9a5d6..5eff36808eea 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt_packet.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt_packet.hrl @@ -53,14 +53,16 @@ %% Packet identifier is a non zero two byte integer. -type packet_id() :: 1..16#ffff. --record(mqtt_packet, {fixed, - variable, - payload}). - -record(mqtt_packet_fixed, {type = 0, dup = 0, qos = 0, - retain = 0}). + retain = 0 + }). + +-record(mqtt_packet, {fixed :: #mqtt_packet_fixed{}, + variable :: tuple(), + payload :: iodata() + }). -record(mqtt_packet_connect, {proto_ver, will_retain, @@ -90,8 +92,6 @@ -record(mqtt_packet_suback, {packet_id :: packet_id(), qos_table = []}). --record(mqtt_packet_other, {other}). - -record(mqtt_msg, {retain :: boolean(), qos :: qos(), topic :: string(), diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_packet.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_packet.erl index 113374b3d166..846227cbbc3f 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_packet.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_packet.erl @@ -24,9 +24,9 @@ parse(<<>>, none) -> {more, fun(Bin) -> parse(Bin, none) end}; parse(<>, none) -> parse_remaining_len(Rest, #mqtt_packet_fixed{ type = MessageType, - dup = bool(Dup), - qos = QoS, - retain = bool(Retain) }); + dup = bool(Dup), + qos = QoS, + retain = bool(Retain) }); parse(Bin, Cont) -> Cont(Bin). parse_remaining_len(<<>>, Fixed) -> @@ -45,7 +45,7 @@ parse_remaining_len(<<0:1, Len:7, Rest/binary>>, Fixed, Multiplier, Value) -> parse_packet(Rest, Fixed, Value + Len * Multiplier). parse_packet(Bin, #mqtt_packet_fixed{ type = Type, - qos = Qos } = Fixed, Length) -> + qos = Qos } = Fixed, Length) -> case {Type, Bin} of {?CONNECT, <>} -> {ProtoName, Rest1} = parse_utf(PacketBin), @@ -90,7 +90,7 @@ parse_packet(Bin, #mqtt_packet_fixed{ type = Type, {M, R} end, wrap(Fixed, #mqtt_packet_publish { topic_name = TopicName, - packet_id = PacketId }, + packet_id = PacketId }, Payload, Rest); {?PUBACK, <>} -> <> = PacketBin, @@ -101,7 +101,7 @@ parse_packet(Bin, #mqtt_packet_fixed{ type = Type, <> = PacketBin, Topics = parse_topics(Subs, Rest1, []), wrap(Fixed, #mqtt_packet_subscribe { packet_id = PacketId, - topic_table = Topics }, Rest); + topic_table = Topics }, Rest); {Minimal, Rest} when Minimal =:= ?DISCONNECT orelse Minimal =:= ?PINGREQ -> Length = 0, @@ -109,7 +109,7 @@ parse_packet(Bin, #mqtt_packet_fixed{ type = Type, {_, TooShortBin} -> {more, fun(BinMore) -> parse_packet(<>, - Fixed, Length) + Fixed, Length) end} end. @@ -150,24 +150,29 @@ bool(1) -> true. %% serialisation +-spec serialise(#mqtt_packet{}, ?MQTT_PROTO_V3 | ?MQTT_PROTO_V4) -> + iodata(). serialise(#mqtt_packet{fixed = Fixed, - variable = Variable, - payload = Payload}, Vsn) -> + variable = Variable, + payload = Payload}, Vsn) -> serialise_variable(Fixed, Variable, serialise_payload(Payload), Vsn). -serialise_payload(undefined) -> <<>>; -serialise_payload(B) when is_binary(B) -> B. +serialise_payload(undefined) -> + <<>>; +serialise_payload(P) + when is_binary(P) orelse is_list(P) -> + P. serialise_variable(#mqtt_packet_fixed { type = ?CONNACK } = Fixed, #mqtt_packet_connack { session_present = SessionPresent, - return_code = ReturnCode }, + return_code = ReturnCode }, <<>> = PayloadBin, _Vsn) -> VariableBin = <>, serialise_fixed(Fixed, VariableBin, PayloadBin); serialise_variable(#mqtt_packet_fixed { type = SubAck } = Fixed, #mqtt_packet_suback { packet_id = PacketId, - qos_table = Qos }, + qos_table = Qos }, <<>> = _PayloadBin, Vsn) when SubAck =:= ?SUBACK orelse SubAck =:= ?UNSUBACK -> VariableBin = <>, @@ -181,16 +186,16 @@ serialise_variable(#mqtt_packet_fixed { type = SubAck } = Fixed, serialise_fixed(Fixed, VariableBin, QosBin); serialise_variable(#mqtt_packet_fixed { type = ?PUBLISH, - qos = Qos } = Fixed, + qos = Qos } = Fixed, #mqtt_packet_publish { topic_name = TopicName, - packet_id = PacketId }, - PayloadBin, _Vsn) -> + packet_id = PacketId }, + Payload, _Vsn) -> TopicBin = serialise_utf(TopicName), PacketIdBin = case Qos of 0 -> <<>>; 1 -> <> end, - serialise_fixed(Fixed, <>, PayloadBin); + serialise_fixed(Fixed, <>, Payload); serialise_variable(#mqtt_packet_fixed { type = ?PUBACK } = Fixed, #mqtt_packet_publish { packet_id = PacketId }, @@ -204,15 +209,15 @@ serialise_variable(#mqtt_packet_fixed {} = Fixed, serialise_fixed(Fixed, <<>>, <<>>). serialise_fixed(#mqtt_packet_fixed{ type = Type, - dup = Dup, - qos = Qos, - retain = Retain }, VariableBin, PayloadBin) + dup = Dup, + qos = Qos, + retain = Retain }, VariableBin, Payload) when is_integer(Type) andalso ?CONNECT =< Type andalso Type =< ?DISCONNECT -> - Len = size(VariableBin) + size(PayloadBin), + Len = size(VariableBin) + iolist_size(Payload), true = (Len =< ?MAX_LEN), LenBin = serialise_len(Len), - <>. + [<>, Payload]. serialise_utf(String) -> StringBin = unicode:characters_to_binary(String), diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 41604cb663d4..661492b68948 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -1604,8 +1604,6 @@ maybe_publish_to_client( content = #content{payload_fragments_rev = FragmentsRev}}}, QoS, State0 = #state{send_fun = SendFun}) -> {PacketId, State} = queue_packet_id_to_packet_id(QMsgId, QoS, State0), - %%TODO support iolists when sending to client - Payload = list_to_binary(lists:reverse(FragmentsRev)), Packet = #mqtt_packet{ fixed = #mqtt_packet_fixed{ @@ -1621,7 +1619,7 @@ maybe_publish_to_client( variable = #mqtt_packet_publish{ packet_id = PacketId, topic_name = amqp_to_mqtt(RoutingKey)}, - payload = Payload}, + payload = lists:reverse(FragmentsRev)}, SendFun(Packet, State), message_delivered(QNameOrType, Redelivered, QoS, State), State. diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/integration_SUITE.erl index ddf14d8155d1..8437d7209867 100644 --- a/deps/rabbitmq_mqtt/test/integration_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/integration_SUITE.erl @@ -37,7 +37,7 @@ groups() -> [ %% separate RMQ so global counters start from 0 {global_counters, [], [global_counters_v3, global_counters_v4]}, - {common_tests, [], common_tests()} + {common_tests, [], tests()} ]}, {cluster_size_3, [], [queue_down_qos1, @@ -46,11 +46,11 @@ groups() -> flow_classic_mirrored_queue, flow_quorum_queue, flow_stream, - rabbit_mqtt_qos0_queue] ++ common_tests() + rabbit_mqtt_qos0_queue] ++ tests() } ]. -common_tests() -> +tests() -> [delete_create_queue ,quorum_queue_rejects ,publish_to_all_queue_types_qos0 @@ -62,6 +62,8 @@ common_tests() -> ,subscribe_same_topic_same_qos ,subscribe_same_topic_different_qos ,subscribe_multiple + ,large_message_mqtt_to_mqtt + ,large_message_amqp_to_mqtt ]. suite() -> @@ -157,7 +159,7 @@ publish_to_all_queue_types_qos1(Config) -> publish_to_all_queue_types(Config, qos1). publish_to_all_queue_types(Config, QoS) -> - {Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0), + Ch = rabbit_ct_client_helpers:open_channel(Config, 0), CQ = <<"classic-queue">>, CMQ = <<"classic-mirrored-queue">>, @@ -214,7 +216,7 @@ publish_to_all_queue_types(Config, QoS) -> ok = emqtt:disconnect(C), ?awaitMatch([], all_connection_pids(Config), 10_000, 1000), - ok = rabbit_ct_client_helpers:close_connection_and_channel(Conn, Ch). + ok = rabbit_ct_client_helpers:close_channel(Ch). flow_classic_mirrored_queue(Config) -> QueueName = <<"flow">>, @@ -234,7 +236,7 @@ flow(Config, {App, Par, Val}, QueueType) Result = rpc_all(Config, application, set_env, [App, Par, Val]), ?assert(lists:all(fun(R) -> R =:= ok end, Result)), - {Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0), + Ch = rabbit_ct_client_helpers:open_channel(Config, 0), QueueName = Topic = atom_to_binary(?FUNCTION_NAME), declare_queue(Ch, QueueName, [{<<"x-queue-type">>, longstr, QueueType}]), bind(Ch, QueueName, Topic), @@ -261,7 +263,7 @@ flow(Config, {App, Par, Val}, QueueType) ok = emqtt:disconnect(C), ?awaitMatch([], all_connection_pids(Config), 10_000, 1000), - ok = rabbit_ct_client_helpers:close_connection_and_channel(Conn, Ch), + ok = rabbit_ct_client_helpers:close_channel(Ch), Result = rpc_all(Config, application, set_env, [App, Par, DefaultVal]), ok. @@ -461,11 +463,11 @@ global_counters(Config, ProtoVer) -> get_global_counters(Config, ProtoVer)). queue_down_qos1(Config) -> - {Conn1, Ch1} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 1), + Ch1 = rabbit_ct_client_helpers:open_channel(Config, 1), CQ = Topic = atom_to_binary(?FUNCTION_NAME), declare_queue(Ch1, CQ, []), bind(Ch1, CQ, Topic), - ok = rabbit_ct_client_helpers:close_connection_and_channel(Conn1, Ch1), + ok = rabbit_ct_client_helpers:close_channel(Ch1), ok = rabbit_ct_broker_helpers:stop_node(Config, 1), C = connect(?FUNCTION_NAME, Config, [{retry_interval, 2}]), @@ -479,9 +481,9 @@ queue_down_qos1(Config) -> rabbitmqctl_list(Config, 1, ["list_queues", "messages", "--no-table-headers"])), 500, 20), - {Conn0, Ch0} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0), + Ch0 = rabbit_ct_client_helpers:open_channel(Config, 0), delete_queue(Ch0, CQ), - ok = rabbit_ct_client_helpers:close_connection_and_channel(Conn0, Ch0), + ok = rabbit_ct_client_helpers:close_channel(Ch0), ok = emqtt:disconnect(C). %% Even though classic mirrored queues are deprecated, we know that some users have set up @@ -581,7 +583,7 @@ consuming_classic_queue_down(Config) -> ok. delete_create_queue(Config) -> - {Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0), + Ch = rabbit_ct_client_helpers:open_channel(Config, 0), CQ1 = <<"classic-queue-1-delete-create">>, CQ2 = <<"classic-queue-2-delete-create">>, QQ = <<"quorum-queue-delete-create">>, @@ -639,7 +641,7 @@ delete_create_queue(Config) -> 1000, 10), delete_queue(Ch, [CQ1, CQ2, QQ]), - ok = rabbit_ct_client_helpers:close_connection_and_channel(Conn, Ch), + ok = rabbit_ct_client_helpers:close_channel(Ch), ok = emqtt:disconnect(C). non_clean_sess_disconnect(Config) -> @@ -727,6 +729,35 @@ subscribe_multiple(Config) -> {<<"topic1">>, qos1}])), ok = emqtt:disconnect(C). +large_message_mqtt_to_mqtt(Config) -> + Topic = ClientId = atom_to_binary(?FUNCTION_NAME), + C = connect(ClientId, Config), + {ok, _, [1]} = emqtt:subscribe(C, {Topic, qos1}), + + Payload0 = binary:copy(<<"x">>, 1_000_000), + Payload = <>, + {ok, _} = emqtt:publish(C, Topic, Payload, qos1), + ok = expect_publishes(Topic, [Payload]), + + ok = emqtt:disconnect(C). + +large_message_amqp_to_mqtt(Config) -> + Topic = ClientId = atom_to_binary(?FUNCTION_NAME), + C = connect(ClientId, Config), + {ok, _, [1]} = emqtt:subscribe(C, {Topic, qos1}), + + Ch = rabbit_ct_client_helpers:open_channel(Config, 0), + Payload0 = binary:copy(<<"x">>, 1_000_000), + Payload = <>, + amqp_channel:call(Ch, + #'basic.publish'{exchange = <<"amq.topic">>, + routing_key = Topic}, + #amqp_msg{payload = Payload}), + ok = expect_publishes(Topic, [Payload]), + + ok = rabbit_ct_client_helpers:close_channel(Ch), + ok = emqtt:disconnect(C). + %% This test is mostly interesting in mixed version mode where feature flag %% rabbit_mqtt_qos0_queue is disabled and therefore a classic queue gets created. rabbit_mqtt_qos0_queue(Config) -> From b3215ebaf197ad4eee066cdf998b38c0461e4ce0 Mon Sep 17 00:00:00 2001 From: Chunyi Lyu Date: Tue, 6 Dec 2022 10:50:10 +0000 Subject: [PATCH 075/118] Set close header when close connection in web_mqtt init/2 --- deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl | 2 +- deps/rabbitmq_web_mqtt/test/system_SUITE.erl | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index a03e9093e17c..e4c21acd294a 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -231,7 +231,7 @@ terminate(_Reason, _Request, no_supported_sub_protocol(Protocol, Req) -> rabbit_log_connection:error("Web MQTT: mqtt not found in client supported protocol, protocol: ~tp", [Protocol]), - {ok, cowboy_req:reply(400, Req), #state{}}. + {ok, cowboy_req:reply(400, #{<<"connection">> => <<"close">>}, Req), #state{}}. handle_data(Data, State0 = #state{conn_name = ConnName}) -> case handle_data1(Data, State0) of diff --git a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl index bb8ec77a156d..bc0ea787cd4d 100644 --- a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl +++ b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl @@ -193,8 +193,7 @@ client_protocol_test(Config, Protocol) -> {_, [{http_response, Res}]} = rfc6455_client:open(WS), {'HTTP/1.1', 400, <<"Bad Request">>, _} = cow_http:parse_status_line(rabbit_data_coercion:to_binary(Res)), rfc6455_client:send_binary(WS, rabbit_ws_test_util:mqtt_3_1_1_connect_packet()), - {close, _P} = rfc6455_client:recv(WS), - ok. + {close, _P} = rfc6455_client:recv(WS). %% Web mqtt connections are tracked together with mqtt connections num_mqtt_connections(Config, Node) -> From 2a7f22fa26be912d5f8782cd9babdbe123a4a856 Mon Sep 17 00:00:00 2001 From: Chunyi Lyu Date: Tue, 6 Dec 2022 14:08:18 +0000 Subject: [PATCH 076/118] Web mqtt: close connection when receive invalid data - from https://www.rfc-editor.org/rfc/rfc6455#section-7.4.1, code `1003` is for "an endpoint is terminating the connection because it has received a type of data it cannot accept" --- deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl | 8 +++----- deps/rabbitmq_web_mqtt/test/system_SUITE.erl | 10 +++++++++- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index e4c21acd294a..48c490e976e4 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -39,6 +39,7 @@ %% Close frame status codes as defined in https://www.rfc-editor.org/rfc/rfc6455#section-7.4.1 -define(CLOSE_NORMAL, 1000). -define(CLOSE_PROTOCOL_ERROR, 1002). +-define(CLOSE_INVALID_DATA, 1003). -define(CLOSE_INCONSISTENT_MSG_TYPE, 1007). %% cowboy_sub_protcol @@ -135,14 +136,11 @@ websocket_handle({Ping, _}, State) websocket_handle(Ping, State) when Ping =:= ping orelse Ping =:= pong -> {[], State, hibernate}; -%% Log any other unexpected frames. +%% Log and close connection when receiving any other unexpected frames. websocket_handle(Frame, State) -> rabbit_log_connection:info("Web MQTT: unexpected WebSocket frame ~tp", [Frame]), - %%TODO close connection instead? - %%"MQTT Control Packets MUST be sent in WebSocket binary data frames. - %% If any other type of data frame is received the recipient MUST close the Network Connection" - {[], State, hibernate}. + stop(State, ?CLOSE_INVALID_DATA, "unexpected WebSocket frame"). -spec websocket_info(any(), State) -> {cowboy_websocket:commands(), State} | diff --git a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl index bc0ea787cd4d..a033582faf53 100644 --- a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl +++ b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl @@ -32,6 +32,7 @@ groups() -> , maintenance , client_no_supported_protocol , client_not_support_mqtt + , invalid_data ]} ]. @@ -193,7 +194,14 @@ client_protocol_test(Config, Protocol) -> {_, [{http_response, Res}]} = rfc6455_client:open(WS), {'HTTP/1.1', 400, <<"Bad Request">>, _} = cow_http:parse_status_line(rabbit_data_coercion:to_binary(Res)), rfc6455_client:send_binary(WS, rabbit_ws_test_util:mqtt_3_1_1_connect_packet()), - {close, _P} = rfc6455_client:recv(WS). + {close, _} = rfc6455_client:recv(WS, timer:seconds(1)). + +invalid_data(Config) -> + PortStr = rabbit_ws_test_util:get_web_mqtt_port_str(Config), + WS = rfc6455_client:new("ws://localhost:" ++ PortStr ++ "/ws", self(), undefined, ["mqtt"]), + {ok, _} = rfc6455_client:open(WS), + rfc6455_client:send(WS, "not-binary-data"), + {close, {1003, _}} = rfc6455_client:recv(WS, timer:seconds(1)). %% Web mqtt connections are tracked together with mqtt connections num_mqtt_connections(Config, Node) -> From aa8d16fe946bb7fd4132e972930b446e698b9290 Mon Sep 17 00:00:00 2001 From: Chunyi Lyu Date: Tue, 6 Dec 2022 15:41:05 +0000 Subject: [PATCH 077/118] Web mqtt: return hibernate in handle_credits --- deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index 48c490e976e4..f50195fb795a 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -292,12 +292,11 @@ stop(State, CloseCode, Error0) -> {[{close, CloseCode, Error}], State}. handle_credits(State0) -> - %%TODO return hibernate? case control_throttle(State0) of State = #state{state = running} -> - {[{active, true}], State}; + {[{active, true}], State, hibernate}; State -> - {[], State} + {[], State, hibernate} end. control_throttle(State = #state{state = CS, From d5e497f8b68b56aa142e94412a04ff0a5fdbafbb Mon Sep 17 00:00:00 2001 From: Chunyi Lyu Date: Tue, 6 Dec 2022 16:16:49 +0000 Subject: [PATCH 078/118] Web mqtt: backfill test for connecting with duplicated id --- deps/rabbitmq_web_mqtt/test/system_SUITE.erl | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl index a033582faf53..1c0420a36d1c 100644 --- a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl +++ b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl @@ -33,6 +33,7 @@ groups() -> , client_no_supported_protocol , client_not_support_mqtt , invalid_data + , duplicate_id ]} ]. @@ -203,6 +204,20 @@ invalid_data(Config) -> rfc6455_client:send(WS, "not-binary-data"), {close, {1003, _}} = rfc6455_client:recv(WS, timer:seconds(1)). +duplicate_id(Config) -> + C1 = ws_connect(?FUNCTION_NAME, Config), + process_flag(trap_exit, true), + eventually(?_assertEqual(1, num_mqtt_connections(Config, 0))), + C2 = ws_connect(?FUNCTION_NAME, Config), + receive + {'EXIT', C1, _Reason} -> + ok + after 5000 -> + ct:fail("server did not disconnect a client with duplicate ID") + end, + eventually(?_assertEqual(1, num_mqtt_connections(Config, 0))), + ok = emqtt:disconnect(C2). + %% Web mqtt connections are tracked together with mqtt connections num_mqtt_connections(Config, Node) -> length(rpc(Config, Node, rabbit_mqtt, local_connection_pids, [])). From aea7ff8f8da828b6e913755ac464182ec582ddbe Mon Sep 17 00:00:00 2001 From: Chunyi Lyu Date: Thu, 8 Dec 2022 14:59:43 +0000 Subject: [PATCH 079/118] Use helper to connect to node in mqtt cluster suite --- deps/rabbitmq_mqtt/test/cluster_SUITE.erl | 17 +++-------------- deps/rabbitmq_mqtt/test/util.erl | 8 ++++++-- 2 files changed, 9 insertions(+), 16 deletions(-) diff --git a/deps/rabbitmq_mqtt/test/cluster_SUITE.erl b/deps/rabbitmq_mqtt/test/cluster_SUITE.erl index d1c7dd8d69d9..e91b3eb59905 100644 --- a/deps/rabbitmq_mqtt/test/cluster_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/cluster_SUITE.erl @@ -9,7 +9,7 @@ -include_lib("common_test/include/ct.hrl"). -include_lib("eunit/include/eunit.hrl"). --import(util, [expect_publishes/2]). +-import(util, [expect_publishes/2, connect/4]). -import(rabbit_ct_broker_helpers, [setup_steps/0, @@ -205,22 +205,11 @@ assert_connection_count(Config, Retries, NodeId, NumElements) -> end. connect_to_node(Config, Node, ClientID) -> - Port = get_node_config(Config, Node, tcp_port_mqtt), - {ok, C} = connect(Port, ClientID), + C = connect(ClientID, Config, Node, [{connect_timeout, 1}, {ack_timeout, 1}]), + unlink(C), MRef = erlang:monitor(process, C), {ok, MRef, C}. -connect(Port, ClientID) -> - {ok, C} = emqtt:start_link([{host, "localhost"}, - {port, Port}, - {clientid, ClientID}, - {proto_ver, v4}, - {connect_timeout, 1}, - {ack_timeout, 1}]), - {ok, _Properties} = emqtt:connect(C), - unlink(C), - {ok, C}. - await_disconnection(Ref) -> receive {'DOWN', Ref, _, _, _} -> ok diff --git a/deps/rabbitmq_mqtt/test/util.erl b/deps/rabbitmq_mqtt/test/util.erl index 71fe7f9bc362..15af7750dc00 100644 --- a/deps/rabbitmq_mqtt/test/util.erl +++ b/deps/rabbitmq_mqtt/test/util.erl @@ -10,7 +10,8 @@ get_global_counters/4, expect_publishes/2, connect/2, - connect/3]). + connect/3, + connect/4]). all_connection_pids(Config) -> Nodes = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), @@ -69,7 +70,10 @@ connect(ClientId, Config) -> connect(ClientId, Config, []). connect(ClientId, Config, AdditionalOpts) -> - P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), + connect(ClientId, Config, 0, AdditionalOpts). + +connect(ClientId, Config, Node, AdditionalOpts) -> + P = rabbit_ct_broker_helpers:get_node_config(Config, Node, tcp_port_mqtt), Options = [{host, "localhost"}, {port, P}, {clientid, rabbit_data_coercion:to_binary(ClientId)}, From de28560d8f1f4051b0a3f45c4a6fe8c62dc2199a Mon Sep 17 00:00:00 2001 From: Chunyi Lyu Date: Thu, 8 Dec 2022 15:35:21 +0000 Subject: [PATCH 080/118] Extract connect to node helper in rmq mqtt tests --- deps/rabbitmq_mqtt/test/cluster_SUITE.erl | 10 +++------- deps/rabbitmq_mqtt/test/ff_SUITE.erl | 21 +++++---------------- deps/rabbitmq_mqtt/test/util.erl | 11 +++++++++-- 3 files changed, 17 insertions(+), 25 deletions(-) diff --git a/deps/rabbitmq_mqtt/test/cluster_SUITE.erl b/deps/rabbitmq_mqtt/test/cluster_SUITE.erl index e91b3eb59905..7f18325ffca7 100644 --- a/deps/rabbitmq_mqtt/test/cluster_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/cluster_SUITE.erl @@ -9,7 +9,9 @@ -include_lib("common_test/include/ct.hrl"). -include_lib("eunit/include/eunit.hrl"). --import(util, [expect_publishes/2, connect/4]). +-import(util, [expect_publishes/2, + connect/4, + connect_to_node/3]). -import(rabbit_ct_broker_helpers, [setup_steps/0, @@ -204,12 +206,6 @@ assert_connection_count(Config, Retries, NodeId, NumElements) -> assert_connection_count(Config, Retries-1, NodeId, NumElements) end. -connect_to_node(Config, Node, ClientID) -> - C = connect(ClientID, Config, Node, [{connect_timeout, 1}, {ack_timeout, 1}]), - unlink(C), - MRef = erlang:monitor(process, C), - {ok, MRef, C}. - await_disconnection(Ref) -> receive {'DOWN', Ref, _, _, _} -> ok diff --git a/deps/rabbitmq_mqtt/test/ff_SUITE.erl b/deps/rabbitmq_mqtt/test/ff_SUITE.erl index bb98108fb4e0..ae85690c1152 100644 --- a/deps/rabbitmq_mqtt/test/ff_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/ff_SUITE.erl @@ -14,8 +14,8 @@ -import(rabbit_ct_broker_helpers, [rpc/5]). -import(rabbit_ct_helpers, [eventually/1]). -import(util, [expect_publishes/2, - get_global_counters/4 - ]). + get_global_counters/4, + connect_to_node/3]). -define(PROTO_VER, v4). @@ -70,7 +70,7 @@ end_per_testcase(_TestCase, Config) -> delete_ra_cluster_mqtt_node(Config) -> FeatureFlag = ?FUNCTION_NAME, - C = connect_to_node(Config, 1, <<"my-client">>), + {ok, _, C} = connect_to_node(Config, 1, <<"my-client">>), timer:sleep(500), %% old client ID tracking works ?assertEqual(1, length(util:all_connection_pids(Config))), @@ -94,7 +94,7 @@ rabbit_mqtt_qos0_queue(Config) -> FeatureFlag = ?FUNCTION_NAME, Msg = Topic = ClientId = atom_to_binary(?FUNCTION_NAME), - C1 = connect_to_node(Config, 0, ClientId), + {ok, _, C1} = connect_to_node(Config, 0, ClientId), {ok, _, [0]} = emqtt:subscribe(C1, Topic, qos0), ok = emqtt:publish(C1, Topic, Msg, qos0), ok = expect_publishes(Topic, [Msg]), @@ -115,7 +115,7 @@ rabbit_mqtt_qos0_queue(Config) -> %% Reconnecting with the same client ID will terminate the old connection. true = unlink(C1), - C2 = connect_to_node(Config, 0, ClientId), + {ok, _, C2} = connect_to_node(Config, 0, ClientId), {ok, _, [0]} = emqtt:subscribe(C2, Topic, qos0), %% This time, we get the new queue type. eventually( @@ -129,14 +129,3 @@ rabbit_mqtt_qos0_queue(Config) -> messages_delivered_consume_auto_ack_total := 1}, get_global_counters(Config, ?PROTO_VER, 0, [{queue_type, FeatureFlag}])), ok = emqtt:disconnect(C2). - -connect_to_node(Config, Node, ClientID) -> - Port = rabbit_ct_broker_helpers:get_node_config(Config, Node, tcp_port_mqtt), - {ok, C} = emqtt:start_link([{host, "localhost"}, - {port, Port}, - {clientid, ClientID}, - {proto_ver, ?PROTO_VER}, - {connect_timeout, 1}, - {ack_timeout, 1}]), - {ok, _Properties} = emqtt:connect(C), - C. diff --git a/deps/rabbitmq_mqtt/test/util.erl b/deps/rabbitmq_mqtt/test/util.erl index 15af7750dc00..9abe67a5135f 100644 --- a/deps/rabbitmq_mqtt/test/util.erl +++ b/deps/rabbitmq_mqtt/test/util.erl @@ -11,7 +11,8 @@ expect_publishes/2, connect/2, connect/3, - connect/4]). + connect/4, + connect_to_node/3]). all_connection_pids(Config) -> Nodes = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), @@ -81,4 +82,10 @@ connect(ClientId, Config, Node, AdditionalOpts) -> ] ++ AdditionalOpts, {ok, C} = emqtt:start_link(Options), {ok, _Properties} = emqtt:connect(C), - C. \ No newline at end of file + C. + +connect_to_node(Config, Node, ClientID) -> + C = connect(ClientID, Config, Node, [{connect_timeout, 1}, {ack_timeout, 1}]), + unlink(C), + MRef = erlang:monitor(process, C), + {ok, MRef, C}. \ No newline at end of file From 97fefff0fe6c178fb88a06c3f07a1532f8c68739 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Mon, 12 Dec 2022 08:57:33 +0000 Subject: [PATCH 081/118] Add overflow drop-head to rabbit_mqtt_qos_queue type Traditionally, queue types implement flow control by keeping state in both sending and receiving Erlang processes (for example credit based flow control keeps the number of credits within the process dictionary). The rabbit_mqtt_qos0_queue cannot keep such state in sending or receiving Erlang process because such state would consume a large amount of memory in case of large fan-ins or large fan-outs. The whole idea of the rabbit_mqtt_qos_queue type is to not keep any state in the rabbit_queue_type client. This makes this new queue type scale so well. Therefore the new queue type cannot (easily) implement flow control throttling individual senders. In this commit, we take a different approach: Instead of implementing flow control throttling individual senders, the receiving MQTT connection process drops QoS 0 messages from the rabbit_mqtt_qos_queue if it is overflowed with messages AND its MQTT client is not capable of receiving messages fast enough. This is a simple and sufficient solution because it's better to drop QoS 0 (at most once) messages instead of causing cluster-wide memory alarms. The user can opt out of dropping messages by setting the new env setting mailbox_soft_limit to 0. Additionally, we reduce the send_timeout from 30 seconds default in Ranch to 15 seconds default in MQTT. This will detect hanging MQTT clients faster causing the MQTT connection to be closed. --- deps/rabbit/docs/rabbitmq.conf.example | 4 +- deps/rabbitmq_mqtt/BUILD.bazel | 8 +- deps/rabbitmq_mqtt/Makefile | 8 +- deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl | 6 +- deps/rabbitmq_mqtt/src/rabbit_mqtt.erl | 12 ++- .../src/rabbit_mqtt_processor.erl | 60 ++++++++++++--- .../src/rabbit_mqtt_qos0_queue.erl | 6 -- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 4 +- .../src/rabbit_mqtt_retainer.erl | 3 +- deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl | 8 +- deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl | 2 +- deps/rabbitmq_mqtt/test/integration_SUITE.erl | 77 +++++++++++++++++-- deps/rabbitmq_mqtt/test/util.erl | 2 +- 13 files changed, 157 insertions(+), 43 deletions(-) diff --git a/deps/rabbit/docs/rabbitmq.conf.example b/deps/rabbit/docs/rabbitmq.conf.example index c3358bcd6b5d..00dec1473ce7 100644 --- a/deps/rabbit/docs/rabbitmq.conf.example +++ b/deps/rabbit/docs/rabbitmq.conf.example @@ -755,7 +755,7 @@ # stomp.tcp_listen_options.nodelay = true # # stomp.tcp_listen_options.exit_on_close = true -# stomp.tcp_listen_options.send_timeout = 120 +# stomp.tcp_listen_options.send_timeout = 120000 ## Proxy protocol support ## @@ -838,7 +838,7 @@ # mqtt.tcp_listen_options.nodelay = true # # mqtt.tcp_listen_options.exit_on_close = true -# mqtt.tcp_listen_options.send_timeout = 120 +# mqtt.tcp_listen_options.send_timeout = 120000 ## TLS listener settings ## ## See https://rabbitmq.com/mqtt.html and https://rabbitmq.com/ssl.html for details. diff --git a/deps/rabbitmq_mqtt/BUILD.bazel b/deps/rabbitmq_mqtt/BUILD.bazel index 8636e92d40a6..3872527f961f 100644 --- a/deps/rabbitmq_mqtt/BUILD.bazel +++ b/deps/rabbitmq_mqtt/BUILD.bazel @@ -36,9 +36,13 @@ APP_ENV = """[ {num_tcp_acceptors, 10}, {num_ssl_acceptors, 10}, {tcp_listen_options, [{backlog, 128}, - {nodelay, true}]}, + {nodelay, true}, + {send_timeout, 15000}, + {send_timeout_close, true} + ]}, {proxy_protocol, false}, - {sparkplug, false} + {sparkplug, false}, + {mailbox_soft_limit, 1000} ]""" BUILD_DEPS = [ diff --git a/deps/rabbitmq_mqtt/Makefile b/deps/rabbitmq_mqtt/Makefile index d07c6ba04b61..e13eb95e345a 100644 --- a/deps/rabbitmq_mqtt/Makefile +++ b/deps/rabbitmq_mqtt/Makefile @@ -21,9 +21,13 @@ define PROJECT_ENV {num_tcp_acceptors, 10}, {num_ssl_acceptors, 10}, {tcp_listen_options, [{backlog, 128}, - {nodelay, true}]}, + {nodelay, true}, + {send_timeout, 15000}, + {send_timeout_close, true} + ]}, {proxy_protocol, false}, - {sparkplug, false} + {sparkplug, false}, + {mailbox_soft_limit, 1000} ] endef diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl index 4bfab130b743..577d36734e25 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl @@ -5,15 +5,13 @@ %% Copyright (c) 2020-2023 VMware, Inc. or its affiliates. All rights reserved. %% +-define(APP_NAME, rabbitmq_mqtt). -define(PG_SCOPE, pg_scope_rabbitmq_mqtt_clientid). - -define(QUEUE_TYPE_QOS_0, rabbit_mqtt_qos0_queue). - +-define(PERSISTENT_TERM_MAILBOX_SOFT_LIMIT, mqtt_mailbox_soft_limit). -define(MQTT_GUIDE_URL, <<"https://rabbitmq.com/mqtt.html">>). - -define(MQTT_PROTO_V3, mqtt310). -define(MQTT_PROTO_V4, mqtt311). - -define(INFO_ITEMS, [protocol, host, diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl index e8861de33340..63e120a0c91d 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl @@ -10,6 +10,7 @@ -behaviour(application). -include("rabbit_mqtt.hrl"). +-include_lib("stdlib/include/assert.hrl"). -export([start/2, stop/1]). -export([emit_connection_info_all/4, @@ -20,7 +21,7 @@ start(normal, []) -> init_global_counters(), - rabbit_mqtt_util:init_sparkplug(), + persist_static_configuration(), {ok, Listeners} = application:get_env(tcp_listeners), {ok, SslListeners} = application:get_env(ssl_listeners), case rabbit_mqtt_ff:track_client_id_in_ra() of @@ -50,7 +51,7 @@ emit_connection_info_all(Nodes, Items, Ref, AggregatorPid) -> %% remaining nodes, we send back 'finished' so that the CLI does not time out. [AggregatorPid ! {Ref, finished} || _ <- lists:seq(1, length(Nodes) - 1)]; false -> - Pids = [spawn_link(Node, rabbit_mqtt, emit_connection_info_local, + Pids = [spawn_link(Node, ?MODULE, emit_connection_info_local, [Items, Ref, AggregatorPid]) || Node <- Nodes], rabbit_control_misc:await_emitters_termination(Pids) @@ -98,3 +99,10 @@ init_global_counters(ProtoVer) -> rabbit_global_counters:init([Proto, {queue_type, ?QUEUE_TYPE_QOS_0}]), rabbit_global_counters:init([Proto, {queue_type, rabbit_classic_queue}]), rabbit_global_counters:init([Proto, {queue_type, rabbit_quorum_queue}]). + +persist_static_configuration() -> + rabbit_mqtt_util:init_sparkplug(), + + {ok, MailboxSoftLimit} = application:get_env(?APP_NAME, mailbox_soft_limit), + ?assert(is_number(MailboxSoftLimit)), + ok = persistent_term:put(?PERSISTENT_TERM_MAILBOX_SOFT_LIMIT, MailboxSoftLimit). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 661492b68948..44643dbd955e 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -27,7 +27,6 @@ -include("rabbit_mqtt.hrl"). -include("rabbit_mqtt_packet.hrl"). --define(APP, rabbitmq_mqtt). -define(MAX_PERMISSION_CACHE_SIZE, 12). -define(CONSUMER_TAG, mqtt). @@ -74,7 +73,8 @@ info, delivery_flow :: flow | noflow, %% quorum queues and streams whose soft limit has been exceeded - soft_limit_exceeded = sets:new([{version, 2}]) :: sets:set() + soft_limit_exceeded = sets:new([{version, 2}]) :: sets:set(), + qos0_messages_dropped = 0 :: non_neg_integer() }). -opaque state() :: #state{}. @@ -913,7 +913,7 @@ get_vhost_ssl(UserBin, SslLoginName, Port) -> end. vhost_in_username(UserBin) -> - case application:get_env(?APP, ignore_colons_in_username) of + case application:get_env(?APP_NAME, ignore_colons_in_username) of {ok, true} -> false; _ -> %% split at the last colon, disallowing colons in username @@ -925,7 +925,7 @@ vhost_in_username(UserBin) -> get_vhost_username(UserBin) -> Default = {rabbit_mqtt_util:env(vhost), UserBin}, - case application:get_env(?APP, ignore_colons_in_username) of + case application:get_env(?APP_NAME, ignore_colons_in_username) of {ok, true} -> Default; _ -> %% split at the last colon, disallowing colons in username @@ -972,8 +972,8 @@ human_readable_vhost_lookup_strategy(Val) -> creds(User, Pass, SSLLoginName) -> DefaultUser = rabbit_mqtt_util:env(default_user), DefaultPass = rabbit_mqtt_util:env(default_pass), - {ok, Anon} = application:get_env(?APP, allow_anonymous), - {ok, TLSAuth} = application:get_env(?APP, ssl_cert_login), + {ok, Anon} = application:get_env(?APP_NAME, allow_anonymous), + {ok, TLSAuth} = application:get_env(?APP_NAME, ssl_cert_login), HaveDefaultCreds = Anon =:= true andalso is_binary(DefaultUser) andalso is_binary(DefaultPass), @@ -1320,7 +1320,7 @@ send_puback(PktId, State = #state{send_fun = SendFun, rabbit_global_counters:messages_confirmed(ProtoVer, 1), SendFun( #mqtt_packet{fixed = #mqtt_packet_fixed{type = ?PUBACK}, - variable = #mqtt_packet_publish{packet_id = PktId}}, + variable = #mqtt_packet_publish{packet_id = PktId}}, State). serialise_and_send_to_client(Packet, #state{proto_ver = ProtoVer, socket = Sock}) -> @@ -1487,8 +1487,14 @@ handle_down({{'DOWN', QName}, _MRef, process, QPid, Reason}, {ok, State} end. -handle_queue_event({queue_event, ?QUEUE_TYPE_QOS_0, Msg}, State0) -> - State = deliver_one_to_client(Msg, false, State0), +handle_queue_event({queue_event, ?QUEUE_TYPE_QOS_0, Msg}, + State0 = #state{qos0_messages_dropped = N}) -> + State = case drop_qos0_message(State0) of + false -> + deliver_one_to_client(Msg, false, State0); + true -> + State0#state{qos0_messages_dropped = N + 1} + end, {ok, State}; handle_queue_event({queue_event, QName, Evt}, State0 = #state{queue_states = QStates0, @@ -1738,6 +1744,36 @@ check_topic_access(TopicName, Access, end end. +-spec drop_qos0_message(state()) -> + boolean(). +drop_qos0_message(State) -> + mailbox_soft_limit_exceeded() andalso + is_socket_busy(State#state.socket). + +-spec mailbox_soft_limit_exceeded() -> + boolean(). +mailbox_soft_limit_exceeded() -> + case persistent_term:get(?PERSISTENT_TERM_MAILBOX_SOFT_LIMIT) of + Limit when Limit > 0 -> + case erlang:process_info(self(), message_queue_len) of + {message_queue_len, Len} when Len > Limit -> + true; + _ -> + false + end; + _ -> + false + end. + +is_socket_busy(Socket) -> + case rabbit_net:getstat(Socket, [send_pend]) of + {ok, [{send_pend, NumBytes}]} + when is_number(NumBytes) andalso NumBytes > 0 -> + true; + _ -> + false + end. + info(protocol, #state{info = #info{proto_human = Val}}) -> Val; info(host, #state{info = #info{host = Val}}) -> Val; info(port, #state{info = #info{port = Val}}) -> Val; @@ -1787,7 +1823,8 @@ format_status(#state{queue_states = QState, peer_addr = PeerAddr, register_state = RegisterState, conn_name = ConnName, - info = Info + info = Info, + qos0_messages_dropped = Qos0MsgsDropped } = State) -> #{queue_states => rabbit_queue_type:format_status(QState), proto_ver => ProtoVersion, @@ -1805,7 +1842,8 @@ format_status(#state{queue_states = QState, register_state => RegisterState, conn_name => ConnName, info => Info, - soft_limit_exceeded => soft_limit_exceeded(State)}. + soft_limit_exceeded => soft_limit_exceeded(State), + qos0_messages_dropped => Qos0MsgsDropped}. soft_limit_exceeded(#state{soft_limit_exceeded = SLE}) -> not sets:is_empty(SLE). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl index 332f55e9945f..8f50df26de74 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl @@ -78,12 +78,6 @@ delete(Q, _IfUnused, _IfEmpty, ActingUser) -> ok = rabbit_amqqueue:internal_delete(QName, ActingUser), {ok, 0}. -%%TODO implement some form of flow control to not overwhelm the destination -%% MQTT connection process? -%% E.g. drop this message if destination queue is already long? -% erlang:process_info(Pid, message_queue_len) -%% However that seems to be expensive due to locking (see lcnt). -%% Alternatively, use credits? use rabbit_amqqueue_common:notify_sent/2 on the consuming side? -spec deliver([{amqqueue:amqqueue(), stateless}], Delivery :: term()) -> {[], rabbit_queue_type:actions()}. deliver([{Q, stateless}], Delivery = #delivery{message = BasicMessage}) -> diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index b750ab61e37e..ce83edc73652 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -63,7 +63,7 @@ close_connection(Pid, Reason) -> init(Ref) -> process_flag(trap_exit, true), {ok, Sock} = rabbit_networking:handshake(Ref, - application:get_env(rabbitmq_mqtt, proxy_protocol, false)), + application:get_env(?APP_NAME, proxy_protocol, false)), RealSocket = rabbit_net:unwrap_socket(Sock), case rabbit_net:connection_string(Sock, inbound) of {ok, ConnStr} -> @@ -71,7 +71,7 @@ init(Ref) -> rabbit_log_connection:debug("MQTT accepting TCP connection ~tp (~ts)", [self(), ConnName]), rabbit_alarm:register( self(), {?MODULE, conserve_resources, []}), - LoginTimeout = application:get_env(rabbitmq_mqtt, login_timeout, 10_000), + LoginTimeout = application:get_env(?APP_NAME, login_timeout, 10_000), erlang:send_after(LoginTimeout, self(), login_timeout), ProcessorState = rabbit_mqtt_processor:initial_state(RealSocket, ConnName), gen_server:enter_loop(?MODULE, [], diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer.erl index 906087338724..212c9d3a005f 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer.erl @@ -9,6 +9,7 @@ -behaviour(gen_server2). -include("rabbit_mqtt_retain.hrl"). +-include("rabbit_mqtt.hrl"). -include("rabbit_mqtt_packet.hrl"). -export([init/1, handle_call/3, handle_cast/2, handle_info/2, @@ -56,7 +57,7 @@ init([StoreMod, VHost]) -> {ok, State}. store_module() -> - case application:get_env(rabbitmq_mqtt, retained_message_store) of + case application:get_env(?APP_NAME, retained_message_store) of {ok, Mod} -> Mod; undefined -> undefined end. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl index 900559781fb1..a7cd43865251 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl @@ -20,14 +20,14 @@ start_link(Listeners, []) -> supervisor:start_link({local, ?MODULE}, ?MODULE, [Listeners]). init([{Listeners, SslListeners0}]) -> - NumTcpAcceptors = application:get_env(rabbitmq_mqtt, num_tcp_acceptors, 10), - ConcurrentConnsSups = application:get_env(rabbitmq_mqtt, num_conns_sups, 1), - {ok, SocketOpts} = application:get_env(rabbitmq_mqtt, tcp_listen_options), + NumTcpAcceptors = application:get_env(?APP_NAME, num_tcp_acceptors, 10), + ConcurrentConnsSups = application:get_env(?APP_NAME, num_conns_sups, 1), + {ok, SocketOpts} = application:get_env(?APP_NAME, tcp_listen_options), {SslOpts, NumSslAcceptors, SslListeners} = case SslListeners0 of [] -> {none, 0, []}; _ -> {rabbit_networking:ensure_ssl(), - application:get_env(rabbitmq_mqtt, num_ssl_acceptors, 10), + application:get_env(?APP_NAME, num_ssl_acceptors, 10), case rabbit_networking:poodle_check('MQTT') of ok -> SslListeners0; danger -> [] diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl index 11769b4e6371..165e896957fa 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl @@ -138,7 +138,7 @@ gen_client_id() -> rabbit_misc:base64url(rabbit_guid:gen_secure()). env(Key) -> - case application:get_env(rabbitmq_mqtt, Key) of + case application:get_env(?APP_NAME, Key) of {ok, Val} -> coerce_env_value(Key, Val); undefined -> undefined end. diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/integration_SUITE.erl index 8437d7209867..6e3ddbda5dfd 100644 --- a/deps/rabbitmq_mqtt/test/integration_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/integration_SUITE.erl @@ -35,9 +35,9 @@ groups() -> [ {cluster_size_1, [], [ - %% separate RMQ so global counters start from 0 + %% separate node so global counters start from 0 {global_counters, [], [global_counters_v3, global_counters_v4]}, - {common_tests, [], tests()} + {tests, [], tests()} ]}, {cluster_size_3, [], [queue_down_qos1, @@ -64,6 +64,7 @@ tests() -> ,subscribe_multiple ,large_message_mqtt_to_mqtt ,large_message_amqp_to_mqtt + ,rabbit_mqtt_qos0_queue_overflow ]. suite() -> @@ -85,11 +86,10 @@ init_per_group(cluster_size_1, Config) -> init_per_group(cluster_size_3 = Group, Config) -> init_per_group0(Group, rabbit_ct_helpers:set_config(Config, [{rmq_nodes_count, 3}])); - init_per_group(Group, Config) when Group =:= global_counters orelse - Group =:= common_tests -> - init_per_group0(Group,Config). + Group =:= tests -> + init_per_group0(Group, Config). init_per_group0(Group, Config0) -> Config1 = rabbit_ct_helpers:set_config( @@ -782,10 +782,77 @@ rabbit_mqtt_qos0_queue(Config) -> ok = emqtt:disconnect(Sub), ok = emqtt:disconnect(Pub). +%% Test that queue type rabbit_mqtt_qos0_queue drops QoS 0 messages when its +%% max length is reached. +rabbit_mqtt_qos0_queue_overflow(Config) -> + Topic = atom_to_binary(?FUNCTION_NAME), + Msg = binary:copy(<<"x">>, 1000), + NumMsgs = 10_000, + + %% Provoke TCP back-pressure from client to server by using small buffers. + Opts = [{tcp_opts, [{recbuf, 1500}, + {buffer, 1500}]}], + Sub = connect(<<"subscriber">>, Config, Opts), + {ok, _, [0]} = emqtt:subscribe(Sub, Topic, qos0), + [ServerConnectionPid] = all_connection_pids(Config), + + %% Suspend the receiving client such that it stops reading from its socket + %% causing TCP back-pressure to the server being applied. + true = erlang:suspend_process(Sub), + + %% Let's overflow the receiving server MQTT connection process + %% (i.e. the rabbit_mqtt_qos0_queue) by sending many large messages. + Pub = connect(<<"publisher">>, Config), + lists:foreach(fun(_) -> + ok = emqtt:publish(Pub, Topic, Msg, qos0) + end, lists:seq(1, NumMsgs)), + + %% Give the server some time to process (either send or drop) the messages. + timer:sleep(2000), + + %% Let's resume the receiving client to receive any remaining messages that did + %% not get dropped. + true = erlang:resume_process(Sub), + NumReceived = num_received(Topic, Msg, 0), + + {status, _, _, [_, _, _, _, Misc]} = sys:get_status(ServerConnectionPid), + [State] = [S || {data, [{"State", S}]} <- Misc], + #{proc_state := #{qos0_messages_dropped := NumDropped}} = State, + ct:pal("NumReceived=~b~nNumDropped=~b", [NumReceived, NumDropped]), + + %% We expect that + %% 1. all sent messages were either received or dropped + ?assertEqual(NumMsgs, NumReceived + NumDropped), + case rabbit_ct_helpers:is_mixed_versions(Config) of + false -> + %% 2. at least one message was dropped (otherwise our whole test case did not + %% test what it was supposed to test: that messages are dropped due to the + %% server being overflowed with messages while the client receives too slowly) + ?assert(NumDropped >= 1); + true -> + %% Feature flag rabbit_mqtt_qos0_queue is disabled. + ?assertEqual(0, NumDropped) + end, + %% 3. we received at least 1000 messages because everything below the default + %% of mailbox_soft_limit=1000 should not be dropped + ?assert(NumReceived >= 1000), + + ok = emqtt:disconnect(Sub), + ok = emqtt:disconnect(Pub). + %% ------------------------------------------------------------------- %% Internal helpers %% ------------------------------------------------------------------- +num_received(Topic, Payload, N) -> + receive + {publish, #{topic := Topic, + payload := Payload}} -> + num_received(Topic, Payload, N + 1) + after 1000 -> + N + end. + await_confirms_ordered(_, To, To) -> ok; await_confirms_ordered(From, N, To) -> diff --git a/deps/rabbitmq_mqtt/test/util.erl b/deps/rabbitmq_mqtt/test/util.erl index 9abe67a5135f..24f9f60ffc5d 100644 --- a/deps/rabbitmq_mqtt/test/util.erl +++ b/deps/rabbitmq_mqtt/test/util.erl @@ -88,4 +88,4 @@ connect_to_node(Config, Node, ClientID) -> C = connect(ClientID, Config, Node, [{connect_timeout, 1}, {ack_timeout, 1}]), unlink(C), MRef = erlang:monitor(process, C), - {ok, MRef, C}. \ No newline at end of file + {ok, MRef, C}. From bd0acb33e407ed01c76763daa4b1cb2fb97c2b47 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Mon, 12 Dec 2022 10:05:43 +0000 Subject: [PATCH 082/118] Remove test helper util:connect_to_node/3 because this method is superfluous given that util:connect already exists. --- deps/rabbitmq_mqtt/test/cluster_SUITE.erl | 50 +++++++++++-------- deps/rabbitmq_mqtt/test/ff_SUITE.erl | 9 ++-- deps/rabbitmq_mqtt/test/util.erl | 10 +--- .../src/rabbit_web_mqtt_handler.erl | 10 ++-- deps/rabbitmq_web_mqtt/test/system_SUITE.erl | 6 +-- 5 files changed, 44 insertions(+), 41 deletions(-) diff --git a/deps/rabbitmq_mqtt/test/cluster_SUITE.erl b/deps/rabbitmq_mqtt/test/cluster_SUITE.erl index 7f18325ffca7..13266fe4a946 100644 --- a/deps/rabbitmq_mqtt/test/cluster_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/cluster_SUITE.erl @@ -10,8 +10,8 @@ -include_lib("common_test/include/ct.hrl"). -include_lib("eunit/include/eunit.hrl"). -import(util, [expect_publishes/2, - connect/4, - connect_to_node/3]). + connect/3, + connect/4]). -import(rabbit_ct_broker_helpers, [setup_steps/0, @@ -23,6 +23,9 @@ drain_node/2, revive_node/2]). +-define(OPTS, [{connect_timeout, 1}, + {ack_timeout, 1}]). + all() -> [ {group, cluster_size_3}, @@ -105,9 +108,10 @@ end_per_testcase(Testcase, Config) -> %% ------------------------------------------------------------------- maintenance(Config) -> - {ok, MRef0, C0} = connect_to_node(Config, 0, <<"client-0">>), - {ok, MRef1a, C1a} = connect_to_node(Config, 1, <<"client-1a">>), - {ok, MRef1b, C1b} = connect_to_node(Config, 1, <<"client-1b">>), + C0 = connect(<<"client-0">>, Config, 0, ?OPTS), + C1a = connect(<<"client-1a">>, Config, 1, ?OPTS), + C1b = connect(<<"client-1b">>, Config, 1, ?OPTS), + timer:sleep(500), ok = drain_node(Config, 2), @@ -115,13 +119,14 @@ maintenance(Config) -> timer:sleep(500), [?assert(erlang:is_process_alive(C)) || C <- [C0, C1a, C1b]], + process_flag(trap_exit, true), ok = drain_node(Config, 1), - [await_disconnection(Ref) || Ref <- [MRef1a, MRef1b]], + [await_disconnection(Pid) || Pid <- [C1a, C1b]], ok = revive_node(Config, 1), ?assert(erlang:is_process_alive(C0)), ok = drain_node(Config, 0), - await_disconnection(MRef0), + await_disconnection(C0), ok = revive_node(Config, 0). %% Note about running this testsuite in a mixed-versions cluster: @@ -138,8 +143,8 @@ maintenance(Config) -> %% nodes, the minimum to use Ra in proper conditions. connection_id_tracking(Config) -> - ID = <<"duplicate-id">>, - {ok, MRef1, C1} = connect_to_node(Config, 0, ID), + Id = <<"duplicate-id">>, + C1 = connect(Id, Config, 0, ?OPTS), {ok, _, _} = emqtt:subscribe(C1, <<"TopicA">>, qos0), ok = emqtt:publish(C1, <<"TopicA">>, <<"Payload">>), ok = expect_publishes(<<"TopicA">>, [<<"Payload">>]), @@ -148,27 +153,27 @@ connection_id_tracking(Config) -> assert_connection_count(Config, 4, 2, 1), %% connect to the same node (A or 0) - {ok, MRef2, _C2} = connect_to_node(Config, 0, ID), - %% C1 is disconnected - await_disconnection(MRef1), + process_flag(trap_exit, true), + C2 = connect(Id, Config, 0, ?OPTS), + await_disconnection(C1), assert_connection_count(Config, 4, 2, 1), %% connect to a different node (C or 2) - {ok, _, C3} = connect_to_node(Config, 2, ID), - %% C2 is disconnected - await_disconnection(MRef2), + C3 = connect(Id, Config, 2, ?OPTS), + await_disconnection(C2), assert_connection_count(Config, 4, 2, 1), ok = emqtt:disconnect(C3). connection_id_tracking_on_nodedown(Config) -> Server = get_node_config(Config, 0, nodename), - {ok, MRef, C} = connect_to_node(Config, 0, <<"simpleClient">>), + C = connect(<<"simpleClient">>, Config, ?OPTS), {ok, _, _} = emqtt:subscribe(C, <<"TopicA">>, qos0), ok = emqtt:publish(C, <<"TopicA">>, <<"Payload">>), ok = expect_publishes(<<"TopicA">>, [<<"Payload">>]), assert_connection_count(Config, 4, 2, 1), + process_flag(trap_exit, true), ok = stop_node(Config, Server), - await_disconnection(MRef), + await_disconnection(C), assert_connection_count(Config, 4, 2, 0), ok. @@ -178,14 +183,15 @@ connection_id_tracking_with_decommissioned_node(Config) -> {skip, "This test requires client ID tracking in Ra"}; true -> Server = get_node_config(Config, 0, nodename), - {ok, MRef, C} = connect_to_node(Config, 0, <<"simpleClient">>), + C = connect(<<"simpleClient">>, Config, ?OPTS), {ok, _, _} = emqtt:subscribe(C, <<"TopicA">>, qos0), ok = emqtt:publish(C, <<"TopicA">>, <<"Payload">>), ok = expect_publishes(<<"TopicA">>, [<<"Payload">>]), assert_connection_count(Config, 4, 2, 1), + process_flag(trap_exit, true), {ok, _} = rabbitmqctl(Config, 0, ["decommission_mqtt_node", Server]), - await_disconnection(MRef), + await_disconnection(C), assert_connection_count(Config, 4, 2, 0), ok end. @@ -206,9 +212,9 @@ assert_connection_count(Config, Retries, NodeId, NumElements) -> assert_connection_count(Config, Retries-1, NodeId, NumElements) end. -await_disconnection(Ref) -> +await_disconnection(Client) -> receive - {'DOWN', Ref, _, _, _} -> ok + {'EXIT', Client, _} -> ok after - 20_000 -> exit(missing_down_message) + 20_000 -> ct:fail({missing_exit, Client}) end. diff --git a/deps/rabbitmq_mqtt/test/ff_SUITE.erl b/deps/rabbitmq_mqtt/test/ff_SUITE.erl index ae85690c1152..04ff01a5352d 100644 --- a/deps/rabbitmq_mqtt/test/ff_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/ff_SUITE.erl @@ -15,7 +15,8 @@ -import(rabbit_ct_helpers, [eventually/1]). -import(util, [expect_publishes/2, get_global_counters/4, - connect_to_node/3]). + connect/2, + connect/4]). -define(PROTO_VER, v4). @@ -70,7 +71,7 @@ end_per_testcase(_TestCase, Config) -> delete_ra_cluster_mqtt_node(Config) -> FeatureFlag = ?FUNCTION_NAME, - {ok, _, C} = connect_to_node(Config, 1, <<"my-client">>), + C = connect(<<"my-client">>, Config, 1, []), timer:sleep(500), %% old client ID tracking works ?assertEqual(1, length(util:all_connection_pids(Config))), @@ -94,7 +95,7 @@ rabbit_mqtt_qos0_queue(Config) -> FeatureFlag = ?FUNCTION_NAME, Msg = Topic = ClientId = atom_to_binary(?FUNCTION_NAME), - {ok, _, C1} = connect_to_node(Config, 0, ClientId), + C1 = connect(ClientId, Config), {ok, _, [0]} = emqtt:subscribe(C1, Topic, qos0), ok = emqtt:publish(C1, Topic, Msg, qos0), ok = expect_publishes(Topic, [Msg]), @@ -115,7 +116,7 @@ rabbit_mqtt_qos0_queue(Config) -> %% Reconnecting with the same client ID will terminate the old connection. true = unlink(C1), - {ok, _, C2} = connect_to_node(Config, 0, ClientId), + C2 = connect(ClientId, Config), {ok, _, [0]} = emqtt:subscribe(C2, Topic, qos0), %% This time, we get the new queue type. eventually( diff --git a/deps/rabbitmq_mqtt/test/util.erl b/deps/rabbitmq_mqtt/test/util.erl index 24f9f60ffc5d..e9257bda1a43 100644 --- a/deps/rabbitmq_mqtt/test/util.erl +++ b/deps/rabbitmq_mqtt/test/util.erl @@ -11,8 +11,8 @@ expect_publishes/2, connect/2, connect/3, - connect/4, - connect_to_node/3]). + connect/4 + ]). all_connection_pids(Config) -> Nodes = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), @@ -83,9 +83,3 @@ connect(ClientId, Config, Node, AdditionalOpts) -> {ok, C} = emqtt:start_link(Options), {ok, _Properties} = emqtt:connect(C), C. - -connect_to_node(Config, Node, ClientID) -> - C = connect(ClientID, Config, Node, [{connect_timeout, 1}, {ack_timeout, 1}]), - unlink(C), - MRef = erlang:monitor(process, C), - {ok, MRef, C}. diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index f50195fb795a..254ae35803fa 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -39,7 +39,7 @@ %% Close frame status codes as defined in https://www.rfc-editor.org/rfc/rfc6455#section-7.4.1 -define(CLOSE_NORMAL, 1000). -define(CLOSE_PROTOCOL_ERROR, 1002). --define(CLOSE_INVALID_DATA, 1003). +-define(CLOSE_UNACCEPTABLE_DATA_TYPE, 1003). -define(CLOSE_INCONSISTENT_MSG_TYPE, 1007). %% cowboy_sub_protcol @@ -139,8 +139,8 @@ websocket_handle(Ping, State) %% Log and close connection when receiving any other unexpected frames. websocket_handle(Frame, State) -> rabbit_log_connection:info("Web MQTT: unexpected WebSocket frame ~tp", - [Frame]), - stop(State, ?CLOSE_INVALID_DATA, "unexpected WebSocket frame"). + [Frame]), + stop(State, ?CLOSE_UNACCEPTABLE_DATA_TYPE, <<"unexpected WebSocket frame">>). -spec websocket_info(any(), State) -> {cowboy_websocket:commands(), State} | @@ -228,7 +228,9 @@ terminate(_Reason, _Request, %% Internal. no_supported_sub_protocol(Protocol, Req) -> - rabbit_log_connection:error("Web MQTT: mqtt not found in client supported protocol, protocol: ~tp", [Protocol]), + %% The client MUST include “mqtt” in the list of WebSocket Sub Protocols it offers [MQTT-6.0.0-3]. + rabbit_log_connection:error( + "Web MQTT: 'mqtt' not included in client offered subprotocols: ~tp", [Protocol]), {ok, cowboy_req:reply(400, #{<<"connection">> => <<"close">>}, Req), #state{}}. handle_data(Data, State0 = #state{conn_name = ConnName}) -> diff --git a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl index 1c0420a36d1c..d06ab94ea812 100644 --- a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl +++ b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl @@ -32,7 +32,7 @@ groups() -> , maintenance , client_no_supported_protocol , client_not_support_mqtt - , invalid_data + , unacceptable_data_type , duplicate_id ]} ]. @@ -197,7 +197,7 @@ client_protocol_test(Config, Protocol) -> rfc6455_client:send_binary(WS, rabbit_ws_test_util:mqtt_3_1_1_connect_packet()), {close, _} = rfc6455_client:recv(WS, timer:seconds(1)). -invalid_data(Config) -> +unacceptable_data_type(Config) -> PortStr = rabbit_ws_test_util:get_web_mqtt_port_str(Config), WS = rfc6455_client:new("ws://localhost:" ++ PortStr ++ "/ws", self(), undefined, ["mqtt"]), {ok, _} = rfc6455_client:open(WS), @@ -206,8 +206,8 @@ invalid_data(Config) -> duplicate_id(Config) -> C1 = ws_connect(?FUNCTION_NAME, Config), - process_flag(trap_exit, true), eventually(?_assertEqual(1, num_mqtt_connections(Config, 0))), + process_flag(trap_exit, true), C2 = ws_connect(?FUNCTION_NAME, Config), receive {'EXIT', C1, _Reason} -> From fb913009c4545b02f6441803995f6d8e580f454d Mon Sep 17 00:00:00 2001 From: Chunyi Lyu Date: Tue, 13 Dec 2022 14:45:46 +0000 Subject: [PATCH 083/118] Add func specs for mqtt process_packet and process_request - removed return matching for {error, Error} when calling process_packet because that's not the return type --- deps/rabbitmq_mqtt/include/rabbit_mqtt_packet.hrl | 7 +++++++ deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl | 6 ++++++ deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 4 ---- deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl | 2 -- 4 files changed, 13 insertions(+), 6 deletions(-) diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt_packet.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt_packet.hrl index 5eff36808eea..516b45f866a6 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt_packet.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt_packet.hrl @@ -26,6 +26,11 @@ -define(PINGRESP, 13). -define(DISCONNECT, 14). +-type packet_types() :: ?CONNECT | ?CONNACK | + ?PUBLISH | ?PUBACK | ?PUBREC | ?PUBREL | ?PUBCOMP | + ?SUBSCRIBE | ?SUBACK | ?UNSUBSCRIBE | ?UNSUBACK | + ?PINGREQ | ?PINGRESP | ?DISCONNECT. + %% connect return codes %% Connection accepted. @@ -64,6 +69,8 @@ payload :: iodata() }). +-type mqtt_packet() :: #mqtt_packet{}. + -record(mqtt_packet_connect, {proto_ver, will_retain, will_qos, diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 44643dbd955e..cdffae97479c 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -98,6 +98,9 @@ initial_state(Socket, ConnectionName, SendFun, PeerAddr) -> send_fun = SendFun, delivery_flow = Flow}. +-spec process_packet(mqtt_packet(), state()) -> + {ok, state()} | + {error, Reason :: term(), state()}. process_packet(#mqtt_packet{fixed = #mqtt_packet_fixed{type = Type}}, State = #state{auth_state = undefined}) when Type =/= ?CONNECT -> @@ -105,6 +108,9 @@ process_packet(#mqtt_packet{fixed = #mqtt_packet_fixed{type = Type}}, process_packet(Packet = #mqtt_packet{fixed = #mqtt_packet_fixed{type = Type}}, State) -> process_request(Type, Packet, State). +-spec process_request(packet_types(), mqtt_packet(), state()) -> + {ok, state()} | + {error, Reason :: term(), state()}. process_request(?CONNECT, Packet, State = #state{socket = Socket}) -> %% Check whether peer closed the connection. %% For example, this can happen when connection was blocked because of resource diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index ce83edc73652..f80f84f4c537 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -342,10 +342,6 @@ process_received_bytes(Bytes, rabbit_log_connection:error("MQTT protocol error on connection ~ts: ~tp", [ConnName, Reason]), {stop, {shutdown, Reason}, pstate(State, ProcState1)}; - {error, Error} -> - rabbit_log_connection:error("MQTT detected a framing error on connection ~ts: ~tp", - [ConnName, Error]), - {stop, {shutdown, Error}, State}; {stop, disconnect, ProcState1} -> {stop, normal, {_SendWill = false, pstate(State, ProcState1)}} end; diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index 254ae35803fa..10167a8e10e6 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -271,8 +271,6 @@ handle_data1(Data, State = #state{ parse_state = ParseState, rabbit_log_connection:info("MQTT protocol error ~tp for connection ~tp", [Reason, ConnName]), stop(State, ?CLOSE_PROTOCOL_ERROR, Reason); - {error, Error} -> - stop_with_framing_error(State, Error, ConnName); {stop, disconnect, ProcState1} -> stop({_SendWill = false, State#state{proc_state = ProcState1}}) end; From 482bef519d99ee2d0e939f2da253a3dba85fb456 Mon Sep 17 00:00:00 2001 From: Chunyi Lyu Date: Wed, 14 Dec 2022 12:09:17 +0000 Subject: [PATCH 084/118] Stop rabbit_mqtt_packet:parse from crashing in web mqtt - add a wrapper for mqtt parse/2 to prevent crashing when parse/2 fails to parse a packet - add invalid packet test case for web mqtt; server will respond status code 1007, same status code as framing error --- .../src/rabbit_web_mqtt_handler.erl | 26 +++++++++++-------- deps/rabbitmq_web_mqtt/test/system_SUITE.erl | 9 +++++++ 2 files changed, 24 insertions(+), 11 deletions(-) diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index 10167a8e10e6..b2ce5ee371e0 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -233,12 +233,10 @@ no_supported_sub_protocol(Protocol, Req) -> "Web MQTT: 'mqtt' not included in client offered subprotocols: ~tp", [Protocol]), {ok, cowboy_req:reply(400, #{<<"connection">> => <<"close">>}, Req), #state{}}. -handle_data(Data, State0 = #state{conn_name = ConnName}) -> +handle_data(Data, State0 = #state{}) -> case handle_data1(Data, State0) of {ok, State1 = #state{state = blocked}, hibernate} -> {[{active, false}], State1, hibernate}; - {error, Error} -> - stop_with_framing_error(State0, Error, ConnName); Other -> Other end. @@ -255,7 +253,7 @@ handle_data1(<<>>, State) -> handle_data1(Data, State = #state{ parse_state = ParseState, proc_state = ProcState, conn_name = ConnName }) -> - case rabbit_mqtt_packet:parse(Data, ParseState) of + case parse(Data, ParseState) of {more, ParseState1} -> {ok, ensure_stats_timer(control_throttle( State #state{ parse_state = ParseState1 })), hibernate}; @@ -274,15 +272,21 @@ handle_data1(Data, State = #state{ parse_state = ParseState, {stop, disconnect, ProcState1} -> stop({_SendWill = false, State#state{proc_state = ProcState1}}) end; - Other -> - Other + {error, Error} -> + rabbit_log_connection:error("MQTT parsing error ~tp for connection ~tp", + [Error, ConnName]), + stop(State, ?CLOSE_INCONSISTENT_MSG_TYPE, Error) end. -stop_with_framing_error(State, Error0, ConnName) -> - Error1 = rabbit_misc:format("~tp", [Error0]), - rabbit_log_connection:error("MQTT detected framing error '~ts' for connection ~tp", - [Error1, ConnName]), - stop(State, ?CLOSE_INCONSISTENT_MSG_TYPE, Error1). +parse(Data, ParseState) -> + try + rabbit_mqtt_packet:parse(Data, ParseState) + catch + _:Error:Stacktrace -> + rabbit_log_connection:error("MQTT cannot parse a packet; payload: ~tp, error: {~tp, ~tp} ", + [Data, Error, Stacktrace]), + {error, cannot_parse} + end. stop(State) -> stop(State, ?CLOSE_NORMAL, "MQTT died"). diff --git a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl index d06ab94ea812..bd74cb304d2e 100644 --- a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl +++ b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl @@ -34,6 +34,7 @@ groups() -> , client_not_support_mqtt , unacceptable_data_type , duplicate_id + , handle_invalid_packets ]} ]. @@ -218,6 +219,14 @@ duplicate_id(Config) -> eventually(?_assertEqual(1, num_mqtt_connections(Config, 0))), ok = emqtt:disconnect(C2). +handle_invalid_packets(Config) -> + PortStr = rabbit_ws_test_util:get_web_mqtt_port_str(Config), + WS = rfc6455_client:new("ws://localhost:" ++ PortStr ++ "/ws", self(), undefined, ["mqtt"]), + {ok, _} = rfc6455_client:open(WS), + Bin = <<"GET / HTTP/1.1\r\nHost: www.rabbitmq.com\r\nUser-Agent: curl/7.43.0\r\nAccept: */*">>, + rfc6455_client:send_binary(WS, Bin), + {close, {1007, _}} = rfc6455_client:recv(WS, timer:seconds(1)). + %% Web mqtt connections are tracked together with mqtt connections num_mqtt_connections(Config, Node) -> length(rpc(Config, Node, rabbit_mqtt, local_connection_pids, [])). From 4ca12b767a9b3974e80dcd09c3dc6b7814ed46d3 Mon Sep 17 00:00:00 2001 From: Chunyi Lyu Date: Thu, 15 Dec 2022 11:16:46 +0000 Subject: [PATCH 085/118] Fix func spec for mqtt process_request - it also returns {stop, disconnect, state()} when receiving a disconnect packet - remove match for a {timeout, _} return when calling register_client. register_client only returns {ok, _} and {error, _} according to its function spec --- deps/rabbitmq_mqtt/include/rabbit_mqtt_packet.hrl | 5 +---- deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl | 7 ++----- 2 files changed, 3 insertions(+), 9 deletions(-) diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt_packet.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt_packet.hrl index 516b45f866a6..26bb2ad733c7 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt_packet.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt_packet.hrl @@ -26,10 +26,7 @@ -define(PINGRESP, 13). -define(DISCONNECT, 14). --type packet_types() :: ?CONNECT | ?CONNACK | - ?PUBLISH | ?PUBACK | ?PUBREC | ?PUBREL | ?PUBCOMP | - ?SUBSCRIBE | ?SUBACK | ?UNSUBSCRIBE | ?UNSUBACK | - ?PINGREQ | ?PINGRESP | ?DISCONNECT. +-type packet_type() :: ?CONNECT .. ?DISCONNECT. %% connect return codes diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index cdffae97479c..df2b9361cd21 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -108,8 +108,9 @@ process_packet(#mqtt_packet{fixed = #mqtt_packet_fixed{type = Type}}, process_packet(Packet = #mqtt_packet{fixed = #mqtt_packet_fixed{type = Type}}, State) -> process_request(Type, Packet, State). --spec process_request(packet_types(), mqtt_packet(), state()) -> +-spec process_request(packet_type(), mqtt_packet(), state()) -> {ok, state()} | + {stop, disconnect, state()} | {error, Reason :: term(), state()}. process_request(?CONNECT, Packet, State = #state{socket = Socket}) -> %% Check whether peer closed the connection. @@ -444,10 +445,6 @@ register_client(Packet = #mqtt_packet_connect{proto_ver = ProtoVersion}, %% e.g. this node was removed from the MQTT cluster members rabbit_log_connection:error("MQTT cannot accept a connection: " "client ID tracker is unavailable: ~p", [Err]), - {error, ?CONNACK_SERVER_UNAVAILABLE}; - {timeout, _} -> - rabbit_log_connection:error("MQTT cannot accept a connection: " - "client ID registration timed out"), {error, ?CONNACK_SERVER_UNAVAILABLE} end; false -> From 340e930d281ccc62194b06df06e63301be4dbf78 Mon Sep 17 00:00:00 2001 From: Chunyi Lyu Date: Thu, 15 Dec 2022 11:54:10 +0000 Subject: [PATCH 086/118] Web mqtt returns 1002 with mqtt parsing error - it is a mqtt protocol error --- deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl | 1 + .../src/rabbit_web_mqtt_handler.erl | 16 ++++++++-------- deps/rabbitmq_web_mqtt/test/system_SUITE.erl | 4 ++-- 3 files changed, 11 insertions(+), 10 deletions(-) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index df2b9361cd21..306a82c5d986 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -100,6 +100,7 @@ initial_state(Socket, ConnectionName, SendFun, PeerAddr) -> -spec process_packet(mqtt_packet(), state()) -> {ok, state()} | + {stop, disconnect, state()} | {error, Reason :: term(), state()}. process_packet(#mqtt_packet{fixed = #mqtt_packet_fixed{type = Type}}, State = #state{auth_state = undefined}) diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index b2ce5ee371e0..b73626addd28 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -40,7 +40,6 @@ -define(CLOSE_NORMAL, 1000). -define(CLOSE_PROTOCOL_ERROR, 1002). -define(CLOSE_UNACCEPTABLE_DATA_TYPE, 1003). --define(CLOSE_INCONSISTENT_MSG_TYPE, 1007). %% cowboy_sub_protcol upgrade(Req, Env, Handler, HandlerState) -> @@ -266,16 +265,12 @@ handle_data1(Data, State = #state{ parse_state = ParseState, State#state{parse_state = PS, proc_state = ProcState1}); {error, Reason, _} -> - rabbit_log_connection:info("MQTT protocol error ~tp for connection ~tp", - [Reason, ConnName]), - stop(State, ?CLOSE_PROTOCOL_ERROR, Reason); + stop_mqtt_protocol_error(State, Reason, ConnName); {stop, disconnect, ProcState1} -> stop({_SendWill = false, State#state{proc_state = ProcState1}}) end; - {error, Error} -> - rabbit_log_connection:error("MQTT parsing error ~tp for connection ~tp", - [Error, ConnName]), - stop(State, ?CLOSE_INCONSISTENT_MSG_TYPE, Error) + {error, Reason} -> + stop_mqtt_protocol_error(State, Reason, ConnName) end. parse(Data, ParseState) -> @@ -288,6 +283,11 @@ parse(Data, ParseState) -> {error, cannot_parse} end. +stop_mqtt_protocol_error(State, Reason, ConnName) -> + rabbit_log_connection:info("MQTT protocol error ~tp for connection ~tp", + [Reason, ConnName]), + stop(State, ?CLOSE_PROTOCOL_ERROR, Reason). + stop(State) -> stop(State, ?CLOSE_NORMAL, "MQTT died"). diff --git a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl index bd74cb304d2e..11a0d5daa0df 100644 --- a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl +++ b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl @@ -214,7 +214,7 @@ duplicate_id(Config) -> {'EXIT', C1, _Reason} -> ok after 5000 -> - ct:fail("server did not disconnect a client with duplicate ID") + ct:fail("server did not disconnect a client with duplicate ID") end, eventually(?_assertEqual(1, num_mqtt_connections(Config, 0))), ok = emqtt:disconnect(C2). @@ -225,7 +225,7 @@ handle_invalid_packets(Config) -> {ok, _} = rfc6455_client:open(WS), Bin = <<"GET / HTTP/1.1\r\nHost: www.rabbitmq.com\r\nUser-Agent: curl/7.43.0\r\nAccept: */*">>, rfc6455_client:send_binary(WS, Bin), - {close, {1007, _}} = rfc6455_client:recv(WS, timer:seconds(1)). + {close, {1002, _}} = rfc6455_client:recv(WS, timer:seconds(1)). %% Web mqtt connections are tracked together with mqtt connections num_mqtt_connections(Config, Node) -> From 4fa8e830ada0676e4193cbbb0abc0c788cf770f8 Mon Sep 17 00:00:00 2001 From: Chunyi Lyu Date: Fri, 16 Dec 2022 14:30:16 +0000 Subject: [PATCH 087/118] Allow undefined in some mqtt record type fields - to get rid of dialyzer warnings like "Record construction... violates the declared type of field XYZ" --- deps/rabbitmq_mqtt/include/rabbit_mqtt_packet.hrl | 10 ++++++---- deps/rabbitmq_mqtt/src/rabbit_mqtt_keepalive.erl | 4 +++- deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl | 10 +++++----- 3 files changed, 14 insertions(+), 10 deletions(-) diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt_packet.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt_packet.hrl index 26bb2ad733c7..49f531f7e3b9 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt_packet.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt_packet.hrl @@ -55,6 +55,8 @@ %% Packet identifier is a non zero two byte integer. -type packet_id() :: 1..16#ffff. +-type option(T) :: undefined | T. + -record(mqtt_packet_fixed, {type = 0, dup = 0, qos = 0, @@ -62,8 +64,8 @@ }). -record(mqtt_packet, {fixed :: #mqtt_packet_fixed{}, - variable :: tuple(), - payload :: iodata() + variable :: option(tuple()), + payload :: option(iodata()) }). -type mqtt_packet() :: #mqtt_packet{}. @@ -99,8 +101,8 @@ -record(mqtt_msg, {retain :: boolean(), qos :: qos(), topic :: string(), - dup :: boolean(), - packet_id :: packet_id(), + dup :: option(boolean()), + packet_id :: option(packet_id()), payload :: binary()}). -type mqtt_msg() :: #mqtt_msg{}. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_keepalive.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_keepalive.erl index 46ab4a129277..696d6517e4f4 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_keepalive.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_keepalive.erl @@ -7,7 +7,9 @@ -export_type([state/0]). --record(state, {timer :: reference(), +-type option(T) :: undefined | T. + +-record(state, {timer :: option(reference()), interval_ms :: pos_integer(), socket :: inet:socket(), recv_oct :: non_neg_integer(), diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 306a82c5d986..0304bc2c8a6a 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -44,7 +44,7 @@ -record(state, {socket, - proto_ver :: mqtt310 | mqtt311, + proto_ver :: option(mqtt310 | mqtt311), queue_states = rabbit_queue_type:init() :: rabbit_queue_type:state(), %% Packet IDs published to queues but not yet confirmed. unacked_client_pubs = rabbit_mqtt_confirms:init() :: rabbit_mqtt_confirms:state(), @@ -54,12 +54,12 @@ %% (Not to be confused with packet IDs sent from client to server which can be the %% same IDs because client and server assign IDs independently of each other.) packet_id = 1 :: packet_id(), - client_id :: binary(), - clean_sess :: boolean(), + client_id :: option(binary()), + clean_sess :: option(boolean()), will_msg, - exchange :: rabbit_exchange:name(), + exchange :: option(rabbit_exchange:name()), %% Set if client has at least one subscription with QoS 1. - queue_qos1 :: rabbit_amqqueue:name(), + queue_qos1 :: option(rabbit_amqqueue:name()), has_published = false :: boolean(), ssl_login_name, %% Retained messages handler. See rabbit_mqtt_retainer_sup From cb68e4866e429473babff2aa6f513ae1f71be36e Mon Sep 17 00:00:00 2001 From: Chunyi Lyu Date: Fri, 16 Dec 2022 15:38:23 +0000 Subject: [PATCH 088/118] Resolve some dialyzer issues - mqtt processor publish_to_queue/2 is called in process_request(?PUBLISH,_, _) and maybe_send_will/3. In both places #mqtt_msg{} is initialized with value so it will never be 'undefined'. - all possible value are already matched in mqtt_processor human_readable_vhost_lookup_strategy/1; deleted the unneeded catch all function clause. - Removed a unnecessary case matching in mqtt_reader init/1. Return values for 'rabbit_net:connection_string' are {ok, _} or {error, _}. {'network_error', Reason} will never match. - Fix function spec for mqtt_util gen_client_id/1. Return type of rabbit_misc:base64url is string(), not binary(). --- deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl | 6 +----- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 3 --- deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl | 2 +- 3 files changed, 2 insertions(+), 9 deletions(-) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 0304bc2c8a6a..86ebf17d7540 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -969,9 +969,7 @@ human_readable_vhost_lookup_strategy(port_to_vhost_mapping) -> human_readable_vhost_lookup_strategy(cert_to_vhost_mapping) -> "client certificate to vhost mapping"; human_readable_vhost_lookup_strategy(default_vhost) -> - "plugin configuration or default"; -human_readable_vhost_lookup_strategy(Val) -> - atom_to_list(Val). + "plugin configuration or default". creds(User, Pass, SSLLoginName) -> DefaultUser = rabbit_mqtt_util:env(default_user), @@ -1208,8 +1206,6 @@ binding_action( key = RoutingKey}, BindingFun(Binding, Username). -publish_to_queues(undefined, State) -> - {ok, State}; publish_to_queues( #mqtt_msg{qos = Qos, topic = Topic, diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index f80f84f4c537..bf6fe8a13649 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -87,9 +87,6 @@ init(Ref) -> parse_state = rabbit_mqtt_packet:initial_state(), proc_state = ProcessorState }), #state.stats_timer) ); - {network_error, Reason} -> - rabbit_net:fast_close(RealSocket), - terminate({shutdown, Reason}, undefined); {error, enotconn} -> rabbit_net:fast_close(RealSocket), terminate(shutdown, undefined); diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl index 165e896957fa..4183829d3a93 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl @@ -133,7 +133,7 @@ to_mqtt(T0) -> T2 = string:replace(T1, ".", "/", all), erlang:iolist_to_binary(T2). --spec gen_client_id() -> binary(). +-spec gen_client_id() -> string(). gen_client_id() -> rabbit_misc:base64url(rabbit_guid:gen_secure()). From 56e97a91426a2dcef6f06a9233af11e02235b3c4 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Mon, 19 Dec 2022 18:03:18 +0000 Subject: [PATCH 089/118] Fix MQTT in management plugin 1. Allow to inspect an (web) MQTT connection. 2. Show MQTT client ID on connection page as part of client_properties. 3. Handle force_event_refresh (when management_plugin gets enabled after (web) MQTT connections got created). 4. Reduce code duplication between protocol readers. 5. Display '?' instead of 'NaN' in UI for absent queue metrics. 6. Allow an (web) MQTT connection to be closed via management_plugin. For 6. this commit takes the same approach as already done for the stream plugin: The stream plugin registers neither with {type, network} nor {type, direct}. We cannot use gen_server:call/3 anymore to close the connection because the web MQTT connection cannot handle gen_server calls (only casts). Strictly speaking, this commit requires a feature flag to allow to force closing stream connections from the management plugin during a rolling update. However, given that this is rather an edge case, and there is a workaround (connect to the node directly hosting the stream connection), this commit will not introduce a new feature flag. --- deps/rabbit/src/rabbit_reader.erl | 38 ++--- deps/rabbit/src/rabbit_ssl.erl | 32 ++++ .../src/rabbit_amqp1_0_reader.erl | 40 ++--- .../priv/www/js/formatters.js | 9 +- .../rabbitmq_management/priv/www/js/global.js | 2 +- .../priv/www/js/tmpl/connection.ejs | 2 +- .../priv/www/js/tmpl/connections.ejs | 2 +- .../src/rabbit_mgmt_wm_connection.erl | 17 +- deps/rabbitmq_mqtt/BUILD.bazel | 8 +- deps/rabbitmq_mqtt/Makefile | 2 +- deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl | 50 +++++- deps/rabbitmq_mqtt/src/rabbit_mqtt_ff.erl | 2 +- .../src/rabbit_mqtt_keepalive.erl | 76 +++++---- .../src/rabbit_mqtt_processor.erl | 102 ++++++----- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 156 +++++++++-------- deps/rabbitmq_mqtt/test/command_SUITE.erl | 31 ++-- deps/rabbitmq_mqtt/test/config_SUITE.erl | 2 +- deps/rabbitmq_mqtt/test/event_recorder.erl | 6 + deps/rabbitmq_mqtt/test/integration_SUITE.erl | 137 +++++++++++---- deps/rabbitmq_mqtt/test/reader_SUITE.erl | 4 +- .../src/rabbit_stream_reader.erl | 71 +++----- .../src/rabbit_stream_connection_mgmt.erl | 2 +- deps/rabbitmq_web_mqtt/BUILD.bazel | 9 +- deps/rabbitmq_web_mqtt/Makefile | 2 +- .../src/rabbit_web_mqtt_handler.erl | 158 ++++++++++++------ deps/rabbitmq_web_mqtt/test/system_SUITE.erl | 108 ++++++++++-- 26 files changed, 652 insertions(+), 416 deletions(-) diff --git a/deps/rabbit/src/rabbit_reader.erl b/deps/rabbit/src/rabbit_reader.erl index 6503748cb83a..050cd12c24dd 100644 --- a/deps/rabbit/src/rabbit_reader.erl +++ b/deps/rabbit/src/rabbit_reader.erl @@ -1505,13 +1505,18 @@ i(SockStat, S) when SockStat =:= recv_oct; fun ([{_, I}]) -> I end, S); i(ssl, #v1{sock = Sock, proxy_socket = ProxySock}) -> rabbit_net:proxy_ssl_info(Sock, ProxySock) /= nossl; -i(ssl_protocol, S) -> ssl_info(fun ({P, _}) -> P end, S); -i(ssl_key_exchange, S) -> ssl_info(fun ({_, {K, _, _}}) -> K end, S); -i(ssl_cipher, S) -> ssl_info(fun ({_, {_, C, _}}) -> C end, S); -i(ssl_hash, S) -> ssl_info(fun ({_, {_, _, H}}) -> H end, S); -i(peer_cert_issuer, S) -> cert_info(fun rabbit_ssl:peer_cert_issuer/1, S); -i(peer_cert_subject, S) -> cert_info(fun rabbit_ssl:peer_cert_subject/1, S); -i(peer_cert_validity, S) -> cert_info(fun rabbit_ssl:peer_cert_validity/1, S); +i(SSL, #v1{sock = Sock, proxy_socket = ProxySock}) + when SSL =:= ssl; + SSL =:= ssl_protocol; + SSL =:= ssl_key_exchange; + SSL =:= ssl_cipher; + SSL =:= ssl_hash -> + rabbit_ssl:info(SSL, {Sock, ProxySock}); +i(Cert, #v1{sock = Sock}) + when Cert =:= peer_cert_issuer; + Cert =:= peer_cert_subject; + Cert =:= peer_cert_validity -> + rabbit_ssl:cert_info(Cert, Sock); i(channels, #v1{channel_count = ChannelCount}) -> ChannelCount; i(state, #v1{connection_state = ConnectionState, throttle = #throttle{blocked_by = Reasons, @@ -1574,25 +1579,6 @@ socket_info(Get, Select, #v1{sock = Sock}) -> {error, _} -> 0 end. -ssl_info(F, #v1{sock = Sock, proxy_socket = ProxySock}) -> - case rabbit_net:proxy_ssl_info(Sock, ProxySock) of - nossl -> ''; - {error, _} -> ''; - {ok, Items} -> - P = proplists:get_value(protocol, Items), - #{cipher := C, - key_exchange := K, - mac := H} = proplists:get_value(selected_cipher_suite, Items), - F({P, {K, C, H}}) - end. - -cert_info(F, #v1{sock = Sock}) -> - case rabbit_net:peercert(Sock) of - nossl -> ''; - {error, _} -> ''; - {ok, Cert} -> list_to_binary(F(Cert)) - end. - maybe_emit_stats(State) -> rabbit_event:if_enabled(State, #v1.stats_timer, fun() -> emit_stats(State) end). diff --git a/deps/rabbit/src/rabbit_ssl.erl b/deps/rabbit/src/rabbit_ssl.erl index 7c6873e6d5ce..abd94f4ba733 100644 --- a/deps/rabbit/src/rabbit_ssl.erl +++ b/deps/rabbit/src/rabbit_ssl.erl @@ -14,6 +14,7 @@ -export([cipher_suites_erlang/2, cipher_suites_erlang/1, cipher_suites_openssl/2, cipher_suites_openssl/1, cipher_suites/1]). +-export([info/2, cert_info/2]). %%-------------------------------------------------------------------------- @@ -181,3 +182,34 @@ otp_san_type(uri) -> uniformResourceIdentifier; otp_san_type(other_name) -> otherName; otp_san_type(Other) -> Other. +info(ssl_protocol, Socks) -> info0(fun ({P, _}) -> P end, Socks); +info(ssl_key_exchange, Socks) -> info0(fun ({_, {K, _, _}}) -> K end, Socks); +info(ssl_cipher, Socks) -> info0(fun ({_, {_, C, _}}) -> C end, Socks); +info(ssl_hash, Socks) -> info0(fun ({_, {_, _, H}}) -> H end, Socks); +info(ssl, {Sock, ProxySock}) -> rabbit_net:proxy_ssl_info(Sock, ProxySock) /= nossl. + +info0(F, {Sock, ProxySock}) -> + case rabbit_net:proxy_ssl_info(Sock, ProxySock) of + nossl -> ''; + {error, _} -> ''; + {ok, Items} -> + P = proplists:get_value(protocol, Items), + #{cipher := C, + key_exchange := K, + mac := H} = proplists:get_value(selected_cipher_suite, Items), + F({P, {K, C, H}}) + end. + +cert_info(peer_cert_issuer, Sock) -> + cert_info0(fun peer_cert_issuer/1, Sock); +cert_info(peer_cert_subject, Sock) -> + cert_info0(fun peer_cert_subject/1, Sock); +cert_info(peer_cert_validity, Sock) -> + cert_info0(fun peer_cert_validity/1, Sock). + +cert_info0(F, Sock) -> + case rabbit_net:peercert(Sock) of + nossl -> ''; + {error, _} -> ''; + {ok, Cert} -> list_to_binary(F(Cert)) + end. diff --git a/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_reader.erl b/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_reader.erl index a780d00df9d0..86d78890c3ee 100644 --- a/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_reader.erl +++ b/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_reader.erl @@ -785,16 +785,17 @@ info_internal(SockStat, S) when SockStat =:= recv_oct; socket_info(fun (Sock) -> rabbit_net:getstat(Sock, [SockStat]) end, fun ([{_, I}]) -> I end, S); info_internal(ssl, #v1{sock = Sock}) -> rabbit_net:is_ssl(Sock); -info_internal(ssl_protocol, S) -> ssl_info(fun ({P, _}) -> P end, S); -info_internal(ssl_key_exchange, S) -> ssl_info(fun ({_, {K, _, _}}) -> K end, S); -info_internal(ssl_cipher, S) -> ssl_info(fun ({_, {_, C, _}}) -> C end, S); -info_internal(ssl_hash, S) -> ssl_info(fun ({_, {_, _, H}}) -> H end, S); -info_internal(peer_cert_issuer, S) -> - cert_info(fun rabbit_ssl:peer_cert_issuer/1, S); -info_internal(peer_cert_subject, S) -> - cert_info(fun rabbit_ssl:peer_cert_subject/1, S); -info_internal(peer_cert_validity, S) -> - cert_info(fun rabbit_ssl:peer_cert_validity/1, S). +info_internal(SSL, #v1{sock = Sock, proxy_socket = ProxySock}) + when SSL =:= ssl_protocol; + SSL =:= ssl_key_exchange; + SSL =:= ssl_cipher; + SSL =:= ssl_hash -> + rabbit_ssl:info(SSL, {Sock, ProxySock}); +info_internal(Cert, #v1{sock = Sock}) + when Cert =:= peer_cert_issuer; + Cert =:= peer_cert_subject; + Cert =:= peer_cert_validity -> + rabbit_ssl:cert_info(Cert, Sock). %% From rabbit_reader socket_info(Get, Select, #v1{sock = Sock}) -> @@ -802,22 +803,3 @@ socket_info(Get, Select, #v1{sock = Sock}) -> {ok, T} -> Select(T); {error, _} -> '' end. - -ssl_info(F, #v1{sock = Sock, proxy_socket = ProxySock}) -> - case rabbit_net:proxy_ssl_info(Sock, ProxySock) of - nossl -> ''; - {error, _} -> ''; - {ok, Items} -> - P = proplists:get_value(protocol, Items), - #{cipher := C, - key_exchange := K, - mac := H} = proplists:get_value(selected_cipher_suite, Items), - F({P, {K, C, H}}) - end. - -cert_info(F, #v1{sock = Sock}) -> - case rabbit_net:peercert(Sock) of - nossl -> ''; - {error, _} -> ''; - {ok, Cert} -> list_to_binary(F(Cert)) - end. diff --git a/deps/rabbitmq_management/priv/www/js/formatters.js b/deps/rabbitmq_management/priv/www/js/formatters.js index 9ef81a8ac7d1..132afb6afc69 100644 --- a/deps/rabbitmq_management/priv/www/js/formatters.js +++ b/deps/rabbitmq_management/priv/www/js/formatters.js @@ -275,6 +275,7 @@ function fmt_rate_num(num) { } function fmt_num_thousands(num) { + if (num == undefined) return UNKNOWN_REPR; var conv_num = parseFloat(num); // to avoid errors, if someone calls fmt_num_thousands(someNumber.toFixed(0)) return fmt_num_thousands_unfixed(conv_num.toFixed(0)); } @@ -288,11 +289,9 @@ function fmt_num_thousands_unfixed(num) { } function fmt_percent(num) { - if (num === '') { - return 'N/A'; - } else { - return Math.round(num * 100) + '%'; - } + if (num == undefined) return UNKNOWN_REPR; + if (num === '') return UNKNOWN_REPR; + return Math.round(num * 100) + '%'; } function pick_rate(fmt, obj, name, mode) { diff --git a/deps/rabbitmq_management/priv/www/js/global.js b/deps/rabbitmq_management/priv/www/js/global.js index 2b81dd8c9ded..023b5b3c57d8 100644 --- a/deps/rabbitmq_management/priv/www/js/global.js +++ b/deps/rabbitmq_management/priv/www/js/global.js @@ -112,7 +112,7 @@ var ALL_COLUMNS = ['channels', 'Channels', true], ['channel_max', 'Channel max', false], ['frame_max', 'Frame max', false], - ['auth_mechanism', 'Auth mechanism', false], + ['auth_mechanism', 'SASL auth mechanism', false], ['client', 'Client', false]], 'Network': [['from_client', 'From client', true], ['to_client', 'To client', true], diff --git a/deps/rabbitmq_management/priv/www/js/tmpl/connection.ejs b/deps/rabbitmq_management/priv/www/js/tmpl/connection.ejs index f107a3981c1a..f834b02fb5e0 100644 --- a/deps/rabbitmq_management/priv/www/js/tmpl/connection.ejs +++ b/deps/rabbitmq_management/priv/www/js/tmpl/connection.ejs @@ -44,7 +44,7 @@ <% if (connection.auth_mechanism) { %> - Authentication + SASL auth mechanism <%= connection.auth_mechanism %> <% } %> diff --git a/deps/rabbitmq_management/priv/www/js/tmpl/connections.ejs b/deps/rabbitmq_management/priv/www/js/tmpl/connections.ejs index a75fa7f56011..464894d20876 100644 --- a/deps/rabbitmq_management/priv/www/js/tmpl/connections.ejs +++ b/deps/rabbitmq_management/priv/www/js/tmpl/connections.ejs @@ -52,7 +52,7 @@ <%= fmt_sort('Frame max', 'frame_max') %> <% } %> <% if (show_column('connections', 'auth_mechanism')) { %> - <%= fmt_sort('Auth mechanism', 'auth_mechanism') %> + <%= fmt_sort('SASL auth mechanism', 'auth_mechanism') %> <% } %> <% if (show_column('connections', 'client')) { %> <%= fmt_sort('Client', 'properties') %> diff --git a/deps/rabbitmq_management/src/rabbit_mgmt_wm_connection.erl b/deps/rabbitmq_management/src/rabbit_mgmt_wm_connection.erl index 5e14eb710465..820506400128 100644 --- a/deps/rabbitmq_management/src/rabbit_mgmt_wm_connection.erl +++ b/deps/rabbitmq_management/src/rabbit_mgmt_wm_connection.erl @@ -90,11 +90,14 @@ force_close_connection(ReqData, Conn, Pid) -> undefined -> "Closed via management plugin"; V -> binary_to_list(V) end, - case proplists:get_value(type, Conn) of - direct -> amqp_direct_connection:server_close(Pid, 320, Reason); - network -> rabbit_networking:close_connection(Pid, Reason); - _ -> - % best effort, this will work for connections to the stream plugin - gen_server:call(Pid, {shutdown, Reason}, infinity) - end, + case proplists:get_value(type, Conn) of + direct -> + amqp_direct_connection:server_close(Pid, 320, Reason); + network -> + rabbit_networking:close_connection(Pid, Reason); + _ -> + %% Best effort will work for following plugins: + %% rabbitmq_stream, rabbitmq_mqtt, rabbitmq_web_mqtt + Pid ! {shutdown, Reason} + end, ok. diff --git a/deps/rabbitmq_mqtt/BUILD.bazel b/deps/rabbitmq_mqtt/BUILD.bazel index 3872527f961f..bfb6255e4a67 100644 --- a/deps/rabbitmq_mqtt/BUILD.bazel +++ b/deps/rabbitmq_mqtt/BUILD.bazel @@ -83,7 +83,11 @@ dialyze( warnings_as_errors = False, ) -broker_for_integration_suites() +broker_for_integration_suites( + extra_plugins = [ + "//deps/rabbitmq_management:erlang_app", + ], +) rabbitmq_test_helper( name = "rabbit_auth_backend_mqtt_mock", @@ -158,7 +162,6 @@ rabbitmq_integration_suite( rabbitmq_integration_suite( name = "config_SUITE", - size = "small", ) rabbitmq_integration_suite( @@ -227,6 +230,7 @@ rabbitmq_integration_suite( size = "large", runtime_deps = [ "@emqtt//:erlang_app", + "//deps/rabbitmq_management_agent:erlang_app", ], additional_beam = [ ":event_recorder", diff --git a/deps/rabbitmq_mqtt/Makefile b/deps/rabbitmq_mqtt/Makefile index e13eb95e345a..179a5ab709ba 100644 --- a/deps/rabbitmq_mqtt/Makefile +++ b/deps/rabbitmq_mqtt/Makefile @@ -40,7 +40,7 @@ BUILD_WITHOUT_QUIC=1 export BUILD_WITHOUT_QUIC DEPS = ranch rabbit_common rabbit amqp_client ra -TEST_DEPS = emqtt ct_helper rabbitmq_ct_helpers rabbitmq_ct_client_helpers +TEST_DEPS = emqtt ct_helper rabbitmq_ct_helpers rabbitmq_ct_client_helpers rabbitmq_management dep_ct_helper = git https://github.com/extend/ct_helper.git master dep_emqtt = git https://github.com/emqx/emqtt.git 1.7.0-rc.2 diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl index 577d36734e25..c845240e55d3 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl @@ -12,24 +12,30 @@ -define(MQTT_GUIDE_URL, <<"https://rabbitmq.com/mqtt.html">>). -define(MQTT_PROTO_V3, mqtt310). -define(MQTT_PROTO_V4, mqtt311). --define(INFO_ITEMS, - [protocol, + +-define(ITEMS, + [pid, + protocol, host, port, peer_host, peer_port, - connection, - conn_name, - connection_state, ssl, ssl_protocol, ssl_key_exchange, ssl_cipher, ssl_hash, - ssl_login_name, - client_id, vhost, - user, + user + ]). + +-define(INFO_ITEMS, + ?ITEMS ++ + [ + client_id, + conn_name, + connection_state, + ssl_login_name, recv_cnt, recv_oct, send_cnt, @@ -43,3 +49,31 @@ messages_unconfirmed, messages_unacknowledged ]). + +-define(CREATION_EVENT_KEYS, + ?ITEMS ++ + [name, + client_properties, + peer_cert_issuer, + peer_cert_subject, + peer_cert_validity, + auth_mechanism, + timeout, + frame_max, + channel_max, + connected_at, + node, + user_who_performed_action + ]). + +-define(SIMPLE_METRICS, + [pid, + recv_oct, + send_oct, + reductions]). +-define(OTHER_METRICS, + [recv_cnt, + send_cnt, + send_pend, + garbage_collection, + state]). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_ff.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_ff.erl index 892fe8fec7a0..2ebde6e4d1ef 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_ff.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_ff.erl @@ -13,7 +13,7 @@ -rabbit_feature_flag( {?QUEUE_TYPE_QOS_0, - #{desc => "Support pseudo queue type for MQTT QoS 0 omitting a queue process", + #{desc => "Support pseudo queue type for MQTT QoS 0 subscribers omitting a queue process", stability => stable }}). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_keepalive.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_keepalive.erl index 696d6517e4f4..348501640102 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_keepalive.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_keepalive.erl @@ -3,17 +3,20 @@ -export([start/2, handle/2, start_timer/1, - cancel_timer/1]). + cancel_timer/1, + interval_secs/1]). -export_type([state/0]). -type option(T) :: undefined | T. --record(state, {timer :: option(reference()), - interval_ms :: pos_integer(), - socket :: inet:socket(), - recv_oct :: non_neg_integer(), - received :: boolean()}). +-record(state, { + %% Keep Alive value as sent in the CONNECT packet. + interval_secs :: pos_integer(), + timer :: option(reference()), + socket :: inet:socket(), + recv_oct :: non_neg_integer(), + received :: boolean()}). -opaque(state() :: undefined | #state{}). @@ -30,28 +33,8 @@ start(Seconds, Sock) handle({init, IntervalSecs, Sock}, _State) -> case rabbit_net:getstat(Sock, [recv_oct]) of {ok, [{recv_oct, RecvOct}]} -> - %% "If the Keep Alive value is non-zero and the Server does not receive a Control - %% Packet from the Client within one and a half times the Keep Alive time period, - %% it MUST disconnect the Network Connection to the Client as if the network had - %% failed" [MQTT-3.1.2-24] - %% - %% We check every (1.5 / 2 = 0.75) * KeepaliveInterval whether we received - %% any data from the client. If there was no activity for two consecutive times, - %% we close the connection. - %% We choose 0.75 (instead of a larger or smaller factor) to have the right balance - %% between not checking too often (since it could become expensive when there are - %% millions of clients) and not checking too rarely (to detect dead clients promptly). - %% - %% See https://github.com/emqx/emqx/issues/460 - %% PING - %% | DOWN - %% | |<-------Delay Time---------> - %% t0---->|----------|----------|----------|---->tn - %% | | | - %% Ok Retry Timeout - IntervalMs = round(0.75 * timer:seconds(IntervalSecs)), State = #state{socket = Sock, - interval_ms = IntervalMs, + interval_secs = IntervalSecs, recv_oct = RecvOct, received = true}, {ok, start_timer(State)}; @@ -78,11 +61,11 @@ handle(check, State = #state{socket = Sock, end. -spec start_timer(state()) -> state(). +start_timer(#state{interval_secs = Seconds} = State) -> + Ref = erlang:send_after(timer_ms(Seconds), self(), {keepalive, check}), + State#state{timer = Ref}; start_timer(undefined) -> - undefined; -start_timer(#state{interval_ms = IntervalMs} = State) -> - Ref = erlang:send_after(IntervalMs, self(), {keepalive, check}), - State#state{timer = Ref}. + undefined. -spec cancel_timer(state()) -> state(). cancel_timer(undefined) -> @@ -92,3 +75,34 @@ cancel_timer(#state{timer = Ref} = State) ok = erlang:cancel_timer(Ref, [{async, true}, {info, false}]), State. + +%% "If the Keep Alive value is non-zero and the Server does not receive a Control +%% Packet from the Client within one and a half times the Keep Alive time period, +%% it MUST disconnect the Network Connection to the Client as if the network had +%% failed" [MQTT-3.1.2-24] +%% +%% We check every (1.5 / 2 = 0.75) * KeepaliveInterval whether we received +%% any data from the client. If there was no activity for two consecutive times, +%% we close the connection. +%% We choose 0.75 (instead of a larger or smaller factor) to have the right balance +%% between not checking too often (since it could become expensive when there are +%% millions of clients) and not checking too rarely (to detect dead clients promptly). +%% +%% See https://github.com/emqx/emqx/issues/460 +%% PING +%% | DOWN +%% | |<-------Delay Time---------> +%% t0---->|----------|----------|----------|---->tn +%% | | | +%% Ok Retry Timeout +-spec timer_ms(pos_integer()) -> + pos_integer(). +timer_ms(KeepaliveSeconds) -> + round(0.75 * timer:seconds(KeepaliveSeconds)). + +-spec interval_secs(state()) -> + non_neg_integer(). +interval_secs(#state{interval_secs = Seconds}) -> + Seconds; +interval_secs(undefined) -> + 0. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 86ebf17d7540..fb157c5befaa 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -5,13 +5,15 @@ %% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved. %% +%% This module contains code that is common to MQTT and Web MQTT connections. -module(rabbit_mqtt_processor). -export([info/2, initial_state/2, initial_state/4, process_packet/2, serialise/2, - terminate/3, handle_pre_hibernate/0, + terminate/4, handle_pre_hibernate/0, handle_ra_event/2, handle_down/2, handle_queue_event/2, - soft_limit_exceeded/1, format_status/1]). + soft_limit_exceeded/1, proto_version_tuple/1, + format_status/1]). %% for testing purposes -export([get_vhost_username/1, get_vhost/3, get_vhost_from_user_mapping/2]). @@ -40,7 +42,7 @@ port, peer_host, peer_port, - proto_human}). + connected_at}). -record(state, {socket, @@ -335,8 +337,8 @@ process_connect(#mqtt_packet{ fun check_credentials/2, fun login/2, fun register_client/2, - fun notify_connection_created/2, fun start_keepalive/2, + fun notify_connection_created/1, fun handle_clean_session/2], PacketConnect, State0) of {ok, SessionPresent0, State1} -> @@ -422,19 +424,18 @@ register_client(Packet = #mqtt_packet_connect{proto_ver = ProtoVersion}, {ok, {PeerHost, PeerPort, Host, Port}} = rabbit_net:socket_ends(Socket, inbound), ExchangeBin = rabbit_mqtt_util:env(exchange), ExchangeName = rabbit_misc:r(VHost, exchange, ExchangeBin), - ProtoHumanReadable = {'MQTT', human_readable_mqtt_version(ProtoVersion)}, State#state{ exchange = ExchangeName, will_msg = make_will_msg(Packet), retainer_pid = RetainerPid, register_state = RegisterState, - proto_ver = protocol_integer_to_atom(ProtoVersion), + proto_ver = proto_integer_to_atom(ProtoVersion), info = #info{prefetch = Prefetch, peer_host = PeerHost, peer_port = PeerPort, host = Host, port = Port, - proto_human = ProtoHumanReadable + connected_at = os:system_time(milli_seconds) }} end, case rabbit_mqtt_ff:track_client_id_in_ra() of @@ -452,41 +453,12 @@ register_client(Packet = #mqtt_packet_connect{proto_ver = ProtoVersion}, {ok, NewProcState(undefined)} end. -notify_connection_created(already_connected, _State) -> +notify_connection_created(already_connected) -> ok; -notify_connection_created( - _Packet, - #state{socket = Sock, - conn_name = ConnName, - info = #info{proto_human = {ProtoName, ProtoVsn}}, - auth_state = #auth_state{vhost = VHost, - username = Username}} = State) -> - {ok, {PeerHost, PeerPort, Host, Port}} = rabbit_net:socket_ends(Sock, inbound), - ConnectedAt = os:system_time(milli_seconds), - Infos = [{host, Host}, - {port, Port}, - {peer_host, PeerHost}, - {peer_port, PeerPort}, - {vhost, VHost}, - {node, node()}, - {user, Username}, - {name, ConnName}, - {connected_at, ConnectedAt}, - {pid, self()}, - {protocol, {ProtoName, binary_to_list(ProtoVsn)}}, - {type, network} - ], - rabbit_core_metrics:connection_created(self(), Infos), - rabbit_event:notify(connection_created, Infos), +notify_connection_created(#mqtt_packet_connect{}) -> rabbit_networking:register_non_amqp_connection(self()), - {ok, State#state{ - %% We won't need conn_name anymore. Use less memmory by setting to undefined. - conn_name = undefined}}. - -human_readable_mqtt_version(3) -> - <<"3.1.0">>; -human_readable_mqtt_version(4) -> - <<"3.1.1">>. + self() ! connection_created, + ok. return_connack(?CONNACK_ACCEPT, S) -> {ok, S}; @@ -781,7 +753,7 @@ check_user_login(#{vhost := VHost, username_bin := UsernameBin, pass_bin := PassBin, client_id := ClientId - } = In, State) -> + } = In, State0) -> AuthProps = case PassBin of none -> %% SSL user name provided. @@ -795,7 +767,11 @@ check_user_login(#{vhost := VHost, case rabbit_access_control:check_user_login( UsernameBin, AuthProps) of {ok, User = #user{username = Username}} -> - notify_auth_result(user_authentication_success, Username, State), + notify_auth_result(user_authentication_success, Username, State0), + State = State0#state{ + %% We won't need conn_name anymore. + %% Use less memmory by setting to undefined. + conn_name = undefined}, {ok, maps:put(user, User, In), State}; {refused, Username, Msg, Args} -> rabbit_log_connection:error( @@ -803,7 +779,7 @@ check_user_login(#{vhost := VHost, "access refused for user '~s' in vhost '~s' " ++ Msg, [self(), Username, VHost] ++ Args), - notify_auth_result(user_authentication_failure, Username, State), + notify_auth_result(user_authentication_failure, Username, State0), {error, ?CONNACK_BAD_CREDENTIALS} end. @@ -1336,13 +1312,14 @@ serialise_and_send_to_client(Packet, #state{proto_ver = ProtoVer, socket = Sock} serialise(Packet, #state{proto_ver = ProtoVer}) -> rabbit_mqtt_packet:serialise(Packet, ProtoVer). -terminate(SendWill, ConnName, State) -> +terminate(SendWill, ConnName, ProtoFamily, State) -> maybe_send_will(SendWill, ConnName, State), Infos = [{name, ConnName}, {node, node()}, {pid, self()}, {disconnected_at, os:system_time(milli_seconds)} - ] ++ additional_connection_closed_info(State), + ] ++ additional_connection_closed_info(ProtoFamily, State), + rabbit_core_metrics:connection_closed(self()), rabbit_event:notify(connection_closed, Infos), rabbit_networking:unregister_non_amqp_connection(self()), maybe_unregister_client(State), @@ -1373,13 +1350,13 @@ maybe_send_will(_, _, _) -> ok. additional_connection_closed_info( - #state{info = #info{proto_human = {ProtoName, ProtoVsn}}, - auth_state = #auth_state{vhost = VHost, - username = Username}}) -> - [{protocol, {ProtoName, binary_to_list(ProtoVsn)}}, + ProtoFamily, + State = #state{auth_state = #auth_state{vhost = VHost, + username = Username}}) -> + [{protocol, {ProtoFamily, proto_version_tuple(State)}}, {vhost, VHost}, {user, Username}]; -additional_connection_closed_info(_) -> +additional_connection_closed_info(_, _) -> []. maybe_unregister_client(#state{client_id = ClientId}) @@ -1774,15 +1751,15 @@ is_socket_busy(Socket) -> false end. -info(protocol, #state{info = #info{proto_human = Val}}) -> Val; info(host, #state{info = #info{host = Val}}) -> Val; info(port, #state{info = #info{port = Val}}) -> Val; info(peer_host, #state{info = #info{peer_host = Val}}) -> Val; info(peer_port, #state{info = #info{peer_port = Val}}) -> Val; +info(connected_at, #state{info = #info{connected_at = Val}}) -> Val; info(ssl_login_name, #state{ssl_login_name = Val}) -> Val; -info(client_id, #state{client_id = Val}) -> - rabbit_data_coercion:to_binary(Val); info(vhost, #state{auth_state = #auth_state{vhost = Val}}) -> Val; +info(user_who_performed_action, S) -> + info(user, S); info(user, #state{auth_state = #auth_state{username = Val}}) -> Val; info(clean_sess, #state{clean_sess = Val}) -> Val; info(will_msg, #state{will_msg = Val}) -> Val; @@ -1793,6 +1770,16 @@ info(messages_unconfirmed, #state{unacked_client_pubs = Val}) -> rabbit_mqtt_confirms:size(Val); info(messages_unacknowledged, #state{unacked_server_pubs = Val}) -> maps:size(Val); +info(node, _) -> node(); +info(client_id, #state{client_id = Val}) -> Val; +%% for rabbitmq_management/priv/www/js/tmpl/connection.ejs +info(client_properties, #state{client_id = Val}) -> + [{client_id, longstr, Val}]; +info(channel_max, _) -> 0; +%% Maximum packet size supported only in MQTT 5.0. +info(frame_max, _) -> 0; +%% SASL supported only in MQTT 5.0. +info(auth_mechanism, _) -> none; info(Other, _) -> throw({bad_argument, Other}). -spec ssl_login_name(rabbit_net:socket()) -> @@ -1848,11 +1835,16 @@ format_status(#state{queue_states = QState, soft_limit_exceeded(#state{soft_limit_exceeded = SLE}) -> not sets:is_empty(SLE). -protocol_integer_to_atom(3) -> +proto_integer_to_atom(3) -> ?MQTT_PROTO_V3; -protocol_integer_to_atom(4) -> +proto_integer_to_atom(4) -> ?MQTT_PROTO_V4. +proto_version_tuple(#state{proto_ver = ?MQTT_PROTO_V3}) -> + {3, 1, 0}; +proto_version_tuple(#state{proto_ver = ?MQTT_PROTO_V4}) -> + {3, 1, 1}. + maybe_increment_publisher(State = #state{has_published = false, proto_ver = ProtoVer}) -> rabbit_global_counters:publisher_created(ProtoVer), diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index bf6fe8a13649..9229f4d7c37b 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -21,9 +21,8 @@ -include("rabbit_mqtt.hrl"). --define(SIMPLE_METRICS, [pid, recv_oct, send_oct, reductions]). --define(OTHER_METRICS, [recv_cnt, send_cnt, send_pend, garbage_collection, state]). -define(HIBERNATE_AFTER, 1000). +-define(PROTO_FAMILY, 'MQTT'). -record(state, {socket, @@ -49,11 +48,10 @@ conserve_resources(Pid, _, {_, Conserve, _}) -> Pid ! {conserve_resources, Conserve}, ok. -info(Pid, InfoItems) -> - case InfoItems -- ?INFO_ITEMS of - [] -> gen_server:call(Pid, {info, InfoItems}); - UnknownItems -> throw({bad_argument, UnknownItems}) - end. +-spec info(pid(), rabbit_types:info_keys()) -> + rabbit_types:infos(). +info(Pid, Items) -> + gen_server:call(Pid, {info, Items}). close_connection(Pid, Reason) -> gen_server:cast(Pid, {close_connection, Reason}). @@ -74,19 +72,18 @@ init(Ref) -> LoginTimeout = application:get_env(?APP_NAME, login_timeout, 10_000), erlang:send_after(LoginTimeout, self(), login_timeout), ProcessorState = rabbit_mqtt_processor:initial_state(RealSocket, ConnName), - gen_server:enter_loop(?MODULE, [], - rabbit_event:init_stats_timer( - control_throttle( - #state{socket = RealSocket, - proxy_socket = rabbit_net:maybe_get_proxy_socket(Sock), - conn_name = ConnName, - await_recv = false, - connection_state = running, - received_connect_packet = false, - conserve = false, - parse_state = rabbit_mqtt_packet:initial_state(), - proc_state = ProcessorState }), #state.stats_timer) - ); + State0 = #state{socket = RealSocket, + proxy_socket = rabbit_net:maybe_get_proxy_socket(Sock), + conn_name = ConnName, + await_recv = false, + connection_state = running, + received_connect_packet = false, + conserve = false, + parse_state = rabbit_mqtt_packet:initial_state(), + proc_state = ProcessorState}, + State1 = control_throttle(State0), + State = rabbit_event:init_stats_timer(State1, #state.stats_timer), + gen_server:enter_loop(?MODULE, [], State); {error, enotconn} -> rabbit_net:fast_close(RealSocket), terminate(shutdown, undefined); @@ -96,11 +93,7 @@ init(Ref) -> end. handle_call({info, InfoItems}, _From, State) -> - Infos = lists:map( - fun(InfoItem) -> - {InfoItem, info_internal(InfoItem, State)} - end, InfoItems), - {reply, Infos, State, ?HIBERNATE_AFTER}; + {reply, infos(InfoItems, State), State, ?HIBERNATE_AFTER}; handle_call(Msg, From, State) -> {stop, {mqtt_unexpected_call, Msg, From}, State}. @@ -135,9 +128,21 @@ handle_cast(QueueEvent = {queue_event, _, _}, {stop, Reason, pstate(State, PState)} end; +handle_cast({force_event_refresh, Ref}, State0) -> + Infos = infos(?CREATION_EVENT_KEYS, State0), + rabbit_event:notify(connection_created, Infos, Ref), + State = rabbit_event:init_stats_timer(State0, #state.stats_timer), + {noreply, State, ?HIBERNATE_AFTER}; + handle_cast(Msg, State) -> {stop, {mqtt_unexpected_cast, Msg}, State}. +handle_info(connection_created, State) -> + Infos = infos(?CREATION_EVENT_KEYS, State), + rabbit_core_metrics:connection_created(self(), Infos), + rabbit_event:notify(connection_created, Infos), + {noreply, State, ?HIBERNATE_AFTER}; + handle_info(timeout, State) -> rabbit_mqtt_processor:handle_pre_hibernate(), {noreply, State, hibernate}; @@ -224,6 +229,11 @@ handle_info({'DOWN', _MRef, process, QPid, _Reason}, State) -> rabbit_amqqueue_common:notify_sent_queue_down(QPid), {noreply, State, ?HIBERNATE_AFTER}; +handle_info({shutdown, Explanation} = Reason, State = #state{conn_name = ConnName}) -> + %% rabbitmq_management plugin requests to close connection. + rabbit_log_connection:info("MQTT closing connection ~tp: ~p", [ConnName, Explanation]), + {stop, Reason, State}; + handle_info(Msg, State) -> {stop, {mqtt_unexpected_msg, Msg}, State}. @@ -234,7 +244,7 @@ terminate(Reason, {SendWill, State = #state{conn_name = ConnName, proc_state = PState}}) -> KState = rabbit_mqtt_keepalive:cancel_timer(KState0), maybe_emit_stats(State#state{keepalive = KState}), - rabbit_mqtt_processor:terminate(SendWill, ConnName, PState), + rabbit_mqtt_processor:terminate(SendWill, ConnName, ?PROTO_FAMILY, PState), log_terminate(Reason, State). log_terminate({network_error, {ssl_upgrade_error, closed}, ConnName}, _State) -> @@ -430,73 +440,77 @@ emit_stats(State=#state{received_connect_packet = false}) -> ensure_stats_timer(State1); emit_stats(State) -> [{_, Pid}, - {_, Recv_oct}, - {_, Send_oct}, + {_, RecvOct}, + {_, SendOct}, {_, Reductions}] = infos(?SIMPLE_METRICS, State), Infos = infos(?OTHER_METRICS, State), rabbit_core_metrics:connection_stats(Pid, Infos), - rabbit_core_metrics:connection_stats(Pid, Recv_oct, Send_oct, Reductions), + rabbit_core_metrics:connection_stats(Pid, RecvOct, SendOct, Reductions), State1 = rabbit_event:reset_stats_timer(State, #state.stats_timer), ensure_stats_timer(State1). ensure_stats_timer(State = #state{}) -> rabbit_event:ensure_stats_timer(State, #state.stats_timer, emit_stats). -infos(Items, State) -> [{Item, info_internal(Item, State)} || Item <- Items]. +infos(Items, State) -> + [{Item, i(Item, State)} || Item <- Items]. -info_internal(pid, State) -> info_internal(connection, State); -info_internal(SockStat, #state{socket = Sock}) when SockStat =:= recv_oct; - SockStat =:= recv_cnt; - SockStat =:= send_oct; - SockStat =:= send_cnt; - SockStat =:= send_pend -> +i(SockStat, #state{socket = Sock}) + when SockStat =:= recv_oct; + SockStat =:= recv_cnt; + SockStat =:= send_oct; + SockStat =:= send_cnt; + SockStat =:= send_pend -> case rabbit_net:getstat(Sock, [SockStat]) of - {ok, [{_, N}]} when is_number(N) -> N; - _ -> 0 + {ok, [{_, N}]} when is_number(N) -> + N; + _ -> + 0 end; -info_internal(state, State) -> info_internal(connection_state, State); -info_internal(garbage_collection, _State) -> +i(state, S) -> + i(connection_state, S); +i(garbage_collection, _) -> rabbit_misc:get_gc_info(self()); -info_internal(reductions, _State) -> +i(reductions, _) -> {reductions, Reductions} = erlang:process_info(self(), reductions), Reductions; -info_internal(conn_name, #state{conn_name = Val}) -> - rabbit_data_coercion:to_binary(Val); -info_internal(connection_state, #state{received_connect_packet = false}) -> +i(name, S) -> + i(conn_name, S); +i(conn_name, #state{conn_name = Val}) -> + Val; +i(connection_state, #state{received_connect_packet = false}) -> starting; -info_internal(connection_state, #state{connection_state = Val}) -> +i(connection_state, #state{connection_state = Val}) -> Val; -info_internal(connection, _State) -> +i(pid, _) -> self(); -info_internal(ssl, #state{socket = Sock, proxy_socket = ProxySock}) -> - rabbit_net:proxy_ssl_info(Sock, ProxySock) /= nossl; -info_internal(ssl_protocol, S) -> ssl_info(fun ({P, _}) -> P end, S); -info_internal(ssl_key_exchange, S) -> ssl_info(fun ({_, {K, _, _}}) -> K end, S); -info_internal(ssl_cipher, S) -> ssl_info(fun ({_, {_, C, _}}) -> C end, S); -info_internal(ssl_hash, S) -> ssl_info(fun ({_, {_, _, H}}) -> H end, S); -info_internal(Key, #state{proc_state = ProcState}) -> +i(SSL, #state{socket = Sock, proxy_socket = ProxySock}) + when SSL =:= ssl; + SSL =:= ssl_protocol; + SSL =:= ssl_key_exchange; + SSL =:= ssl_cipher; + SSL =:= ssl_hash -> + rabbit_ssl:info(SSL, {Sock, ProxySock}); +i(Cert, #state{socket = Sock}) + when Cert =:= peer_cert_issuer; + Cert =:= peer_cert_subject; + Cert =:= peer_cert_validity -> + rabbit_ssl:cert_info(Cert, Sock); +i(timeout, #state{keepalive = KState}) -> + rabbit_mqtt_keepalive:interval_secs(KState); +i(protocol, #state{proc_state = ProcState}) -> + {?PROTO_FAMILY, rabbit_mqtt_processor:proto_version_tuple(ProcState)}; +i(Key, #state{proc_state = ProcState}) -> rabbit_mqtt_processor:info(Key, ProcState). -ssl_info(F, #state{socket = Sock, proxy_socket = ProxySock}) -> - case rabbit_net:proxy_ssl_info(Sock, ProxySock) of - nossl -> ''; - {error, _} -> ''; - {ok, Items} -> - P = proplists:get_value(protocol, Items), - #{cipher := C, - key_exchange := K, - mac := H} = proplists:get_value(selected_cipher_suite, Items), - F({P, {K, C, H}}) - end. - -spec format_status(map()) -> map(). format_status(Status) -> - maps:map( - fun(state,State) -> - format_state(State); - (_,Value) -> - Value - end, Status). + maps:map( + fun(state, State) -> + format_state(State); + (_, Value) -> + Value + end, Status). format_state(#state{proc_state = PState, socket = Socket, diff --git a/deps/rabbitmq_mqtt/test/command_SUITE.erl b/deps/rabbitmq_mqtt/test/command_SUITE.erl index cae8df7e4233..76421287d312 100644 --- a/deps/rabbitmq_mqtt/test/command_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/command_SUITE.erl @@ -109,31 +109,20 @@ run(Config) -> timer:sleep(200), %% Still two MQTT connections - [[{client_id, <<"simpleClient">>}], - [{client_id, <<"simpleClient1">>}]] = - lists:sort('Elixir.Enum':to_list(?COMMAND:run([<<"client_id">>], Opts))), + ?assertEqual( + [[{client_id, <<"simpleClient">>}], + [{client_id, <<"simpleClient1">>}]], + lists:sort('Elixir.Enum':to_list(?COMMAND:run([<<"client_id">>], Opts)))), %% Verbose returns all keys - Infos = lists:map(fun(El) -> atom_to_binary(El, utf8) end, ?INFO_ITEMS), - AllKeys1 = 'Elixir.Enum':to_list(?COMMAND:run(Infos, Opts)), - AllKeys2 = 'Elixir.Enum':to_list(?COMMAND:run([], Opts#{verbose => true})), - - %% There are two connections - [FirstPL, _] = AllKeys1, - [SecondPL, _] = AllKeys2, - - First = maps:from_list(lists:usort(FirstPL)), - Second = maps:from_list(lists:usort(SecondPL)), + AllKeys = lists:map(fun(I) -> atom_to_binary(I) end, ?INFO_ITEMS), + [AllInfos1Con1, _AllInfos1Con2] = 'Elixir.Enum':to_list(?COMMAND:run(AllKeys, Opts)), + [AllInfos2Con1, _AllInfos2Con2] = 'Elixir.Enum':to_list(?COMMAND:run([], Opts#{verbose => true})), %% Keys are INFO_ITEMS - KeysCount = length(?INFO_ITEMS), - ?assert(KeysCount =:= maps:size(First)), - ?assert(KeysCount =:= maps:size(Second)), - - Keys = maps:keys(First), - - [] = Keys -- ?INFO_ITEMS, - [] = ?INFO_ITEMS -- Keys, + InfoItemsSorted = lists:sort(?INFO_ITEMS), + ?assertEqual(InfoItemsSorted, lists:sort(proplists:get_keys(AllInfos1Con1))), + ?assertEqual(InfoItemsSorted, lists:sort(proplists:get_keys(AllInfos2Con1))), ok = emqtt:disconnect(C1), ok = emqtt:disconnect(C2). diff --git a/deps/rabbitmq_mqtt/test/config_SUITE.erl b/deps/rabbitmq_mqtt/test/config_SUITE.erl index edcfc4934190..0761d3e49a96 100644 --- a/deps/rabbitmq_mqtt/test/config_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/config_SUITE.erl @@ -28,7 +28,7 @@ groups() -> ]. suite() -> - [{timetrap, {seconds, 30}}]. + [{timetrap, {minutes, 2}}]. %% ------------------------------------------------------------------- %% Testsuite setup/teardown. diff --git a/deps/rabbitmq_mqtt/test/event_recorder.erl b/deps/rabbitmq_mqtt/test/event_recorder.erl index a9e219f476f1..592f13454d19 100644 --- a/deps/rabbitmq_mqtt/test/event_recorder.erl +++ b/deps/rabbitmq_mqtt/test/event_recorder.erl @@ -10,9 +10,15 @@ -export([init/1, handle_event/2, handle_call/2]). -define(INIT_STATE, []). +-include_lib("rabbit_common/include/rabbit.hrl"). + init(_) -> {ok, ?INIT_STATE}. +handle_event(#event{type = node_stats}, State) -> + {ok, State}; +handle_event(#event{type = node_node_stats}, State) -> + {ok, State}; handle_event(Event, State) -> {ok, [Event | State]}. diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/integration_SUITE.erl index 6e3ddbda5dfd..1c17a1090f05 100644 --- a/deps/rabbitmq_mqtt/test/integration_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/integration_SUITE.erl @@ -11,19 +11,31 @@ -include_lib("eunit/include/eunit.hrl"). -include_lib("amqp_client/include/amqp_client.hrl"). -include_lib("rabbitmq_ct_helpers/include/rabbit_assert.hrl"). - --import(rabbit_ct_broker_helpers, [rabbitmqctl_list/3, - rpc/5, - rpc_all/4]). --import(rabbit_ct_helpers, [eventually/3, - eventually/1]). --import(util, [all_connection_pids/1, - get_global_counters/2, - get_global_counters/3, - get_global_counters/4, - expect_publishes/2, - connect/2, - connect/3]). +-include_lib("rabbitmq_ct_helpers/include/rabbit_mgmt_test.hrl"). + +-import(rabbit_ct_broker_helpers, + [rabbitmqctl_list/3, + rpc/5, + rpc_all/4, + get_node_config/3]). +-import(rabbit_ct_helpers, + [eventually/3, + eventually/1]). +-import(util, + [all_connection_pids/1, + get_global_counters/2, + get_global_counters/3, + get_global_counters/4, + expect_publishes/2, + connect/2, + connect/3]). +-import(rabbit_mgmt_test_util, + [http_get/2, + http_delete/3]). + +-define(MANAGEMENT_PLUGIN_TESTS, + [management_plugin_connection, + management_plugin_enable]). all() -> [ @@ -46,11 +58,13 @@ groups() -> flow_classic_mirrored_queue, flow_quorum_queue, flow_stream, - rabbit_mqtt_qos0_queue] ++ tests() + rabbit_mqtt_qos0_queue + ] ++ tests() } ]. tests() -> + ?MANAGEMENT_PLUGIN_TESTS ++ [delete_create_queue ,quorum_queue_rejects ,publish_to_all_queue_types_qos0 @@ -114,11 +128,30 @@ end_per_group(_, Config) -> rabbit_ct_broker_helpers:teardown_steps()). init_per_testcase(Testcase, Config) -> + maybe_start_inets(Testcase), rabbit_ct_helpers:testcase_started(Config, Testcase). end_per_testcase(Testcase, Config) -> + maybe_stop_inets(Testcase), + rabbit_ct_client_helpers:close_channels_and_connection(Config, 0), rabbit_ct_helpers:testcase_finished(Config, Testcase). +maybe_start_inets(Testcase) -> + case lists:member(Testcase, ?MANAGEMENT_PLUGIN_TESTS) of + true -> + ok = inets:start(); + false -> + ok + end. + +maybe_stop_inets(Testcase) -> + case lists:member(Testcase, ?MANAGEMENT_PLUGIN_TESTS) of + true -> + ok = inets:stop(); + false -> + ok + end. + %% ------------------------------------------------------------------- %% Testsuite cases %% ------------------------------------------------------------------- @@ -215,8 +248,7 @@ publish_to_all_queue_types(Config, QoS) -> ok = rabbit_ct_broker_helpers:clear_policy(Config, 0, CMQ), ok = emqtt:disconnect(C), ?awaitMatch([], - all_connection_pids(Config), 10_000, 1000), - ok = rabbit_ct_client_helpers:close_channel(Ch). + all_connection_pids(Config), 10_000, 1000). flow_classic_mirrored_queue(Config) -> QueueName = <<"flow">>, @@ -263,13 +295,12 @@ flow(Config, {App, Par, Val}, QueueType) ok = emqtt:disconnect(C), ?awaitMatch([], all_connection_pids(Config), 10_000, 1000), - ok = rabbit_ct_client_helpers:close_channel(Ch), Result = rpc_all(Config, application, set_env, [App, Par, DefaultVal]), ok. events(Config) -> ok = rabbit_ct_broker_helpers:add_code_path_to_all_nodes(Config, event_recorder), - Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + Server = get_node_config(Config, 0, nodename), ok = gen_event:add_handler({rabbit_event, Server}, event_recorder, []), ClientId = atom_to_binary(?FUNCTION_NAME), @@ -282,7 +313,7 @@ events(Config) -> E0), assert_event_type(connection_created, E1), [ConnectionPid] = all_connection_pids(Config), - ExpectedConnectionProps = [{protocol, {'MQTT', "3.1.1"}}, + ExpectedConnectionProps = [{protocol, {'MQTT', {3,1,1}}}, {node, Server}, {vhost, <<"/">>}, {user, <<"guest">>}, @@ -351,7 +382,7 @@ events(Config) -> ok = gen_event:delete_handler({rabbit_event, Server}, event_recorder, []). event_authentication_failure(Config) -> - P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), + P = get_node_config(Config, 0, tcp_port_mqtt), ClientId = atom_to_binary(?FUNCTION_NAME), {ok, C} = emqtt:start_link([{username, <<"Trudy">>}, {password, <<"fake-password">>}, @@ -362,7 +393,7 @@ event_authentication_failure(Config) -> true = unlink(C), ok = rabbit_ct_broker_helpers:add_code_path_to_all_nodes(Config, event_recorder), - Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + Server = get_node_config(Config, 0, nodename), ok = gen_event:add_handler({rabbit_event, Server}, event_recorder, []), ?assertMatch({error, _}, emqtt:connect(C)), @@ -376,7 +407,7 @@ event_authentication_failure(Config) -> ok = gen_event:delete_handler({rabbit_event, Server}, event_recorder, []). internal_event_handler(Config) -> - Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + Server = get_node_config(Config, 0, nodename), ok = gen_event:call({rabbit_event, Server}, rabbit_mqtt_internal_event_handler, ignored_request, 1000). global_counters_v3(Config) -> @@ -463,11 +494,11 @@ global_counters(Config, ProtoVer) -> get_global_counters(Config, ProtoVer)). queue_down_qos1(Config) -> - Ch1 = rabbit_ct_client_helpers:open_channel(Config, 1), + {Conn1, Ch1} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 1), CQ = Topic = atom_to_binary(?FUNCTION_NAME), declare_queue(Ch1, CQ, []), bind(Ch1, CQ, Topic), - ok = rabbit_ct_client_helpers:close_channel(Ch1), + ok = rabbit_ct_client_helpers:close_connection_and_channel(Conn1, Ch1), ok = rabbit_ct_broker_helpers:stop_node(Config, 1), C = connect(?FUNCTION_NAME, Config, [{retry_interval, 2}]), @@ -483,7 +514,6 @@ queue_down_qos1(Config) -> Ch0 = rabbit_ct_client_helpers:open_channel(Config, 0), delete_queue(Ch0, CQ), - ok = rabbit_ct_client_helpers:close_channel(Ch0), ok = emqtt:disconnect(C). %% Even though classic mirrored queues are deprecated, we know that some users have set up @@ -505,7 +535,7 @@ consuming_classic_mirrored_queue_down(Config) -> %% Consume from Server2. Options = [{host, "localhost"}, - {port, rabbit_ct_broker_helpers:get_node_config(Config, Server2, tcp_port_mqtt)}, + {port, get_node_config(Config, Server2, tcp_port_mqtt)}, {clientid, atom_to_binary(?FUNCTION_NAME)}, {proto_ver, v4}], {ok, C2} = emqtt:start_link([{clean_start, false} | Options]), @@ -549,7 +579,7 @@ consuming_classic_queue_down(Config) -> ProtoVer = v4, %% Consume from Server3. Options = [{host, "localhost"}, - {port, rabbit_ct_broker_helpers:get_node_config(Config, Server3, tcp_port_mqtt)}, + {port, get_node_config(Config, Server3, tcp_port_mqtt)}, {clientid, ClientId}, {proto_ver, ProtoVer}], {ok, C2} = emqtt:start_link([{clean_start, false} | Options]), @@ -565,7 +595,8 @@ consuming_classic_queue_down(Config) -> %% When the dedicated MQTT connection (non-mirrored classic) queue goes down, it is reasonable %% that the server closes the MQTT connection because the MQTT client cannot consume anymore. eventually(?_assertMatch(#{consumers := 0}, - get_global_counters(Config, ProtoVer, Server3))), + get_global_counters(Config, ProtoVer, Server3)), + 1000, 5), receive {'EXIT', C2, _} -> ok @@ -641,7 +672,6 @@ delete_create_queue(Config) -> 1000, 10), delete_queue(Ch, [CQ1, CQ2, QQ]), - ok = rabbit_ct_client_helpers:close_channel(Ch), ok = emqtt:disconnect(C). non_clean_sess_disconnect(Config) -> @@ -755,7 +785,6 @@ large_message_amqp_to_mqtt(Config) -> #amqp_msg{payload = Payload}), ok = expect_publishes(Topic, [Payload]), - ok = rabbit_ct_client_helpers:close_channel(Ch), ok = emqtt:disconnect(C). %% This test is mostly interesting in mixed version mode where feature flag @@ -769,7 +798,7 @@ rabbit_mqtt_qos0_queue(Config) -> %% Place MQTT publisher process on old node in mixed version. {ok, Pub} = emqtt:start_link( - [{port, rabbit_ct_broker_helpers:get_node_config(Config, 1, tcp_port_mqtt)}, + [{port, get_node_config(Config, 1, tcp_port_mqtt)}, {clientid, <<"publisher">>}, {proto_ver, v4} ]), @@ -789,9 +818,9 @@ rabbit_mqtt_qos0_queue_overflow(Config) -> Msg = binary:copy(<<"x">>, 1000), NumMsgs = 10_000, - %% Provoke TCP back-pressure from client to server by using small buffers. - Opts = [{tcp_opts, [{recbuf, 1500}, - {buffer, 1500}]}], + %% Provoke TCP back-pressure from client to server by using very small buffers. + Opts = [{tcp_opts, [{recbuf, 512}, + {buffer, 512}]}], Sub = connect(<<"subscriber">>, Config, Opts), {ok, _, [0]} = emqtt:subscribe(Sub, Topic, qos0), [ServerConnectionPid] = all_connection_pids(Config), @@ -840,6 +869,46 @@ rabbit_mqtt_qos0_queue_overflow(Config) -> ok = emqtt:disconnect(Sub), ok = emqtt:disconnect(Pub). +%% Test that MQTT connection can be listed and closed via the rabbitmq_management plugin. +management_plugin_connection(Config) -> + KeepaliveSecs = 99, + ClientId = atom_to_binary(?FUNCTION_NAME), + Node = atom_to_binary(get_node_config(Config, 0, nodename)), + C = connect(ClientId, Config, [{keepalive, KeepaliveSecs}]), + + eventually(?_assertEqual(1, length(http_get(Config, "/connections"))), 1000, 10), + [#{client_properties := #{client_id := ClientId}, + timeout := KeepaliveSecs, + node := Node, + name := ConnectionName}] = http_get(Config, "/connections"), + + process_flag(trap_exit, true), + http_delete(Config, + "/connections/" ++ binary_to_list(uri_string:quote((ConnectionName))), + ?NO_CONTENT), + receive + {'EXIT', C, _} -> + ok + after 5000 -> + ct:fail("server did not close connection") + end, + ?assertEqual([], http_get(Config, "/connections")), + ?assertEqual([], all_connection_pids(Config)). + +management_plugin_enable(Config) -> + ?assertEqual(0, length(http_get(Config, "/connections"))), + ok = rabbit_ct_broker_helpers:disable_plugin(Config, 0, rabbitmq_management), + ok = rabbit_ct_broker_helpers:disable_plugin(Config, 0, rabbitmq_management_agent), + + %% If the MQTT connection is established **before** the management plugin is enabled, + %% the management plugin should still list the MQTT connection. + C = connect(?FUNCTION_NAME, Config), + ok = rabbit_ct_broker_helpers:enable_plugin(Config, 0, rabbitmq_management_agent), + ok = rabbit_ct_broker_helpers:enable_plugin(Config, 0, rabbitmq_management), + eventually(?_assertEqual(1, length(http_get(Config, "/connections"))), 1000, 10), + + ok = emqtt:disconnect(C). + %% ------------------------------------------------------------------- %% Internal helpers %% ------------------------------------------------------------------- diff --git a/deps/rabbitmq_mqtt/test/reader_SUITE.erl b/deps/rabbitmq_mqtt/test/reader_SUITE.erl index c3956deb8ac8..85adc75591ae 100644 --- a/deps/rabbitmq_mqtt/test/reader_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/reader_SUITE.erl @@ -240,8 +240,8 @@ stats(Config) -> %% Wait for stats being emitted (every 100ms) timer:sleep(300), %% Retrieve the connection Pid - [Reader] = all_connection_pids(Config), - [{_, Pid}] = rpc(Config, rabbit_mqtt_reader, info, [Reader, [connection]]), + [Pid] = all_connection_pids(Config), + [{pid, Pid}] = rpc(Config, rabbit_mqtt_reader, info, [Pid, [pid]]), %% Verify the content of the metrics, garbage_collection must be present [{Pid, Props}] = rpc(Config, ets, lookup, [connection_metrics, Pid]), true = proplists:is_defined(garbage_collection, Props), diff --git a/deps/rabbitmq_stream/src/rabbit_stream_reader.erl b/deps/rabbitmq_stream/src/rabbit_stream_reader.erl index 06d8ce3b9794..6b88001a8f64 100644 --- a/deps/rabbitmq_stream/src/rabbit_stream_reader.erl +++ b/deps/rabbitmq_stream/src/rabbit_stream_reader.erl @@ -119,7 +119,6 @@ ssl_key_exchange, ssl_cipher, ssl_hash, - protocol, user, vhost, protocol, @@ -925,6 +924,14 @@ open(info, emit_stats, StatemData) -> Connection1 = emit_stats(Connection, State), {keep_state, StatemData#statem_data{connection = Connection1}}; +open(info, {shutdown, Explanation} = Reason, + #statem_data{connection = Connection}) -> + %% rabbitmq_management or rabbitmq_stream_management plugin + %% requests to close connection. + rabbit_log_connection:info("Forcing stream connection ~tp closing: ~tp", + [self(), Explanation]), + _ = demonitor_all_streams(Connection), + {stop, Reason}; open(info, Unknown, _StatemData) -> rabbit_log_connection:warning("Received unknown message ~tp in state ~ts", [Unknown, ?FUNCTION_NAME]), @@ -944,13 +951,6 @@ open({call, From}, {publishers_info, Items}, #statem_data{connection = Connection}) -> {keep_state_and_data, {reply, From, publishers_infos(Items, Connection)}}; -open({call, From}, {shutdown, Explanation}, - #statem_data{connection = Connection}) -> - % likely closing call from the management plugin - rabbit_log_connection:info("Forcing stream connection ~tp closing: ~tp", - [self(), Explanation]), - _ = demonitor_all_streams(Connection), - {stop_and_reply, normal, {reply, From, ok}}; open(cast, {queue_event, _, {osiris_written, _, undefined, CorrelationList}}, #statem_data{transport = Transport, @@ -3565,23 +3565,18 @@ i(host, #stream_connection{host = Host}, _) -> Host; i(peer_host, #stream_connection{peer_host = PeerHost}, _) -> PeerHost; -i(ssl, #stream_connection{socket = Socket, proxy_socket = ProxySock}, - _) -> - rabbit_net:proxy_ssl_info(Socket, ProxySock) /= nossl; -i(peer_cert_subject, S, _) -> - cert_info(fun rabbit_ssl:peer_cert_subject/1, S); -i(peer_cert_issuer, S, _) -> - cert_info(fun rabbit_ssl:peer_cert_issuer/1, S); -i(peer_cert_validity, S, _) -> - cert_info(fun rabbit_ssl:peer_cert_validity/1, S); -i(ssl_protocol, S, _) -> - ssl_info(fun({P, _}) -> P end, S); -i(ssl_key_exchange, S, _) -> - ssl_info(fun({_, {K, _, _}}) -> K end, S); -i(ssl_cipher, S, _) -> - ssl_info(fun({_, {_, C, _}}) -> C end, S); -i(ssl_hash, S, _) -> - ssl_info(fun({_, {_, _, H}}) -> H end, S); +i(SSL, #stream_connection{socket = Sock, proxy_socket = ProxySock}, _) + when SSL =:= ssl; + SSL =:= ssl_protocol; + SSL =:= ssl_key_exchange; + SSL =:= ssl_cipher; + SSL =:= ssl_hash -> + rabbit_ssl:info(SSL, {Sock, ProxySock}); +i(Cert, #stream_connection{socket = Sock},_) + when Cert =:= peer_cert_issuer; + Cert =:= peer_cert_subject; + Cert =:= peer_cert_validity -> + rabbit_ssl:cert_info(Cert, Sock); i(channels, _, _) -> 0; i(protocol, _, _) -> @@ -3623,32 +3618,6 @@ i(_Unknown, _, _) -> send(Transport, Socket, Data) when is_atom(Transport) -> Transport:send(Socket, Data). -cert_info(F, #stream_connection{socket = Sock}) -> - case rabbit_net:peercert(Sock) of - nossl -> - ''; - {error, _} -> - ''; - {ok, Cert} -> - list_to_binary(F(Cert)) - end. - -ssl_info(F, - #stream_connection{socket = Sock, proxy_socket = ProxySock}) -> - case rabbit_net:proxy_ssl_info(Sock, ProxySock) of - nossl -> - ''; - {error, _} -> - ''; - {ok, Items} -> - P = proplists:get_value(protocol, Items), - #{cipher := C, - key_exchange := K, - mac := H} = - proplists:get_value(selected_cipher_suite, Items), - F({P, {K, C, H}}) - end. - get_chunk_selector(Properties) -> binary_to_atom(maps:get(<<"chunk_selector">>, Properties, <<"user_data">>)). diff --git a/deps/rabbitmq_stream_management/src/rabbit_stream_connection_mgmt.erl b/deps/rabbitmq_stream_management/src/rabbit_stream_connection_mgmt.erl index db1a3e4a518d..22483391a7b4 100644 --- a/deps/rabbitmq_stream_management/src/rabbit_stream_connection_mgmt.erl +++ b/deps/rabbitmq_stream_management/src/rabbit_stream_connection_mgmt.erl @@ -156,5 +156,5 @@ force_close_connection(ReqData, Pid) -> V -> binary_to_list(V) end, - gen_server:call(Pid, {shutdown, Reason}, infinity), + Pid ! {shutdown, Reason}, ok. diff --git a/deps/rabbitmq_web_mqtt/BUILD.bazel b/deps/rabbitmq_web_mqtt/BUILD.bazel index 472e0018bac4..6ac2d1251a5b 100644 --- a/deps/rabbitmq_web_mqtt/BUILD.bazel +++ b/deps/rabbitmq_web_mqtt/BUILD.bazel @@ -34,11 +34,11 @@ BUILD_DEPS = [ DEPS = [ "//deps/rabbit_common:erlang_app", "@cowboy//:erlang_app", + "//deps/rabbitmq_mqtt:erlang_app", ] RUNTIME_DEPS = [ "//deps/rabbit:erlang_app", - "//deps/rabbitmq_mqtt:erlang_app", ] rabbitmq_app( @@ -64,7 +64,11 @@ dialyze( plt = ":base_plt", ) -broker_for_integration_suites() +broker_for_integration_suites( + extra_plugins = [ + "//deps/rabbitmq_management:erlang_app", + ], +) rabbitmq_test_helper( name = "test_utils", @@ -91,6 +95,7 @@ rabbitmq_integration_suite( "@emqtt//:erlang_app", "@gun//:erlang_app", "@cowlib//:erlang_app", + "//deps/rabbitmq_management_agent:erlang_app", ], ) diff --git a/deps/rabbitmq_web_mqtt/Makefile b/deps/rabbitmq_web_mqtt/Makefile index 95ef34e7663a..25405eae04c3 100644 --- a/deps/rabbitmq_web_mqtt/Makefile +++ b/deps/rabbitmq_web_mqtt/Makefile @@ -18,7 +18,7 @@ BUILD_WITHOUT_QUIC=1 export BUILD_WITHOUT_QUIC DEPS = rabbit_common rabbit cowboy rabbitmq_mqtt -TEST_DEPS = emqtt rabbitmq_ct_helpers rabbitmq_ct_client_helpers +TEST_DEPS = emqtt rabbitmq_ct_helpers rabbitmq_ct_client_helpers rabbitmq_management # FIXME: Add Ranch as a BUILD_DEPS to be sure the correct version is picked. # See rabbitmq-components.mk. diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index b73626addd28..1603a0b22b78 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -9,6 +9,8 @@ -behaviour(cowboy_websocket). -behaviour(cowboy_sub_protocol). +-include_lib("rabbitmq_mqtt/include/rabbit_mqtt.hrl"). + -export([ init/2, websocket_init/1, @@ -27,7 +29,7 @@ conn_name, parse_state, proc_state, - state, + connection_state, conserve_resources, socket, peername, @@ -40,6 +42,7 @@ -define(CLOSE_NORMAL, 1000). -define(CLOSE_PROTOCOL_ERROR, 1002). -define(CLOSE_UNACCEPTABLE_DATA_TYPE, 1003). +-define(PROTO_FAMILY, 'Web MQTT'). %% cowboy_sub_protcol upgrade(Req, Env, Handler, HandlerState) -> @@ -64,27 +67,28 @@ takeover(Parent, Ref, Socket, Transport, Opts, Buffer, {Handler, HandlerState}) {module(), Req, any(), any()} when Req::cowboy_req:req(). init(Req, Opts) -> - {PeerAddr, _PeerPort} = maps:get(peer, Req), - SockInfo = maps:get(proxy_header, Req, undefined), - WsOpts0 = proplists:get_value(ws_opts, Opts, #{}), - WsOpts = maps:merge(#{compress => true}, WsOpts0), case cowboy_req:parse_header(<<"sec-websocket-protocol">>, Req) of undefined -> no_supported_sub_protocol(undefined, Req); Protocol -> + {PeerAddr, _PeerPort} = maps:get(peer, Req), + WsOpts0 = proplists:get_value(ws_opts, Opts, #{}), + WsOpts = maps:merge(#{compress => true}, WsOpts0), case lists:member(<<"mqtt">>, Protocol) of false -> no_supported_sub_protocol(Protocol, Req); true -> - {?MODULE, cowboy_req:set_resp_header(<<"sec-websocket-protocol">>, <<"mqtt">>, Req), - #state{ + {?MODULE, + cowboy_req:set_resp_header(<<"sec-websocket-protocol">>, <<"mqtt">>, Req), + #state{ parse_state = rabbit_mqtt_packet:initial_state(), - state = running, + connection_state = running, conserve_resources = false, - socket = SockInfo, + socket = maps:get(proxy_header, Req, undefined), peername = PeerAddr, received_connect_packet = false - }, WsOpts} + }, + WsOpts} end end. @@ -96,22 +100,17 @@ websocket_init(State0 = #state{socket = Sock, peername = PeerAddr}) -> case rabbit_net:connection_string(Sock, inbound) of {ok, ConnStr} -> ConnName = rabbit_data_coercion:to_binary(ConnStr), - State = State0#state{ - conn_name = ConnName, - socket = Sock - }, - rabbit_log_connection:info("Accepting Web MQTT connection ~p (~s)", [self(), ConnName]), - RealSocket = rabbit_net:unwrap_socket(Sock), - ProcessorState = rabbit_mqtt_processor:initial_state(RealSocket, - ConnName, - fun send_reply/2, - PeerAddr), + rabbit_log_connection:info("Accepting Web MQTT connection ~s", [ConnName]), + PState = rabbit_mqtt_processor:initial_state( + rabbit_net:unwrap_socket(Sock), + ConnName, + fun send_reply/2, + PeerAddr), + State1 = State0#state{conn_name = ConnName, + proc_state = PState}, + State = rabbit_event:init_stats_timer(State1, #state.stats_timer), process_flag(trap_exit, true), - {[], - rabbit_event:init_stats_timer( - State#state{proc_state = ProcessorState}, - #state.stats_timer), - hibernate}; + {[], State, hibernate}; {error, Reason} -> {[{shutdown_reason, Reason}], State0} end. @@ -174,6 +173,11 @@ websocket_info({'$gen_cast', {close_connection, Reason}}, State = #state{ proc_s rabbit_log_connection:warning("Web MQTT disconnecting client with ID '~s' (~p), reason: ~s", [rabbit_mqtt_processor:info(client_id, ProcState), ConnName, Reason]), stop(State); +websocket_info({'$gen_cast', {force_event_refresh, Ref}}, State0) -> + Infos = infos(?CREATION_EVENT_KEYS, State0), + rabbit_event:notify(connection_created, Infos, Ref), + State = rabbit_event:init_stats_timer(State0, #state.stats_timer), + {[], State, hibernate}; websocket_info({keepalive, Req}, State = #state{keepalive = KState0, conn_name = ConnName}) -> case rabbit_mqtt_keepalive:handle(Req, KState0) of @@ -205,12 +209,21 @@ websocket_info({{'DOWN', _QName}, _MRef, process, _Pid, _Reason} = Evt, websocket_info({'DOWN', _MRef, process, QPid, _Reason}, State) -> rabbit_amqqueue_common:notify_sent_queue_down(QPid), {[], State, hibernate}; +websocket_info({shutdown, Reason}, #state{conn_name = ConnName} = State) -> + %% rabbitmq_management plugin requests to close connection. + rabbit_log_connection:info("Web MQTT closing connection ~tp: ~tp", [ConnName, Reason]), + stop(State, ?CLOSE_NORMAL, Reason); +websocket_info(connection_created, State) -> + Infos = infos(?CREATION_EVENT_KEYS, State), + rabbit_core_metrics:connection_created(self(), Infos), + rabbit_event:notify(connection_created, Infos), + {[], State, hibernate}; websocket_info(Msg, State) -> - rabbit_log_connection:warning("Web MQTT: unexpected message ~p", [Msg]), + rabbit_log_connection:warning("Web MQTT: unexpected message ~tp", [Msg]), {[], State, hibernate}. -spec terminate(any(), cowboy_req:req(), any()) -> ok. -terminate(_Reason, _Req, #state{state = undefined}) -> +terminate(_Reason, _Req, #state{connection_state = undefined}) -> ok; terminate(Reason, Request, #state{} = State) -> terminate(Reason, Request, {true, State}); @@ -218,11 +231,11 @@ terminate(_Reason, _Request, {SendWill, #state{conn_name = ConnName, proc_state = PState, keepalive = KState} = State}) -> - rabbit_log_connection:info("closing Web MQTT connection ~p (~s)", [self(), ConnName]), + rabbit_log_connection:info("Web MQTT closing connection ~ts", [ConnName]), maybe_emit_stats(State), rabbit_mqtt_keepalive:cancel_timer(KState), ok = file_handle_cache:release(), - rabbit_mqtt_processor:terminate(SendWill, ConnName, PState). + rabbit_mqtt_processor:terminate(SendWill, ConnName, ?PROTO_FAMILY, PState). %% Internal. @@ -234,7 +247,7 @@ no_supported_sub_protocol(Protocol, Req) -> handle_data(Data, State0 = #state{}) -> case handle_data1(Data, State0) of - {ok, State1 = #state{state = blocked}, hibernate} -> + {ok, State1 = #state{connection_state = blocked}, hibernate} -> {[{active, false}], State1, hibernate}; Other -> Other @@ -250,8 +263,8 @@ handle_data1(<<>>, State0 = #state{received_connect_packet = false, handle_data1(<<>>, State) -> {ok, ensure_stats_timer(control_throttle(State)), hibernate}; handle_data1(Data, State = #state{ parse_state = ParseState, - proc_state = ProcState, - conn_name = ConnName }) -> + proc_state = ProcState, + conn_name = ConnName }) -> case parse(Data, ParseState) of {more, ParseState1} -> {ok, ensure_stats_timer(control_throttle( @@ -297,13 +310,13 @@ stop(State, CloseCode, Error0) -> handle_credits(State0) -> case control_throttle(State0) of - State = #state{state = running} -> + State = #state{connection_state = running} -> {[{active, true}], State, hibernate}; State -> {[], State, hibernate} end. -control_throttle(State = #state{state = CS, +control_throttle(State = #state{connection_state = CS, conserve_resources = Conserve, keepalive = KState, proc_state = PState}) -> @@ -312,10 +325,10 @@ control_throttle(State = #state{state = CS, credit_flow:blocked(), case {CS, Throttle} of {running, true} -> - State#state{state = blocked, + State#state{connection_state = blocked, keepalive = rabbit_mqtt_keepalive:cancel_timer(KState)}; {blocked,false} -> - State#state{state = running, + State#state{connection_state = running, keepalive = rabbit_mqtt_keepalive:start_timer(KState)}; {_, _} -> State @@ -336,15 +349,66 @@ maybe_emit_stats(State) -> emit_stats(State=#state{received_connect_packet = false}) -> %% Avoid emitting stats on terminate when the connection has not yet been %% established, as this causes orphan entries on the stats database + rabbit_event:reset_stats_timer(State, #state.stats_timer); +emit_stats(State) -> + [{_, Pid}, + {_, RecvOct}, + {_, SendOct}, + {_, Reductions}] = infos(?SIMPLE_METRICS, State), + Infos = infos(?OTHER_METRICS, State), + rabbit_core_metrics:connection_stats(Pid, Infos), + rabbit_core_metrics:connection_stats(Pid, RecvOct, SendOct, Reductions), State1 = rabbit_event:reset_stats_timer(State, #state.stats_timer), - State1; -emit_stats(State=#state{socket=Sock, state=RunningState}) -> - SockInfos = case rabbit_net:getstat(Sock, - [recv_oct, recv_cnt, send_oct, send_cnt, send_pend]) of - {ok, SI} -> SI; - {error, _} -> [] - end, - Infos = [{pid, self()}, {state, RunningState}|SockInfos], - rabbit_core_metrics:connection_stats(self(), Infos), - State1 = rabbit_event:reset_stats_timer(State, #state.stats_timer), - State1. + ensure_stats_timer(State1). + +infos(Items, State) -> + [{Item, i(Item, State)} || Item <- Items]. + +i(pid, _) -> + self(); +i(SockStat, #state{socket = Sock}) + when SockStat =:= recv_oct; + SockStat =:= recv_cnt; + SockStat =:= send_oct; + SockStat =:= send_cnt; + SockStat =:= send_pend -> + case rabbit_net:getstat(Sock, [SockStat]) of + {ok, [{_, N}]} when is_number(N) -> + N; + _ -> + 0 + end; +i(reductions, _) -> + {reductions, Reductions} = erlang:process_info(self(), reductions), + Reductions; +i(garbage_collection, _) -> + rabbit_misc:get_gc_info(self()); +i(protocol, #state{proc_state = PState}) -> + {?PROTO_FAMILY, rabbit_mqtt_processor:proto_version_tuple(PState)}; +i(SSL, #state{socket = Sock}) + when SSL =:= ssl; + SSL =:= ssl_protocol; + SSL =:= ssl_key_exchange; + SSL =:= ssl_cipher; + SSL =:= ssl_hash -> + rabbit_ssl:info(SSL, {rabbit_net:unwrap_socket(Sock), + rabbit_net:maybe_get_proxy_socket(Sock)}); +i(name, S) -> + i(conn_name, S); +i(conn_name, #state{conn_name = Val}) -> + Val; +i(Cert, #state{socket = Sock}) + when Cert =:= peer_cert_issuer; + Cert =:= peer_cert_subject; + Cert =:= peer_cert_validity -> + rabbit_ssl:cert_info(Cert, rabbit_net:unwrap_socket(Sock)); +i(state, S) -> + i(connection_state, S); +i(connection_state, #state{received_connect_packet = false}) -> + starting; +i(connection_state, #state{connection_state = Val}) -> + Val; +i(timeout, #state{keepalive = KState}) -> + rabbit_mqtt_keepalive:interval_secs(KState); +i(Key, #state{proc_state = PState}) -> + rabbit_mqtt_processor:info(Key, PState). diff --git a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl index 11a0d5daa0df..1224fc2bcc94 100644 --- a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl +++ b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl @@ -8,11 +8,22 @@ -module(system_SUITE). -include_lib("eunit/include/eunit.hrl"). +-include_lib("rabbitmq_ct_helpers/include/rabbit_mgmt_test.hrl"). -compile([export_all, nowarn_export_all]). --import(rabbit_ct_broker_helpers, [rpc/5]). --import(rabbit_ct_helpers, [eventually/1]). +-import(rabbit_ct_broker_helpers, + [rpc/5]). +-import(rabbit_ct_helpers, + [eventually/1, + eventually/3]). +-import(rabbit_mgmt_test_util, + [http_get/2, + http_delete/3]). + +-define(MANAGEMENT_PLUGIN_TESTS, + [management_plugin_connection, + management_plugin_enable]). all() -> [ @@ -21,23 +32,27 @@ all() -> groups() -> [ - {non_parallel_tests, [], - [connection - , pubsub_shared_connection - , pubsub_separate_connections - , last_will_enabled_disconnect - , last_will_enabled_no_disconnect - , disconnect - , keepalive - , maintenance - , client_no_supported_protocol - , client_not_support_mqtt - , unacceptable_data_type - , duplicate_id - , handle_invalid_packets - ]} + {non_parallel_tests, [], + [connection + , pubsub_shared_connection + , pubsub_separate_connections + , last_will_enabled_disconnect + , last_will_enabled_no_disconnect + , disconnect + , keepalive + , maintenance + , client_no_supported_protocol + , client_not_support_mqtt + , unacceptable_data_type + , duplicate_id + , handle_invalid_packets + ] ++ ?MANAGEMENT_PLUGIN_TESTS + } ]. +suite() -> + [{timetrap, {minutes, 5}}]. + init_per_suite(Config) -> rabbit_ct_helpers:log_environment(), Config1 = rabbit_ct_helpers:set_config(Config, [ @@ -60,11 +75,30 @@ end_per_group(_, Config) -> Config. init_per_testcase(Testcase, Config) -> + maybe_start_inets(Testcase), rabbit_ct_helpers:testcase_started(Config, Testcase). end_per_testcase(Testcase, Config) -> + maybe_stop_inets(Testcase), + rabbit_ct_client_helpers:close_channels_and_connection(Config, 0), rabbit_ct_helpers:testcase_finished(Config, Testcase). +maybe_start_inets(Testcase) -> + case lists:member(Testcase, ?MANAGEMENT_PLUGIN_TESTS) of + true -> + ok = inets:start(); + false -> + ok + end. + +maybe_stop_inets(Testcase) -> + case lists:member(Testcase, ?MANAGEMENT_PLUGIN_TESTS) of + true -> + ok = inets:stop(); + false -> + ok + end. + connection(Config) -> C = ws_connect(?FUNCTION_NAME, Config), ok = emqtt:disconnect(C). @@ -227,6 +261,46 @@ handle_invalid_packets(Config) -> rfc6455_client:send_binary(WS, Bin), {close, {1002, _}} = rfc6455_client:recv(WS, timer:seconds(1)). +%% Test that Web MQTT connection can be listed and closed via the rabbitmq_management plugin. +management_plugin_connection(Config) -> + KeepaliveSecs = 99, + ClientId = atom_to_binary(?FUNCTION_NAME), + Node = atom_to_binary(rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename)), + C = ws_connect(ClientId, Config, [{keepalive, KeepaliveSecs}]), + + eventually(?_assertEqual(1, length(http_get(Config, "/connections"))), 1000, 10), + [#{client_properties := #{client_id := ClientId}, + timeout := KeepaliveSecs, + node := Node, + name := ConnectionName}] = http_get(Config, "/connections"), + + process_flag(trap_exit, true), + http_delete(Config, + "/connections/" ++ binary_to_list(uri_string:quote((ConnectionName))), + ?NO_CONTENT), + receive + {'EXIT', C, _} -> + ok + after 5000 -> + ct:fail("server did not close connection") + end, + ?assertEqual([], http_get(Config, "/connections")), + ?assertEqual(0, num_mqtt_connections(Config, 0)). + +management_plugin_enable(Config) -> + ?assertEqual(0, length(http_get(Config, "/connections"))), + ok = rabbit_ct_broker_helpers:disable_plugin(Config, 0, rabbitmq_management), + ok = rabbit_ct_broker_helpers:disable_plugin(Config, 0, rabbitmq_management_agent), + + %% If the Web MQTT connection is established **before** the management plugin is enabled, + %% the management plugin should still list the Web MQTT connection. + C = ws_connect(?FUNCTION_NAME, Config), + ok = rabbit_ct_broker_helpers:enable_plugin(Config, 0, rabbitmq_management_agent), + ok = rabbit_ct_broker_helpers:enable_plugin(Config, 0, rabbitmq_management), + eventually(?_assertEqual(1, length(http_get(Config, "/connections"))), 1000, 10), + + ok = emqtt:disconnect(C). + %% Web mqtt connections are tracked together with mqtt connections num_mqtt_connections(Config, Node) -> length(rpc(Config, Node, rabbit_mqtt, local_connection_pids, [])). From 1720aa0e753339f6cdb92dc7fecdc7c7e2376e9b Mon Sep 17 00:00:00 2001 From: David Ansari Date: Thu, 22 Dec 2022 23:17:01 +0000 Subject: [PATCH 090/118] Allow CLI listing rabbit_mqtt_qos0_queue queues --- .../src/rabbit_mqtt_qos0_queue.erl | 52 ++++++++++++++++--- deps/rabbitmq_mqtt/test/integration_SUITE.erl | 31 ++++++++++- 2 files changed, 76 insertions(+), 7 deletions(-) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl index 8f50df26de74..77a9057d9a32 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl @@ -20,6 +20,7 @@ -behaviour(rabbit_queue_type). -include_lib("rabbit_common/include/rabbit.hrl"). +-include_lib("rabbit/include/amqqueue.hrl"). %% rabbit_queue_type callbacks -export([ @@ -39,6 +40,9 @@ notify_decorators/1 ]). +-define(INFO_KEYS, [type, name, durable, auto_delete, arguments, + pid, owner_pid, state, messages]). + -spec is_stateful() -> boolean(). is_stateful() -> @@ -159,12 +163,6 @@ purge(_Q) -> policy_changed(_Q) -> ok. -%% general queue info --spec info(amqqueue:amqqueue(), all_keys | rabbit_types:info_keys()) -> - rabbit_types:infos(). -info(_Q, _Items) -> - []. - -spec notify_decorators(amqqueue:amqqueue()) -> ok. notify_decorators(_) -> @@ -179,3 +177,45 @@ stat(_Q) -> #{atom() := term()}. capabilities() -> #{}. + +-spec info(amqqueue:amqqueue(), all_keys | rabbit_types:info_keys()) -> + rabbit_types:infos(). +info(Q, all_keys) + when ?is_amqqueue(Q) -> + info(Q, ?INFO_KEYS); +info(Q, Items) + when ?is_amqqueue(Q) -> + [{Item, i(Item, Q)} || Item <- Items]. + +i(type, _) -> + 'MQTT QoS 0'; +i(name, Q) -> + amqqueue:get_name(Q); +i(durable, Q) -> + amqqueue:is_durable(Q); +i(auto_delete, Q) -> + amqqueue:is_auto_delete(Q); +i(arguments, Q) -> + amqqueue:get_arguments(Q); +i(pid, Q) -> + amqqueue:get_pid(Q); +i(owner_pid, Q) -> + amqqueue:get_exclusive_owner(Q); +i(state, Q) -> + Pid = amqqueue:get_pid(Q), + case erpc:call(node(Pid), erlang, is_process_alive, [Pid]) of + true -> + running; + false -> + down + end; +i(messages, Q) -> + Pid = amqqueue:get_pid(Q), + case erpc:call(node(Pid), erlang, process_info, [Pid, message_queue_len]) of + {message_queue_len, N} -> + N; + _ -> + 0 + end; +i(_, _) -> + ''. diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/integration_SUITE.erl index 1c17a1090f05..e101ca8dd900 100644 --- a/deps/rabbitmq_mqtt/test/integration_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/integration_SUITE.erl @@ -58,7 +58,8 @@ groups() -> flow_classic_mirrored_queue, flow_quorum_queue, flow_stream, - rabbit_mqtt_qos0_queue + rabbit_mqtt_qos0_queue, + cli_list_queues ] ++ tests() } ]. @@ -909,6 +910,34 @@ management_plugin_enable(Config) -> ok = emqtt:disconnect(C). +%% Test that queues of type rabbit_mqtt_qos0_queue can be listed via rabbitmqctl. +cli_list_queues(Config) -> + C = connect(?FUNCTION_NAME, Config), + {ok, _, _} = emqtt:subscribe(C, <<"a/b/c">>, qos0), + + Qs = rabbit_ct_broker_helpers:rabbitmqctl_list( + Config, 1, + ["list_queues", "--no-table-headers", + "type", "name", "state", "durable", "auto_delete", + "arguments", "pid", "owner_pid", "messages", "exclusive_consumer_tag" + ]), + ExpectedQueueType = case rabbit_ct_helpers:is_mixed_versions(Config) of + false -> + <<"MQTT QoS 0">>; + true -> + <<"classic">> + end, + ?assertMatch([[ExpectedQueueType, <<"mqtt-subscription-cli_list_queuesqos0">>, + <<"running">>, <<"true">>, <<"false">>, <<"[]">>, _, _, <<"0">>, <<"">>]], + Qs), + + ?assertEqual([], + rabbit_ct_broker_helpers:rabbitmqctl_list( + Config, 1, ["list_queues", "--local", "--no-table-headers"]) + ), + + ok = emqtt:disconnect(C). + %% ------------------------------------------------------------------- %% Internal helpers %% ------------------------------------------------------------------- From a8b69b43c161c1f46a8198b330687842ba659e17 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Tue, 27 Dec 2022 11:00:33 +0100 Subject: [PATCH 091/118] Fix dialyzer issues and add function specs Fix all dialyzer warnings in rabbitmq_mqtt and rabbitmq_web_mqtt. Add more function specs. --- deps/rabbit/src/rabbit_alarm.erl | 6 ++ deps/rabbit/src/rabbit_limiter.erl | 2 +- deps/rabbit/src/rabbit_memory_monitor.erl | 3 + deps/rabbit/src/rabbit_mirror_queue_sync.erl | 3 + deps/rabbit/src/rabbit_queue_consumers.erl | 2 +- deps/rabbit/src/rabbit_queue_type.erl | 10 +- deps/rabbit/src/rabbit_reader.erl | 10 +- .../src/rabbit_amqp1_0_reader.erl | 3 + .../include/rabbit_mqtt_packet.hrl | 10 +- .../include/rabbit_mqtt_retain.hrl | 9 -- .../src/rabbit_mqtt_keepalive.erl | 42 +++++---- .../src/rabbit_mqtt_processor.erl | 92 +++++++++++-------- .../src/rabbit_mqtt_qos0_queue.erl | 2 +- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 46 ++++++---- .../src/rabbit_mqtt_retained_msg_store.erl | 30 +++--- .../rabbit_mqtt_retained_msg_store_dets.erl | 6 +- .../rabbit_mqtt_retained_msg_store_ets.erl | 4 +- .../src/rabbit_mqtt_retainer.erl | 27 ++---- deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl | 12 +-- deps/rabbitmq_mqtt/test/integration_SUITE.erl | 2 +- .../test/proxy_protocol_SUITE.erl | 8 +- .../src/rabbit_stomp_reader.erl | 3 + .../src/rabbit_stream_reader.erl | 10 +- .../src/rabbit_web_mqtt_handler.erl | 48 +++++----- 24 files changed, 209 insertions(+), 181 deletions(-) delete mode 100644 deps/rabbitmq_mqtt/include/rabbit_mqtt_retain.hrl diff --git a/deps/rabbit/src/rabbit_alarm.erl b/deps/rabbit/src/rabbit_alarm.erl index 4e908200772b..750b3f222d2b 100644 --- a/deps/rabbit/src/rabbit_alarm.erl +++ b/deps/rabbit/src/rabbit_alarm.erl @@ -29,6 +29,9 @@ -export([remote_conserve_resources/3]). %% Internal use only +-export_type([resource_alarm_source/0, + resource_alert/0]). + -define(SERVER, ?MODULE). -define(FILE_DESCRIPTOR_RESOURCE, <<"file descriptors">>). @@ -46,6 +49,9 @@ -type resource_alarm_source() :: 'disk' | 'memory'. -type resource_alarm() :: {resource_limit, resource_alarm_source(), node()}. -type alarm() :: local_alarm() | resource_alarm(). +-type resource_alert() :: {WasAlarmSetForNode :: boolean(), + IsThereAnyAlarmsWithSameSourceInTheCluster :: boolean(), + NodeForWhichAlarmWasSetOrCleared :: node()}. %%---------------------------------------------------------------------------- diff --git a/deps/rabbit/src/rabbit_limiter.erl b/deps/rabbit/src/rabbit_limiter.erl index dfba86b3127d..eb4df6268554 100644 --- a/deps/rabbit/src/rabbit_limiter.erl +++ b/deps/rabbit/src/rabbit_limiter.erl @@ -133,7 +133,7 @@ -type lstate() :: #lstate{pid :: pid(), prefetch_limited :: boolean()}. --type qstate() :: #qstate{pid :: pid(), +-type qstate() :: #qstate{pid :: pid() | none, state :: 'dormant' | 'active' | 'suspended'}. -type credit_mode() :: 'manual' | 'drain' | 'auto'. diff --git a/deps/rabbit/src/rabbit_memory_monitor.erl b/deps/rabbit/src/rabbit_memory_monitor.erl index 73f86a79f920..4e71c35a1c5d 100644 --- a/deps/rabbit/src/rabbit_memory_monitor.erl +++ b/deps/rabbit/src/rabbit_memory_monitor.erl @@ -75,6 +75,9 @@ report_ram_duration(Pid, QueueDuration) -> stop() -> gen_server2:cast(?SERVER, stop). +-spec conserve_resources(pid(), + rabbit_alarm:resource_alarm_source(), + rabbit_alarm:resource_alert()) -> ok. %% Paging should be enabled/disabled only in response to disk resource alarms %% for the current node. conserve_resources(Pid, disk, {_, Conserve, Node}) when node(Pid) =:= Node -> diff --git a/deps/rabbit/src/rabbit_mirror_queue_sync.erl b/deps/rabbit/src/rabbit_mirror_queue_sync.erl index f99d8a2fe0da..bf422e2fcf09 100644 --- a/deps/rabbit/src/rabbit_mirror_queue_sync.erl +++ b/deps/rabbit/src/rabbit_mirror_queue_sync.erl @@ -314,6 +314,9 @@ broadcast(SPids, Msg) -> SPid ! Msg end || SPid <- SPids]. +-spec conserve_resources(pid(), + rabbit_alarm:resource_alarm_source(), + rabbit_alarm:resource_alert()) -> ok. conserve_resources(Pid, Source, {_, Conserve, _}) -> Pid ! {conserve_resources, Source, Conserve}, ok. diff --git a/deps/rabbit/src/rabbit_queue_consumers.erl b/deps/rabbit/src/rabbit_queue_consumers.erl index d98c26a4d93d..4bcb897f9881 100644 --- a/deps/rabbit/src/rabbit_queue_consumers.erl +++ b/deps/rabbit/src/rabbit_queue_consumers.erl @@ -120,7 +120,7 @@ count() -> lists:sum([Count || #cr{consumer_count = Count} <- all_ch_record()]). unacknowledged_message_count() -> lists:sum([?QUEUE:len(C#cr.acktags) || C <- all_ch_record()]). --spec add(rabbit_amqqueue:name(), ch(), rabbit_types:ctag(), boolean(), pid(), boolean(), +-spec add(rabbit_amqqueue:name(), ch(), rabbit_types:ctag(), boolean(), pid() | none, boolean(), non_neg_integer(), rabbit_framing:amqp_table(), boolean(), rabbit_types:username(), state()) -> state(). diff --git a/deps/rabbit/src/rabbit_queue_type.erl b/deps/rabbit/src/rabbit_queue_type.erl index 3865b736d4ae..d675522f388f 100644 --- a/deps/rabbit/src/rabbit_queue_type.erl +++ b/deps/rabbit/src/rabbit_queue_type.erl @@ -97,7 +97,7 @@ -type consume_spec() :: #{no_ack := boolean(), channel_pid := pid(), - limiter_pid => pid(), + limiter_pid => pid() | none, limiter_active => boolean(), prefetch_count => non_neg_integer(), consumer_tag := rabbit_types:ctag(), @@ -108,9 +108,6 @@ -% copied from rabbit_amqqueue --type absent_reason() :: 'nodedown' | 'crashed' | stopped | timeout. - -type settle_op() :: 'complete' | 'requeue' | 'discard'. -export_type([state/0, @@ -128,7 +125,7 @@ -callback declare(amqqueue:amqqueue(), node()) -> {'new' | 'existing' | 'owner_died', amqqueue:amqqueue()} | - {'absent', amqqueue:amqqueue(), absent_reason()} | + {'absent', amqqueue:amqqueue(), rabbit_amqqueue:absent_reason()} | {'protocol_error', Type :: atom(), Reason :: string(), Args :: term()} | {'error', Err :: term() }. @@ -262,7 +259,7 @@ is_compatible(Type, Durable, Exclusive, AutoDelete) -> -spec declare(amqqueue:amqqueue(), node()) -> {'new' | 'existing' | 'owner_died', amqqueue:amqqueue()} | - {'absent', amqqueue:amqqueue(), absent_reason()} | + {'absent', amqqueue:amqqueue(), rabbit_amqqueue:absent_reason()} | {protocol_error, Type :: atom(), Reason :: string(), Args :: term()} | {'error', Err :: term() }. declare(Q0, Node) -> @@ -324,6 +321,7 @@ state_info(#ctx{state = S, state_info(_) -> #{}. +-spec format_status(state()) -> map(). format_status(#?STATE{ctxs = Ctxs}) -> #{num_queue_clients => maps:size(Ctxs)}. diff --git a/deps/rabbit/src/rabbit_reader.erl b/deps/rabbit/src/rabbit_reader.erl index 050cd12c24dd..c605f9039161 100644 --- a/deps/rabbit/src/rabbit_reader.erl +++ b/deps/rabbit/src/rabbit_reader.erl @@ -143,12 +143,6 @@ %%-------------------------------------------------------------------------- --type resource_alert() :: {WasAlarmSetForNode :: boolean(), - IsThereAnyAlarmsWithSameSourceInTheCluster :: boolean(), - NodeForWhichAlarmWasSetOrCleared :: node()}. - -%%-------------------------------------------------------------------------- - -spec start_link(pid(), any()) -> rabbit_types:ok(pid()). start_link(HelperSup, Ref) -> @@ -210,7 +204,9 @@ info(Pid, Items) -> force_event_refresh(Pid, Ref) -> gen_server:cast(Pid, {force_event_refresh, Ref}). --spec conserve_resources(pid(), atom(), resource_alert()) -> 'ok'. +-spec conserve_resources(pid(), + rabbit_alarm:resource_alarm_source(), + rabbit_alarm:resource_alert()) -> 'ok'. conserve_resources(Pid, Source, {_, Conserve, _}) -> Pid ! {conserve_resources, Source, Conserve}, diff --git a/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_reader.erl b/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_reader.erl index 86d78890c3ee..e0f0c91db816 100644 --- a/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_reader.erl +++ b/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_reader.erl @@ -81,6 +81,9 @@ system_terminate(Reason, _Parent, _Deb, _State) -> system_code_change(Misc, _Module, _OldVsn, _Extra) -> {ok, Misc}. +-spec conserve_resources(pid(), + rabbit_alarm:resource_alarm_source(), + rabbit_alarm:resource_alert()) -> ok. conserve_resources(Pid, Source, {_, Conserve, _}) -> Pid ! {conserve_resources, Source, Conserve}, ok. diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt_packet.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt_packet.hrl index 49f531f7e3b9..a07b8d34e2ac 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt_packet.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt_packet.hrl @@ -5,6 +5,8 @@ %% Copyright (c) 2020-2023 VMware, Inc. or its affiliates. All rights reserved. %% +-type option(T) :: undefined | T. + -define(PROTOCOL_NAMES, [{3, "MQIsdp"}, {4, "MQTT"}]). @@ -55,8 +57,6 @@ %% Packet identifier is a non zero two byte integer. -type packet_id() :: 1..16#ffff. --type option(T) :: undefined | T. - -record(mqtt_packet_fixed, {type = 0, dup = 0, qos = 0, @@ -106,3 +106,9 @@ payload :: binary()}). -type mqtt_msg() :: #mqtt_msg{}. + +%% does not include vhost because vhost is used in the (D)ETS table name +-record(retained_message, {topic :: string(), + mqtt_msg :: mqtt_msg()}). + +-type retained_message() :: #retained_message{}. diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt_retain.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt_retain.hrl deleted file mode 100644 index 3cd38eb69a8a..000000000000 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt_retain.hrl +++ /dev/null @@ -1,9 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2020-2023 VMware, Inc. or its affiliates. All rights reserved. - -%% does not include vhost because vhost is used in the (D)ETS table name --record(retained_message, {topic, - mqtt_msg}). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_keepalive.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_keepalive.erl index 348501640102..6b7b94b54c23 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_keepalive.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_keepalive.erl @@ -1,6 +1,7 @@ -module(rabbit_mqtt_keepalive). --export([start/2, +-export([init/0, + start/2, handle/2, start_timer/1, cancel_timer/1, @@ -8,17 +9,19 @@ -export_type([state/0]). --type option(T) :: undefined | T. - -record(state, { %% Keep Alive value as sent in the CONNECT packet. interval_secs :: pos_integer(), - timer :: option(reference()), + timer :: reference(), socket :: inet:socket(), recv_oct :: non_neg_integer(), received :: boolean()}). --opaque(state() :: undefined | #state{}). +-opaque(state() :: disabled | #state{}). + +-spec init() -> state(). +init() -> + disabled. -spec start(IntervalSeconds :: non_neg_integer(), inet:socket()) -> ok. start(0, _Sock) -> @@ -33,11 +36,11 @@ start(Seconds, Sock) handle({init, IntervalSecs, Sock}, _State) -> case rabbit_net:getstat(Sock, [recv_oct]) of {ok, [{recv_oct, RecvOct}]} -> - State = #state{socket = Sock, - interval_secs = IntervalSecs, - recv_oct = RecvOct, - received = true}, - {ok, start_timer(State)}; + {ok, #state{interval_secs = IntervalSecs, + timer = start_timer0(IntervalSecs), + socket = Sock, + recv_oct = RecvOct, + received = true}}; {error, _} = Err -> Err end; @@ -62,19 +65,22 @@ handle(check, State = #state{socket = Sock, -spec start_timer(state()) -> state(). start_timer(#state{interval_secs = Seconds} = State) -> - Ref = erlang:send_after(timer_ms(Seconds), self(), {keepalive, check}), - State#state{timer = Ref}; -start_timer(undefined) -> - undefined. + State#state{timer = start_timer0(Seconds)}; +start_timer(disabled) -> + disabled. + +-spec start_timer0(pos_integer()) -> reference(). +start_timer0(KeepAliveSeconds) -> + erlang:send_after(timer_ms(KeepAliveSeconds), self(), {keepalive, check}). -spec cancel_timer(state()) -> state(). -cancel_timer(undefined) -> - undefined; cancel_timer(#state{timer = Ref} = State) when is_reference(Ref) -> ok = erlang:cancel_timer(Ref, [{async, true}, {info, false}]), - State. + State; +cancel_timer(disabled) -> + disabled. %% "If the Keep Alive value is non-zero and the Server does not receive a Control %% Packet from the Client within one and a half times the Keep Alive time period, @@ -104,5 +110,5 @@ timer_ms(KeepaliveSeconds) -> non_neg_integer(). interval_secs(#state{interval_secs = Seconds}) -> Seconds; -interval_secs(undefined) -> +interval_secs(disabled) -> 0. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index fb157c5befaa..802699a1a6c8 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -30,19 +30,20 @@ -include("rabbit_mqtt_packet.hrl"). -define(MAX_PERMISSION_CACHE_SIZE, 12). --define(CONSUMER_TAG, mqtt). +-define(CONSUMER_TAG, <<"mqtt">>). --record(auth_state, {username, - user, - vhost, - authz_ctx}). +-record(auth_state, {username :: binary(), + user :: #user{}, + vhost :: rabbit_types:vhost(), + authz_ctx :: #{binary() := binary()} + }). --record(info, {prefetch, - host, - port, - peer_host, - peer_port, - connected_at}). +-record(info, {prefetch :: non_neg_integer(), + host :: inet:ip_address(), + port :: inet:port_number(), + peer_host :: inet:ip_address(), + peer_port :: inet:port_number(), + connected_at :: pos_integer()}). -record(state, {socket, @@ -58,21 +59,19 @@ packet_id = 1 :: packet_id(), client_id :: option(binary()), clean_sess :: option(boolean()), - will_msg, + will_msg :: option(mqtt_msg()), exchange :: option(rabbit_exchange:name()), %% Set if client has at least one subscription with QoS 1. queue_qos1 :: option(rabbit_amqqueue:name()), has_published = false :: boolean(), - ssl_login_name, - %% Retained messages handler. See rabbit_mqtt_retainer_sup - %% and rabbit_mqtt_retainer. - retainer_pid, - auth_state, - peer_addr, + ssl_login_name :: none | binary(), + retainer_pid :: option(pid()), + auth_state :: option(#auth_state{}), + peer_addr :: inet:ip_address(), send_fun :: fun((Packet :: tuple(), state()) -> term()), register_state, - conn_name, - info, + conn_name :: option(binary()), + info :: option(#info{}), delivery_flow :: flow | noflow, %% quorum queues and streams whose soft limit has been exceeded soft_limit_exceeded = sets:new([{version, 2}]) :: sets:set(), @@ -81,6 +80,8 @@ -opaque state() :: #state{}. +-spec initial_state(Socket :: any(), ConnectionName :: binary()) -> + state(). initial_state(Socket, ConnectionName) -> {ok, {PeerAddr, _PeerPort}} = rabbit_net:peername(Socket), initial_state(Socket, @@ -88,6 +89,11 @@ initial_state(Socket, ConnectionName) -> fun serialise_and_send_to_client/2, PeerAddr). +-spec initial_state(Socket :: any(), + ConnectionName :: binary(), + SendFun :: fun((mqtt_packet(), state()) -> any()), + PeerAddr :: binary()) -> + state(). initial_state(Socket, ConnectionName, SendFun, PeerAddr) -> Flow = case rabbit_misc:get_env(rabbit, mirroring_flow_control, true) of true -> flow; @@ -105,7 +111,7 @@ initial_state(Socket, ConnectionName, SendFun, PeerAddr) -> {stop, disconnect, state()} | {error, Reason :: term(), state()}. process_packet(#mqtt_packet{fixed = #mqtt_packet_fixed{type = Type}}, - State = #state{auth_state = undefined}) + State = #state{auth_state = undefined}) when Type =/= ?CONNECT -> {error, connect_expected, State}; process_packet(Packet = #mqtt_packet{fixed = #mqtt_packet_fixed{type = Type}}, State) -> @@ -149,7 +155,7 @@ process_request(?PUBACK, process_request(?PUBLISH, Packet = #mqtt_packet{ - fixed = Fixed = #mqtt_packet_fixed{qos = ?QOS_2}}, + fixed = Fixed = #mqtt_packet_fixed{qos = ?QOS_2}}, State) -> % Downgrade QOS_2 to QOS_1 process_request(?PUBLISH, @@ -280,7 +286,7 @@ process_request(?SUBSCRIBE, process_request(?UNSUBSCRIBE, #mqtt_packet{variable = #mqtt_packet_subscribe{packet_id = PacketId, topic_table = Topics}, - payload = undefined}, + payload = undefined}, State0 = #state{send_fun = SendFun}) -> rabbit_log_connection:debug("Received an UNSUBSCRIBE for topic(s) ~p", [Topics]), HasSubsBefore = has_subs(State0), @@ -353,12 +359,6 @@ process_connect(#mqtt_packet{ SendFun(ResponsePacket, State), return_connack(ReturnCode, State). -client_id(<<>>) -> - rabbit_mqtt_util:gen_client_id(); -client_id(ClientId) - when is_binary(ClientId) -> - ClientId. - check_protocol_version(#mqtt_packet_connect{proto_ver = ProtoVersion}) -> case lists:member(ProtoVersion, proplists:get_keys(?PROTOCOL_NAMES)) of true -> @@ -397,19 +397,28 @@ check_credentials(Packet = #mqtt_packet_connect{username = Username, login({UserBin, PassBin, Packet = #mqtt_packet_connect{client_id = ClientId0, - clean_sess = CleanSess}}, + clean_sess = CleanSess}}, State0) -> - ClientId = client_id(ClientId0), + ClientId = ensure_client_id(ClientId0), case process_login(UserBin, PassBin, ClientId, State0) of already_connected -> {ok, already_connected}; {ok, State} -> {ok, Packet, State#state{clean_sess = CleanSess, - client_id = ClientId}}; + client_id = ClientId}}; {error, _Reason, _State} = Err -> Err end. +-spec ensure_client_id(binary()) -> binary(). +ensure_client_id(<<>>) -> + rabbit_data_coercion:to_binary( + rabbit_misc:base64url( + rabbit_guid:gen_secure())); +ensure_client_id(ClientId) + when is_binary(ClientId) -> + ClientId. + register_client(already_connected, _State) -> ok; register_client(Packet = #mqtt_packet_connect{proto_ver = ProtoVersion}, @@ -647,7 +656,7 @@ maybe_send_retained_message(RPid, #mqtt_topic{name = Topic0, qos = SubscribeQos} packet_id = PacketId, topic_name = Topic1 }, - payload = Msg#mqtt_msg.payload}, + payload = Msg#mqtt_msg.payload}, State), State end. @@ -786,14 +795,9 @@ check_user_login(#{vhost := VHost, notify_auth_result(AuthResult, Username, #state{conn_name = ConnName}) -> rabbit_event:notify( AuthResult, - [ - {name, case Username of - none -> ''; - _ -> Username - end}, + [{name, Username}, {connection_name, ConnName}, - {connection_type, network} - ]). + {connection_type, network}]). check_user_connection_limit(#{user := #user{username = Username}}) -> case rabbit_auth_backend_internal:is_over_connection_limit(Username) of @@ -1442,6 +1446,8 @@ handle_ra_event(Evt, State) -> rabbit_log:debug("unhandled ra_event: ~w ", [Evt]), State. +-spec handle_down(term(), state()) -> + {ok, state()} | {error, Reason :: any()}. handle_down({{'DOWN', QName}, _MRef, process, QPid, Reason}, State0 = #state{queue_states = QStates0, unacked_client_pubs = U0}) -> @@ -1464,6 +1470,9 @@ handle_down({{'DOWN', QName}, _MRef, process, QPid, Reason}, {ok, State} end. +-spec handle_queue_event( + {queue_event, rabbit_amqqueue:name() | ?QUEUE_TYPE_QOS_0, term()}, state()) -> + {ok, state()} | {error, Reason :: any(), state()}. handle_queue_event({queue_event, ?QUEUE_TYPE_QOS_0, Msg}, State0 = #state{qos0_messages_dropped = N}) -> State = case drop_qos0_message(State0) of @@ -1795,6 +1804,7 @@ ssl_login_name(Sock) -> nossl -> none end. +-spec format_status(state()) -> map(). format_status(#state{queue_states = QState, proto_ver = ProtoVersion, unacked_client_pubs = UnackClientPubs, @@ -1832,6 +1842,7 @@ format_status(#state{queue_states = QState, soft_limit_exceeded => soft_limit_exceeded(State), qos0_messages_dropped => Qos0MsgsDropped}. +-spec soft_limit_exceeded(state()) -> boolean(). soft_limit_exceeded(#state{soft_limit_exceeded = SLE}) -> not sets:is_empty(SLE). @@ -1840,6 +1851,7 @@ proto_integer_to_atom(3) -> proto_integer_to_atom(4) -> ?MQTT_PROTO_V4. +-spec proto_version_tuple(state()) -> tuple(). proto_version_tuple(#state{proto_ver = ?MQTT_PROTO_V3}) -> {3, 1, 0}; proto_version_tuple(#state{proto_ver = ?MQTT_PROTO_V4}) -> diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl index 77a9057d9a32..e757fec4b60b 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl @@ -50,7 +50,7 @@ is_stateful() -> -spec declare(amqqueue:amqqueue(), node()) -> {'new' | 'existing' | 'owner_died', amqqueue:amqqueue()} | - {'absent', amqqueue:amqqueue(), rabbit_queue_type:absent_reason()}. + {'absent', amqqueue:amqqueue(), rabbit_amqqueue:absent_reason()}. declare(Q0, _Node) -> %% The queue gets persisted such that routing to this %% queue (via the topic exchange) works as usual. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index 9229f4d7c37b..4dbe0ec026f5 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -26,17 +26,20 @@ -record(state, {socket, - proxy_socket, - conn_name, - await_recv, - deferred_recv, - received_connect_packet, - connection_state, - conserve, + proxy_socket :: undefined | {rabbit_proxy_soket, any(), any()}, + await_recv :: boolean(), + deferred_recv :: undefined | binary(), parse_state, - proc_state, + proc_state :: rabbit_mqtt_processor:state(), + connection_state :: running | blocked, + conserve :: boolean(), stats_timer, - keepalive :: rabbit_mqtt_keepalive:state()}). + keepalive = rabbit_mqtt_keepalive:init() :: rabbit_mqtt_keepalive:state(), + conn_name :: binary(), + received_connect_packet :: boolean() + }). + +-type(state() :: #state{}). %%---------------------------------------------------------------------------- @@ -44,6 +47,9 @@ start_link(Ref, _Transport, []) -> Pid = proc_lib:spawn_link(?MODULE, init, [Ref]), {ok, Pid}. +-spec conserve_resources(pid(), + rabbit_alarm:resource_alarm_source(), + rabbit_alarm:resource_alert()) -> ok. conserve_resources(Pid, _, {_, Conserve, _}) -> Pid ! {conserve_resources, Conserve}, ok. @@ -53,6 +59,7 @@ conserve_resources(Pid, _, {_, Conserve, _}) -> info(Pid, Items) -> gen_server:call(Pid, {info, Items}). +-spec close_connection(pid(), Reason :: any()) -> ok. close_connection(Pid, Reason) -> gen_server:cast(Pid, {close_connection, Reason}). @@ -84,12 +91,16 @@ init(Ref) -> State1 = control_throttle(State0), State = rabbit_event:init_stats_timer(State1, #state.stats_timer), gen_server:enter_loop(?MODULE, [], State); - {error, enotconn} -> + {error, Reason = enotconn} -> + rabbit_log_connection:info( + "MQTT could not get connection string: ~s", [Reason]), rabbit_net:fast_close(RealSocket), - terminate(shutdown, undefined); + ignore; {error, Reason} -> + rabbit_log_connection:error( + "MQTT could not get connection string: ~p", [Reason]), rabbit_net:fast_close(RealSocket), - terminate({network_error, Reason}, undefined) + {stop, Reason} end. handle_call({info, InfoItems}, _From, State) -> @@ -362,8 +373,7 @@ process_received_bytes(Bytes, {stop, {shutdown, Error}, State} end. --spec pstate(#state{}, rabbit_mqtt_processor:state()) - -> #state{}. +-spec pstate(state(), rabbit_mqtt_processor:state()) -> state(). pstate(State = #state {}, PState) -> State #state{ proc_state = PState }. @@ -408,8 +418,8 @@ control_throttle(State = #state{connection_state = Flow, keepalive = KState, proc_state = PState}) -> Throttle = Conserve orelse - rabbit_mqtt_processor:soft_limit_exceeded(PState) orelse - credit_flow:blocked(), + rabbit_mqtt_processor:soft_limit_exceeded(PState) orelse + credit_flow:blocked(), case {Flow, Throttle} of {running, true} -> State#state{connection_state = blocked, @@ -503,7 +513,8 @@ i(protocol, #state{proc_state = ProcState}) -> i(Key, #state{proc_state = ProcState}) -> rabbit_mqtt_processor:info(Key, ProcState). --spec format_status(map()) -> map(). +-spec format_status(gen_server:format_status()) -> + gen_server:format_status(). format_status(Status) -> maps:map( fun(state, State) -> @@ -512,6 +523,7 @@ format_status(Status) -> Value end, Status). +-spec format_state(state()) -> map(). format_state(#state{proc_state = PState, socket = Socket, proxy_socket = ProxySock, diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store.erl index 5c016eb87741..5204c3089bb1 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store.erl @@ -7,20 +7,25 @@ -module(rabbit_mqtt_retained_msg_store). --export([behaviour_info/1, table_name_for/1]). +-include("rabbit_mqtt_packet.hrl"). -behaviour_info(callbacks) -> - [{new, 2}, - {recover, 2}, - {insert, 3}, - {lookup, 2}, - {delete, 2}, - {terminate, 1}]; -behaviour_info(_Other) -> - undefined. +-callback new(Directory :: file:name_all(), rabbit_types:vhost()) -> + State :: any(). -table_name_for(VHost) -> - rabbit_mqtt_util:vhost_name_to_table_name(VHost). +-callback recover(Directory :: file:name_all(), rabbit_types:vhost()) -> + {ok, State :: any()} | {error, Reason :: term()}. + +-callback insert(Topic :: string(), mqtt_msg(), State :: any()) -> + ok. + +-callback lookup(Topic :: string(), State :: any()) -> + retained_message() | not_found. + +-callback delete(Topic :: string(), State :: any()) -> + ok. + +-callback terminate(State :: any()) -> + ok. %% TODO Support retained messages in RabbitMQ cluster, for %% 1. support PUBLISH with retain on a different node than SUBSCRIBE @@ -30,6 +35,5 @@ table_name_for(VHost) -> %% * retained message store backend does RPCs to peer nodes to lookup and delete %% %% Possible solutions for 2. -%% * rabbitmq_mqtt_retained_msg_store_mnesia %% * rabbitmq_mqtt_retained_msg_store_khepri %% * rabbitmq_mqtt_retained_msg_store_ra (implementing our own ra machine) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_dets.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_dets.erl index 9900129bd634..b057e499ad50 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_dets.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_dets.erl @@ -8,7 +8,7 @@ -module(rabbit_mqtt_retained_msg_store_dets). -behaviour(rabbit_mqtt_retained_msg_store). --include("rabbit_mqtt_retain.hrl"). +-include("rabbit_mqtt_packet.hrl"). -export([new/2, recover/2, insert/3, lookup/2, delete/2, terminate/1]). @@ -44,8 +44,8 @@ terminate(#store_state{table = T}) -> ok = dets:close(T). open_table(Dir, VHost) -> - dets:open_file(rabbit_mqtt_retained_msg_store:table_name_for(VHost), - table_options(rabbit_mqtt_util:path_for(Dir, VHost, ".dets"))). + dets:open_file(rabbit_mqtt_util:vhost_name_to_table_name(VHost), + table_options(rabbit_mqtt_util:path_for(Dir, VHost, ".dets"))). table_options(Path) -> [{type, set}, diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_ets.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_ets.erl index 93e8305123c0..88a384d74e07 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_ets.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_ets.erl @@ -8,7 +8,7 @@ -module(rabbit_mqtt_retained_msg_store_ets). -behaviour(rabbit_mqtt_retained_msg_store). --include("rabbit_mqtt_retain.hrl"). +-include("rabbit_mqtt_packet.hrl"). -export([new/2, recover/2, insert/3, lookup/2, delete/2, terminate/1]). @@ -22,7 +22,7 @@ new(Dir, VHost) -> Path = rabbit_mqtt_util:path_for(Dir, VHost), - TableName = rabbit_mqtt_retained_msg_store:table_name_for(VHost), + TableName = rabbit_mqtt_util:vhost_name_to_table_name(VHost), file:delete(Path), Tid = ets:new(TableName, [set, public, {keypos, #retained_message.topic}]), #store_state{table = Tid, filename = Path}. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer.erl index 212c9d3a005f..8f0308e01aa6 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer.erl @@ -7,13 +7,13 @@ -module(rabbit_mqtt_retainer). --behaviour(gen_server2). --include("rabbit_mqtt_retain.hrl"). -include("rabbit_mqtt.hrl"). -include("rabbit_mqtt_packet.hrl"). +-behaviour(gen_server). + -export([init/1, handle_call/3, handle_cast/2, handle_info/2, - terminate/2, code_change/3, start_link/2]). + terminate/2, start_link/2]). -export([retain/3, fetch/2, clear/2, store_module/0]). @@ -22,27 +22,22 @@ -record(retainer_state, {store_mod, store}). --spec retain(pid(), string(), mqtt_msg()) -> - {noreply, NewState :: term()} | - {noreply, NewState :: term(), timeout() | hibernate} | - {stop, Reason :: term(), NewState :: term()}. - %%---------------------------------------------------------------------------- start_link(RetainStoreMod, VHost) -> - gen_server2:start_link(?MODULE, [RetainStoreMod, VHost], []). + gen_server:start_link(?MODULE, [RetainStoreMod, VHost], []). +-spec retain(pid(), string(), mqtt_msg()) -> ok. retain(Pid, Topic, Msg = #mqtt_msg{retain = true}) -> - gen_server2:cast(Pid, {retain, Topic, Msg}); - + gen_server:cast(Pid, {retain, Topic, Msg}); retain(_Pid, _Topic, Msg = #mqtt_msg{retain = false}) -> throw({error, {retain_is_false, Msg}}). fetch(Pid, Topic) -> - gen_server2:call(Pid, {fetch, Topic}, ?TIMEOUT). + gen_server:call(Pid, {fetch, Topic}, ?TIMEOUT). clear(Pid, Topic) -> - gen_server2:cast(Pid, {clear, Topic}). + gen_server:cast(Pid, {clear, Topic}). %%---------------------------------------------------------------------------- @@ -91,8 +86,4 @@ store_dir() -> rabbit:data_dir(). terminate(_Reason, #retainer_state{store = Store, store_mod = Mod}) -> - Mod:terminate(Store), - ok. - -code_change(_OldVsn, State, _Extra) -> - {ok, State}. + Mod:terminate(Store). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl index 4183829d3a93..4749b6423e50 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl @@ -13,7 +13,6 @@ -export([queue_name_bin/2, qos_from_queue_name/2, - gen_client_id/0, env/1, table_lookup/2, path_for/2, @@ -61,6 +60,7 @@ qos_from_queue_name(#resource{name = Name}, ClientId) -> queue_name_prefix(ClientId) -> <<"mqtt-subscription-", ClientId/binary, "qos">>. +-spec init_sparkplug() -> ok. init_sparkplug() -> case env(sparkplug) of true -> @@ -133,10 +133,6 @@ to_mqtt(T0) -> T2 = string:replace(T1, ".", "/", all), erlang:iolist_to_binary(T2). --spec gen_client_id() -> string(). -gen_client_id() -> - rabbit_misc:base64url(rabbit_guid:gen_secure()). - env(Key) -> case application:get_env(?APP_NAME, Key) of {ok, Val} -> coerce_env_value(Key, Val); @@ -167,9 +163,11 @@ path_for(Dir, VHost, Suffix) -> filename:join(Dir, vhost_name_to_dir_name(VHost, Suffix)). +-spec vhost_name_to_table_name(rabbit_types:vhost()) -> + atom(). vhost_name_to_table_name(VHost) -> - <> = erlang:md5(VHost), - list_to_atom("rabbit_mqtt_retained_" ++ rabbit_misc:format("~36.16.0b", [Num])). + <> = erlang:md5(VHost), + list_to_atom("rabbit_mqtt_retained_" ++ rabbit_misc:format("~36.16.0b", [Num])). -spec register_clientid(rabbit_types:vhost(), binary()) -> ok. register_clientid(Vhost, ClientId) diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/integration_SUITE.erl index e101ca8dd900..07c7e5279b41 100644 --- a/deps/rabbitmq_mqtt/test/integration_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/integration_SUITE.erl @@ -894,7 +894,7 @@ management_plugin_connection(Config) -> ct:fail("server did not close connection") end, ?assertEqual([], http_get(Config, "/connections")), - ?assertEqual([], all_connection_pids(Config)). + eventually(?_assertEqual([], all_connection_pids(Config)), 500, 3). management_plugin_enable(Config) -> ?assertEqual(0, length(http_get(Config, "/connections"))), diff --git a/deps/rabbitmq_mqtt/test/proxy_protocol_SUITE.erl b/deps/rabbitmq_mqtt/test/proxy_protocol_SUITE.erl index df3ee52dd6e1..551b14c86588 100644 --- a/deps/rabbitmq_mqtt/test/proxy_protocol_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/proxy_protocol_SUITE.erl @@ -66,6 +66,7 @@ proxy_protocol(Config) -> ok = inet:send(Socket, "PROXY TCP4 192.168.1.1 192.168.1.2 80 81\r\n"), ok = inet:send(Socket, mqtt_3_1_1_connect_packet()), {ok, _Packet} = gen_tcp:recv(Socket, 0, ?TIMEOUT), + timer:sleep(10), ConnectionName = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, connection_name, []), match = re:run(ConnectionName, <<"^192.168.1.1:80 -> 192.168.1.2:81$">>, [{capture, none}]), gen_tcp:close(Socket), @@ -80,15 +81,14 @@ proxy_protocol_tls(Config) -> {ok, SslSocket} = ssl:connect(Socket, [], ?TIMEOUT), ok = ssl:send(SslSocket, mqtt_3_1_1_connect_packet()), {ok, _Packet} = ssl:recv(SslSocket, 0, ?TIMEOUT), - ConnectionName = rabbit_ct_broker_helpers:rpc(Config, 0, - ?MODULE, connection_name, []), + timer:sleep(10), + ConnectionName = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, connection_name, []), match = re:run(ConnectionName, <<"^192.168.1.1:80 -> 192.168.1.2:81$">>, [{capture, none}]), gen_tcp:close(Socket), ok. connection_name() -> - Connections = ets:tab2list(connection_created), - {_Key, Values} = lists:nth(1, Connections), + [{_Key, Values}] = ets:tab2list(connection_created), {_, Name} = lists:keyfind(name, 1, Values), Name. diff --git a/deps/rabbitmq_stomp/src/rabbit_stomp_reader.erl b/deps/rabbitmq_stomp/src/rabbit_stomp_reader.erl index 1c6b6fa23b54..9b85444769c1 100644 --- a/deps/rabbitmq_stomp/src/rabbit_stomp_reader.erl +++ b/deps/rabbitmq_stomp/src/rabbit_stomp_reader.erl @@ -253,6 +253,9 @@ process_received_bytes(Bytes, {stop, normal, State} end. +-spec conserve_resources(pid(), + rabbit_alarm:resource_alarm_source(), + rabbit_alarm:resource_alert()) -> ok. conserve_resources(Pid, _Source, {_, Conserve, _}) -> Pid ! {conserve_resources, Conserve}, ok. diff --git a/deps/rabbitmq_stream/src/rabbit_stream_reader.erl b/deps/rabbitmq_stream/src/rabbit_stream_reader.erl index 6b88001a8f64..16f947b1f663 100644 --- a/deps/rabbitmq_stream/src/rabbit_stream_reader.erl +++ b/deps/rabbitmq_stream/src/rabbit_stream_reader.erl @@ -550,11 +550,11 @@ invalid_transition(Transport, Socket, From, To) -> close_immediately(Transport, Socket), stop. -resource_alarm(ConnectionPid, disk, - {_WasAlarmSetForNode, - IsThereAnyAlarmsForSameResourceInTheCluster, _Node}) -> - ConnectionPid - ! {resource_alarm, IsThereAnyAlarmsForSameResourceInTheCluster}, +-spec resource_alarm(pid(), + rabbit_alarm:resource_alarm_source(), + rabbit_alarm:resource_alert()) -> ok. +resource_alarm(ConnectionPid, disk, {_, Conserve, _}) -> + ConnectionPid ! {resource_alarm, Conserve}, ok; resource_alarm(_ConnectionPid, _Resource, _Alert) -> ok. diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index 1603a0b22b78..1531d7bba0ae 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -26,18 +26,19 @@ takeover/7]). -record(state, { - conn_name, + socket, parse_state, proc_state, - connection_state, - conserve_resources, - socket, - peername, + connection_state = running :: running | blocked, + conserve = false :: boolean(), stats_timer, - received_connect_packet, - keepalive :: rabbit_mqtt_keepalive:state() + keepalive = rabbit_mqtt_keepalive:init() :: rabbit_mqtt_keepalive:state(), + conn_name, + received_connect_packet = false :: boolean() }). +-type state() :: #state{}. + %% Close frame status codes as defined in https://www.rfc-editor.org/rfc/rfc6455#section-7.4.1 -define(CLOSE_NORMAL, 1000). -define(CLOSE_PROTOCOL_ERROR, 1002). @@ -51,7 +52,7 @@ upgrade(Req, Env, Handler, HandlerState) -> upgrade(Req, Env, Handler, HandlerState, Opts) -> cowboy_websocket:upgrade(Req, Env, Handler, HandlerState, Opts). -takeover(Parent, Ref, Socket, Transport, Opts, Buffer, {Handler, HandlerState}) -> +takeover(Parent, Ref, Socket, Transport, Opts, Buffer, {Handler, {HandlerState, PeerAddr}}) -> Sock = case HandlerState#state.socket of undefined -> Socket; @@ -59,7 +60,7 @@ takeover(Parent, Ref, Socket, Transport, Opts, Buffer, {Handler, HandlerState}) {rabbit_proxy_socket, Socket, ProxyInfo} end, cowboy_websocket:takeover(Parent, Ref, Socket, Transport, Opts, Buffer, - {Handler, HandlerState#state{socket = Sock}}). + {Handler, {HandlerState#state{socket = Sock}, PeerAddr}}). %% cowboy_websocket -spec init(Req, any()) -> @@ -80,22 +81,17 @@ init(Req, Opts) -> true -> {?MODULE, cowboy_req:set_resp_header(<<"sec-websocket-protocol">>, <<"mqtt">>, Req), - #state{ - parse_state = rabbit_mqtt_packet:initial_state(), - connection_state = running, - conserve_resources = false, - socket = maps:get(proxy_header, Req, undefined), - peername = PeerAddr, - received_connect_packet = false - }, + {#state{parse_state = rabbit_mqtt_packet:initial_state(), + socket = maps:get(proxy_header, Req, undefined)}, + PeerAddr}, WsOpts} end end. --spec websocket_init(State) -> - {cowboy_websocket:commands(), State} | - {cowboy_websocket:commands(), State, hibernate}. -websocket_init(State0 = #state{socket = Sock, peername = PeerAddr}) -> +-spec websocket_init({state(), PeerAddr :: binary()}) -> + {cowboy_websocket:commands(), state()} | + {cowboy_websocket:commands(), state(), hibernate}. +websocket_init({State0 = #state{socket = Sock}, PeerAddr}) -> ok = file_handle_cache:obtain(), case rabbit_net:connection_string(Sock, inbound) of {ok, ConnStr} -> @@ -144,7 +140,7 @@ websocket_handle(Frame, State) -> {cowboy_websocket:commands(), State} | {cowboy_websocket:commands(), State, hibernate}. websocket_info({conserve_resources, Conserve}, State) -> - NewState = State#state{conserve_resources = Conserve}, + NewState = State#state{conserve = Conserve}, handle_credits(NewState); websocket_info({bump_credit, Msg}, State) -> credit_flow:handle_bump_msg(Msg), @@ -223,7 +219,7 @@ websocket_info(Msg, State) -> {[], State, hibernate}. -spec terminate(any(), cowboy_req:req(), any()) -> ok. -terminate(_Reason, _Req, #state{connection_state = undefined}) -> +terminate(_Reason, _Req, #state{proc_state = undefined}) -> ok; terminate(Reason, Request, #state{} = State) -> terminate(Reason, Request, {true, State}); @@ -317,12 +313,12 @@ handle_credits(State0) -> end. control_throttle(State = #state{connection_state = CS, - conserve_resources = Conserve, + conserve = Conserve, keepalive = KState, proc_state = PState}) -> Throttle = Conserve orelse - rabbit_mqtt_processor:soft_limit_exceeded(PState) orelse - credit_flow:blocked(), + rabbit_mqtt_processor:soft_limit_exceeded(PState) orelse + credit_flow:blocked(), case {CS, Throttle} of {running, true} -> State#state{connection_state = blocked, From fb6c8da2fca3698060557cc063d914d90576c32e Mon Sep 17 00:00:00 2001 From: David Ansari Date: Wed, 28 Dec 2022 15:24:28 +0000 Subject: [PATCH 092/118] Block Web MQTT connection if memory or disk alarm Previously (until RabbitMQ v3.11.x), a memory or disk alarm did not block the Web MQTT connection because this feature was only implemented half way through: The part that registers the Web MQTT connection with rabbit_alarm was missing. --- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 3 +- deps/rabbitmq_mqtt/test/reader_SUITE.erl | 6 +-- .../src/rabbit_web_mqtt_handler.erl | 27 +++++++---- deps/rabbitmq_web_mqtt/test/system_SUITE.erl | 47 +++++++++++++++++-- 4 files changed, 66 insertions(+), 17 deletions(-) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index 4dbe0ec026f5..764e921f4aca 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -74,8 +74,7 @@ init(Ref) -> {ok, ConnStr} -> ConnName = rabbit_data_coercion:to_binary(ConnStr), rabbit_log_connection:debug("MQTT accepting TCP connection ~tp (~ts)", [self(), ConnName]), - rabbit_alarm:register( - self(), {?MODULE, conserve_resources, []}), + rabbit_alarm:register(self(), {?MODULE, conserve_resources, []}), LoginTimeout = application:get_env(?APP_NAME, login_timeout, 10_000), erlang:send_after(LoginTimeout, self(), login_timeout), ProcessorState = rabbit_mqtt_processor:initial_state(RealSocket, ConnName), diff --git a/deps/rabbitmq_mqtt/test/reader_SUITE.erl b/deps/rabbitmq_mqtt/test/reader_SUITE.erl index 85adc75591ae..17822b8d4136 100644 --- a/deps/rabbitmq_mqtt/test/reader_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/reader_SUITE.erl @@ -110,15 +110,15 @@ block(Config) -> % %% Let it block timer:sleep(100), - %% Blocked, but still will publish + %% Blocked, but still will publish when unblocked puback_timeout = publish_qos1_timeout(C, <<"Topic1">>, <<"Now blocked">>, 1000), puback_timeout = publish_qos1_timeout(C, <<"Topic1">>, <<"Still blocked">>, 1000), %% Unblock rpc(Config, vm_memory_monitor, set_vm_memory_high_watermark, [0.4]), ok = expect_publishes(<<"Topic1">>, [<<"Not blocked yet">>, - <<"Now blocked">>, - <<"Still blocked">>]), + <<"Now blocked">>, + <<"Still blocked">>]), ok = emqtt:disconnect(C). block_connack_timeout(Config) -> diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index 1531d7bba0ae..8901256174b5 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -18,7 +18,9 @@ websocket_info/2, terminate/3 ]). --export([close_connection/2]). + +-export([conserve_resources/3, + close_connection/2]). %% cowboy_sub_protocol -export([upgrade/4, @@ -97,6 +99,7 @@ websocket_init({State0 = #state{socket = Sock}, PeerAddr}) -> {ok, ConnStr} -> ConnName = rabbit_data_coercion:to_binary(ConnStr), rabbit_log_connection:info("Accepting Web MQTT connection ~s", [ConnName]), + rabbit_alarm:register(self(), {?MODULE, conserve_resources, []}), PState = rabbit_mqtt_processor:initial_state( rabbit_net:unwrap_socket(Sock), ConnName, @@ -118,6 +121,13 @@ close_connection(Pid, Reason) -> sys:terminate(Pid, Reason), ok. +-spec conserve_resources(pid(), + rabbit_alarm:resource_alarm_source(), + rabbit_alarm:resource_alert()) -> ok. +conserve_resources(Pid, _, {_, Conserve, _}) -> + Pid ! {conserve_resources, Conserve}, + ok. + -spec websocket_handle(ping | pong | {text | binary | ping | pong, binary()}, State) -> {cowboy_websocket:commands(), State} | {cowboy_websocket:commands(), State, hibernate}. @@ -140,8 +150,7 @@ websocket_handle(Frame, State) -> {cowboy_websocket:commands(), State} | {cowboy_websocket:commands(), State, hibernate}. websocket_info({conserve_resources, Conserve}, State) -> - NewState = State#state{conserve = Conserve}, - handle_credits(NewState); + handle_credits(State#state{conserve = Conserve}); websocket_info({bump_credit, Msg}, State) -> credit_flow:handle_bump_msg(Msg), handle_credits(State); @@ -305,12 +314,12 @@ stop(State, CloseCode, Error0) -> {[{close, CloseCode, Error}], State}. handle_credits(State0) -> - case control_throttle(State0) of - State = #state{connection_state = running} -> - {[{active, true}], State, hibernate}; - State -> - {[], State, hibernate} - end. + State = #state{connection_state = CS} = control_throttle(State0), + Active = case CS of + running -> true; + blocked -> false + end, + {[{active, Active}], State, hibernate}. control_throttle(State = #state{connection_state = CS, conserve = Conserve, diff --git a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl index 1224fc2bcc94..573891623ab5 100644 --- a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl +++ b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl @@ -33,7 +33,7 @@ all() -> groups() -> [ {non_parallel_tests, [], - [connection + [block , pubsub_shared_connection , pubsub_separate_connections , last_will_enabled_disconnect @@ -99,8 +99,30 @@ maybe_stop_inets(Testcase) -> ok end. -connection(Config) -> - C = ws_connect(?FUNCTION_NAME, Config), +%% ------------------------------------------------------------------- +%% Testsuite cases +%% ------------------------------------------------------------------- + +block(Config) -> + Topic = ClientId = atom_to_binary(?FUNCTION_NAME), + C = ws_connect(ClientId, Config), + + {ok, _, _} = emqtt:subscribe(C, Topic), + {ok, _} = emqtt:publish(C, Topic, <<"Not blocked yet">>, [{qos, 1}]), + + ok = rpc(Config, 0, vm_memory_monitor, set_vm_memory_high_watermark, [0.00000001]), + %% Let it block + timer:sleep(100), + + %% Blocked, but still will publish when unblocked + puback_timeout = publish_qos1_timeout(C, Topic, <<"Now blocked">>, 1000), + puback_timeout = publish_qos1_timeout(C, Topic, <<"Still blocked">>, 1000), + + %% Unblock + rpc(Config, 0, vm_memory_monitor, set_vm_memory_high_watermark, [0.4]), + ok = expect_publishes(C, Topic, [<<"Not blocked yet">>, + <<"Now blocked">>, + <<"Still blocked">>]), ok = emqtt:disconnect(C). pubsub_shared_connection(Config) -> @@ -332,3 +354,22 @@ expect_publishes(ClientPid, Topic, [Payload|Rest]) -> after 1000 -> {publish_not_received, Payload} end. + +publish_qos1_timeout(Client, Topic, Payload, Timeout) -> + Mref = erlang:monitor(process, Client), + ok = emqtt:publish_async(Client, Topic, #{}, Payload, [{qos, 1}], infinity, + {fun ?MODULE:sync_publish_result/3, [self(), Mref]}), + receive + {Mref, Reply} -> + erlang:demonitor(Mref, [flush]), + Reply; + {'DOWN', Mref, process, Client, Reason} -> + ct:fail("client is down: ~tp", [Reason]) + after + Timeout -> + erlang:demonitor(Mref, [flush]), + puback_timeout + end. + +sync_publish_result(Caller, Mref, Result) -> + erlang:send(Caller, {Mref, Result}). From c9df098f5c8461cf9ea6229707d9c837fa0a7b15 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Wed, 28 Dec 2022 21:22:14 +0000 Subject: [PATCH 093/118] Handle topic, username, password as binaries Topic, username, and password are parsed as binaries. Storing topics as lists or converting between lists and binaries back and forth several times is unnecessary and expensive. --- .../include/rabbit_mqtt_packet.hrl | 12 +++---- deps/rabbitmq_mqtt/src/rabbit_mqtt_packet.erl | 5 +-- .../src/rabbit_mqtt_processor.erl | 35 ++++++++----------- .../src/rabbit_mqtt_retained_msg_store.erl | 6 ++-- .../src/rabbit_mqtt_retainer.erl | 6 +++- .../src/rabbit_mqtt_retainer_sup.erl | 2 -- deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl | 2 ++ deps/rabbitmq_mqtt/test/util_SUITE.erl | 27 +++++++------- 8 files changed, 44 insertions(+), 51 deletions(-) diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt_packet.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt_packet.hrl index a07b8d34e2ac..d222611fda5e 100644 --- a/deps/rabbitmq_mqtt/include/rabbit_mqtt_packet.hrl +++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt_packet.hrl @@ -8,8 +8,8 @@ -type option(T) :: undefined | T. -define(PROTOCOL_NAMES, - [{3, "MQIsdp"}, - {4, "MQTT"}]). + [{3, <<"MQIsdp">>}, + {4, <<"MQTT">>}]). %% packet types @@ -85,10 +85,10 @@ -record(mqtt_packet_connack, {session_present, return_code}). --record(mqtt_packet_publish, {topic_name, +-record(mqtt_packet_publish, {topic_name :: undefined | binary(), packet_id :: packet_id()}). --record(mqtt_topic, {name, +-record(mqtt_topic, {name :: binary(), qos}). -record(mqtt_packet_subscribe,{packet_id :: packet_id(), @@ -100,7 +100,7 @@ -record(mqtt_msg, {retain :: boolean(), qos :: qos(), - topic :: string(), + topic :: binary(), dup :: option(boolean()), packet_id :: option(packet_id()), payload :: binary()}). @@ -108,7 +108,7 @@ -type mqtt_msg() :: #mqtt_msg{}. %% does not include vhost because vhost is used in the (D)ETS table name --record(retained_message, {topic :: string(), +-record(retained_message, {topic :: binary(), mqtt_msg :: mqtt_msg()}). -type retained_message() :: #retained_message{}. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_packet.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_packet.erl index 846227cbbc3f..b094e5b74289 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_packet.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_packet.erl @@ -59,7 +59,7 @@ parse_packet(Bin, #mqtt_packet_fixed{ type = Type, _Reserved : 1, KeepAlive : 16/big, Rest3/binary>> = Rest2, - {ClientId, Rest4} = parse_utf_as_binary(Rest3), + {ClientId, Rest4} = parse_utf(Rest3), {WillTopic, Rest5} = parse_utf(Rest4, WillFlag), {WillMsg, Rest6} = parse_msg(Rest5, WillFlag), {UserName, Rest7} = parse_utf(Rest6, UsernameFlag), @@ -135,9 +135,6 @@ parse_utf(Bin, _) -> parse_utf(Bin). parse_utf(<>) -> - {binary_to_list(Str), Rest}. - -parse_utf_as_binary(<>) -> {Str, Rest}. parse_msg(Bin, 0) -> diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 802699a1a6c8..3c14501b4757 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -92,7 +92,7 @@ initial_state(Socket, ConnectionName) -> -spec initial_state(Socket :: any(), ConnectionName :: binary(), SendFun :: fun((mqtt_packet(), state()) -> any()), - PeerAddr :: binary()) -> + PeerAddr :: inet:ip_address()) -> state(). initial_state(Socket, ConnectionName, SendFun, PeerAddr) -> Flow = case rabbit_misc:get_env(rabbit, mirroring_flow_control, true) of @@ -383,11 +383,11 @@ check_credentials(Packet = #mqtt_packet_connect{username = Username, rabbit_core_metrics:auth_attempt_failed(Ip, <<>>, mqtt), rabbit_log_connection:error("MQTT login failed: no credentials provided"), {error, ?CONNACK_BAD_CREDENTIALS}; - {invalid_creds, {undefined, Pass}} when is_list(Pass) -> + {invalid_creds, {undefined, Pass}} when is_binary(Pass) -> rabbit_core_metrics:auth_attempt_failed(Ip, <<>>, mqtt), rabbit_log_connection:error("MQTT login failed: no username is provided"), {error, ?CONNACK_BAD_CREDENTIALS}; - {invalid_creds, {User, undefined}} when is_list(User) -> + {invalid_creds, {User, undefined}} when is_binary(User) -> rabbit_core_metrics:auth_attempt_failed(Ip, User, mqtt), rabbit_log_connection:error("MQTT login failed for user '~p': no password provided", [User]), {error, ?CONNACK_BAD_CREDENTIALS}; @@ -608,7 +608,7 @@ maybe_replace_old_sub(#mqtt_topic{name = TopicName, qos = QoS}, maybe_unbind(TopicName, OldTopicNamesQos0, QName, State). maybe_unbind(TopicName, TopicNames, QName, State0) -> - case lists:member(list_to_binary(TopicName), TopicNames) of + case lists:member(TopicName, TopicNames) of false -> {ok, State0}; true -> @@ -622,6 +622,7 @@ maybe_unbind(TopicName, TopicNames, QName, State0) -> end end. +-spec hand_off_to_retainer(pid(), binary(), mqtt_msg()) -> ok. hand_off_to_retainer(RetainerPid, Topic0, #mqtt_msg{payload = <<"">>}) -> Topic1 = amqp_to_mqtt(Topic0), rabbit_mqtt_retainer:clear(RetainerPid, Topic1), @@ -680,11 +681,9 @@ process_login(_UserBin, _PassBin, _ClientId, vhost = VHost }}) when Username =/= undefined, User =/= undefined, VHost =/= underfined -> - UsernameStr = rabbit_data_coercion:to_list(Username), - VHostStr = rabbit_data_coercion:to_list(VHost), rabbit_core_metrics:auth_attempt_failed(list_to_binary(inet:ntoa(Addr)), Username, mqtt), - rabbit_log_connection:warning("MQTT detected duplicate connect/login attempt for user ~tp, vhost ~tp", - [UsernameStr, VHostStr]), + rabbit_log_connection:warning("MQTT detected duplicate connect/login attempt for user ~ts, vhost ~ts", + [Username, VHost]), already_connected; process_login(UserBin, PassBin, ClientId, #state{socket = Sock, @@ -850,7 +849,7 @@ check_user_loopback(#{vhost := VHost, not_allowed -> rabbit_log_connection:warning( "MQTT login failed: user '~s' can only connect via localhost", - [binary_to_list(UsernameBin)]), + [UsernameBin]), {error, ?CONNACK_NOT_AUTHORIZED} end. @@ -957,22 +956,16 @@ creds(User, Pass, SSLLoginName) -> {ok, Anon} = application:get_env(?APP_NAME, allow_anonymous), {ok, TLSAuth} = application:get_env(?APP_NAME, ssl_cert_login), HaveDefaultCreds = Anon =:= true andalso - is_binary(DefaultUser) andalso - is_binary(DefaultPass), + is_binary(DefaultUser) andalso + is_binary(DefaultPass), - CredentialsProvided = User =/= undefined orelse - Pass =/= undefined, - - CorrectCredentials = is_list(User) andalso - is_list(Pass), - - SSLLoginProvided = TLSAuth =:= true andalso - SSLLoginName =/= none, + CredentialsProvided = User =/= undefined orelse Pass =/= undefined, + CorrectCredentials = is_binary(User) andalso is_binary(Pass), + SSLLoginProvided = TLSAuth =:= true andalso SSLLoginName =/= none, case {CredentialsProvided, CorrectCredentials, SSLLoginProvided, HaveDefaultCreds} of %% Username and password take priority - {true, true, _, _} -> {list_to_binary(User), - list_to_binary(Pass)}; + {true, true, _, _} -> {User, Pass}; %% Either username or password is provided {true, false, _, _} -> {invalid_creds, {User, Pass}}; %% rabbitmq_mqtt.ssl_cert_login is true. SSL user name provided. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store.erl index 5204c3089bb1..4a5eea9b4286 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store.erl @@ -15,13 +15,13 @@ -callback recover(Directory :: file:name_all(), rabbit_types:vhost()) -> {ok, State :: any()} | {error, Reason :: term()}. --callback insert(Topic :: string(), mqtt_msg(), State :: any()) -> +-callback insert(Topic :: binary(), mqtt_msg(), State :: any()) -> ok. --callback lookup(Topic :: string(), State :: any()) -> +-callback lookup(Topic :: binary(), State :: any()) -> retained_message() | not_found. --callback delete(Topic :: string(), State :: any()) -> +-callback delete(Topic :: binary(), State :: any()) -> ok. -callback terminate(State :: any()) -> diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer.erl index 8f0308e01aa6..1c632eb5976d 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer.erl @@ -27,15 +27,18 @@ start_link(RetainStoreMod, VHost) -> gen_server:start_link(?MODULE, [RetainStoreMod, VHost], []). --spec retain(pid(), string(), mqtt_msg()) -> ok. +-spec retain(pid(), binary(), mqtt_msg()) -> ok. retain(Pid, Topic, Msg = #mqtt_msg{retain = true}) -> gen_server:cast(Pid, {retain, Topic, Msg}); retain(_Pid, _Topic, Msg = #mqtt_msg{retain = false}) -> throw({error, {retain_is_false, Msg}}). +-spec fetch(pid(), binary()) -> + undefined | mqtt_msg(). fetch(Pid, Topic) -> gen_server:call(Pid, {fetch, Topic}, ?TIMEOUT). +-spec clear(pid(), binary()) -> ok. clear(Pid, Topic) -> gen_server:cast(Pid, {clear, Topic}). @@ -51,6 +54,7 @@ init([StoreMod, VHost]) -> end, {ok, State}. +-spec store_module() -> undefined | module(). store_module() -> case application:get_env(?APP_NAME, retained_message_store) of {ok, Mod} -> Mod; diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer_sup.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer_sup.erl index 178bb09b9b7a..9e8c983a9b5b 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer_sup.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer_sup.erl @@ -11,8 +11,6 @@ -export([start_link/1, init/1, start_child/2,start_child/1, child_for_vhost/1, delete_child/1]). --define(ENCODING, utf8). - -spec start_child(binary()) -> supervisor:startchild_ret(). -spec start_child(term(), binary()) -> supervisor:startchild_ret(). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl index 4749b6423e50..152e44c70b46 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl @@ -72,6 +72,7 @@ init_sparkplug() -> ok end. +-spec mqtt_to_amqp(binary()) -> binary(). mqtt_to_amqp(Topic) -> T = case persistent_term:get(?SPARKPLUG_MP_MQTT_TO_AMQP, no_sparkplug) of no_sparkplug -> @@ -86,6 +87,7 @@ mqtt_to_amqp(Topic) -> end, cached(mta_cache, fun to_amqp/1, T). +-spec amqp_to_mqtt(binary()) -> binary(). amqp_to_mqtt(Topic) -> T = cached(atm_cache, fun to_mqtt/1, Topic), case persistent_term:get(?SPARKPLUG_MP_AMQP_TO_MQTT, no_sparkplug) of diff --git a/deps/rabbitmq_mqtt/test/util_SUITE.erl b/deps/rabbitmq_mqtt/test/util_SUITE.erl index f9d7bc4bd122..3d058500abce 100644 --- a/deps/rabbitmq_mqtt/test/util_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/util_SUITE.erl @@ -12,19 +12,19 @@ all() -> [ - {group, util_tests} + {group, tests} ]. groups() -> [ - {util_tests, [parallel], [ - coerce_exchange, - coerce_vhost, - coerce_default_user, - coerce_default_pass, - mqtt_amqp_topic_translation - ] - } + {tests, [parallel], [ + coerce_exchange, + coerce_vhost, + coerce_default_user, + coerce_default_pass, + mqtt_amqp_topic_translation + ] + } ]. suite() -> @@ -57,23 +57,22 @@ mqtt_amqp_topic_translation(_) -> ok = application:set_env(rabbitmq_mqtt, sparkplug, true), ok = rabbit_mqtt_util:init_sparkplug(), - T0 = "/foo/bar/+/baz", + T0 = <<"/foo/bar/+/baz">>, T0_As_Amqp = <<".foo.bar.*.baz">>, T0_As_Mqtt = <<"/foo/bar/+/baz">>, ?assertEqual(T0_As_Amqp, rabbit_mqtt_util:mqtt_to_amqp(T0)), ?assertEqual(T0_As_Mqtt, rabbit_mqtt_util:amqp_to_mqtt(T0_As_Amqp)), - T1 = "spAv1.0/foo/bar/+/baz", + T1 = <<"spAv1.0/foo/bar/+/baz">>, T1_As_Amqp = <<"spAv1___0.foo.bar.*.baz">>, T1_As_Mqtt = <<"spAv1.0/foo/bar/+/baz">>, ?assertEqual(T1_As_Amqp, rabbit_mqtt_util:mqtt_to_amqp(T1)), ?assertEqual(T1_As_Mqtt, rabbit_mqtt_util:amqp_to_mqtt(T1_As_Amqp)), - T2 = "spBv2.90/foo/bar/+/baz", + T2 = <<"spBv2.90/foo/bar/+/baz">>, T2_As_Amqp = <<"spBv2___90.foo.bar.*.baz">>, T2_As_Mqtt = <<"spBv2.90/foo/bar/+/baz">>, ?assertEqual(T2_As_Amqp, rabbit_mqtt_util:mqtt_to_amqp(T2)), ?assertEqual(T2_As_Mqtt, rabbit_mqtt_util:amqp_to_mqtt(T2_As_Amqp)), - ok = application:unset_env(rabbitmq_mqtt, sparkplug), - ok. + ok = application:unset_env(rabbitmq_mqtt, sparkplug). From 7c1aa4936193c79e100ea1e5567eba27b93a8860 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Thu, 29 Dec 2022 13:01:55 +0000 Subject: [PATCH 094/118] Increase MQTT test coverage and fix edge cases --- .../src/rabbit_ct_broker_helpers.erl | 7 +- .../src/rabbit_mqtt_processor.erl | 88 ++++++--------- deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl | 2 - deps/rabbitmq_mqtt/test/auth_SUITE.erl | 105 +++++++++++++++++- deps/rabbitmq_mqtt/test/integration_SUITE.erl | 37 +++++- .../java/com/rabbitmq/mqtt/test/MqttTest.java | 20 ++++ deps/rabbitmq_mqtt/test/reader_SUITE.erl | 32 +++++- deps/rabbitmq_web_mqtt/test/system_SUITE.erl | 30 ++++- 8 files changed, 249 insertions(+), 72 deletions(-) diff --git a/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl b/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl index b38aaab20a4b..af1afc880cfe 100644 --- a/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl +++ b/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl @@ -1461,11 +1461,10 @@ set_vhost_limit(Config, Node, VHost, Limit0, Value) -> max_queues -> <<"max-queues">>; Other -> rabbit_data_coercion:to_binary(Other) end, - Definition = rabbit_json:encode(#{Limit => Value}), + Limits = [{Limit, Value}], rpc(Config, Node, - rabbit_vhost_limit, - set, - [VHost, Definition, <<"ct-tests">>]). + rabbit_vhost_limit, set, + [VHost, Limits, <<"ct-tests">>]). set_user_limits(Config, Username, Limits) -> set_user_limits(Config, 0, Username, Limits). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 3c14501b4757..a0e91204df6f 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -142,7 +142,6 @@ process_request(?PUBACK, {QMsgId, U} -> case rabbit_queue_type:settle(QName, complete, ?CONSUMER_TAG, [QMsgId], QStates0) of {ok, QStates, Actions} -> - %%TODO rabbit_channel:incr_queue_stats/3 message_acknowledged(QName, State), {ok, handle_queue_actions(Actions, State#state{unacked_server_pubs = U, queue_states = QStates})}; @@ -349,14 +348,14 @@ process_connect(#mqtt_packet{ PacketConnect, State0) of {ok, SessionPresent0, State1} -> {?CONNACK_ACCEPT, SessionPresent0, State1}; - {error, ReturnCode0, State1} -> - {ReturnCode0, false, State1} + {error, ConnectionRefusedReturnCode, State1} -> + {ConnectionRefusedReturnCode, false, State1} end, - ResponsePacket = #mqtt_packet{fixed = #mqtt_packet_fixed{type = ?CONNACK}, - variable = #mqtt_packet_connack{ - session_present = SessionPresent, - return_code = ReturnCode}}, - SendFun(ResponsePacket, State), + Response = #mqtt_packet{fixed = #mqtt_packet_fixed{type = ?CONNACK}, + variable = #mqtt_packet_connack{ + session_present = SessionPresent, + return_code = ReturnCode}}, + SendFun(Response, State), return_connack(ReturnCode, State). check_protocol_version(#mqtt_packet_connect{proto_ver = ProtoVersion}) -> @@ -368,7 +367,7 @@ check_protocol_version(#mqtt_packet_connect{proto_ver = ProtoVersion}) -> end. check_client_id(#mqtt_packet_connect{clean_sess = false, - client_id = []}) -> + client_id = <<>>}) -> {error, ?CONNACK_ID_REJECTED}; check_client_id(_) -> ok. @@ -401,12 +400,10 @@ login({UserBin, PassBin, State0) -> ClientId = ensure_client_id(ClientId0), case process_login(UserBin, PassBin, ClientId, State0) of - already_connected -> - {ok, already_connected}; {ok, State} -> {ok, Packet, State#state{clean_sess = CleanSess, client_id = ClientId}}; - {error, _Reason, _State} = Err -> + {error, _ConnectionRefusedReturnCode, _State} = Err -> Err end. @@ -419,8 +416,6 @@ ensure_client_id(ClientId) when is_binary(ClientId) -> ClientId. -register_client(already_connected, _State) -> - ok; register_client(Packet = #mqtt_packet_connect{proto_ver = ProtoVersion}, State = #state{client_id = ClientId, socket = Socket, @@ -462,8 +457,6 @@ register_client(Packet = #mqtt_packet_connect{proto_ver = ProtoVersion}, {ok, NewProcState(undefined)} end. -notify_connection_created(already_connected) -> - ok; notify_connection_created(#mqtt_packet_connect{}) -> rabbit_networking:register_non_amqp_connection(self()), self() ! connection_created, @@ -616,7 +609,7 @@ maybe_unbind(TopicName, TopicNames, QName, State0) -> {ok, _Output, State} -> {ok, State}; {error, Reason, _State} = Err -> - rabbit_log:error("Failed to unbind ~s with topic ~s: ~p", + rabbit_log:error("Failed to unbind ~s with topic '~s': ~p", [rabbit_misc:rs(QName), TopicName, Reason]), Err end @@ -674,17 +667,18 @@ make_will_msg(#mqtt_packet_connect{will_retain = Retain, dup = false, payload = Msg}. -process_login(_UserBin, _PassBin, _ClientId, +process_login(_UserBin, _PassBin, ClientId, #state{peer_addr = Addr, auth_state = #auth_state{username = Username, user = User, vhost = VHost - }}) + }} = State) when Username =/= undefined, User =/= undefined, VHost =/= underfined -> rabbit_core_metrics:auth_attempt_failed(list_to_binary(inet:ntoa(Addr)), Username, mqtt), - rabbit_log_connection:warning("MQTT detected duplicate connect/login attempt for user ~ts, vhost ~ts", - [Username, VHost]), - already_connected; + rabbit_log_connection:error( + "MQTT detected duplicate connect attempt for client ID '~ts', user '~ts', vhost '~ts'", + [ClientId, Username, VHost]), + {error, ?CONNACK_ID_REJECTED, State}; process_login(UserBin, PassBin, ClientId, #state{socket = Sock, ssl_login_name = SslLoginName, @@ -713,7 +707,7 @@ process_login(UserBin, PassBin, ClientId, {ok, _Output, State} -> rabbit_core_metrics:auth_attempt_succeeded(RemoteIpAddressBin, UsernameBin, mqtt), {ok, State}; - {error, _Reason, _State} = Err -> + {error, _ConnectionRefusedReturnCode, _State} = Err -> rabbit_core_metrics:auth_attempt_failed(RemoteIpAddressBin, UsernameBin, mqtt), Err end. @@ -730,30 +724,30 @@ check_vhost_exists(#{vhost := VHost, end. check_vhost_connection_limit(#{vhost := VHost, - username_bin := UsernameBin}) -> + client_id := ClientId, + username_bin := Username}) -> case rabbit_vhost_limit:is_over_connection_limit(VHost) of false -> ok; {true, Limit} -> rabbit_log_connection:error( - "Error on MQTT connection ~p~n" - "access to vhost '~s' refused for user '~s': " - "vhost connection limit (~p) is reached", - [self(), VHost, UsernameBin, Limit]), + "Failed to create MQTT connection because vhost connection limit is reached; " + "vhost: '~s'; connection limit: ~p; user: '~s'; client ID '~s'", + [VHost, Limit, Username, ClientId]), {error, ?CONNACK_NOT_AUTHORIZED} end. check_vhost_alive(#{vhost := VHost, + client_id := ClientId, username_bin := UsernameBin}) -> case rabbit_vhost_sup_sup:is_vhost_alive(VHost) of true -> ok; false -> rabbit_log_connection:error( - "Error on MQTT connection ~p~n" - "access refused for user '~s': " - "vhost is down", - [self(), UsernameBin, VHost]), + "Failed to create MQTT connection because vhost is down; " + "vhost: ~s; user: ~s; client ID: ~s", + [VHost, UsernameBin, ClientId]), {error, ?CONNACK_NOT_AUTHORIZED} end. @@ -798,16 +792,16 @@ notify_auth_result(AuthResult, Username, #state{conn_name = ConnName}) -> {connection_name, ConnName}, {connection_type, network}]). -check_user_connection_limit(#{user := #user{username = Username}}) -> +check_user_connection_limit(#{user := #user{username = Username}, + client_id := ClientId}) -> case rabbit_auth_backend_internal:is_over_connection_limit(Username) of false -> ok; {true, Limit} -> rabbit_log_connection:error( - "Error on MQTT connection ~p~n" - "access refused for user '~s': " - "user connection limit (~p) is reached", - [self(), Username, Limit]), + "Failed to create MQTT connection because user connection limit is reached; " + "user: '~s'; connection limit: ~p; client ID '~s'", + [Username, Limit, ClientId]), {error, ?CONNACK_NOT_AUTHORIZED} end. @@ -1230,8 +1224,6 @@ deliver_to_queues(Delivery, RoutedToQNames, State0 = #state{queue_states = QStates0, proto_ver = ProtoVer}) -> - %% TODO only lookup fields that are needed using ets:select / match? - %% TODO Use ETS continuations to be more space efficient Qs0 = rabbit_amqqueue:lookup(RoutedToQNames), Qs = rabbit_amqqueue:prepend_extra_bcc(Qs0), case rabbit_queue_type:deliver(Qs, Delivery, QStates0) of @@ -1588,7 +1580,7 @@ maybe_publish_to_client( routing_keys = [RoutingKey | _CcRoutes], content = #content{payload_fragments_rev = FragmentsRev}}}, QoS, State0 = #state{send_fun = SendFun}) -> - {PacketId, State} = queue_packet_id_to_packet_id(QMsgId, QoS, State0), + {PacketId, State} = msg_id_to_packet_id(QMsgId, QoS, State0), Packet = #mqtt_packet{ fixed = #mqtt_packet_fixed{ @@ -1609,11 +1601,11 @@ maybe_publish_to_client( message_delivered(QNameOrType, Redelivered, QoS, State), State. -queue_packet_id_to_packet_id(_, ?QOS_0, State) -> +msg_id_to_packet_id(_, ?QOS_0, State) -> %% "A PUBLISH packet MUST NOT contain a Packet Identifier if its QoS value is set to 0 [MQTT-2.2.1-2]." {undefined, State}; -queue_packet_id_to_packet_id(QMsgId, ?QOS_1, #state{packet_id = PktId, - unacked_server_pubs = U} = State) -> +msg_id_to_packet_id(QMsgId, ?QOS_1, #state{packet_id = PktId, + unacked_server_pubs = U} = State) -> {PktId, State#state{packet_id = increment_packet_id(PktId), unacked_server_pubs = maps:put(PktId, QMsgId, U)}}. @@ -1626,14 +1618,8 @@ increment_packet_id(Id) -> maybe_auto_ack(_AckRequired = true, ?QOS_0, QName, QMsgId, State = #state{queue_states = QStates0}) -> - case rabbit_queue_type:settle(QName, complete, ?CONSUMER_TAG, [QMsgId], QStates0) of - {ok, QStates, Actions} -> - %%TODO rabbit_channel:incr_queue_stats/3 - handle_queue_actions(Actions, State#state{queue_states = QStates}); - {protocol_error, _ErrorType, _Reason, _ReasonArgs} = Err -> - %%TODO handle error - throw(Err) - end; + {ok, QStates, Actions} = rabbit_queue_type:settle(QName, complete, ?CONSUMER_TAG, [QMsgId], QStates0), + handle_queue_actions(Actions, State#state{queue_states = QStates}); maybe_auto_ack(_, _, _, _, State) -> State. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl index 152e44c70b46..db884212d96d 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl @@ -37,8 +37,6 @@ queue_name_bin(ClientId, QoS) -> queue_name0(Prefix, QoS). queue_name0(Prefix, ?QOS_0) -> - %%TODO consider shortening the QoS0 queue name to save memory - %%(can't change QoS1 name to not break rolling updates) <>; queue_name0(Prefix, ?QOS_1) -> <>. diff --git a/deps/rabbitmq_mqtt/test/auth_SUITE.erl b/deps/rabbitmq_mqtt/test/auth_SUITE.erl index 672409e03875..63ac362842e2 100644 --- a/deps/rabbitmq_mqtt/test/auth_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/auth_SUITE.erl @@ -24,7 +24,8 @@ all() -> {group, no_ssl_user}, {group, ssl_user}, {group, client_id_propagation}, - {group, authz}]. + {group, authz}, + {group, limit}]. groups() -> [{anonymous_ssl_user, [], @@ -70,6 +71,7 @@ groups() -> }, {authz, [], [no_queue_bind_permission, + no_queue_unbind_permission, no_queue_consume_permission, no_queue_consume_permission_on_connect, no_queue_delete_permission, @@ -79,7 +81,12 @@ groups() -> no_topic_write_permission, loopback_user_connects_from_remote_host ] - } + }, + {limit, [], + [vhost_connection_limit, + vhost_queue_limit, + user_connection_limit + ]} ]. init_per_suite(Config) -> @@ -115,7 +122,11 @@ init_per_group(Group, Config) -> MqttConfig = mqtt_config(Group), AuthConfig = auth_config(Group), rabbit_ct_helpers:run_setup_steps(Config1, - [ fun(Conf) -> merge_app_env(MqttConfig, Conf) end ] ++ + [ fun(Conf) -> case MqttConfig of + undefined -> Conf; + _ -> merge_app_env(MqttConfig, Conf) + end + end] ++ [ fun(Conf) -> case AuthConfig of undefined -> Conf; _ -> merge_app_env(AuthConfig, Conf) @@ -146,7 +157,9 @@ mqtt_config(no_ssl_user) -> {allow_anonymous, false}]}; mqtt_config(client_id_propagation) -> {rabbitmq_mqtt, [{ssl_cert_login, true}, - {allow_anonymous, true}]}. + {allow_anonymous, true}]}; +mqtt_config(_) -> + undefined. auth_config(client_id_propagation) -> {rabbit, [ @@ -319,6 +332,7 @@ end_per_testcase(ssl_user_port_vhost_mapping_takes_precedence_over_cert_vhost_ma ok = rabbit_ct_broker_helpers:clear_global_parameter(Config, mqtt_port_to_vhost_mapping), rabbit_ct_helpers:testcase_finished(Config, ssl_user_port_vhost_mapping_takes_precedence_over_cert_vhost_mapping); end_per_testcase(Testcase, Config) when Testcase == no_queue_bind_permission; + Testcase == no_queue_unbind_permission; Testcase == no_queue_consume_permission; Testcase == no_queue_consume_permission_on_connect; Testcase == no_queue_delete_permission; @@ -432,10 +446,13 @@ ssl_user_port_vhost_mapping_takes_precedence_over_cert_vhost_mapping(Config) -> expect_successful_connection(fun connect_ssl/1, Config). connect_anonymous(Config) -> + connect_anonymous(Config, <<"simpleClient">>). + +connect_anonymous(Config, ClientId) -> P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), emqtt:start_link([{host, "localhost"}, {port, P}, - {clientid, <<"simpleClient">>}, + {clientid, ClientId}, {proto_ver, v4}]). connect_ssl(Config) -> @@ -522,6 +539,48 @@ no_queue_bind_permission(Config) -> ], test_subscribe_permissions_combination(<<".*">>, <<"">>, <<".*">>, Config, ExpectedLogs). +no_queue_unbind_permission(Config) -> + User = ?config(mqtt_user, Config), + Vhost = ?config(mqtt_vhost, Config), + rabbit_ct_broker_helpers:set_permissions(Config, User, Vhost, <<".*">>, <<".*">>, <<".*">>), + P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), + Opts = [{host, "localhost"}, + {port, P}, + {proto_ver, v4}, + {clientid, User}, + {username, User}, + {password, ?config(mqtt_password, Config)}], + {ok, C1} = emqtt:start_link([{clean_start, false} | Opts]), + {ok, _} = emqtt:connect(C1), + Topic = <<"my/topic">>, + ?assertMatch({ok, _Properties, [1]}, + emqtt:subscribe(C1, Topic, qos1)), + ok = emqtt:disconnect(C1), + + %% Revoke read access to amq.topic exchange. + rabbit_ct_broker_helpers:set_permissions(Config, User, Vhost, <<".*">>, <<".*">>, <<"^(?!amq\.topic$)">>), + {ok, C2} = emqtt:start_link([{clean_start, false} | Opts]), + {ok, _} = emqtt:connect(C2), + process_flag(trap_exit, true), + %% We subscribe with the same client ID to the same topic again, but this time with QoS 0. + %% Therefore we trigger the qos1 queue to be unbound (and the qos0 queue to be bound). + %% However, unbinding requires read access to the exchange, which we don't have anymore. + ?assertMatch({ok, _Properties, [?SUBACK_FAILURE]}, + emqtt:subscribe(C2, Topic, qos0)), + ok = assert_connection_closed(C2), + ExpectedLogs = + ["MQTT resource access refused: read access to exchange 'amq.topic' in vhost 'mqtt-vhost' refused for user 'mqtt-user'", + "Failed to unbind queue 'mqtt-subscription-mqtt-userqos1' in vhost 'mqtt-vhost' with topic 'my/topic': access_refused", + "MQTT protocol error on connection.*: subscribe_error" + ], + wait_log(Config, [?FAIL_IF_CRASH_LOG, {ExpectedLogs, fun () -> stop end}]), + + %% Clean up the qos1 queue by connecting with clean session. + rabbit_ct_broker_helpers:set_permissions(Config, User, Vhost, <<".*">>, <<".*">>, <<".*">>), + {ok, C3} = emqtt:start_link([{clean_start, true} | Opts]), + {ok, _} = emqtt:connect(C3), + ok = emqtt:disconnect(C3). + no_queue_consume_permission(Config) -> ExpectedLogs = ["MQTT resource access refused: read access to queue " @@ -803,6 +862,42 @@ expect_authentication_failure(ConnectFun, Config) -> ?assertEqual(proplists:get_value(auth_attempts_succeeded, Attempt), 0), ok. +vhost_connection_limit(Config) -> + ok = rabbit_ct_broker_helpers:set_vhost_limit(Config, 0, <<"/">>, max_connections, 2), + {ok, C1} = connect_anonymous(Config, <<"client1">>), + {ok, _} = emqtt:connect(C1), + {ok, C2} = connect_anonymous(Config, <<"client2">>), + {ok, _} = emqtt:connect(C2), + {ok, C3} = connect_anonymous(Config, <<"client3">>), + unlink(C3), + ?assertMatch({error, {unauthorized_client, _}}, emqtt:connect(C3)), + ok = emqtt:disconnect(C1), + ok = emqtt:disconnect(C2). + +vhost_queue_limit(Config) -> + ok = rabbit_ct_broker_helpers:set_vhost_limit(Config, 0, <<"/">>, max_queues, 1), + {ok, C} = connect_anonymous(Config), + {ok, _} = emqtt:connect(C), + process_flag(trap_exit, true), + %% qos0 queue can be created, qos1 queue fails to be created. + %% (RabbitMQ creates subscriptions in the reverse order of the SUBSCRIBE packet.) + ?assertMatch({ok, _Properties, [?SUBACK_FAILURE, ?SUBACK_FAILURE, 0]}, + emqtt:subscribe(C, [{<<"topic1">>, qos1}, + {<<"topic2">>, qos1}, + {<<"topic3">>, qos0}])), + ok = assert_connection_closed(C). + +user_connection_limit(Config) -> + DefaultUser = <<"guest">>, + ok = rabbit_ct_broker_helpers:set_user_limits(Config, DefaultUser, #{max_connections => 1}), + {ok, C1} = connect_anonymous(Config, <<"client1">>), + {ok, _} = emqtt:connect(C1), + {ok, C2} = connect_anonymous(Config, <<"client2">>), + unlink(C2), + ?assertMatch({error, {unauthorized_client, _}}, emqtt:connect(C2)), + ok = emqtt:disconnect(C1), + ok = rabbit_ct_broker_helpers:clear_user_limits(Config, DefaultUser, max_connections). + wait_log(Config, Clauses) -> wait_log(Config, Clauses, erlang:monotonic_time(millisecond) + 1000). diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/integration_SUITE.erl index 07c7e5279b41..7f60f9a573e2 100644 --- a/deps/rabbitmq_mqtt/test/integration_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/integration_SUITE.erl @@ -79,7 +79,9 @@ tests() -> ,subscribe_multiple ,large_message_mqtt_to_mqtt ,large_message_amqp_to_mqtt + ,many_qos1_messages ,rabbit_mqtt_qos0_queue_overflow + ,non_clean_sess_empty_client_id ]. suite() -> @@ -765,11 +767,10 @@ large_message_mqtt_to_mqtt(Config) -> C = connect(ClientId, Config), {ok, _, [1]} = emqtt:subscribe(C, {Topic, qos1}), - Payload0 = binary:copy(<<"x">>, 1_000_000), + Payload0 = binary:copy(<<"x">>, 8_000_000), Payload = <>, {ok, _} = emqtt:publish(C, Topic, Payload, qos1), ok = expect_publishes(Topic, [Payload]), - ok = emqtt:disconnect(C). large_message_amqp_to_mqtt(Config) -> @@ -778,14 +779,27 @@ large_message_amqp_to_mqtt(Config) -> {ok, _, [1]} = emqtt:subscribe(C, {Topic, qos1}), Ch = rabbit_ct_client_helpers:open_channel(Config, 0), - Payload0 = binary:copy(<<"x">>, 1_000_000), + Payload0 = binary:copy(<<"x">>, 8_000_000), Payload = <>, amqp_channel:call(Ch, #'basic.publish'{exchange = <<"amq.topic">>, routing_key = Topic}, #amqp_msg{payload = Payload}), ok = expect_publishes(Topic, [Payload]), + ok = emqtt:disconnect(C). +%% Packet identifier is a non zero two byte integer. +%% Test that the server wraps around the packet identifier. +many_qos1_messages(Config) -> + Topic = ClientId = atom_to_binary(?FUNCTION_NAME), + C = connect(ClientId, Config), + {ok, _, [1]} = emqtt:subscribe(C, {Topic, qos1}), + NumMsgs = 16#ffff + 100, + Payload = <<>>, + lists:foreach(fun(_) -> + {ok, _} = emqtt:publish(C, Topic, Payload, qos1) + end, lists:seq(1, NumMsgs)), + ?assertEqual(NumMsgs, num_received(Topic, Payload, 0)), ok = emqtt:disconnect(C). %% This test is mostly interesting in mixed version mode where feature flag @@ -938,6 +952,23 @@ cli_list_queues(Config) -> ok = emqtt:disconnect(C). +%% "If the Client supplies a zero-byte ClientId with CleanSession set to 0, +%% the Server MUST respond to the CONNECT Packet with a CONNACK return code 0x02 +%% (Identifier rejected) and then close the Network Connection" [MQTT-3.1.3-8]. +non_clean_sess_empty_client_id(Config) -> + {ok, C} = emqtt:start_link([{clientid, <<>>}, + {clean_start, false}, + {proto_ver, v4}, + {host, "localhost"}, + {port, get_node_config(Config, 0, tcp_port_mqtt)} + ]), + process_flag(trap_exit, true), + ?assertMatch({error, {client_identifier_not_valid, _}}, + emqtt:connect(C)), + receive {'EXIT', C, _} -> ok + after 500 -> ct:fail("server did not close connection") + end. + %% ------------------------------------------------------------------- %% Internal helpers %% ------------------------------------------------------------------- diff --git a/deps/rabbitmq_mqtt/test/java_SUITE_data/src/test/java/com/rabbitmq/mqtt/test/MqttTest.java b/deps/rabbitmq_mqtt/test/java_SUITE_data/src/test/java/com/rabbitmq/mqtt/test/MqttTest.java index f31497d1c705..42dfd61af17c 100644 --- a/deps/rabbitmq_mqtt/test/java_SUITE_data/src/test/java/com/rabbitmq/mqtt/test/MqttTest.java +++ b/deps/rabbitmq_mqtt/test/java_SUITE_data/src/test/java/com/rabbitmq/mqtt/test/MqttTest.java @@ -962,6 +962,26 @@ public void handleDelivery(String consumerTag, Envelope envelope, AMQP.BasicProp client.disconnect(); } + // "A Server MAY allow a Client to supply a ClientId that has a length of zero bytes, however if it does so + // the Server MUST treat this as a special case and assign a unique ClientId to that Client." [MQTT-3.1.3-6] + // RabbitMQ allows a Client to supply a ClientId that has a length of zero bytes. + @Test public void emptyClientId(TestInfo info) throws MqttException, InterruptedException { + String emptyClientId = ""; + MqttConnectOptions client_opts = new TestMqttConnectOptions(); + MqttClient client = newConnectedClient(emptyClientId, client_opts); + MqttClient client2 = newConnectedClient(emptyClientId, client_opts); + client.setCallback(this); + client2.setCallback(this); + client.subscribe("/test-topic/#"); + client2.subscribe("/test-topic/#"); + + publish(client, "/test-topic/1", 0, "my-message".getBytes()); + waitAtMost(() -> receivedMessagesSize() == 2); + + disconnect(client); + disconnect(client2); + } + private void publish(MqttClient client, String topicName, int qos, byte[] payload) throws MqttException { publish(client, topicName, qos, payload, false); } diff --git a/deps/rabbitmq_mqtt/test/reader_SUITE.erl b/deps/rabbitmq_mqtt/test/reader_SUITE.erl index 17822b8d4136..77bd42a559c0 100644 --- a/deps/rabbitmq_mqtt/test/reader_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/reader_SUITE.erl @@ -193,9 +193,15 @@ keepalive(Config) -> KeepaliveSecs = 1, KeepaliveMs = timer:seconds(KeepaliveSecs), ProtoVer = v4, - C = connect(?FUNCTION_NAME, Config, [{keepalive, KeepaliveSecs}, - {proto_ver, ProtoVer}]), - ok = emqtt:publish(C, <<"ignored">>, <<"msg">>), + WillTopic = <<"will/topic">>, + WillPayload = <<"will-payload">>, + C1 = connect(?FUNCTION_NAME, Config, [{keepalive, KeepaliveSecs}, + {proto_ver, ProtoVer}, + {will_topic, WillTopic}, + {will_payload, WillPayload}, + {will_retain, true}, + {will_qos, 0}]), + ok = emqtt:publish(C1, <<"ignored">>, <<"msg">>), %% Connection should stay up when client sends PING requests. timer:sleep(KeepaliveMs), @@ -214,8 +220,24 @@ keepalive(Config) -> util:get_global_counters(Config, ProtoVer)), KeepaliveMs, 3 * KeepaliveSecs), + receive {'EXIT', C1, _} -> ok + after 1000 -> ct:fail("missing client exit") + end, + true = rpc(Config, 0, meck, validate, [Mod]), - ok = rpc(Config, 0, meck, unload, [Mod]). + ok = rpc(Config, 0, meck, unload, [Mod]), + + C2 = connect(<<"client2">>, Config), + {ok, _, [0]} = emqtt:subscribe(C2, WillTopic), + receive {publish, #{client_pid := C2, + dup := false, + qos := 0, + retain := true, + topic := WillTopic, + payload := WillPayload}} -> ok + after 3000 -> ct:fail("missing will") + end, + ok = emqtt:disconnect(C2). keepalive_turned_off(Config) -> %% "A Keep Alive value of zero (0) has the effect of turning off the keep alive mechanism." @@ -344,7 +366,7 @@ will(Config) -> Msg = <<"will msg">>, Publisher = connect(<<"will-publisher">>, Config, [{will_topic, Topic}, {will_payload, Msg}, - {will_qos, qos0}, + {will_qos, 0}, {will_retain, false}]), timer:sleep(100), [ServerPublisherPid] = all_connection_pids(Config), diff --git a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl index 573891623ab5..8a39edbc78cc 100644 --- a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl +++ b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl @@ -27,12 +27,12 @@ all() -> [ - {group, non_parallel_tests} + {group, tests} ]. groups() -> [ - {non_parallel_tests, [], + {tests, [], [block , pubsub_shared_connection , pubsub_separate_connections @@ -46,6 +46,7 @@ groups() -> , unacceptable_data_type , duplicate_id , handle_invalid_packets + , duplicate_connect ] ++ ?MANAGEMENT_PLUGIN_TESTS } ]. @@ -323,6 +324,31 @@ management_plugin_enable(Config) -> ok = emqtt:disconnect(C). +%% "A Client can only send the CONNECT Packet once over a Network Connection. +%% The Server MUST process a second CONNECT Packet sent from a Client as a protocol +%% violation and disconnect the Client [MQTT-3.1.0-2]. +duplicate_connect(Config) -> + Url = "ws://127.0.0.1:" ++ rabbit_ws_test_util:get_web_mqtt_port_str(Config) ++ "/ws", + WS = rfc6455_client:new(Url, self(), undefined, ["mqtt"]), + {ok, _} = rfc6455_client:open(WS), + + %% 1st CONNECT should succeed. + rfc6455_client:send_binary(WS, rabbit_ws_test_util:mqtt_3_1_1_connect_packet()), + {binary, _P} = rfc6455_client:recv(WS), + eventually(?_assertEqual(1, num_mqtt_connections(Config, 0))), + + %% 2nd CONNECT should fail. + process_flag(trap_exit, true), + rfc6455_client:send_binary(WS, rabbit_ws_test_util:mqtt_3_1_1_connect_packet()), + eventually(?_assertEqual(0, num_mqtt_connections(Config, 0))), + receive {'EXIT', WS, _} -> ok + after 500 -> ct:fail("expected web socket to exit") + end. + +%% ------------------------------------------------------------------- +%% Internal helpers +%% ------------------------------------------------------------------- + %% Web mqtt connections are tracked together with mqtt connections num_mqtt_connections(Config, Node) -> length(rpc(Config, Node, rabbit_mqtt, local_connection_pids, [])). From d651f87ea7a7a5bb93325d9fe3c5b4e0372a7634 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Mon, 2 Jan 2023 17:02:43 +0000 Subject: [PATCH 095/118] Share tests between MQTT and Web MQTT New test suite deps/rabbitmq_mqtt/test/shared_SUITE contains tests that are executed against both MQTT and Web MQTT. This has two major advantages: 1. Eliminates test code duplication between rabbitmq_mqtt and rabbitmq_web_mqtt making the tests easier to maintain and to understand. 2. Increases test coverage of Web MQTT. It's acceptable to add a **test** dependency from rabbitmq_mqtt to rabbitmq_web_mqtt. Obviously, there should be no such dependency for non-test code. --- .../src/rabbit_ct_broker_helpers.erl | 5 +- deps/rabbitmq_mqtt/BUILD.bazel | 24 +- deps/rabbitmq_mqtt/Makefile | 2 +- .../src/rabbit_mqtt_processor.erl | 2 +- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 4 +- deps/rabbitmq_mqtt/test/cluster_SUITE.erl | 98 +-- deps/rabbitmq_mqtt/test/event_recorder.erl | 7 +- deps/rabbitmq_mqtt/test/ff_SUITE.erl | 8 +- deps/rabbitmq_mqtt/test/reader_SUITE.erl | 290 ++++---- deps/rabbitmq_mqtt/test/retainer_SUITE.erl | 12 +- ...integration_SUITE.erl => shared_SUITE.erl} | 624 +++++++++++------- deps/rabbitmq_mqtt/test/util.erl | 87 ++- .../src/rabbit_web_mqtt_app.erl | 10 +- .../src/rabbit_web_mqtt_handler.erl | 14 +- .../test/src/rabbit_ws_test_util.erl | 14 +- deps/rabbitmq_web_mqtt/test/system_SUITE.erl | 316 +-------- 16 files changed, 661 insertions(+), 856 deletions(-) rename deps/rabbitmq_mqtt/test/{integration_SUITE.erl => shared_SUITE.erl} (69%) diff --git a/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl b/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl index af1afc880cfe..fcb6302cc90b 100644 --- a/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl +++ b/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl @@ -38,7 +38,7 @@ add_code_path_to_node/2, add_code_path_to_all_nodes/2, - rpc/5, rpc/6, + rpc/4, rpc/5, rpc/6, rpc_all/4, rpc_all/5, start_node/2, @@ -1558,6 +1558,9 @@ add_code_path_to_all_nodes(Config, Module) -> || Nodename <- Nodenames], ok. +rpc(Config, Module, Function, Args) -> + rpc(Config, 0, Module, Function, Args). + rpc(Config, Node, Module, Function, Args) when is_atom(Node) andalso Node =/= undefined -> rpc(Config, Node, Module, Function, Args, infinity); diff --git a/deps/rabbitmq_mqtt/BUILD.bazel b/deps/rabbitmq_mqtt/BUILD.bazel index bfb6255e4a67..7fc1732f1cb7 100644 --- a/deps/rabbitmq_mqtt/BUILD.bazel +++ b/deps/rabbitmq_mqtt/BUILD.bazel @@ -86,6 +86,7 @@ dialyze( broker_for_integration_suites( extra_plugins = [ "//deps/rabbitmq_management:erlang_app", + "//deps/rabbitmq_web_mqtt:erlang_app", ], ) @@ -193,6 +194,7 @@ rabbitmq_integration_suite( "@emqtt//:erlang_app", ], additional_beam = [ + ":event_recorder", ":util", ], ) @@ -226,16 +228,18 @@ rabbitmq_integration_suite( ) rabbitmq_integration_suite( - name = "integration_SUITE", - size = "large", - runtime_deps = [ - "@emqtt//:erlang_app", - "//deps/rabbitmq_management_agent:erlang_app", - ], - additional_beam = [ - ":event_recorder", - ":util", - ], + name = "shared_SUITE", + size = "large", + timeout = "eternal", + runtime_deps = [ + "@emqtt//:erlang_app", + "//deps/rabbitmq_management_agent:erlang_app", + "@gun//:erlang_app", + ], + additional_beam = [ + ":event_recorder", + ":util", + ], ) assert_suites() diff --git a/deps/rabbitmq_mqtt/Makefile b/deps/rabbitmq_mqtt/Makefile index 179a5ab709ba..774101913185 100644 --- a/deps/rabbitmq_mqtt/Makefile +++ b/deps/rabbitmq_mqtt/Makefile @@ -40,7 +40,7 @@ BUILD_WITHOUT_QUIC=1 export BUILD_WITHOUT_QUIC DEPS = ranch rabbit_common rabbit amqp_client ra -TEST_DEPS = emqtt ct_helper rabbitmq_ct_helpers rabbitmq_ct_client_helpers rabbitmq_management +TEST_DEPS = emqtt ct_helper rabbitmq_ct_helpers rabbitmq_ct_client_helpers rabbitmq_management rabbitmq_web_mqtt dep_ct_helper = git https://github.com/extend/ct_helper.git master dep_emqtt = git https://github.com/emqx/emqtt.git 1.7.0-rc.2 diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index a0e91204df6f..309de9a973e8 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -76,7 +76,7 @@ %% quorum queues and streams whose soft limit has been exceeded soft_limit_exceeded = sets:new([{version, 2}]) :: sets:set(), qos0_messages_dropped = 0 :: non_neg_integer() - }). + }). -opaque state() :: #state{}. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index 764e921f4aca..2af6125e2515 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -412,14 +412,14 @@ run_socket(State = #state{ socket = Sock }) -> rabbit_net:setopts(Sock, [{active, once}]), State#state{ await_recv = true }. -control_throttle(State = #state{connection_state = Flow, +control_throttle(State = #state{connection_state = ConnState, conserve = Conserve, keepalive = KState, proc_state = PState}) -> Throttle = Conserve orelse rabbit_mqtt_processor:soft_limit_exceeded(PState) orelse credit_flow:blocked(), - case {Flow, Throttle} of + case {ConnState, Throttle} of {running, true} -> State#state{connection_state = blocked, keepalive = rabbit_mqtt_keepalive:cancel_timer(KState)}; diff --git a/deps/rabbitmq_mqtt/test/cluster_SUITE.erl b/deps/rabbitmq_mqtt/test/cluster_SUITE.erl index 13266fe4a946..7ec2240cc713 100644 --- a/deps/rabbitmq_mqtt/test/cluster_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/cluster_SUITE.erl @@ -9,54 +9,52 @@ -include_lib("common_test/include/ct.hrl"). -include_lib("eunit/include/eunit.hrl"). --import(util, [expect_publishes/2, +-import(util, [expect_publishes/3, connect/3, - connect/4]). + connect/4, + await_exit/1]). -import(rabbit_ct_broker_helpers, [setup_steps/0, teardown_steps/0, get_node_config/3, rabbitmqctl/3, - rpc/5, - stop_node/2, - drain_node/2, - revive_node/2]). + rpc/4, + stop_node/2 + ]). -define(OPTS, [{connect_timeout, 1}, {ack_timeout, 1}]). all() -> [ - {group, cluster_size_3}, {group, cluster_size_5} ]. groups() -> [ - {cluster_size_3, [], [ - maintenance - ]}, - {cluster_size_5, [], [ - connection_id_tracking, - connection_id_tracking_on_nodedown, - connection_id_tracking_with_decommissioned_node - ]} + {cluster_size_5, [], + [ + connection_id_tracking, + connection_id_tracking_on_nodedown, + connection_id_tracking_with_decommissioned_node + ]} ]. suite() -> - [{timetrap, {minutes, 5}}]. + [{timetrap, {minutes, 3}}]. %% ------------------------------------------------------------------- %% Testsuite setup/teardown. %% ------------------------------------------------------------------- merge_app_env(Config) -> - rabbit_ct_helpers:merge_app_env(Config, - {rabbit, [ - {collect_statistics, basic}, - {collect_statistics_interval, 100} - ]}). + rabbit_ct_helpers:merge_app_env( + Config, + {rabbit, [ + {collect_statistics, basic}, + {collect_statistics_interval, 100} + ]}). init_per_suite(Config) -> rabbit_ct_helpers:log_environment(), @@ -65,20 +63,9 @@ init_per_suite(Config) -> end_per_suite(Config) -> rabbit_ct_helpers:run_teardown_steps(Config). -init_per_group(cluster_size_3, Config) -> - case rabbit_ct_helpers:is_mixed_versions() of - true -> - {skip, "maintenance mode wrongly closes cluster-wide MQTT connections " - " in RMQ < 3.11.2 and < 3.10.10"}; - false -> - set_cluster_size(3, Config) - end; init_per_group(cluster_size_5, Config) -> - set_cluster_size(5, Config). - -set_cluster_size(NodesCount, Config) -> rabbit_ct_helpers:set_config( - Config, [{rmq_nodes_count, NodesCount}]). + Config, [{rmq_nodes_count, 5}]). end_per_group(_, Config) -> Config. @@ -107,28 +94,6 @@ end_per_testcase(Testcase, Config) -> %% Test cases %% ------------------------------------------------------------------- -maintenance(Config) -> - C0 = connect(<<"client-0">>, Config, 0, ?OPTS), - C1a = connect(<<"client-1a">>, Config, 1, ?OPTS), - C1b = connect(<<"client-1b">>, Config, 1, ?OPTS), - - timer:sleep(500), - - ok = drain_node(Config, 2), - ok = revive_node(Config, 2), - timer:sleep(500), - [?assert(erlang:is_process_alive(C)) || C <- [C0, C1a, C1b]], - - process_flag(trap_exit, true), - ok = drain_node(Config, 1), - [await_disconnection(Pid) || Pid <- [C1a, C1b]], - ok = revive_node(Config, 1), - ?assert(erlang:is_process_alive(C0)), - - ok = drain_node(Config, 0), - await_disconnection(C0), - ok = revive_node(Config, 0). - %% Note about running this testsuite in a mixed-versions cluster: %% All even-numbered nodes will use the same code base when using a %% secondary Umbrella. Odd-numbered nodes might use an incompatible code @@ -147,7 +112,7 @@ connection_id_tracking(Config) -> C1 = connect(Id, Config, 0, ?OPTS), {ok, _, _} = emqtt:subscribe(C1, <<"TopicA">>, qos0), ok = emqtt:publish(C1, <<"TopicA">>, <<"Payload">>), - ok = expect_publishes(<<"TopicA">>, [<<"Payload">>]), + ok = expect_publishes(C1, <<"TopicA">>, [<<"Payload">>]), %% there's one connection assert_connection_count(Config, 4, 2, 1), @@ -155,12 +120,12 @@ connection_id_tracking(Config) -> %% connect to the same node (A or 0) process_flag(trap_exit, true), C2 = connect(Id, Config, 0, ?OPTS), - await_disconnection(C1), + await_exit(C1), assert_connection_count(Config, 4, 2, 1), %% connect to a different node (C or 2) C3 = connect(Id, Config, 2, ?OPTS), - await_disconnection(C2), + await_exit(C2), assert_connection_count(Config, 4, 2, 1), ok = emqtt:disconnect(C3). @@ -169,16 +134,16 @@ connection_id_tracking_on_nodedown(Config) -> C = connect(<<"simpleClient">>, Config, ?OPTS), {ok, _, _} = emqtt:subscribe(C, <<"TopicA">>, qos0), ok = emqtt:publish(C, <<"TopicA">>, <<"Payload">>), - ok = expect_publishes(<<"TopicA">>, [<<"Payload">>]), + ok = expect_publishes(C, <<"TopicA">>, [<<"Payload">>]), assert_connection_count(Config, 4, 2, 1), process_flag(trap_exit, true), ok = stop_node(Config, Server), - await_disconnection(C), + await_exit(C), assert_connection_count(Config, 4, 2, 0), ok. connection_id_tracking_with_decommissioned_node(Config) -> - case rpc(Config, 0, rabbit_mqtt_ff, track_client_id_in_ra, []) of + case rpc(Config, rabbit_mqtt_ff, track_client_id_in_ra, []) of false -> {skip, "This test requires client ID tracking in Ra"}; true -> @@ -186,12 +151,12 @@ connection_id_tracking_with_decommissioned_node(Config) -> C = connect(<<"simpleClient">>, Config, ?OPTS), {ok, _, _} = emqtt:subscribe(C, <<"TopicA">>, qos0), ok = emqtt:publish(C, <<"TopicA">>, <<"Payload">>), - ok = expect_publishes(<<"TopicA">>, [<<"Payload">>]), + ok = expect_publishes(C, <<"TopicA">>, [<<"Payload">>]), assert_connection_count(Config, 4, 2, 1), process_flag(trap_exit, true), {ok, _} = rabbitmqctl(Config, 0, ["decommission_mqtt_node", Server]), - await_disconnection(C), + await_exit(C), assert_connection_count(Config, 4, 2, 0), ok end. @@ -211,10 +176,3 @@ assert_connection_count(Config, Retries, NodeId, NumElements) -> timer:sleep(500), assert_connection_count(Config, Retries-1, NodeId, NumElements) end. - -await_disconnection(Client) -> - receive - {'EXIT', Client, _} -> ok - after - 20_000 -> ct:fail({missing_exit, Client}) - end. diff --git a/deps/rabbitmq_mqtt/test/event_recorder.erl b/deps/rabbitmq_mqtt/test/event_recorder.erl index 592f13454d19..c7175310bc1e 100644 --- a/deps/rabbitmq_mqtt/test/event_recorder.erl +++ b/deps/rabbitmq_mqtt/test/event_recorder.erl @@ -15,9 +15,10 @@ init(_) -> {ok, ?INIT_STATE}. -handle_event(#event{type = node_stats}, State) -> - {ok, State}; -handle_event(#event{type = node_node_stats}, State) -> +handle_event(#event{type = T}, State) + when T =:= node_stats orelse + T =:= node_node_stats orelse + T =:= node_node_deleted -> {ok, State}; handle_event(Event, State) -> {ok, [Event | State]}. diff --git a/deps/rabbitmq_mqtt/test/ff_SUITE.erl b/deps/rabbitmq_mqtt/test/ff_SUITE.erl index 04ff01a5352d..138a18fe59c5 100644 --- a/deps/rabbitmq_mqtt/test/ff_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/ff_SUITE.erl @@ -13,7 +13,7 @@ -import(rabbit_ct_broker_helpers, [rpc/5]). -import(rabbit_ct_helpers, [eventually/1]). --import(util, [expect_publishes/2, +-import(util, [expect_publishes/3, get_global_counters/4, connect/2, connect/4]). @@ -98,7 +98,7 @@ rabbit_mqtt_qos0_queue(Config) -> C1 = connect(ClientId, Config), {ok, _, [0]} = emqtt:subscribe(C1, Topic, qos0), ok = emqtt:publish(C1, Topic, Msg, qos0), - ok = expect_publishes(Topic, [Msg]), + ok = expect_publishes(C1, Topic, [Msg]), ?assertEqual(1, length(rpc(Config, 0, rabbit_amqqueue, list_by_type, [rabbit_classic_queue]))), @@ -109,7 +109,7 @@ rabbit_mqtt_qos0_queue(Config) -> ?assertEqual(1, length(rpc(Config, 0, rabbit_amqqueue, list_by_type, [rabbit_classic_queue]))), ok = emqtt:publish(C1, Topic, Msg, qos0), - ok = expect_publishes(Topic, [Msg]), + ok = expect_publishes(C1, Topic, [Msg]), ?assertMatch(#{messages_delivered_total := 2, messages_delivered_consume_auto_ack_total := 2}, get_global_counters(Config, ?PROTO_VER, 0, [{queue_type, rabbit_classic_queue}])), @@ -125,7 +125,7 @@ rabbit_mqtt_qos0_queue(Config) -> ?assertEqual(1, length(rpc(Config, 0, rabbit_amqqueue, list_by_type, [FeatureFlag]))), ok = emqtt:publish(C2, Topic, Msg, qos0), - ok = expect_publishes(Topic, [Msg]), + ok = expect_publishes(C2, Topic, [Msg]), ?assertMatch(#{messages_delivered_total := 1, messages_delivered_consume_auto_ack_total := 1}, get_global_counters(Config, ?PROTO_VER, 0, [{queue_type, FeatureFlag}])), diff --git a/deps/rabbitmq_mqtt/test/reader_SUITE.erl b/deps/rabbitmq_mqtt/test/reader_SUITE.erl index 77bd42a559c0..32a9b239fa75 100644 --- a/deps/rabbitmq_mqtt/test/reader_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/reader_SUITE.erl @@ -11,38 +11,35 @@ -include_lib("common_test/include/ct.hrl"). -include_lib("eunit/include/eunit.hrl"). --import(rabbit_ct_broker_helpers, [rpc/5]). --import(rabbit_ct_helpers, [consistently/1, - eventually/3]). +-import(rabbit_ct_broker_helpers, [rpc/4]). +-import(rabbit_ct_helpers, [eventually/3]). -import(util, [all_connection_pids/1, publish_qos1_timeout/4, - expect_publishes/2, + expect_publishes/3, connect/2, - connect/3]). + connect/3, + await_exit/1]). all() -> [ - {group, non_parallel_tests} + {group, tests} ]. groups() -> [ - {non_parallel_tests, [], + {tests, [], [ - block, block_connack_timeout, handle_invalid_packets, login_timeout, - keepalive, - keepalive_turned_off, stats, - will, - clean_session_disconnect_client, - clean_session_kill_node, quorum_clean_session_false, quorum_clean_session_true, classic_clean_session_true, - classic_clean_session_false + classic_clean_session_false, + non_clean_sess_empty_client_id, + event_authentication_failure, + rabbit_mqtt_qos0_queue_overflow ]} ]. @@ -94,33 +91,6 @@ end_per_testcase(Testcase, Config) -> %% Testsuite cases %% ------------------------------------------------------------------- -block(Config) -> - C = connect(?FUNCTION_NAME, Config), - - %% Only here to ensure the connection is really up - {ok, _, _} = emqtt:subscribe(C, <<"TopicA">>), - ok = emqtt:publish(C, <<"TopicA">>, <<"Payload">>), - ok = expect_publishes(<<"TopicA">>, [<<"Payload">>]), - {ok, _, _} = emqtt:unsubscribe(C, <<"TopicA">>), - - {ok, _, _} = emqtt:subscribe(C, <<"Topic1">>), - {ok, _} = emqtt:publish(C, <<"Topic1">>, <<"Not blocked yet">>, [{qos, 1}]), - - ok = rpc(Config, vm_memory_monitor, set_vm_memory_high_watermark, [0.00000001]), - % %% Let it block - timer:sleep(100), - - %% Blocked, but still will publish when unblocked - puback_timeout = publish_qos1_timeout(C, <<"Topic1">>, <<"Now blocked">>, 1000), - puback_timeout = publish_qos1_timeout(C, <<"Topic1">>, <<"Still blocked">>, 1000), - - %% Unblock - rpc(Config, vm_memory_monitor, set_vm_memory_high_watermark, [0.4]), - ok = expect_publishes(<<"Topic1">>, [<<"Not blocked yet">>, - <<"Now blocked">>, - <<"Still blocked">>]), - ok = emqtt:disconnect(C). - block_connack_timeout(Config) -> P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), Ports0 = rpc(Config, erlang, ports, []), @@ -189,74 +159,6 @@ login_timeout(Config) -> rpc(Config, application, unset_env, [rabbitmq_mqtt, login_timeout]) end. -keepalive(Config) -> - KeepaliveSecs = 1, - KeepaliveMs = timer:seconds(KeepaliveSecs), - ProtoVer = v4, - WillTopic = <<"will/topic">>, - WillPayload = <<"will-payload">>, - C1 = connect(?FUNCTION_NAME, Config, [{keepalive, KeepaliveSecs}, - {proto_ver, ProtoVer}, - {will_topic, WillTopic}, - {will_payload, WillPayload}, - {will_retain, true}, - {will_qos, 0}]), - ok = emqtt:publish(C1, <<"ignored">>, <<"msg">>), - - %% Connection should stay up when client sends PING requests. - timer:sleep(KeepaliveMs), - ?assertMatch(#{publishers := 1}, - util:get_global_counters(Config, ProtoVer)), - - %% Mock the server socket to not have received any bytes. - rabbit_ct_broker_helpers:setup_meck(Config), - Mod = rabbit_net, - ok = rpc(Config, 0, meck, new, [Mod, [no_link, passthrough]]), - ok = rpc(Config, 0, meck, expect, [Mod, getstat, 2, {ok, [{recv_oct, 999}]} ]), - process_flag(trap_exit, true), - - %% We expect the server to respect the keepalive closing the connection. - eventually(?_assertMatch(#{publishers := 0}, - util:get_global_counters(Config, ProtoVer)), - KeepaliveMs, 3 * KeepaliveSecs), - - receive {'EXIT', C1, _} -> ok - after 1000 -> ct:fail("missing client exit") - end, - - true = rpc(Config, 0, meck, validate, [Mod]), - ok = rpc(Config, 0, meck, unload, [Mod]), - - C2 = connect(<<"client2">>, Config), - {ok, _, [0]} = emqtt:subscribe(C2, WillTopic), - receive {publish, #{client_pid := C2, - dup := false, - qos := 0, - retain := true, - topic := WillTopic, - payload := WillPayload}} -> ok - after 3000 -> ct:fail("missing will") - end, - ok = emqtt:disconnect(C2). - -keepalive_turned_off(Config) -> - %% "A Keep Alive value of zero (0) has the effect of turning off the keep alive mechanism." - KeepaliveSecs = 0, - C = connect(?FUNCTION_NAME, Config, [{keepalive, KeepaliveSecs}]), - ok = emqtt:publish(C, <<"TopicB">>, <<"Payload">>), - - %% Mock the server socket to not have received any bytes. - rabbit_ct_broker_helpers:setup_meck(Config), - Mod = rabbit_net, - ok = rpc(Config, 0, meck, new, [Mod, [no_link, passthrough]]), - ok = rpc(Config, 0, meck, expect, [Mod, getstat, 2, {ok, [{recv_oct, 999}]} ]), - - consistently(?_assert(erlang:is_process_alive(C))), - - true = rpc(Config, 0, meck, validate, [Mod]), - ok = rpc(Config, 0, meck, unload, [Mod]), - ok = emqtt:disconnect(C). - stats(Config) -> C = connect(?FUNCTION_NAME, Config), %% Wait for stats being emitted (every 100ms) @@ -288,7 +190,7 @@ validate_durable_queue_type(Config, ClientName, CleanSession, ExpectedQueueType) C = connect(ClientName, Config, [{clean_start, CleanSession}]), {ok, _, _} = emqtt:subscribe(C, <<"TopicB">>, qos1), ok = emqtt:publish(C, <<"TopicB">>, <<"Payload">>), - ok = expect_publishes(<<"TopicB">>, [<<"Payload">>]), + ok = expect_publishes(C, <<"TopicB">>, [<<"Payload">>]), {ok, _, _} = emqtt:unsubscribe(C, <<"TopicB">>), Prefix = <<"mqtt-subscription-">>, Suffix = <<"qos1">>, @@ -296,51 +198,6 @@ validate_durable_queue_type(Config, ClientName, CleanSession, ExpectedQueueType) ?assertEqual(ExpectedQueueType, get_durable_queue_type(Server, QNameBin)), ok = emqtt:disconnect(C). -clean_session_disconnect_client(Config) -> - C = connect(?FUNCTION_NAME, Config), - {ok, _, _} = emqtt:subscribe(C, <<"topic0">>, qos0), - {ok, _, _} = emqtt:subscribe(C, <<"topic1">>, qos1), - QsQos0 = rpc(Config, rabbit_amqqueue, list_by_type, [rabbit_mqtt_qos0_queue]), - QsClassic = rpc(Config, rabbit_amqqueue, list_by_type, [rabbit_classic_queue]), - case rabbit_ct_helpers:is_mixed_versions(Config) of - false -> - ?assertEqual(1, length(QsQos0)), - ?assertEqual(1, length(QsClassic)); - true -> - ?assertEqual(0, length(QsQos0)), - ?assertEqual(2, length(QsClassic)) - end, - - ok = emqtt:disconnect(C), - %% After terminating a clean session, we expect any session state to be cleaned up on the server. - timer:sleep(200), %% Give some time to clean up exclusive classic queue. - L = rpc(Config, rabbit_amqqueue, list, []), - ?assertEqual(0, length(L)). - -clean_session_kill_node(Config) -> - C = connect(?FUNCTION_NAME, Config), - {ok, _, _} = emqtt:subscribe(C, <<"topic0">>, qos0), - {ok, _, _} = emqtt:subscribe(C, <<"topic1">>, qos1), - QsQos0 = rpc(Config, rabbit_amqqueue, list_by_type, [rabbit_mqtt_qos0_queue]), - QsClassic = rpc(Config, rabbit_amqqueue, list_by_type, [rabbit_classic_queue]), - case rabbit_ct_helpers:is_mixed_versions(Config) of - false -> - ?assertEqual(1, length(QsQos0)), - ?assertEqual(1, length(QsClassic)); - true -> - ?assertEqual(0, length(QsQos0)), - ?assertEqual(2, length(QsClassic)) - end, - ?assertEqual(2, rpc(Config, ets, info, [rabbit_durable_queue, size])), - - process_flag(trap_exit, true), - ok = rabbit_ct_broker_helpers:kill_node(Config, 0), - ok = rabbit_ct_broker_helpers:start_node(Config, 0), - - %% After terminating a clean session by a node crash, we expect any session - %% state to be cleaned up on the server once the server comes back up. - ?assertEqual(0, rpc(Config, ets, info, [rabbit_durable_queue, size])). - quorum_clean_session_false(Config) -> Default = rpc(Config, reader_SUITE, get_env, []), rpc(Config, reader_SUITE, set_env, [quorum]), @@ -361,24 +218,109 @@ classic_clean_session_true(Config) -> classic_clean_session_false(Config) -> validate_durable_queue_type(Config, <<"classicCleanSessionFalse">>, false, rabbit_classic_queue). -will(Config) -> - Topic = <<"will/topic">>, - Msg = <<"will msg">>, - Publisher = connect(<<"will-publisher">>, Config, [{will_topic, Topic}, - {will_payload, Msg}, - {will_qos, 0}, - {will_retain, false}]), - timer:sleep(100), - [ServerPublisherPid] = all_connection_pids(Config), - - Subscriber = connect(<<"will-subscriber">>, Config), - {ok, _, _} = emqtt:subscribe(Subscriber, Topic, qos0), - - true = unlink(Publisher), - erlang:exit(ServerPublisherPid, test_will), - ok = expect_publishes(Topic, [Msg]), +%% "If the Client supplies a zero-byte ClientId with CleanSession set to 0, +%% the Server MUST respond to the CONNECT Packet with a CONNACK return code 0x02 +%% (Identifier rejected) and then close the Network Connection" [MQTT-3.1.3-8]. +non_clean_sess_empty_client_id(Config) -> + {ok, C} = emqtt:start_link( + [{clientid, <<>>}, + {clean_start, false}, + {proto_ver, v4}, + {host, "localhost"}, + {port, rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt)} + ]), + process_flag(trap_exit, true), + ?assertMatch({error, {client_identifier_not_valid, _}}, + emqtt:connect(C)), + ok = await_exit(C). + +event_authentication_failure(Config) -> + {ok, C} = emqtt:start_link( + [{username, <<"Trudy">>}, + {password, <<"fake-password">>}, + {host, "localhost"}, + {port, rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt)}, + {clientid, atom_to_binary(?FUNCTION_NAME)}, + {proto_ver, v4}]), + true = unlink(C), + + ok = rabbit_ct_broker_helpers:add_code_path_to_all_nodes(Config, event_recorder), + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + ok = gen_event:add_handler({rabbit_event, Server}, event_recorder, []), + + ?assertMatch({error, _}, emqtt:connect(C)), + + [E, _ConnectionClosedEvent] = util:get_events(Server), + util:assert_event_type(user_authentication_failure, E), + util:assert_event_prop([{name, <<"Trudy">>}, + {connection_type, network}], + E), + + ok = gen_event:delete_handler({rabbit_event, Server}, event_recorder, []). + +%% Test that queue type rabbit_mqtt_qos0_queue drops QoS 0 messages when its +%% max length is reached. +rabbit_mqtt_qos0_queue_overflow(Config) -> + Topic = atom_to_binary(?FUNCTION_NAME), + Msg = binary:copy(<<"x">>, 1000), + NumMsgs = 10_000, + + %% Provoke TCP back-pressure from client to server by using very small buffers. + Opts = [{tcp_opts, [{recbuf, 512}, + {buffer, 512}]}], + Sub = connect(<<"subscriber">>, Config, Opts), + {ok, _, [0]} = emqtt:subscribe(Sub, Topic, qos0), + [ServerConnectionPid] = all_connection_pids(Config), + + %% Suspend the receiving client such that it stops reading from its socket + %% causing TCP back-pressure to the server being applied. + true = erlang:suspend_process(Sub), + + %% Let's overflow the receiving server MQTT connection process + %% (i.e. the rabbit_mqtt_qos0_queue) by sending many large messages. + Pub = connect(<<"publisher">>, Config), + lists:foreach(fun(_) -> + ok = emqtt:publish(Pub, Topic, Msg, qos0) + end, lists:seq(1, NumMsgs)), + + %% Give the server some time to process (either send or drop) the messages. + timer:sleep(2000), + + %% Let's resume the receiving client to receive any remaining messages that did + %% not get dropped. + true = erlang:resume_process(Sub), + NumReceived = num_received(Topic, Msg, 0), + + {status, _, _, [_, _, _, _, Misc]} = sys:get_status(ServerConnectionPid), + [State] = [S || {data, [{"State", S}]} <- Misc], + #{proc_state := #{qos0_messages_dropped := NumDropped}} = State, + ct:pal("NumReceived=~b~nNumDropped=~b", [NumReceived, NumDropped]), + + %% We expect that + %% 1. all sent messages were either received or dropped + ?assertEqual(NumMsgs, NumReceived + NumDropped), + case rabbit_ct_helpers:is_mixed_versions(Config) of + false -> + %% 2. at least one message was dropped (otherwise our whole test case did not + %% test what it was supposed to test: that messages are dropped due to the + %% server being overflowed with messages while the client receives too slowly) + ?assert(NumDropped >= 1); + true -> + %% Feature flag rabbit_mqtt_qos0_queue is disabled. + ?assertEqual(0, NumDropped) + end, + %% 3. we received at least 1000 messages because everything below the default + %% of mailbox_soft_limit=1000 should not be dropped + ?assert(NumReceived >= 1000), - ok = emqtt:disconnect(Subscriber). + ok = emqtt:disconnect(Sub), + ok = emqtt:disconnect(Pub). -rpc(Config, M, F, A) -> - rpc(Config, 0, M, F, A). +num_received(Topic, Payload, N) -> + receive + {publish, #{topic := Topic, + payload := Payload}} -> + num_received(Topic, Payload, N + 1) + after 1000 -> + N + end. diff --git a/deps/rabbitmq_mqtt/test/retainer_SUITE.erl b/deps/rabbitmq_mqtt/test/retainer_SUITE.erl index 5a329fe6b702..3a2585e8fe5b 100644 --- a/deps/rabbitmq_mqtt/test/retainer_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/retainer_SUITE.erl @@ -8,7 +8,7 @@ -compile([export_all, nowarn_export_all]). -include_lib("common_test/include/ct.hrl"). --import(util, [expect_publishes/2, +-import(util, [expect_publishes/3, connect/3]). all() -> @@ -91,7 +91,7 @@ coerce_configuration_data(Config) -> {ok, _, _} = emqtt:subscribe(C, <<"TopicA">>, qos0), ok = emqtt:publish(C, <<"TopicA">>, <<"Payload">>), - ok = expect_publishes(<<"TopicA">>, [<<"Payload">>]), + ok = expect_publishes(C, <<"TopicA">>, [<<"Payload">>]), ok = emqtt:disconnect(C). @@ -105,7 +105,7 @@ should_translate_amqp2mqtt_on_publish(Config) -> %% there's an active consumer {ok, _, _} = emqtt:subscribe(C, <<"TopicA/Device.Field">>, qos1), ok = emqtt:publish(C, <<"TopicA/Device.Field">>, #{}, <<"Payload">>, [{retain, true}]), - ok = expect_publishes(<<"TopicA/Device/Field">>, [<<"Payload">>]), + ok = expect_publishes(C, <<"TopicA/Device/Field">>, [<<"Payload">>]), ok = emqtt:disconnect(C). %% ------------------------------------------------------------------- @@ -118,7 +118,7 @@ should_translate_amqp2mqtt_on_retention(Config) -> %% publish with retain = true before a consumer comes around ok = emqtt:publish(C, <<"TopicA/Device.Field">>, #{}, <<"Payload">>, [{retain, true}]), {ok, _, _} = emqtt:subscribe(C, <<"TopicA/Device.Field">>, qos1), - ok = expect_publishes(<<"TopicA/Device/Field">>, [<<"Payload">>]), + ok = expect_publishes(C, <<"TopicA/Device/Field">>, [<<"Payload">>]), ok = emqtt:disconnect(C). %% ------------------------------------------------------------------- @@ -130,7 +130,7 @@ should_translate_amqp2mqtt_on_retention_search(Config) -> C = connect(<<"simpleClientRetainer">>, Config, [{ack_timeout, 1}]), ok = emqtt:publish(C, <<"TopicA/Device.Field">>, #{}, <<"Payload">>, [{retain, true}]), {ok, _, _} = emqtt:subscribe(C, <<"TopicA/Device/Field">>, qos1), - ok = expect_publishes(<<"TopicA/Device/Field">>, [<<"Payload">>]), + ok = expect_publishes(C, <<"TopicA/Device/Field">>, [<<"Payload">>]), ok = emqtt:disconnect(C). does_not_retain(Config) -> @@ -143,4 +143,4 @@ does_not_retain(Config) -> after 1000 -> ok end, - ok = emqtt:disconnect(C). \ No newline at end of file + ok = emqtt:disconnect(C). diff --git a/deps/rabbitmq_mqtt/test/integration_SUITE.erl b/deps/rabbitmq_mqtt/test/shared_SUITE.erl similarity index 69% rename from deps/rabbitmq_mqtt/test/integration_SUITE.erl rename to deps/rabbitmq_mqtt/test/shared_SUITE.erl index 7f60f9a573e2..6a4897f9e4fb 100644 --- a/deps/rabbitmq_mqtt/test/integration_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/shared_SUITE.erl @@ -3,11 +3,13 @@ %% file, You can obtain one at https://mozilla.org/MPL/2.0/. %% %% Copyright (c) 2007-2022 VMware, Inc. or its affiliates. All rights reserved. -%% --module(integration_SUITE). + +%% Test suite shared between rabbitmq_mqtt and rabbitmq_web_mqtt. +-module(shared_SUITE). -compile([export_all, nowarn_export_all]). +-include_lib("common_test/include/ct.hrl"). -include_lib("eunit/include/eunit.hrl"). -include_lib("amqp_client/include/amqp_client.hrl"). -include_lib("rabbitmq_ct_helpers/include/rabbit_assert.hrl"). @@ -15,63 +17,82 @@ -import(rabbit_ct_broker_helpers, [rabbitmqctl_list/3, + rpc/4, rpc/5, rpc_all/4, - get_node_config/3]). + get_node_config/3, + drain_node/2, + revive_node/2 + ]). -import(rabbit_ct_helpers, [eventually/3, eventually/1]). -import(util, [all_connection_pids/1, - get_global_counters/2, - get_global_counters/3, - get_global_counters/4, - expect_publishes/2, - connect/2, - connect/3]). + get_global_counters/2, get_global_counters/3, get_global_counters/4, + expect_publishes/3, + connect/2, connect/3, connect/4, + get_events/1, assert_event_type/2, assert_event_prop/2, + await_exit/1, await_exit/2, + publish_qos1_timeout/4]). -import(rabbit_mgmt_test_util, [http_get/2, http_delete/3]). --define(MANAGEMENT_PLUGIN_TESTS, - [management_plugin_connection, - management_plugin_enable]). - all() -> [ - {group, cluster_size_1}, - {group, cluster_size_3} + {group, mqtt} + ,{group, web_mqtt} ]. groups() -> + [ + {mqtt, [], subgroups()} + ,{web_mqtt, [], subgroups()} + ]. + +subgroups() -> [ {cluster_size_1, [], [ - %% separate node so global counters start from 0 - {global_counters, [], [global_counters_v3, global_counters_v4]}, - {tests, [], tests()} + {global_counters, [], + [ + global_counters_v3, + global_counters_v4 + ]}, + {tests, [], + [ + many_qos1_messages + ] ++ tests()} ]}, {cluster_size_3, [], - [queue_down_qos1, + [ + queue_down_qos1, consuming_classic_mirrored_queue_down, consuming_classic_queue_down, flow_classic_mirrored_queue, flow_quorum_queue, flow_stream, rabbit_mqtt_qos0_queue, - cli_list_queues - ] ++ tests() - } + cli_list_queues, + maintenance + ] ++ tests()} ]. tests() -> - ?MANAGEMENT_PLUGIN_TESTS ++ - [delete_create_queue + [ + management_plugin_connection + ,management_plugin_enable + ,disconnect + ,pubsub_shared_connection + ,pubsub_separate_connections + ,will_with_disconnect + ,will_without_disconnect + ,delete_create_queue ,quorum_queue_rejects ,publish_to_all_queue_types_qos0 ,publish_to_all_queue_types_qos1 ,events - ,event_authentication_failure ,internal_event_handler ,non_clean_sess_disconnect ,subscribe_same_topic_same_qos @@ -79,9 +100,12 @@ tests() -> ,subscribe_multiple ,large_message_mqtt_to_mqtt ,large_message_amqp_to_mqtt - ,many_qos1_messages - ,rabbit_mqtt_qos0_queue_overflow - ,non_clean_sess_empty_client_id + ,keepalive + ,keepalive_turned_off + ,duplicate_client_id + ,block + ,clean_session_disconnect_client + ,clean_session_kill_node ]. suite() -> @@ -98,6 +122,11 @@ init_per_suite(Config) -> end_per_suite(Config) -> rabbit_ct_helpers:run_teardown_steps(Config). +init_per_group(mqtt, Config) -> + rabbit_ct_helpers:set_config(Config, {websocket, false}); +init_per_group(web_mqtt, Config) -> + rabbit_ct_helpers:set_config(Config, {websocket, true}); + init_per_group(cluster_size_1, Config) -> rabbit_ct_helpers:set_config(Config, [{rmq_nodes_count, 1}]); init_per_group(cluster_size_3 = Group, Config) -> @@ -109,20 +138,24 @@ init_per_group(Group, Config) init_per_group0(Group, Config). init_per_group0(Group, Config0) -> + Suffix = lists:flatten(io_lib:format("~s_websocket_~w", [Group, ?config(websocket, Config0)])), Config1 = rabbit_ct_helpers:set_config( Config0, - [{rmq_nodename_suffix, Group}, + [{rmq_nodename_suffix, Suffix}, {rmq_extra_tcp_ports, [tcp_port_mqtt_extra, tcp_port_mqtt_tls_extra]}]), - Config = rabbit_ct_helpers:run_steps( + Config = rabbit_ct_helpers:merge_app_env( Config1, - rabbit_ct_broker_helpers:setup_steps() ++ - rabbit_ct_client_helpers:setup_steps()), - Result = rpc_all(Config, application, set_env, [rabbit, classic_queue_default_version, 2]), - ?assert(lists:all(fun(R) -> R =:= ok end, Result)), - Config. + {rabbit, [{classic_queue_default_version, 2}]}), + rabbit_ct_helpers:run_steps( + Config, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps()). -end_per_group(cluster_size_1, Config) -> +end_per_group(G, Config) + when G =:= mqtt; + G =:= web_mqtt; + G =:= cluster_size_1 -> Config; end_per_group(_, Config) -> rabbit_ct_helpers:run_teardown_steps( @@ -130,35 +163,120 @@ end_per_group(_, Config) -> rabbit_ct_client_helpers:teardown_steps() ++ rabbit_ct_broker_helpers:teardown_steps()). +init_per_testcase(Testcase = maintenance, Config) -> + case rabbit_ct_helpers:is_mixed_versions() of + true -> + {skip, "maintenance mode wrongly closes cluster-wide MQTT connections " + "in RMQ < 3.11.2 and < 3.10.10"}; + false -> + init_per_testcase0(Testcase, Config) + end; +init_per_testcase(T, Config) + when T =:= management_plugin_connection; + T =:= management_plugin_enable -> + ok = inets:start(), + init_per_testcase0(T, Config); init_per_testcase(Testcase, Config) -> - maybe_start_inets(Testcase), + init_per_testcase0(Testcase, Config). + +init_per_testcase0(Testcase, Config) -> + Nodes = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + [ok = rabbit_ct_broker_helpers:enable_plugin(Config, N, rabbitmq_web_mqtt) || N <- Nodes], rabbit_ct_helpers:testcase_started(Config, Testcase). +end_per_testcase(T, Config) + when T =:= management_plugin_connection; + T =:= management_plugin_enable -> + ok = inets:stop(), + end_per_testcase0(T, Config); end_per_testcase(Testcase, Config) -> - maybe_stop_inets(Testcase), + end_per_testcase0(Testcase, Config). + +end_per_testcase0(Testcase, Config) -> rabbit_ct_client_helpers:close_channels_and_connection(Config, 0), rabbit_ct_helpers:testcase_finished(Config, Testcase). -maybe_start_inets(Testcase) -> - case lists:member(Testcase, ?MANAGEMENT_PLUGIN_TESTS) of - true -> - ok = inets:start(); - false -> - ok - end. - -maybe_stop_inets(Testcase) -> - case lists:member(Testcase, ?MANAGEMENT_PLUGIN_TESTS) of - true -> - ok = inets:stop(); - false -> - ok - end. - %% ------------------------------------------------------------------- %% Testsuite cases %% ------------------------------------------------------------------- +disconnect(Config) -> + C = connect(?FUNCTION_NAME, Config), + eventually(?_assertEqual(1, length(all_connection_pids(Config)))), + process_flag(trap_exit, true), + ok = emqtt:disconnect(C), + await_exit(C, normal), + eventually(?_assertEqual([], all_connection_pids(Config))), + ok. + +pubsub_shared_connection(Config) -> + C = connect(?FUNCTION_NAME, Config), + + Topic = <<"/topic/test-mqtt">>, + {ok, _, [1]} = emqtt:subscribe(C, Topic, qos1), + + Payload = <<"a\x00a">>, + ?assertMatch({ok, #{packet_id := _, + reason_code := 0, + reason_code_name := success + }}, + emqtt:publish(C, Topic, Payload, [{qos, 1}])), + ok = expect_publishes(C, Topic, [Payload]), + ok = emqtt:disconnect(C). + +pubsub_separate_connections(Config) -> + Pub = connect(<<(atom_to_binary(?FUNCTION_NAME))/binary, "_publisher">>, Config), + Sub = connect(<<(atom_to_binary(?FUNCTION_NAME))/binary, "_subscriber">>, Config), + + Topic = <<"/topic/test-mqtt">>, + {ok, _, [1]} = emqtt:subscribe(Sub, Topic, qos1), + + Payload = <<"a\x00a">>, + ?assertMatch({ok, #{packet_id := _, + reason_code := 0, + reason_code_name := success + }}, + emqtt:publish(Pub, Topic, Payload, [{qos, 1}])), + ok = expect_publishes(Sub, Topic, [Payload]), + ok = emqtt:disconnect(Pub), + ok = emqtt:disconnect(Sub). + +will_with_disconnect(Config) -> + LastWillTopic = <<"/topic/last-will">>, + LastWillMsg = <<"last will message">>, + PubOpts = [{will_topic, LastWillTopic}, + {will_payload, LastWillMsg}, + {will_qos, 1}], + Pub = connect(<<(atom_to_binary(?FUNCTION_NAME))/binary, "_publisher">>, Config, PubOpts), + Sub = connect(<<(atom_to_binary(?FUNCTION_NAME))/binary, "_subscriber">>, Config), + {ok, _, [1]} = emqtt:subscribe(Sub, LastWillTopic, qos1), + + %% Client sends DISCONNECT packet. Therefore, will message should not be sent. + ok = emqtt:disconnect(Pub), + ?assertEqual({publish_not_received, LastWillMsg}, + expect_publishes(Sub, LastWillTopic, [LastWillMsg])), + + ok = emqtt:disconnect(Sub). + +will_without_disconnect(Config) -> + LastWillTopic = <<"/topic/last-will">>, + LastWillMsg = <<"last will message">>, + PubOpts = [{will_topic, LastWillTopic}, + {will_payload, LastWillMsg}, + {will_qos, 1}], + Pub = connect(<<(atom_to_binary(?FUNCTION_NAME))/binary, "_publisher">>, Config, PubOpts), + timer:sleep(100), + [ServerPublisherPid] = all_connection_pids(Config), + Sub = connect(<<(atom_to_binary(?FUNCTION_NAME))/binary, "_subscriber">>, Config), + {ok, _, [1]} = emqtt:subscribe(Sub, LastWillTopic, qos1), + + %% Client does not send DISCONNECT packet. Therefore, will message should be sent. + unlink(Pub), + erlang:exit(ServerPublisherPid, test_will), + ?assertEqual(ok, expect_publishes(Sub, LastWillTopic, [LastWillMsg])), + + ok = emqtt:disconnect(Sub). + quorum_queue_rejects(Config) -> {_Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0), Name = atom_to_binary(?FUNCTION_NAME), @@ -267,7 +385,7 @@ flow_stream(Config) -> flow(Config, {App, Par, Val}, QueueType) when is_binary(QueueType) -> - {ok, DefaultVal} = rpc(Config, 0, application, get_env, [App, Par]), + {ok, DefaultVal} = rpc(Config, application, get_env, [App, Par]), Result = rpc_all(Config, application, set_env, [App, Par, Val]), ?assert(lists:all(fun(R) -> R =:= ok end, Result)), @@ -278,7 +396,7 @@ flow(Config, {App, Par, Val}, QueueType) NumMsgs = 1000, C = connect(?FUNCTION_NAME, Config, [{retry_interval, 600}, - {max_inflight, NumMsgs}]), + {max_inflight, NumMsgs}]), TestPid = self(), lists:foreach( fun(N) -> @@ -316,7 +434,11 @@ events(Config) -> E0), assert_event_type(connection_created, E1), [ConnectionPid] = all_connection_pids(Config), - ExpectedConnectionProps = [{protocol, {'MQTT', {3,1,1}}}, + Proto = case ?config(websocket, Config) of + true -> 'Web MQTT'; + false -> 'MQTT' + end, + ExpectedConnectionProps = [{protocol, {Proto, {3,1,1}}}, {node, Server}, {vhost, <<"/">>}, {user, <<"guest">>}, @@ -384,31 +506,6 @@ events(Config) -> ok = gen_event:delete_handler({rabbit_event, Server}, event_recorder, []). -event_authentication_failure(Config) -> - P = get_node_config(Config, 0, tcp_port_mqtt), - ClientId = atom_to_binary(?FUNCTION_NAME), - {ok, C} = emqtt:start_link([{username, <<"Trudy">>}, - {password, <<"fake-password">>}, - {host, "localhost"}, - {port, P}, - {clientid, ClientId}, - {proto_ver, v4}]), - true = unlink(C), - - ok = rabbit_ct_broker_helpers:add_code_path_to_all_nodes(Config, event_recorder), - Server = get_node_config(Config, 0, nodename), - ok = gen_event:add_handler({rabbit_event, Server}, event_recorder, []), - - ?assertMatch({error, _}, emqtt:connect(C)), - - [E, _ConnectionClosedEvent] = get_events(Server), - assert_event_type(user_authentication_failure, E), - assert_event_prop([{name, <<"Trudy">>}, - {connection_type, network}], - E), - - ok = gen_event:delete_handler({rabbit_event, Server}, event_recorder, []). - internal_event_handler(Config) -> Server = get_node_config(Config, 0, nodename), ok = gen_event:call({rabbit_event, Server}, rabbit_mqtt_internal_event_handler, ignored_request, 1000). @@ -435,9 +532,9 @@ global_counters(Config, ProtoVer) -> ok = emqtt:publish(C, <<"no/queue/bound">>, <<"msg-dropped">>, qos0), {ok, _} = emqtt:publish(C, <<"no/queue/bound">>, <<"msg-returned">>, qos1), - ok = expect_publishes(Topic0, [<<"testm0">>]), - ok = expect_publishes(Topic1, [<<"testm1">>]), - ok = expect_publishes(Topic2, [<<"testm2">>]), + ok = expect_publishes(C, Topic0, [<<"testm0">>]), + ok = expect_publishes(C, Topic1, [<<"testm1">>]), + ok = expect_publishes(C, Topic2, [<<"testm2">>]), ?assertEqual(#{publishers => 1, consumers => 1, @@ -524,7 +621,7 @@ queue_down_qos1(Config) -> %% and failover consumption when the classic mirrored queue leader fails. consuming_classic_mirrored_queue_down(Config) -> [Server1, Server2, _Server3] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), - Topic = PolicyName = atom_to_binary(?FUNCTION_NAME), + ClientId = Topic = PolicyName = atom_to_binary(?FUNCTION_NAME), ok = rabbit_ct_broker_helpers:set_policy( Config, Server1, PolicyName, <<".*">>, <<"queues">>, @@ -532,38 +629,30 @@ consuming_classic_mirrored_queue_down(Config) -> {<<"queue-master-locator">>, <<"client-local">>}]), %% Declare queue leader on Server1. - C1 = connect(?FUNCTION_NAME, Config, [{clean_start, false}]), + C1 = connect(ClientId, Config, Server1, [{clean_start, false}]), {ok, _, _} = emqtt:subscribe(C1, Topic, qos1), ok = emqtt:disconnect(C1), %% Consume from Server2. - Options = [{host, "localhost"}, - {port, get_node_config(Config, Server2, tcp_port_mqtt)}, - {clientid, atom_to_binary(?FUNCTION_NAME)}, - {proto_ver, v4}], - {ok, C2} = emqtt:start_link([{clean_start, false} | Options]), - {ok, _} = emqtt:connect(C2), + C2 = connect(ClientId, Config, Server2, [{clean_start, false}]), %% Sanity check that consumption works. {ok, _} = emqtt:publish(C2, Topic, <<"m1">>, qos1), - ok = expect_publishes(Topic, [<<"m1">>]), + ok = expect_publishes(C2, Topic, [<<"m1">>]), %% Let's stop the queue leader node. ok = rabbit_ct_broker_helpers:stop_node(Config, Server1), %% Consumption should continue to work. {ok, _} = emqtt:publish(C2, Topic, <<"m2">>, qos1), - ok = expect_publishes(Topic, [<<"m2">>]), + ok = expect_publishes(C2, Topic, [<<"m2">>]), %% Cleanup ok = emqtt:disconnect(C2), ok = rabbit_ct_broker_helpers:start_node(Config, Server1), ?assertMatch([_Q], rpc(Config, Server1, rabbit_amqqueue, list, [])), - %% "When a Client has determined that it has no further use for the session it should do a - %% final connect with CleanSession set to 1 and then disconnect." - {ok, C3} = emqtt:start_link([{clean_start, true} | Options]), - {ok, _} = emqtt:connect(C3), + C3 = connect(ClientId, Config, Server2, [{clean_start, true}]), ok = emqtt:disconnect(C3), ?assertEqual([], rpc(Config, Server1, rabbit_amqqueue, list, [])), @@ -575,19 +664,14 @@ consuming_classic_queue_down(Config) -> ClientId = Topic = atom_to_binary(?FUNCTION_NAME), %% Declare classic queue on Server1. - C1 = connect(?FUNCTION_NAME, Config, [{clean_start, false}]), + C1 = connect(ClientId, Config, [{clean_start, false}]), {ok, _, _} = emqtt:subscribe(C1, Topic, qos1), ok = emqtt:disconnect(C1), - ProtoVer = v4, %% Consume from Server3. - Options = [{host, "localhost"}, - {port, get_node_config(Config, Server3, tcp_port_mqtt)}, - {clientid, ClientId}, - {proto_ver, ProtoVer}], - {ok, C2} = emqtt:start_link([{clean_start, false} | Options]), - {ok, _} = emqtt:connect(C2), + C2 = connect(ClientId, Config, Server3, [{clean_start, false}]), + ProtoVer = v4, ?assertMatch(#{consumers := 1}, get_global_counters(Config, ProtoVer, Server3)), @@ -600,17 +684,11 @@ consuming_classic_queue_down(Config) -> eventually(?_assertMatch(#{consumers := 0}, get_global_counters(Config, ProtoVer, Server3)), 1000, 5), - receive - {'EXIT', C2, _} -> - ok - after 3000 -> - ct:fail("MQTT connection should have been closed") - end, + await_exit(C2), %% Cleanup ok = rabbit_ct_broker_helpers:start_node(Config, Server1), - {ok, C3} = emqtt:start_link([{clean_start, true} | Options]), - {ok, _} = emqtt:connect(C3), + C3 = connect(ClientId, Config, Server3, [{clean_start, true}]), ok = emqtt:disconnect(C3), ?assertEqual([], rpc(Config, Server1, rabbit_amqqueue, list, [])), @@ -699,7 +777,7 @@ non_clean_sess_disconnect(Config) -> get_global_counters(Config, ProtoVer)), Msg = <<"msg">>, {ok, _} = emqtt:publish(C2, Topic, Msg, qos1), - {publish_not_received, Msg} = expect_publishes(Topic, [Msg]), + {publish_not_received, Msg} = expect_publishes(C2, Topic, [Msg]), ok = emqtt:disconnect(C2), %% connect with clean sess true to clean up @@ -720,9 +798,9 @@ subscribe_same_topic_same_qos(Config) -> {ok, _} = emqtt:publish(C, Topic, <<"msg2">>, qos1), %% "Any existing retained messages matching the Topic Filter MUST be re-sent" [MQTT-3.8.4-3] - ok = expect_publishes(Topic, [<<"retained">>, <<"msg1">>, - <<"retained">>, <<"msg2">> - ]), + ok = expect_publishes(C, Topic, [<<"retained">>, <<"msg1">>, + <<"retained">>, <<"msg2">> + ]), ok = emqtt:disconnect(C). subscribe_same_topic_different_qos(Config) -> @@ -742,12 +820,12 @@ subscribe_same_topic_different_qos(Config) -> {ok, _} = emqtt:publish(C, Topic, <<"msg3">>, qos1), %% "Any existing retained messages matching the Topic Filter MUST be re-sent" [MQTT-3.8.4-3] - ok = expect_publishes(Topic, [<<"retained">>, <<"msg1">>, - <<"retained">>, <<"msg2">>, - <<"retained">>, <<"msg3">>]), + ok = expect_publishes(C, Topic, [<<"retained">>, <<"msg1">>, + <<"retained">>, <<"msg2">>, + <<"retained">>, <<"msg3">>]), %% There should be exactly one consumer for each queue: qos0 and qos1 - Consumers = rpc(Config, 0, rabbit_amqqueue, consumers_all, [<<"/">>]), + Consumers = rpc(Config, rabbit_amqqueue, consumers_all, [<<"/">>]), ?assertEqual(2, length(Consumers)), ok = emqtt:disconnect(C), @@ -770,7 +848,7 @@ large_message_mqtt_to_mqtt(Config) -> Payload0 = binary:copy(<<"x">>, 8_000_000), Payload = <>, {ok, _} = emqtt:publish(C, Topic, Payload, qos1), - ok = expect_publishes(Topic, [Payload]), + ok = expect_publishes(C, Topic, [Payload]), ok = emqtt:disconnect(C). large_message_amqp_to_mqtt(Config) -> @@ -785,21 +863,21 @@ large_message_amqp_to_mqtt(Config) -> #'basic.publish'{exchange = <<"amq.topic">>, routing_key = Topic}, #amqp_msg{payload = Payload}), - ok = expect_publishes(Topic, [Payload]), + ok = expect_publishes(C, Topic, [Payload]), ok = emqtt:disconnect(C). %% Packet identifier is a non zero two byte integer. %% Test that the server wraps around the packet identifier. many_qos1_messages(Config) -> Topic = ClientId = atom_to_binary(?FUNCTION_NAME), - C = connect(ClientId, Config), + C = connect(ClientId, Config, 0, [{retry_interval, 600}]), {ok, _, [1]} = emqtt:subscribe(C, {Topic, qos1}), NumMsgs = 16#ffff + 100, - Payload = <<>>, - lists:foreach(fun(_) -> - {ok, _} = emqtt:publish(C, Topic, Payload, qos1) - end, lists:seq(1, NumMsgs)), - ?assertEqual(NumMsgs, num_received(Topic, Payload, 0)), + Payloads = lists:map(fun integer_to_binary/1, lists:seq(1, NumMsgs)), + lists:foreach(fun(P) -> + {ok, _} = emqtt:publish(C, Topic, P, qos1) + end, Payloads), + expect_publishes(C, Topic, Payloads), ok = emqtt:disconnect(C). %% This test is mostly interesting in mixed version mode where feature flag @@ -812,74 +890,11 @@ rabbit_mqtt_qos0_queue(Config) -> {ok, _, [0]} = emqtt:subscribe(Sub, Topic, qos0), %% Place MQTT publisher process on old node in mixed version. - {ok, Pub} = emqtt:start_link( - [{port, get_node_config(Config, 1, tcp_port_mqtt)}, - {clientid, <<"publisher">>}, - {proto_ver, v4} - ]), - {ok, _Properties} = emqtt:connect(Pub), + Pub = connect(<<"publisher">>, Config, 1, []), Msg = <<"msg">>, ok = emqtt:publish(Pub, Topic, Msg, qos0), - ok = expect_publishes(Topic, [Msg]), - - ok = emqtt:disconnect(Sub), - ok = emqtt:disconnect(Pub). - -%% Test that queue type rabbit_mqtt_qos0_queue drops QoS 0 messages when its -%% max length is reached. -rabbit_mqtt_qos0_queue_overflow(Config) -> - Topic = atom_to_binary(?FUNCTION_NAME), - Msg = binary:copy(<<"x">>, 1000), - NumMsgs = 10_000, - - %% Provoke TCP back-pressure from client to server by using very small buffers. - Opts = [{tcp_opts, [{recbuf, 512}, - {buffer, 512}]}], - Sub = connect(<<"subscriber">>, Config, Opts), - {ok, _, [0]} = emqtt:subscribe(Sub, Topic, qos0), - [ServerConnectionPid] = all_connection_pids(Config), - - %% Suspend the receiving client such that it stops reading from its socket - %% causing TCP back-pressure to the server being applied. - true = erlang:suspend_process(Sub), - - %% Let's overflow the receiving server MQTT connection process - %% (i.e. the rabbit_mqtt_qos0_queue) by sending many large messages. - Pub = connect(<<"publisher">>, Config), - lists:foreach(fun(_) -> - ok = emqtt:publish(Pub, Topic, Msg, qos0) - end, lists:seq(1, NumMsgs)), - - %% Give the server some time to process (either send or drop) the messages. - timer:sleep(2000), - - %% Let's resume the receiving client to receive any remaining messages that did - %% not get dropped. - true = erlang:resume_process(Sub), - NumReceived = num_received(Topic, Msg, 0), - - {status, _, _, [_, _, _, _, Misc]} = sys:get_status(ServerConnectionPid), - [State] = [S || {data, [{"State", S}]} <- Misc], - #{proc_state := #{qos0_messages_dropped := NumDropped}} = State, - ct:pal("NumReceived=~b~nNumDropped=~b", [NumReceived, NumDropped]), - - %% We expect that - %% 1. all sent messages were either received or dropped - ?assertEqual(NumMsgs, NumReceived + NumDropped), - case rabbit_ct_helpers:is_mixed_versions(Config) of - false -> - %% 2. at least one message was dropped (otherwise our whole test case did not - %% test what it was supposed to test: that messages are dropped due to the - %% server being overflowed with messages while the client receives too slowly) - ?assert(NumDropped >= 1); - true -> - %% Feature flag rabbit_mqtt_qos0_queue is disabled. - ?assertEqual(0, NumDropped) - end, - %% 3. we received at least 1000 messages because everything below the default - %% of mailbox_soft_limit=1000 should not be dropped - ?assert(NumReceived >= 1000), + ok = expect_publishes(Sub, Topic, [Msg]), ok = emqtt:disconnect(Sub), ok = emqtt:disconnect(Pub). @@ -901,12 +916,7 @@ management_plugin_connection(Config) -> http_delete(Config, "/connections/" ++ binary_to_list(uri_string:quote((ConnectionName))), ?NO_CONTENT), - receive - {'EXIT', C, _} -> - ok - after 5000 -> - ct:fail("server did not close connection") - end, + await_exit(C), ?assertEqual([], http_get(Config, "/connections")), eventually(?_assertEqual([], all_connection_pids(Config)), 500, 3). @@ -915,8 +925,8 @@ management_plugin_enable(Config) -> ok = rabbit_ct_broker_helpers:disable_plugin(Config, 0, rabbitmq_management), ok = rabbit_ct_broker_helpers:disable_plugin(Config, 0, rabbitmq_management_agent), - %% If the MQTT connection is established **before** the management plugin is enabled, - %% the management plugin should still list the MQTT connection. + %% If the (web) MQTT connection is established **before** the management plugin is enabled, + %% the management plugin should still list the (web) MQTT connection. C = connect(?FUNCTION_NAME, Config), ok = rabbit_ct_broker_helpers:enable_plugin(Config, 0, rabbitmq_management_agent), ok = rabbit_ct_broker_helpers:enable_plugin(Config, 0, rabbitmq_management), @@ -952,36 +962,177 @@ cli_list_queues(Config) -> ok = emqtt:disconnect(C). -%% "If the Client supplies a zero-byte ClientId with CleanSession set to 0, -%% the Server MUST respond to the CONNECT Packet with a CONNACK return code 0x02 -%% (Identifier rejected) and then close the Network Connection" [MQTT-3.1.3-8]. -non_clean_sess_empty_client_id(Config) -> - {ok, C} = emqtt:start_link([{clientid, <<>>}, - {clean_start, false}, - {proto_ver, v4}, - {host, "localhost"}, - {port, get_node_config(Config, 0, tcp_port_mqtt)} - ]), +maintenance(Config) -> + C0 = connect(<<"client-0">>, Config, 0, []), + C1a = connect(<<"client-1a">>, Config, 1, []), + C1b = connect(<<"client-1b">>, Config, 1, []), + + timer:sleep(500), + + ok = drain_node(Config, 2), + ok = revive_node(Config, 2), + timer:sleep(500), + [?assert(erlang:is_process_alive(C)) || C <- [C0, C1a, C1b]], + process_flag(trap_exit, true), - ?assertMatch({error, {client_identifier_not_valid, _}}, - emqtt:connect(C)), - receive {'EXIT', C, _} -> ok - after 500 -> ct:fail("server did not close connection") - end. + ok = drain_node(Config, 1), + [await_exit(Pid) || Pid <- [C1a, C1b]], + ok = revive_node(Config, 1), + ?assert(erlang:is_process_alive(C0)), + + ok = drain_node(Config, 0), + await_exit(C0), + ok = revive_node(Config, 0). + +keepalive(Config) -> + KeepaliveSecs = 1, + KeepaliveMs = timer:seconds(KeepaliveSecs), + ProtoVer = v4, + WillTopic = <<"will/topic">>, + WillPayload = <<"will-payload">>, + C1 = connect(?FUNCTION_NAME, Config, [{keepalive, KeepaliveSecs}, + {proto_ver, ProtoVer}, + {will_topic, WillTopic}, + {will_payload, WillPayload}, + {will_retain, true}, + {will_qos, 0}]), + ok = emqtt:publish(C1, <<"ignored">>, <<"msg">>), + + %% Connection should stay up when client sends PING requests. + timer:sleep(KeepaliveMs), + ?assertMatch(#{publishers := 1}, + util:get_global_counters(Config, ProtoVer)), + + %% Mock the server socket to not have received any bytes. + rabbit_ct_broker_helpers:setup_meck(Config), + Mod = rabbit_net, + ok = rpc(Config, meck, new, [Mod, [no_link, passthrough]]), + ok = rpc(Config, meck, expect, [Mod, getstat, 2, {ok, [{recv_oct, 999}]} ]), + process_flag(trap_exit, true), + + %% We expect the server to respect the keepalive closing the connection. + eventually(?_assertMatch(#{publishers := 0}, + util:get_global_counters(Config, ProtoVer)), + KeepaliveMs, 3 * KeepaliveSecs), + await_exit(C1), + + true = rpc(Config, meck, validate, [Mod]), + ok = rpc(Config, meck, unload, [Mod]), + + C2 = connect(<<"client2">>, Config), + {ok, _, [0]} = emqtt:subscribe(C2, WillTopic), + receive {publish, #{client_pid := C2, + dup := false, + qos := 0, + retain := true, + topic := WillTopic, + payload := WillPayload}} -> ok + after 3000 -> ct:fail("missing will") + end, + ok = emqtt:disconnect(C2). + +keepalive_turned_off(Config) -> + %% "A Keep Alive value of zero (0) has the effect of turning off the keep alive mechanism." + KeepaliveSecs = 0, + C = connect(?FUNCTION_NAME, Config, [{keepalive, KeepaliveSecs}]), + ok = emqtt:publish(C, <<"TopicB">>, <<"Payload">>), + + %% Mock the server socket to not have received any bytes. + rabbit_ct_broker_helpers:setup_meck(Config), + Mod = rabbit_net, + ok = rpc(Config, meck, new, [Mod, [no_link, passthrough]]), + ok = rpc(Config, meck, expect, [Mod, getstat, 2, {ok, [{recv_oct, 999}]} ]), + + rabbit_ct_helpers:consistently(?_assert(erlang:is_process_alive(C))), + + true = rpc(Config, meck, validate, [Mod]), + ok = rpc(Config, meck, unload, [Mod]), + ok = emqtt:disconnect(C). + +duplicate_client_id(Config) -> + DuplicateClientId = ?FUNCTION_NAME, + C1 = connect(DuplicateClientId, Config), + eventually(?_assertEqual(1, length(all_connection_pids(Config)))), + + process_flag(trap_exit, true), + C2 = connect(DuplicateClientId, Config), + await_exit(C1), + timer:sleep(200), + ?assertEqual(1, length(all_connection_pids(Config))), + + ok = emqtt:disconnect(C2). + +block(Config) -> + Topic = ClientId = atom_to_binary(?FUNCTION_NAME), + C = connect(ClientId, Config), + + {ok, _, _} = emqtt:subscribe(C, Topic), + {ok, _} = emqtt:publish(C, Topic, <<"Not blocked yet">>, [{qos, 1}]), + + ok = rpc(Config, 0, vm_memory_monitor, set_vm_memory_high_watermark, [0.00000001]), + %% Let it block + timer:sleep(100), + + %% Blocked, but still will publish when unblocked + puback_timeout = publish_qos1_timeout(C, Topic, <<"Now blocked">>, 1000), + puback_timeout = publish_qos1_timeout(C, Topic, <<"Still blocked">>, 1000), + + %% Unblock + rpc(Config, 0, vm_memory_monitor, set_vm_memory_high_watermark, [0.4]), + ok = expect_publishes(C, Topic, [<<"Not blocked yet">>, + <<"Now blocked">>, + <<"Still blocked">>]), + ok = emqtt:disconnect(C). + +clean_session_disconnect_client(Config) -> + C = connect(?FUNCTION_NAME, Config), + {ok, _, _} = emqtt:subscribe(C, <<"topic0">>, qos0), + {ok, _, _} = emqtt:subscribe(C, <<"topic1">>, qos1), + QsQos0 = rpc(Config, rabbit_amqqueue, list_by_type, [rabbit_mqtt_qos0_queue]), + QsClassic = rpc(Config, rabbit_amqqueue, list_by_type, [rabbit_classic_queue]), + case rabbit_ct_helpers:is_mixed_versions(Config) of + false -> + ?assertEqual(1, length(QsQos0)), + ?assertEqual(1, length(QsClassic)); + true -> + ?assertEqual(0, length(QsQos0)), + ?assertEqual(2, length(QsClassic)) + end, + + ok = emqtt:disconnect(C), + %% After terminating a clean session, we expect any session state to be cleaned up on the server. + timer:sleep(200), %% Give some time to clean up exclusive classic queue. + L = rpc(Config, rabbit_amqqueue, list, []), + ?assertEqual(0, length(L)). + +clean_session_kill_node(Config) -> + C = connect(?FUNCTION_NAME, Config), + {ok, _, _} = emqtt:subscribe(C, <<"topic0">>, qos0), + {ok, _, _} = emqtt:subscribe(C, <<"topic1">>, qos1), + QsQos0 = rpc(Config, rabbit_amqqueue, list_by_type, [rabbit_mqtt_qos0_queue]), + QsClassic = rpc(Config, rabbit_amqqueue, list_by_type, [rabbit_classic_queue]), + case rabbit_ct_helpers:is_mixed_versions(Config) of + false -> + ?assertEqual(1, length(QsQos0)), + ?assertEqual(1, length(QsClassic)); + true -> + ?assertEqual(0, length(QsQos0)), + ?assertEqual(2, length(QsClassic)) + end, + ?assertEqual(2, rpc(Config, ets, info, [rabbit_durable_queue, size])), + + unlink(C), + ok = rabbit_ct_broker_helpers:kill_node(Config, 0), + ok = rabbit_ct_broker_helpers:start_node(Config, 0), + + %% After terminating a clean session by a node crash, we expect any session + %% state to be cleaned up on the server once the server comes back up. + ?assertEqual(0, rpc(Config, ets, info, [rabbit_durable_queue, size])). %% ------------------------------------------------------------------- %% Internal helpers %% ------------------------------------------------------------------- -num_received(Topic, Payload, N) -> - receive - {publish, #{topic := Topic, - payload := Payload}} -> - num_received(Topic, Payload, N + 1) - after 1000 -> - N - end. - await_confirms_ordered(_, To, To) -> ok; await_confirms_ordered(From, N, To) -> @@ -1033,20 +1184,3 @@ bind(Ch, QueueName, Topic) Ch, #'queue.bind'{queue = QueueName, exchange = <<"amq.topic">>, routing_key = Topic}). - -get_events(Node) -> - timer:sleep(300), %% events are sent and processed asynchronously - Result = gen_event:call({rabbit_event, Node}, event_recorder, take_state), - ?assert(is_list(Result)), - Result. - -assert_event_type(ExpectedType, #event{type = ActualType}) -> - ?assertEqual(ExpectedType, ActualType). - -assert_event_prop(ExpectedProp = {Key, _Value}, #event{props = Props}) -> - ?assertEqual(ExpectedProp, lists:keyfind(Key, 1, Props)); -assert_event_prop(ExpectedProps, Event) - when is_list(ExpectedProps) -> - lists:foreach(fun(P) -> - assert_event_prop(P, Event) - end, ExpectedProps). diff --git a/deps/rabbitmq_mqtt/test/util.erl b/deps/rabbitmq_mqtt/test/util.erl index e9257bda1a43..44c684a6c5d6 100644 --- a/deps/rabbitmq_mqtt/test/util.erl +++ b/deps/rabbitmq_mqtt/test/util.erl @@ -1,6 +1,8 @@ -module(util). -include("rabbit_mqtt.hrl"). +-include_lib("rabbit_common/include/rabbit.hrl"). +-include_lib("eunit/include/eunit.hrl"). -export([all_connection_pids/1, publish_qos1_timeout/4, @@ -8,10 +10,15 @@ get_global_counters/2, get_global_counters/3, get_global_counters/4, - expect_publishes/2, + expect_publishes/3, connect/2, connect/3, - connect/4 + connect/4, + get_events/1, + assert_event_type/2, + assert_event_prop/2, + await_exit/1, + await_exit/2 ]). all_connection_pids(Config) -> @@ -39,20 +46,27 @@ publish_qos1_timeout(Client, Topic, Payload, Timeout) -> puback_timeout end. -expect_publishes(_Topic, []) -> +sync_publish_result(Caller, Mref, Result) -> + erlang:send(Caller, {Mref, Result}). + +expect_publishes(_, _, []) -> ok; -expect_publishes(Topic, [Payload|Rest]) -> +expect_publishes(Client, Topic, [Payload|Rest]) + when is_pid(Client) -> receive - {publish, #{topic := Topic, + {publish, #{client_pid := Client, + topic := Topic, payload := Payload}} -> - expect_publishes(Topic, Rest) - after 5000 -> + expect_publishes(Client, Topic, Rest); + {publish, #{client_pid := Client, + topic := Topic, + payload := Other}} -> + ct:fail("Received unexpected PUBLISH payload. Expected: ~p Got: ~p", + [Payload, Other]) + after 3000 -> {publish_not_received, Payload} end. -sync_publish_result(Caller, Mref, Result) -> - erlang:send(Caller, {Mref, Result}). - get_global_counters(Config, ProtoVer) -> get_global_counters(Config, ProtoVer, 0, []). @@ -67,6 +81,37 @@ get_global_counters(Config, Proto, Node, QType) -> maps:get([{protocol, Proto}] ++ QType, rabbit_ct_broker_helpers:rpc(Config, Node, rabbit_global_counters, overview, [])). +get_events(Node) -> + timer:sleep(300), %% events are sent and processed asynchronously + Result = gen_event:call({rabbit_event, Node}, event_recorder, take_state), + ?assert(is_list(Result)), + Result. + +assert_event_type(ExpectedType, #event{type = ActualType}) -> + ?assertEqual(ExpectedType, ActualType). + +assert_event_prop(ExpectedProp = {Key, _Value}, #event{props = Props}) -> + ?assertEqual(ExpectedProp, lists:keyfind(Key, 1, Props)); +assert_event_prop(ExpectedProps, Event) + when is_list(ExpectedProps) -> + lists:foreach(fun(P) -> + assert_event_prop(P, Event) + end, ExpectedProps). + +await_exit(Pid) -> + receive + {'EXIT', Pid, _} -> ok + after + 20_000 -> ct:fail({missing_exit, Pid}) + end. + +await_exit(Pid, Reason) -> + receive + {'EXIT', Pid, Reason} -> ok + after + 20_000 -> ct:fail({missing_exit, Pid}) + end. + connect(ClientId, Config) -> connect(ClientId, Config, []). @@ -74,12 +119,22 @@ connect(ClientId, Config, AdditionalOpts) -> connect(ClientId, Config, 0, AdditionalOpts). connect(ClientId, Config, Node, AdditionalOpts) -> - P = rabbit_ct_broker_helpers:get_node_config(Config, Node, tcp_port_mqtt), + {Port, WsOpts, Connect} = + case rabbit_ct_helpers:get_config(Config, websocket, false) of + false -> + {rabbit_ct_broker_helpers:get_node_config(Config, Node, tcp_port_mqtt), + [], + fun emqtt:connect/1}; + true -> + {rabbit_ct_broker_helpers:get_node_config(Config, Node, tcp_port_web_mqtt), + [{ws_path, "/ws"}], + fun emqtt:ws_connect/1} + end, Options = [{host, "localhost"}, - {port, P}, - {clientid, rabbit_data_coercion:to_binary(ClientId)}, - {proto_ver, v4} - ] ++ AdditionalOpts, + {port, Port}, + {proto_ver, v4}, + {clientid, rabbit_data_coercion:to_binary(ClientId)} + ] ++ WsOpts ++ AdditionalOpts, {ok, C} = emqtt:start_link(Options), - {ok, _Properties} = emqtt:connect(C), + {ok, _Properties} = Connect(C), C. diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_app.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_app.erl index 0fde0e4e20d5..1ad2baa3886f 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_app.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_app.erl @@ -12,8 +12,7 @@ start/2, prep_stop/1, stop/1, - list_connections/0, - close_all_client_connections/1 + list_connections/0 ]). %% Dummy supervisor - see Ulf Wiger's comment at @@ -51,15 +50,8 @@ init([]) -> {ok, {{one_for_one, 1, 5}, []}}. list_connections() -> PlainPids = connection_pids_of_protocol(?TCP_PROTOCOL), TLSPids = connection_pids_of_protocol(?TLS_PROTOCOL), - PlainPids ++ TLSPids. --spec close_all_client_connections(string()) -> {'ok', non_neg_integer()}. -close_all_client_connections(Reason) -> - Connections = list_connections(), - [rabbit_web_mqtt_handler:close_connection(Pid, Reason) || Pid <- Connections], - {ok, length(Connections)}. - %% %% Implementation %% diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index 8901256174b5..732c6303e2a5 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -19,8 +19,7 @@ terminate/3 ]). --export([conserve_resources/3, - close_connection/2]). +-export([conserve_resources/3]). %% cowboy_sub_protocol -export([upgrade/4, @@ -114,13 +113,6 @@ websocket_init({State0 = #state{socket = Sock}, PeerAddr}) -> {[{shutdown_reason, Reason}], State0} end. --spec close_connection(pid(), string()) -> 'ok'. -close_connection(Pid, Reason) -> - rabbit_log_connection:info("Web MQTT: will terminate connection process ~tp, reason: ~ts", - [Pid, Reason]), - sys:terminate(Pid, Reason), - ok. - -spec conserve_resources(pid(), rabbit_alarm:resource_alarm_source(), rabbit_alarm:resource_alert()) -> ok. @@ -321,14 +313,14 @@ handle_credits(State0) -> end, {[{active, Active}], State, hibernate}. -control_throttle(State = #state{connection_state = CS, +control_throttle(State = #state{connection_state = ConnState, conserve = Conserve, keepalive = KState, proc_state = PState}) -> Throttle = Conserve orelse rabbit_mqtt_processor:soft_limit_exceeded(PState) orelse credit_flow:blocked(), - case {CS, Throttle} of + case {ConnState, Throttle} of {running, true} -> State#state{connection_state = blocked, keepalive = rabbit_mqtt_keepalive:cancel_timer(KState)}; diff --git a/deps/rabbitmq_web_mqtt/test/src/rabbit_ws_test_util.erl b/deps/rabbitmq_web_mqtt/test/src/rabbit_ws_test_util.erl index b89787781201..9b1f73bf19ab 100644 --- a/deps/rabbitmq_web_mqtt/test/src/rabbit_ws_test_util.erl +++ b/deps/rabbitmq_web_mqtt/test/src/rabbit_ws_test_util.erl @@ -7,6 +7,8 @@ -module(rabbit_ws_test_util). +-include_lib("common_test/include/ct.hrl"). + -export([update_app_env/3, get_web_mqtt_port_str/1, mqtt_3_1_1_connect_packet/0]). @@ -22,12 +24,12 @@ update_app_env(Config, Key, Val) -> [rabbitmq_web_mqtt]). get_web_mqtt_port_str(Config) -> - Port = case rabbit_ct_helpers:get_config(Config, protocol) of - "ws" -> - rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt); - "wss" -> - rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt_tls) - end, + Port = case ?config(protocol, Config) of + "ws" -> + rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt); + "wss" -> + rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt_tls) + end, integer_to_list(Port). mqtt_3_1_1_connect_packet() -> diff --git a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl index 8a39edbc78cc..19b0f873c46a 100644 --- a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl +++ b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl @@ -8,51 +8,27 @@ -module(system_SUITE). -include_lib("eunit/include/eunit.hrl"). --include_lib("rabbitmq_ct_helpers/include/rabbit_mgmt_test.hrl"). -compile([export_all, nowarn_export_all]). --import(rabbit_ct_broker_helpers, - [rpc/5]). --import(rabbit_ct_helpers, - [eventually/1, - eventually/3]). --import(rabbit_mgmt_test_util, - [http_get/2, - http_delete/3]). - --define(MANAGEMENT_PLUGIN_TESTS, - [management_plugin_connection, - management_plugin_enable]). +-import(rabbit_ct_helpers, [eventually/1]). all() -> - [ - {group, tests} - ]. + [{group, tests}]. groups() -> [ {tests, [], - [block - , pubsub_shared_connection - , pubsub_separate_connections - , last_will_enabled_disconnect - , last_will_enabled_no_disconnect - , disconnect - , keepalive - , maintenance - , client_no_supported_protocol - , client_not_support_mqtt - , unacceptable_data_type - , duplicate_id - , handle_invalid_packets - , duplicate_connect - ] ++ ?MANAGEMENT_PLUGIN_TESTS - } + [no_websocket_subprotocol + ,unsupported_websocket_subprotocol + ,unacceptable_data_type + ,handle_invalid_packets + ,duplicate_connect + ]} ]. suite() -> - [{timetrap, {minutes, 5}}]. + [{timetrap, {minutes, 2}}]. init_per_suite(Config) -> rabbit_ct_helpers:log_environment(), @@ -76,185 +52,32 @@ end_per_group(_, Config) -> Config. init_per_testcase(Testcase, Config) -> - maybe_start_inets(Testcase), rabbit_ct_helpers:testcase_started(Config, Testcase). end_per_testcase(Testcase, Config) -> - maybe_stop_inets(Testcase), - rabbit_ct_client_helpers:close_channels_and_connection(Config, 0), rabbit_ct_helpers:testcase_finished(Config, Testcase). -maybe_start_inets(Testcase) -> - case lists:member(Testcase, ?MANAGEMENT_PLUGIN_TESTS) of - true -> - ok = inets:start(); - false -> - ok - end. - -maybe_stop_inets(Testcase) -> - case lists:member(Testcase, ?MANAGEMENT_PLUGIN_TESTS) of - true -> - ok = inets:stop(); - false -> - ok - end. - %% ------------------------------------------------------------------- %% Testsuite cases %% ------------------------------------------------------------------- -block(Config) -> - Topic = ClientId = atom_to_binary(?FUNCTION_NAME), - C = ws_connect(ClientId, Config), - - {ok, _, _} = emqtt:subscribe(C, Topic), - {ok, _} = emqtt:publish(C, Topic, <<"Not blocked yet">>, [{qos, 1}]), - - ok = rpc(Config, 0, vm_memory_monitor, set_vm_memory_high_watermark, [0.00000001]), - %% Let it block - timer:sleep(100), - - %% Blocked, but still will publish when unblocked - puback_timeout = publish_qos1_timeout(C, Topic, <<"Now blocked">>, 1000), - puback_timeout = publish_qos1_timeout(C, Topic, <<"Still blocked">>, 1000), - - %% Unblock - rpc(Config, 0, vm_memory_monitor, set_vm_memory_high_watermark, [0.4]), - ok = expect_publishes(C, Topic, [<<"Not blocked yet">>, - <<"Now blocked">>, - <<"Still blocked">>]), - ok = emqtt:disconnect(C). - -pubsub_shared_connection(Config) -> - C = ws_connect(?FUNCTION_NAME, Config), - - Topic = <<"/topic/test-web-mqtt">>, - {ok, _, [1]} = emqtt:subscribe(C, Topic, qos1), - - Payload = <<"a\x00a">>, - ?assertMatch({ok, #{packet_id := _, - reason_code := 0, - reason_code_name := success - }}, - emqtt:publish(C, Topic, Payload, [{qos, 1}])), - ok = expect_publishes(C, Topic, [Payload]), - ok = emqtt:disconnect(C). - -pubsub_separate_connections(Config) -> - Publisher = ws_connect(<<(atom_to_binary(?FUNCTION_NAME))/binary, "_publisher">>, Config), - Consumer = ws_connect(<<(atom_to_binary(?FUNCTION_NAME))/binary, "_consumer">>, Config), - - Topic = <<"/topic/test-web-mqtt">>, - {ok, _, [1]} = emqtt:subscribe(Consumer, Topic, qos1), - - Payload = <<"a\x00a">>, - ?assertMatch({ok, #{packet_id := _, - reason_code := 0, - reason_code_name := success - }}, - emqtt:publish(Publisher, Topic, Payload, [{qos, 1}])), - ok = expect_publishes(Consumer, Topic, [Payload]), - ok = emqtt:disconnect(Publisher), - ok = emqtt:disconnect(Consumer). - -last_will_enabled_disconnect(Config) -> - LastWillTopic = <<"/topic/web-mqtt-tests-ws1-last-will">>, - LastWillMsg = <<"a last will and testament message">>, - PubOpts = [{will_topic, LastWillTopic}, - {will_payload, LastWillMsg}, - {will_qos, 1}], - Publisher = ws_connect(<<(atom_to_binary(?FUNCTION_NAME))/binary, "_publisher">>, Config, PubOpts), - Consumer = ws_connect(<<(atom_to_binary(?FUNCTION_NAME))/binary, "_consumer">>, Config), - {ok, _, [1]} = emqtt:subscribe(Consumer, LastWillTopic, qos1), - - %% Client sends DISCONNECT packet. Therefore, will message should not be sent. - ok = emqtt:disconnect(Publisher), - ?assertEqual({publish_not_received, LastWillMsg}, - expect_publishes(Consumer, LastWillTopic, [LastWillMsg])), - - ok = emqtt:disconnect(Consumer). - -last_will_enabled_no_disconnect(Config) -> - LastWillTopic = <<"/topic/web-mqtt-tests-ws1-last-will">>, - LastWillMsg = <<"a last will and testament message">>, - PubOpts = [{will_topic, LastWillTopic}, - {will_payload, LastWillMsg}, - {will_qos, 1}], - _Publisher = ws_connect(<<(atom_to_binary(?FUNCTION_NAME))/binary, "_publisher">>, Config, PubOpts), - timer:sleep(100), - [ServerPublisherPid] = rpc(Config, 0, rabbit_mqtt, local_connection_pids, []), - Consumer = ws_connect(<<(atom_to_binary(?FUNCTION_NAME))/binary, "_consumer">>, Config), - {ok, _, [1]} = emqtt:subscribe(Consumer, LastWillTopic, qos1), - - %% Client does not send DISCONNECT packet. Therefore, will message should be sent. - erlang:exit(ServerPublisherPid, test_will), - ?assertEqual(ok, expect_publishes(Consumer, LastWillTopic, [LastWillMsg])), - - ok = emqtt:disconnect(Consumer). - -disconnect(Config) -> - C = ws_connect(?FUNCTION_NAME, Config), - process_flag(trap_exit, true), - eventually(?_assertEqual(1, num_mqtt_connections(Config, 0))), - ok = emqtt:disconnect(C), - receive - {'EXIT', C, normal} -> - ok - after 5000 -> - ct:fail("disconnect didn't terminate client") - end, - eventually(?_assertEqual(0, num_mqtt_connections(Config, 0))), - ok. - -keepalive(Config) -> - KeepaliveSecs = 1, - KeepaliveMs = timer:seconds(KeepaliveSecs), - C = ws_connect(?FUNCTION_NAME, Config, [{keepalive, KeepaliveSecs}]), - - %% Connection should stay up when client sends PING requests. - timer:sleep(KeepaliveMs), - - %% Mock the server socket to not have received any bytes. - rabbit_ct_broker_helpers:setup_meck(Config), - Mod = rabbit_net, - ok = rpc(Config, 0, meck, new, [Mod, [no_link, passthrough]]), - ok = rpc(Config, 0, meck, expect, [Mod, getstat, 2, {ok, [{recv_oct, 999}]} ]), - - process_flag(trap_exit, true), - receive - {'EXIT', C, _Reason} -> - ok - after - ceil(3 * 0.75 * KeepaliveMs) -> - ct:fail("server did not respect keepalive") - end, - - true = rpc(Config, 0, meck, validate, [Mod]), - ok = rpc(Config, 0, meck, unload, [Mod]). - -maintenance(Config) -> - C = ws_connect(?FUNCTION_NAME, Config), - true = unlink(C), - eventually(?_assertEqual(1, num_mqtt_connections(Config, 0))), - ok = rabbit_ct_broker_helpers:drain_node(Config, 0), - eventually(?_assertEqual(0, num_mqtt_connections(Config, 0))), - ok = rabbit_ct_broker_helpers:revive_node(Config, 0). +no_websocket_subprotocol(Config) -> + websocket_subprotocol(Config, []). -client_no_supported_protocol(Config) -> - client_protocol_test(Config, []). +unsupported_websocket_subprotocol(Config) -> + websocket_subprotocol(Config, ["not-mqtt-protocol"]). -client_not_support_mqtt(Config) -> - client_protocol_test(Config, ["not-mqtt-protocol"]). - -client_protocol_test(Config, Protocol) -> +%% "The client MUST include “mqtt” in the list of WebSocket Sub Protocols it offers" [MQTT-6.0.0-3]. +websocket_subprotocol(Config, SubProtocol) -> PortStr = rabbit_ws_test_util:get_web_mqtt_port_str(Config), - WS = rfc6455_client:new("ws://localhost:" ++ PortStr ++ "/ws", self(), undefined, Protocol), + WS = rfc6455_client:new("ws://localhost:" ++ PortStr ++ "/ws", self(), undefined, SubProtocol), {_, [{http_response, Res}]} = rfc6455_client:open(WS), {'HTTP/1.1', 400, <<"Bad Request">>, _} = cow_http:parse_status_line(rabbit_data_coercion:to_binary(Res)), rfc6455_client:send_binary(WS, rabbit_ws_test_util:mqtt_3_1_1_connect_packet()), {close, _} = rfc6455_client:recv(WS, timer:seconds(1)). +%% "MQTT Control Packets MUST be sent in WebSocket binary data frames. If any other type +%% of data frame is received the recipient MUST close the Network Connection" [MQTT-6.0.0-1]. unacceptable_data_type(Config) -> PortStr = rabbit_ws_test_util:get_web_mqtt_port_str(Config), WS = rfc6455_client:new("ws://localhost:" ++ PortStr ++ "/ws", self(), undefined, ["mqtt"]), @@ -262,20 +85,6 @@ unacceptable_data_type(Config) -> rfc6455_client:send(WS, "not-binary-data"), {close, {1003, _}} = rfc6455_client:recv(WS, timer:seconds(1)). -duplicate_id(Config) -> - C1 = ws_connect(?FUNCTION_NAME, Config), - eventually(?_assertEqual(1, num_mqtt_connections(Config, 0))), - process_flag(trap_exit, true), - C2 = ws_connect(?FUNCTION_NAME, Config), - receive - {'EXIT', C1, _Reason} -> - ok - after 5000 -> - ct:fail("server did not disconnect a client with duplicate ID") - end, - eventually(?_assertEqual(1, num_mqtt_connections(Config, 0))), - ok = emqtt:disconnect(C2). - handle_invalid_packets(Config) -> PortStr = rabbit_ws_test_util:get_web_mqtt_port_str(Config), WS = rfc6455_client:new("ws://localhost:" ++ PortStr ++ "/ws", self(), undefined, ["mqtt"]), @@ -284,46 +93,6 @@ handle_invalid_packets(Config) -> rfc6455_client:send_binary(WS, Bin), {close, {1002, _}} = rfc6455_client:recv(WS, timer:seconds(1)). -%% Test that Web MQTT connection can be listed and closed via the rabbitmq_management plugin. -management_plugin_connection(Config) -> - KeepaliveSecs = 99, - ClientId = atom_to_binary(?FUNCTION_NAME), - Node = atom_to_binary(rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename)), - C = ws_connect(ClientId, Config, [{keepalive, KeepaliveSecs}]), - - eventually(?_assertEqual(1, length(http_get(Config, "/connections"))), 1000, 10), - [#{client_properties := #{client_id := ClientId}, - timeout := KeepaliveSecs, - node := Node, - name := ConnectionName}] = http_get(Config, "/connections"), - - process_flag(trap_exit, true), - http_delete(Config, - "/connections/" ++ binary_to_list(uri_string:quote((ConnectionName))), - ?NO_CONTENT), - receive - {'EXIT', C, _} -> - ok - after 5000 -> - ct:fail("server did not close connection") - end, - ?assertEqual([], http_get(Config, "/connections")), - ?assertEqual(0, num_mqtt_connections(Config, 0)). - -management_plugin_enable(Config) -> - ?assertEqual(0, length(http_get(Config, "/connections"))), - ok = rabbit_ct_broker_helpers:disable_plugin(Config, 0, rabbitmq_management), - ok = rabbit_ct_broker_helpers:disable_plugin(Config, 0, rabbitmq_management_agent), - - %% If the Web MQTT connection is established **before** the management plugin is enabled, - %% the management plugin should still list the Web MQTT connection. - C = ws_connect(?FUNCTION_NAME, Config), - ok = rabbit_ct_broker_helpers:enable_plugin(Config, 0, rabbitmq_management_agent), - ok = rabbit_ct_broker_helpers:enable_plugin(Config, 0, rabbitmq_management), - eventually(?_assertEqual(1, length(http_get(Config, "/connections"))), 1000, 10), - - ok = emqtt:disconnect(C). - %% "A Client can only send the CONNECT Packet once over a Network Connection. %% The Server MUST process a second CONNECT Packet sent from a Client as a protocol %% violation and disconnect the Client [MQTT-3.1.0-2]. @@ -351,51 +120,4 @@ duplicate_connect(Config) -> %% Web mqtt connections are tracked together with mqtt connections num_mqtt_connections(Config, Node) -> - length(rpc(Config, Node, rabbit_mqtt, local_connection_pids, [])). - -ws_connect(ClientId, Config) -> - ws_connect(ClientId, Config, []). -ws_connect(ClientId, Config, AdditionalOpts) -> - P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt), - Options = [{host, "localhost"}, - {username, "guest"}, - {password, "guest"}, - {ws_path, "/ws"}, - {port, P}, - {clientid, rabbit_data_coercion:to_binary(ClientId)}, - {proto_ver, v4} - ] ++ AdditionalOpts, - {ok, C} = emqtt:start_link(Options), - {ok, _Properties} = emqtt:ws_connect(C), - C. - -expect_publishes(_ClientPid, _Topic, []) -> - ok; -expect_publishes(ClientPid, Topic, [Payload|Rest]) -> - receive - {publish, #{client_pid := ClientPid, - topic := Topic, - payload := Payload}} -> - expect_publishes(ClientPid, Topic, Rest) - after 1000 -> - {publish_not_received, Payload} - end. - -publish_qos1_timeout(Client, Topic, Payload, Timeout) -> - Mref = erlang:monitor(process, Client), - ok = emqtt:publish_async(Client, Topic, #{}, Payload, [{qos, 1}], infinity, - {fun ?MODULE:sync_publish_result/3, [self(), Mref]}), - receive - {Mref, Reply} -> - erlang:demonitor(Mref, [flush]), - Reply; - {'DOWN', Mref, process, Client, Reason} -> - ct:fail("client is down: ~tp", [Reason]) - after - Timeout -> - erlang:demonitor(Mref, [flush]), - puback_timeout - end. - -sync_publish_result(Caller, Mref, Result) -> - erlang:send(Caller, {Mref, Result}). + length(rabbit_ct_broker_helpers:rpc(Config, Node, rabbit_mqtt, local_connection_pids, [])). From 6ba2dc4afc69bd84d9aba2a78c7e7a8ee32b6505 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Wed, 4 Jan 2023 11:16:36 +0000 Subject: [PATCH 096/118] Switch to Logger macros Convert from the old rabbit_log* API to the new Logger macros for MQTT and Web MQTT connections. Advantages: * metadata mfa, file, line, pid, gl, time is auto-inserted by Logger. * Log lines output by the shared module rabbit_mqtt_processor now include via the domain whether it's a MQTT or Web MQTT connection. Instead of using domain [rabbitmq, connection], this commit now uses the smaller and more specialized domains [rabbitmq, connection, mqtt] and [rabbitmq, connection, web_mqtt] for MQTT and Web MQTT processes respectively, resulting in the following example output: "msg":"Received a CONNECT,", "domain":"rabbitmq.connection.mqtt" or "msg":"Received a CONNECT,", "domain":"rabbitmq.connection.web_mqtt" --- .../src/rabbit_mqtt_processor.erl | 124 +++++++++--------- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 84 ++++++------ .../src/rabbit_web_mqtt_app.erl | 4 +- .../src/rabbit_web_mqtt_handler.erl | 43 +++--- 4 files changed, 123 insertions(+), 132 deletions(-) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 309de9a973e8..5413daa788e6 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -23,6 +23,7 @@ -import(rabbit_mqtt_util, [mqtt_to_amqp/1, amqp_to_mqtt/1]). +-include_lib("kernel/include/logger.hrl"). -include_lib("rabbit_common/include/rabbit.hrl"). -include_lib("rabbit_common/include/rabbit_framing.hrl"). -include_lib("rabbit/include/amqqueue.hrl"). @@ -218,7 +219,7 @@ process_request(?SUBSCRIBE, payload = undefined}, #state{send_fun = SendFun, retainer_pid = RPid} = State0) -> - rabbit_log_connection:debug("Received a SUBSCRIBE for topic(s) ~p", [Topics]), + ?LOG_DEBUG("Received a SUBSCRIBE for topic(s) ~p", [Topics]), TopicNamesQos0 = topic_names(?QOS_0, State0), TopicNamesQos1 = topic_names(?QOS_1, State0), HasSubsBefore = TopicNamesQos0 =/= [] orelse TopicNamesQos1 =/= [], @@ -253,8 +254,8 @@ process_request(?SUBSCRIBE, {[QoS | L], S2} end; {error, Reason, S2} -> - rabbit_log:error("Failed to bind ~s with topic ~s: ~p", - [rabbit_misc:rs(QName), TopicName, Reason]), + ?LOG_ERROR("Failed to bind ~s with topic ~s: ~p", + [rabbit_misc:rs(QName), TopicName, Reason]), {[?SUBACK_FAILURE | L], S2} end; {error, _} -> @@ -287,7 +288,7 @@ process_request(?UNSUBSCRIBE, topic_table = Topics}, payload = undefined}, State0 = #state{send_fun = SendFun}) -> - rabbit_log_connection:debug("Received an UNSUBSCRIBE for topic(s) ~p", [Topics]), + ?LOG_DEBUG("Received an UNSUBSCRIBE for topic(s) ~p", [Topics]), HasSubsBefore = has_subs(State0), State = lists:foldl( fun(#mqtt_topic{name = TopicName}, #state{} = S0) -> @@ -297,8 +298,8 @@ process_request(?UNSUBSCRIBE, {ok, _, S} -> S; {error, Reason, S} -> - rabbit_log:error("Failed to unbind ~s with topic ~s: ~p", - [rabbit_misc:rs(QName), TopicName, Reason]), + ?LOG_ERROR("Failed to unbind ~s with topic ~s: ~p", + [rabbit_misc:rs(QName), TopicName, Reason]), S end; {not_found, _} -> @@ -313,16 +314,17 @@ process_request(?UNSUBSCRIBE, maybe_decrement_consumer(HasSubsBefore, State), {ok, State}; -process_request(?PINGREQ, #mqtt_packet{}, State = #state{send_fun = SendFun}) -> - rabbit_log_connection:debug("Received a PINGREQ"), +process_request(?PINGREQ, #mqtt_packet{}, State = #state{send_fun = SendFun, + client_id = ClientId}) -> + ?LOG_DEBUG("Received a PINGREQ, client ID: ~s", [ClientId]), SendFun( #mqtt_packet{fixed = #mqtt_packet_fixed{type = ?PINGRESP}}, State), - rabbit_log_connection:debug("Sent a PINGRESP"), + ?LOG_DEBUG("Sent a PINGRESP, client ID: ~s", [ClientId]), {ok, State}; process_request(?DISCONNECT, #mqtt_packet{}, State) -> - rabbit_log_connection:debug("Received a DISCONNECT"), + ?LOG_DEBUG("Received a DISCONNECT"), {stop, disconnect, State}. process_connect(#mqtt_packet{ @@ -333,9 +335,9 @@ process_connect(#mqtt_packet{ client_id = ClientId, keep_alive = Keepalive} = PacketConnect}, State0 = #state{send_fun = SendFun}) -> - rabbit_log_connection:debug("Received a CONNECT, client ID: ~s, username: ~s, " - "clean session: ~s, protocol version: ~p, keepalive: ~p", - [ClientId, Username, CleanSess, ProtoVersion, Keepalive]), + ?LOG_DEBUG("Received a CONNECT, client ID: ~s, username: ~s, " + "clean session: ~s, protocol version: ~p, keepalive: ~p", + [ClientId, Username, CleanSess, ProtoVersion, Keepalive]), {ReturnCode, SessionPresent, State} = case rabbit_misc:pipeline([fun check_protocol_version/1, fun check_client_id/1, @@ -380,15 +382,15 @@ check_credentials(Packet = #mqtt_packet_connect{username = Username, case creds(Username, Password, SslLoginName) of nocreds -> rabbit_core_metrics:auth_attempt_failed(Ip, <<>>, mqtt), - rabbit_log_connection:error("MQTT login failed: no credentials provided"), + ?LOG_ERROR("MQTT login failed: no credentials provided"), {error, ?CONNACK_BAD_CREDENTIALS}; {invalid_creds, {undefined, Pass}} when is_binary(Pass) -> rabbit_core_metrics:auth_attempt_failed(Ip, <<>>, mqtt), - rabbit_log_connection:error("MQTT login failed: no username is provided"), + ?LOG_ERROR("MQTT login failed: no username is provided"), {error, ?CONNACK_BAD_CREDENTIALS}; {invalid_creds, {User, undefined}} when is_binary(User) -> rabbit_core_metrics:auth_attempt_failed(Ip, User, mqtt), - rabbit_log_connection:error("MQTT login failed for user '~p': no password provided", [User]), + ?LOG_ERROR("MQTT login failed for user '~p': no password provided", [User]), {error, ?CONNACK_BAD_CREDENTIALS}; {UserBin, PassBin} -> {ok, {UserBin, PassBin, Packet}, State} @@ -449,8 +451,8 @@ register_client(Packet = #mqtt_packet_connect{proto_ver = ProtoVersion}, {ok, NewProcState({pending, Corr})}; {error, _} = Err -> %% e.g. this node was removed from the MQTT cluster members - rabbit_log_connection:error("MQTT cannot accept a connection: " - "client ID tracker is unavailable: ~p", [Err]), + ?LOG_ERROR("MQTT cannot accept a connection: client ID tracker is unavailable: ~p", + [Err]), {error, ?CONNACK_SERVER_UNAVAILABLE} end; false -> @@ -609,8 +611,8 @@ maybe_unbind(TopicName, TopicNames, QName, State0) -> {ok, _Output, State} -> {ok, State}; {error, Reason, _State} = Err -> - rabbit_log:error("Failed to unbind ~s with topic '~s': ~p", - [rabbit_misc:rs(QName), TopicName, Reason]), + ?LOG_ERROR("Failed to unbind ~s with topic '~s': ~p", + [rabbit_misc:rs(QName), TopicName, Reason]), Err end end. @@ -675,9 +677,9 @@ process_login(_UserBin, _PassBin, ClientId, }} = State) when Username =/= undefined, User =/= undefined, VHost =/= underfined -> rabbit_core_metrics:auth_attempt_failed(list_to_binary(inet:ntoa(Addr)), Username, mqtt), - rabbit_log_connection:error( - "MQTT detected duplicate connect attempt for client ID '~ts', user '~ts', vhost '~ts'", - [ClientId, Username, VHost]), + ?LOG_ERROR( + "MQTT detected duplicate connect attempt for client ID '~ts', user '~ts', vhost '~ts'", + [ClientId, Username, VHost]), {error, ?CONNACK_ID_REJECTED, State}; process_login(UserBin, PassBin, ClientId, #state{socket = Sock, @@ -686,9 +688,8 @@ process_login(UserBin, PassBin, ClientId, auth_state = undefined} = State0) -> {ok, {_PeerHost, _PeerPort, _Host, Port}} = rabbit_net:socket_ends(Sock, inbound), {VHostPickedUsing, {VHost, UsernameBin}} = get_vhost(UserBin, SslLoginName, Port), - rabbit_log_connection:debug( - "MQTT vhost picked using ~s", - [human_readable_vhost_lookup_strategy(VHostPickedUsing)]), + ?LOG_DEBUG("MQTT vhost picked using ~s", + [human_readable_vhost_lookup_strategy(VHostPickedUsing)]), RemoteIpAddressBin = list_to_binary(inet:ntoa(Addr)), Input = #{vhost => VHost, username_bin => UsernameBin, @@ -718,8 +719,8 @@ check_vhost_exists(#{vhost := VHost, true -> ok; false -> - rabbit_log_connection:error("MQTT login failed for user '~s': virtual host '~s' does not exist", - [UsernameBin, VHost]), + ?LOG_ERROR("MQTT login failed for user '~s': virtual host '~s' does not exist", + [UsernameBin, VHost]), {error, ?CONNACK_BAD_CREDENTIALS} end. @@ -730,10 +731,10 @@ check_vhost_connection_limit(#{vhost := VHost, false -> ok; {true, Limit} -> - rabbit_log_connection:error( - "Failed to create MQTT connection because vhost connection limit is reached; " - "vhost: '~s'; connection limit: ~p; user: '~s'; client ID '~s'", - [VHost, Limit, Username, ClientId]), + ?LOG_ERROR( + "Failed to create MQTT connection because vhost connection limit is reached; " + "vhost: '~s'; connection limit: ~p; user: '~s'; client ID '~s'", + [VHost, Limit, Username, ClientId]), {error, ?CONNACK_NOT_AUTHORIZED} end. @@ -744,7 +745,7 @@ check_vhost_alive(#{vhost := VHost, true -> ok; false -> - rabbit_log_connection:error( + ?LOG_ERROR( "Failed to create MQTT connection because vhost is down; " "vhost: ~s; user: ~s; client ID: ~s", [VHost, UsernameBin, ClientId]), @@ -776,7 +777,7 @@ check_user_login(#{vhost := VHost, conn_name = undefined}, {ok, maps:put(user, User, In), State}; {refused, Username, Msg, Args} -> - rabbit_log_connection:error( + ?LOG_ERROR( "Error on MQTT connection ~p~n" "access refused for user '~s' in vhost '~s' " ++ Msg, @@ -798,7 +799,7 @@ check_user_connection_limit(#{user := #user{username = Username}, false -> ok; {true, Limit} -> - rabbit_log_connection:error( + ?LOG_ERROR( "Failed to create MQTT connection because user connection limit is reached; " "user: '~s'; connection limit: ~p; client ID '~s'", [Username, Limit, ClientId]), @@ -820,7 +821,7 @@ check_vhost_access(#{vhost := VHost, ok -> {ok, maps:put(authz_ctx, AuthzCtx, In), State} catch exit:#amqp_error{name = not_allowed} -> - rabbit_log_connection:error( + ?LOG_ERROR( "Error on MQTT connection ~p~n" "access refused for user '~s'", [self(), Username]), @@ -841,7 +842,7 @@ check_user_loopback(#{vhost := VHost, authz_ctx = AuthzCtx}, {ok, State#state{auth_state = AuthState}}; not_allowed -> - rabbit_log_connection:warning( + ?LOG_WARNING( "MQTT login failed: user '~s' can only connect via localhost", [UsernameBin]), {error, ?CONNACK_NOT_AUTHORIZED} @@ -984,9 +985,8 @@ ensure_queue(QoS, State = #state{auth_state = #auth_state{user = #user{username {ok, Q}; {error, {resource_locked, Q}} -> QName = amqqueue:get_name(Q), - rabbit_log:debug("MQTT deleting exclusive ~s owned by ~p", - [rabbit_misc:rs(QName), - ?amqqueue_v2_field_exclusive_owner(Q)]), + ?LOG_DEBUG("MQTT deleting exclusive ~s owned by ~p", + [rabbit_misc:rs(QName), ?amqqueue_v2_field_exclusive_owner(Q)]), delete_queue(QName, Username), create_queue(QoS, State); {error, not_found} -> @@ -1025,14 +1025,14 @@ create_queue(QoS, #state{ rabbit_core_metrics:queue_created(QName), {ok, Q}; Other -> - rabbit_log:error("Failed to declare ~s: ~p", - [rabbit_misc:rs(QName), Other]), + ?LOG_ERROR("Failed to declare ~s: ~p", + [rabbit_misc:rs(QName), Other]), {error, queue_declare} end; {true, Limit} -> - rabbit_log:error("cannot declare ~s because " - "queue limit ~p in vhost '~s' is reached", - [rabbit_misc:rs(QName), Limit, VHost]), + ?LOG_ERROR("cannot declare ~s because " + "queue limit ~p in vhost '~s' is reached", + [rabbit_misc:rs(QName), Limit, VHost]), {error, access_refused} end; {error, access_refused} = E -> @@ -1110,8 +1110,8 @@ consume(Q, QoS, #state{ State = maybe_set_queue_qos1(QoS, State1), {ok, State}; {error, Reason} = Err -> - rabbit_log:error("Failed to consume from ~s: ~p", - [rabbit_misc:rs(QName), Reason]), + ?LOG_ERROR("Failed to consume from ~s: ~p", + [rabbit_misc:rs(QName), Reason]), Err end end) @@ -1216,7 +1216,7 @@ publish_to_queues( QNames = rabbit_exchange:route(Exchange, Delivery), deliver_to_queues(Delivery, QNames, State); {error, not_found} -> - rabbit_log:error("~s not found", [rabbit_misc:rs(ExchangeName)]), + ?LOG_ERROR("~s not found", [rabbit_misc:rs(ExchangeName)]), {error, exchange_not_found, State} end. @@ -1235,8 +1235,8 @@ deliver_to_queues(Delivery, %% contain rejections of publishes. {ok, handle_queue_actions(Actions, State)}; {error, Reason} -> - rabbit_log:error("Failed to deliver message with packet_id=~p to queues: ~p", - [Delivery#delivery.msg_seq_no, Reason]), + ?LOG_ERROR("Failed to deliver message with packet_id=~p to queues: ~p", + [Delivery#delivery.msg_seq_no, Reason]), {error, Reason, State0} end. @@ -1292,10 +1292,9 @@ serialise_and_send_to_client(Packet, #state{proto_ver = ProtoVer, socket = Sock} Data = rabbit_mqtt_packet:serialise(Packet, ProtoVer), try rabbit_net:port_command(Sock, Data) catch error:Error -> - rabbit_log_connection:error( - "MQTT: a socket write failed: ~p", [Error]), - rabbit_log_connection:debug( - "Failed to write to socket ~p, error: ~p, packet: ~p", [Sock, Error, Packet]) + ?LOG_ERROR("MQTT: a socket write failed: ~p", [Error]), + ?LOG_DEBUG("Failed to write to socket ~p, error: ~p, packet: ~p", + [Sock, Error, Packet]) end. serialise(Packet, #state{proto_ver = ProtoVer}) -> @@ -1321,8 +1320,8 @@ maybe_send_will(true, ConnStr, topic = Topic}, retainer_pid = RPid } = State) -> - rabbit_log_connection:debug("sending MQTT will message to topic ~s on connection ~s", - [Topic, ConnStr]), + ?LOG_DEBUG("sending MQTT will message to topic ~s on connection ~s", + [Topic, ConnStr]), case check_topic_access(Topic, write, State) of ok -> publish_to_queues(WillMsg, State), @@ -1333,7 +1332,7 @@ maybe_send_will(true, ConnStr, hand_off_to_retainer(RPid, Topic, WillMsg) end; {error, access_refused = Reason} -> - rabbit_log:error("failed to send will message: ~p", [Reason]) + ?LOG_ERROR("failed to send will message: ~p", [Reason]) end; maybe_send_will(_, _, _) -> ok. @@ -1427,8 +1426,7 @@ handle_ra_event(register_timeout, handle_ra_event(register_timeout, State) -> State; handle_ra_event(Evt, State) -> - %% log these? - rabbit_log:debug("unhandled ra_event: ~w ", [Evt]), + ?LOG_DEBUG("unhandled ra_event: ~w ", [Evt]), State. -spec handle_down(term(), state()) -> @@ -1528,8 +1526,8 @@ handle_queue_down(QName, State0 = #state{client_id = ClientId}) -> {ok, State} -> State; {error, _Reason} -> - rabbit_log:info("Terminating MQTT connection because consuming ~s is down.", - [rabbit_misc:rs(QName)]), + ?LOG_INFO("Terminating MQTT connection because consuming ~s is down.", + [rabbit_misc:rs(QName)]), throw(consuming_queue_down) end end; @@ -1668,7 +1666,7 @@ check_resource_access(User, Resource, Perm, Context) -> catch exit:#amqp_error{name = access_refused, explanation = Msg} -> - rabbit_log:error("MQTT resource access refused: ~s", [Msg]), + ?LOG_ERROR("MQTT resource access refused: ~s", [Msg]), {error, access_refused} end end. @@ -1704,7 +1702,7 @@ check_topic_access(TopicName, Access, catch exit:#amqp_error{name = access_refused, explanation = Msg} -> - rabbit_log:error("MQTT topic access refused: ~s", [Msg]), + ?LOG_ERROR("MQTT topic access refused: ~s", [Msg]), {error, access_refused} end end. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index 2af6125e2515..1a96dc557426 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -10,6 +10,9 @@ -behaviour(gen_server). -behaviour(ranch_protocol). +-include_lib("kernel/include/logger.hrl"). +-include_lib("rabbit_common/include/logging.hrl"). + -export([start_link/3]). -export([init/1, handle_call/3, handle_cast/2, handle_info/2, code_change/3, terminate/2, format_status/1]). @@ -67,13 +70,14 @@ close_connection(Pid, Reason) -> init(Ref) -> process_flag(trap_exit, true), + logger:set_process_metadata(#{domain => ?RMQLOG_DOMAIN_CONN ++ [mqtt]}), {ok, Sock} = rabbit_networking:handshake(Ref, application:get_env(?APP_NAME, proxy_protocol, false)), RealSocket = rabbit_net:unwrap_socket(Sock), case rabbit_net:connection_string(Sock, inbound) of {ok, ConnStr} -> ConnName = rabbit_data_coercion:to_binary(ConnStr), - rabbit_log_connection:debug("MQTT accepting TCP connection ~tp (~ts)", [self(), ConnName]), + ?LOG_DEBUG("MQTT accepting TCP connection ~tp (~ts)", [self(), ConnName]), rabbit_alarm:register(self(), {?MODULE, conserve_resources, []}), LoginTimeout = application:get_env(?APP_NAME, login_timeout, 10_000), erlang:send_after(LoginTimeout, self(), login_timeout), @@ -91,13 +95,11 @@ init(Ref) -> State = rabbit_event:init_stats_timer(State1, #state.stats_timer), gen_server:enter_loop(?MODULE, [], State); {error, Reason = enotconn} -> - rabbit_log_connection:info( - "MQTT could not get connection string: ~s", [Reason]), + ?LOG_INFO("MQTT could not get connection string: ~s", [Reason]), rabbit_net:fast_close(RealSocket), ignore; {error, Reason} -> - rabbit_log_connection:error( - "MQTT could not get connection string: ~p", [Reason]), + ?LOG_ERROR("MQTT could not get connection string: ~p", [Reason]), rabbit_net:fast_close(RealSocket), {stop, Reason} end. @@ -111,22 +113,22 @@ handle_call(Msg, From, State) -> handle_cast(duplicate_id, State = #state{ proc_state = PState, conn_name = ConnName }) -> - rabbit_log_connection:warning("MQTT disconnecting client ~tp with duplicate id '~ts'", + ?LOG_WARNING("MQTT disconnecting client ~tp with duplicate id '~ts'", [ConnName, rabbit_mqtt_processor:info(client_id, PState)]), {stop, {shutdown, duplicate_id}, State}; handle_cast(decommission_node, State = #state{ proc_state = PState, conn_name = ConnName }) -> - rabbit_log_connection:warning("MQTT disconnecting client ~tp with client ID '~ts' as its node is about" - " to be decommissioned", + ?LOG_WARNING("MQTT disconnecting client ~tp with client ID '~ts' as its node is about" + " to be decommissioned", [ConnName, rabbit_mqtt_processor:info(client_id, PState)]), {stop, {shutdown, decommission_node}, State}; handle_cast({close_connection, Reason}, State = #state{conn_name = ConnName, proc_state = PState}) -> - rabbit_log_connection:warning("MQTT disconnecting client ~tp with client ID '~ts', reason: ~ts", - [ConnName, rabbit_mqtt_processor:info(client_id, PState), Reason]), + ?LOG_WARNING("MQTT disconnecting client ~tp with client ID '~ts', reason: ~ts", + [ConnName, rabbit_mqtt_processor:info(client_id, PState), Reason]), {stop, {shutdown, server_initiated_close}, State}; handle_cast(QueueEvent = {queue_event, _, _}, @@ -199,7 +201,7 @@ handle_info({keepalive, Req}, State = #state{keepalive = KState0, {ok, KState} -> {noreply, State#state{keepalive = KState}, ?HIBERNATE_AFTER}; {error, timeout} -> - rabbit_log_connection:error("closing MQTT connection ~p (keepalive timeout)", [ConnName]), + ?LOG_ERROR("closing MQTT connection ~p (keepalive timeout)", [ConnName]), {stop, {shutdown, keepalive_timeout}, State}; {error, Reason} -> {stop, Reason, State} @@ -213,7 +215,7 @@ handle_info(login_timeout, State = #state{conn_name = ConnName}) -> %% the connection is blocked because of a resource alarm. However %% we don't know what is in the buffer, it can be arbitrary bytes, %% and we don't want to skip closing the connection in that case. - rabbit_log_connection:error("closing MQTT connection ~tp (login timeout)", [ConnName]), + ?LOG_ERROR("closing MQTT connection ~tp (login timeout)", [ConnName]), {stop, {shutdown, login_timeout}, State}; handle_info(emit_stats, State) -> @@ -241,7 +243,7 @@ handle_info({'DOWN', _MRef, process, QPid, _Reason}, State) -> handle_info({shutdown, Explanation} = Reason, State = #state{conn_name = ConnName}) -> %% rabbitmq_management plugin requests to close connection. - rabbit_log_connection:info("MQTT closing connection ~tp: ~p", [ConnName, Explanation]), + ?LOG_INFO("MQTT closing connection ~tp: ~p", [ConnName, Explanation]), {stop, Reason, State}; handle_info(Msg, State) -> @@ -258,8 +260,7 @@ terminate(Reason, {SendWill, State = #state{conn_name = ConnName, log_terminate(Reason, State). log_terminate({network_error, {ssl_upgrade_error, closed}, ConnName}, _State) -> - rabbit_log_connection:error("MQTT detected TLS upgrade error on ~s: connection closed", - [ConnName]); + ?LOG_ERROR("MQTT detected TLS upgrade error on ~s: connection closed", [ConnName]); log_terminate({network_error, {ssl_upgrade_error, @@ -278,18 +279,16 @@ log_terminate({network_error, {tls_alert, Alert}}, ConnName}, _State) -> log_tls_alert(Alert, ConnName); log_terminate({network_error, {ssl_upgrade_error, Reason}, ConnName}, _State) -> - rabbit_log_connection:error("MQTT detected TLS upgrade error on ~s: ~p", - [ConnName, Reason]); + ?LOG_ERROR("MQTT detected TLS upgrade error on ~s: ~p", [ConnName, Reason]); log_terminate({network_error, Reason, ConnName}, _State) -> - rabbit_log_connection:error("MQTT detected network error on ~s: ~p", - [ConnName, Reason]); + ?LOG_ERROR("MQTT detected network error on ~s: ~p", [ConnName, Reason]); log_terminate({network_error, Reason}, _State) -> - rabbit_log_connection:error("MQTT detected network error: ~p", [Reason]); + ?LOG_ERROR("MQTT detected network error: ~p", [Reason]); log_terminate(normal, #state{conn_name = ConnName}) -> - rabbit_log_connection:info("closing MQTT connection ~p (~s)", [self(), ConnName]), + ?LOG_INFO("closing MQTT connection ~p (~s)", [self(), ConnName]), ok; log_terminate(_Reason, _State) -> @@ -301,20 +300,18 @@ code_change(_OldVsn, State, _Extra) -> %%---------------------------------------------------------------------------- log_tls_alert(handshake_failure, ConnName) -> - rabbit_log_connection:error("MQTT detected TLS upgrade error on ~ts: handshake failure", - [ConnName]); + ?LOG_ERROR("MQTT detected TLS upgrade error on ~ts: handshake failure", [ConnName]); log_tls_alert(unknown_ca, ConnName) -> - rabbit_log_connection:error("MQTT detected TLS certificate verification error on ~ts: alert 'unknown CA'", - [ConnName]); + ?LOG_ERROR("MQTT detected TLS certificate verification error on ~ts: alert 'unknown CA'", + [ConnName]); log_tls_alert(Alert, ConnName) -> - rabbit_log_connection:error("MQTT detected TLS upgrade error on ~ts: alert ~ts", - [ConnName, Alert]). + ?LOG_ERROR("MQTT detected TLS upgrade error on ~ts: alert ~ts", [ConnName, Alert]). process_received_bytes(<<>>, State = #state{received_connect_packet = false, proc_state = PState, conn_name = ConnName}) -> - rabbit_log_connection:info("Accepted MQTT connection ~p (~s, client id: ~s)", - [self(), ConnName, rabbit_mqtt_processor:info(client_id, PState)]), + ?LOG_INFO("Accepted MQTT connection ~p (~s, client ID: ~s)", + [self(), ConnName, rabbit_mqtt_processor:info(client_id, PState)]), {noreply, ensure_stats_timer(State#state{received_connect_packet = true}), ?HIBERNATE_AFTER}; process_received_bytes(<<>>, State) -> {noreply, ensure_stats_timer(State), ?HIBERNATE_AFTER}; @@ -337,38 +334,36 @@ process_received_bytes(Bytes, proc_state = ProcState1}); %% PUBLISH and more {error, unauthorized = Reason, ProcState1} -> - rabbit_log_connection:error("MQTT connection ~ts is closing due to an authorization failure", [ConnName]), + ?LOG_ERROR("MQTT connection ~ts is closing due to an authorization failure", [ConnName]), {stop, {shutdown, Reason}, pstate(State, ProcState1)}; %% CONNECT packets only {error, unauthenticated = Reason, ProcState1} -> - rabbit_log_connection:error("MQTT connection ~ts is closing due to an authentication failure", [ConnName]), + ?LOG_ERROR("MQTT connection ~ts is closing due to an authentication failure", [ConnName]), {stop, {shutdown, Reason}, pstate(State, ProcState1)}; %% CONNECT packets only {error, invalid_client_id = Reason, ProcState1} -> - rabbit_log_connection:error("MQTT cannot accept connection ~ts: client uses an invalid ID", [ConnName]), + ?LOG_ERROR("MQTT cannot accept connection ~ts: client uses an invalid ID", [ConnName]), {stop, {shutdown, Reason}, pstate(State, ProcState1)}; %% CONNECT packets only {error, unsupported_protocol_version = Reason, ProcState1} -> - rabbit_log_connection:error("MQTT cannot accept connection ~ts: incompatible protocol version", [ConnName]), + ?LOG_ERROR("MQTT cannot accept connection ~ts: incompatible protocol version", [ConnName]), {stop, {shutdown, Reason}, pstate(State, ProcState1)}; {error, unavailable = Reason, ProcState1} -> - rabbit_log_connection:error("MQTT cannot accept connection ~ts due to an internal error or unavailable component", - [ConnName]), + ?LOG_ERROR("MQTT cannot accept connection ~ts due to an internal error or unavailable component", + [ConnName]), {stop, {shutdown, Reason}, pstate(State, ProcState1)}; {error, Reason, ProcState1} -> - rabbit_log_connection:error("MQTT protocol error on connection ~ts: ~tp", - [ConnName, Reason]), + ?LOG_ERROR("MQTT protocol error on connection ~ts: ~tp", [ConnName, Reason]), {stop, {shutdown, Reason}, pstate(State, ProcState1)}; {stop, disconnect, ProcState1} -> {stop, normal, {_SendWill = false, pstate(State, ProcState1)}} end; {error, {cannot_parse, Error, Stacktrace}} -> - rabbit_log_connection:error("MQTT cannot parse a packet on connection '~ts', unparseable payload: ~tp, error: {~tp, ~tp} ", - [ConnName, Bytes, Error, Stacktrace]), + ?LOG_ERROR("MQTT cannot parse a packet on connection '~ts', unparseable payload: ~tp, error: {~tp, ~tp} ", + [ConnName, Bytes, Error, Stacktrace]), {stop, {shutdown, Error}, State}; {error, Error} -> - rabbit_log_connection:error("MQTT detected a framing error on connection ~ts: ~tp", - [ConnName, Error]), + ?LOG_ERROR("MQTT detected a framing error on connection ~ts: ~tp", [ConnName, Error]), {stop, {shutdown, Error}, State} end. @@ -391,15 +386,14 @@ network_error(closed, Fmt = "MQTT connection ~p will terminate because peer closed TCP connection", Args = [ConnName], case Connected of - true -> rabbit_log_connection:info(Fmt, Args); - false -> rabbit_log_connection:debug(Fmt, Args) + true -> ?LOG_INFO(Fmt, Args); + false -> ?LOG_DEBUG(Fmt, Args) end, {stop, {shutdown, conn_closed}, State}; network_error(Reason, State = #state{conn_name = ConnName}) -> - rabbit_log_connection:info("MQTT detected network error for ~p: ~p", - [ConnName, Reason]), + ?LOG_INFO("MQTT detected network error for ~p: ~p", [ConnName, Reason]), {stop, {shutdown, conn_closed}, State}. run_socket(State = #state{ connection_state = blocked }) -> diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_app.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_app.erl index 1ad2baa3886f..b93242801a67 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_app.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_app.erl @@ -106,7 +106,7 @@ start_tcp_listener(TCPConf0, CowboyOpts) -> {error, {already_started, _}} -> ok; {error, ErrTCP} -> - rabbit_log_connection:error( + rabbit_log:error( "Failed to start a WebSocket (HTTP) listener. Error: ~p, listener settings: ~p", [ErrTCP, TCPConf]), throw(ErrTCP) @@ -132,7 +132,7 @@ start_tls_listener(TLSConf0, CowboyOpts) -> {error, {already_started, _}} -> ok; {error, ErrTLS} -> - rabbit_log_connection:error( + rabbit_log:error( "Failed to start a TLS WebSocket (HTTPS) listener. Error: ~p, listener settings: ~p", [ErrTLS, TLSConf]), throw(ErrTLS) diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index 732c6303e2a5..9e8f492d2447 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -9,6 +9,8 @@ -behaviour(cowboy_websocket). -behaviour(cowboy_sub_protocol). +-include_lib("kernel/include/logger.hrl"). +-include_lib("rabbit_common/include/logging.hrl"). -include_lib("rabbitmq_mqtt/include/rabbit_mqtt.hrl"). -export([ @@ -93,11 +95,12 @@ init(Req, Opts) -> {cowboy_websocket:commands(), state()} | {cowboy_websocket:commands(), state(), hibernate}. websocket_init({State0 = #state{socket = Sock}, PeerAddr}) -> + logger:set_process_metadata(#{domain => ?RMQLOG_DOMAIN_CONN ++ [web_mqtt]}), ok = file_handle_cache:obtain(), case rabbit_net:connection_string(Sock, inbound) of {ok, ConnStr} -> ConnName = rabbit_data_coercion:to_binary(ConnStr), - rabbit_log_connection:info("Accepting Web MQTT connection ~s", [ConnName]), + ?LOG_INFO("Accepting Web MQTT connection ~s", [ConnName]), rabbit_alarm:register(self(), {?MODULE, conserve_resources, []}), PState = rabbit_mqtt_processor:initial_state( rabbit_net:unwrap_socket(Sock), @@ -134,8 +137,7 @@ websocket_handle(Ping, State) {[], State, hibernate}; %% Log and close connection when receiving any other unexpected frames. websocket_handle(Frame, State) -> - rabbit_log_connection:info("Web MQTT: unexpected WebSocket frame ~tp", - [Frame]), + ?LOG_INFO("Web MQTT: unexpected WebSocket frame ~tp", [Frame]), stop(State, ?CLOSE_UNACCEPTABLE_DATA_TYPE, <<"unexpected WebSocket frame">>). -spec websocket_info(any(), State) -> @@ -156,18 +158,18 @@ websocket_info({'$gen_cast', QueueEvent = {queue_event, _, _}}, {ok, PState} -> handle_credits(State#state{proc_state = PState}); {error, Reason, PState} -> - rabbit_log_connection:error("Web MQTT connection ~p failed to handle queue event: ~p", - [State#state.conn_name, Reason]), + ?LOG_ERROR("Web MQTT connection ~p failed to handle queue event: ~p", + [State#state.conn_name, Reason]), stop(State#state{proc_state = PState}) end; websocket_info({'$gen_cast', duplicate_id}, State = #state{ proc_state = ProcState, conn_name = ConnName }) -> - rabbit_log_connection:warning("Web MQTT disconnecting a client with duplicate ID '~s' (~p)", + ?LOG_WARNING("Web MQTT disconnecting a client with duplicate ID '~s' (~p)", [rabbit_mqtt_processor:info(client_id, ProcState), ConnName]), stop(State); websocket_info({'$gen_cast', {close_connection, Reason}}, State = #state{ proc_state = ProcState, conn_name = ConnName }) -> - rabbit_log_connection:warning("Web MQTT disconnecting client with ID '~s' (~p), reason: ~s", + ?LOG_WARNING("Web MQTT disconnecting client with ID '~s' (~p), reason: ~s", [rabbit_mqtt_processor:info(client_id, ProcState), ConnName, Reason]), stop(State); websocket_info({'$gen_cast', {force_event_refresh, Ref}}, State0) -> @@ -181,12 +183,11 @@ websocket_info({keepalive, Req}, State = #state{keepalive = KState0, {ok, KState} -> {[], State#state{keepalive = KState}, hibernate}; {error, timeout} -> - rabbit_log_connection:error("keepalive timeout in Web MQTT connection ~p", - [ConnName]), + ?LOG_ERROR("keepalive timeout in Web MQTT connection ~p", [ConnName]), stop(State, ?CLOSE_NORMAL, <<"MQTT keepalive timeout">>); {error, Reason} -> - rabbit_log_connection:error("keepalive error in Web MQTT connection ~p: ~p", - [ConnName, Reason]), + ?LOG_ERROR("keepalive error in Web MQTT connection ~p: ~p", + [ConnName, Reason]), stop(State) end; websocket_info(emit_stats, State) -> @@ -208,7 +209,7 @@ websocket_info({'DOWN', _MRef, process, QPid, _Reason}, State) -> {[], State, hibernate}; websocket_info({shutdown, Reason}, #state{conn_name = ConnName} = State) -> %% rabbitmq_management plugin requests to close connection. - rabbit_log_connection:info("Web MQTT closing connection ~tp: ~tp", [ConnName, Reason]), + ?LOG_INFO("Web MQTT closing connection ~tp: ~tp", [ConnName, Reason]), stop(State, ?CLOSE_NORMAL, Reason); websocket_info(connection_created, State) -> Infos = infos(?CREATION_EVENT_KEYS, State), @@ -216,7 +217,7 @@ websocket_info(connection_created, State) -> rabbit_event:notify(connection_created, Infos), {[], State, hibernate}; websocket_info(Msg, State) -> - rabbit_log_connection:warning("Web MQTT: unexpected message ~tp", [Msg]), + ?LOG_WARNING("Web MQTT: unexpected message ~tp", [Msg]), {[], State, hibernate}. -spec terminate(any(), cowboy_req:req(), any()) -> ok. @@ -228,7 +229,7 @@ terminate(_Reason, _Request, {SendWill, #state{conn_name = ConnName, proc_state = PState, keepalive = KState} = State}) -> - rabbit_log_connection:info("Web MQTT closing connection ~ts", [ConnName]), + ?LOG_INFO("Web MQTT closing connection ~ts", [ConnName]), maybe_emit_stats(State), rabbit_mqtt_keepalive:cancel_timer(KState), ok = file_handle_cache:release(), @@ -238,8 +239,7 @@ terminate(_Reason, _Request, no_supported_sub_protocol(Protocol, Req) -> %% The client MUST include “mqtt” in the list of WebSocket Sub Protocols it offers [MQTT-6.0.0-3]. - rabbit_log_connection:error( - "Web MQTT: 'mqtt' not included in client offered subprotocols: ~tp", [Protocol]), + ?LOG_ERROR("Web MQTT: 'mqtt' not included in client offered subprotocols: ~tp", [Protocol]), {ok, cowboy_req:reply(400, #{<<"connection">> => <<"close">>}, Req), #state{}}. handle_data(Data, State0 = #state{}) -> @@ -253,8 +253,8 @@ handle_data(Data, State0 = #state{}) -> handle_data1(<<>>, State0 = #state{received_connect_packet = false, proc_state = PState, conn_name = ConnName}) -> - rabbit_log_connection:info("Accepted web MQTT connection ~p (~s, client id: ~s)", - [self(), ConnName, rabbit_mqtt_processor:info(client_id, PState)]), + ?LOG_INFO("Accepted web MQTT connection ~p (~s, client ID: ~s)", + [self(), ConnName, rabbit_mqtt_processor:info(client_id, PState)]), State = State0#state{received_connect_packet = true}, {ok, ensure_stats_timer(control_throttle(State)), hibernate}; handle_data1(<<>>, State) -> @@ -288,14 +288,13 @@ parse(Data, ParseState) -> rabbit_mqtt_packet:parse(Data, ParseState) catch _:Error:Stacktrace -> - rabbit_log_connection:error("MQTT cannot parse a packet; payload: ~tp, error: {~tp, ~tp} ", - [Data, Error, Stacktrace]), + ?LOG_ERROR("MQTT cannot parse a packet; payload: ~tp, error: {~tp, ~tp} ", + [Data, Error, Stacktrace]), {error, cannot_parse} end. stop_mqtt_protocol_error(State, Reason, ConnName) -> - rabbit_log_connection:info("MQTT protocol error ~tp for connection ~tp", - [Reason, ConnName]), + ?LOG_INFO("MQTT protocol error ~tp for connection ~tp", [Reason, ConnName]), stop(State, ?CLOSE_PROTOCOL_ERROR, Reason). stop(State) -> From fb93a3c17d3363161ab03e8ba5398db1db67460e Mon Sep 17 00:00:00 2001 From: David Ansari Date: Wed, 4 Jan 2023 18:29:28 +0000 Subject: [PATCH 097/118] Block only publishing (Web) MQTT connections When a cluster wide memory or disk alarm is fired, in AMQP 0.9.1 only connections that are publishing messages get blocked. Connections that only consume can continue to empty the queues. Prior to this commit, all MQTT connections got blocked during a memory or disk alarm. This has two downsides: 1. MQTT connections that only consume, but never published, cannot empty queues anymore. 2. If the memory or disk alarm takes long, the MQTT client does not receive a PINGRESP from the server when it sends a PINGREQ potentially leading to mass client disconnection (depending on the MQTT client implementation). This commit makes sure that an MQTT connection that never sent a single PUBLISH packet (e.g. "pure" MQTT subscribers) are not blocked during memory or disk alarms. In contrast to AMQP 0.9.1, new connections are still blocked from being accepted because accepting (many) new MQTT connections also lead to increased resource usage. The implemenation as done in this commit is simpler, but also more naive than the logic in rabbit_reader: rabbit_reader blocks connections more dynamically whereas rabbit_mqtt_reader and rabbit_web_mqtt_handler block a connection if the connection ever sent a single PUBLISH packet during its lifetime. --- .../src/rabbit_mqtt_processor.erl | 38 +++++++------ deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 10 ++-- deps/rabbitmq_mqtt/test/reader_SUITE.erl | 2 +- deps/rabbitmq_mqtt/test/shared_SUITE.erl | 55 ++++++++++++++++++- .../src/rabbit_web_mqtt_handler.erl | 10 ++-- 5 files changed, 84 insertions(+), 31 deletions(-) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 5413daa788e6..f4594d7fdf67 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -12,8 +12,7 @@ process_packet/2, serialise/2, terminate/4, handle_pre_hibernate/0, handle_ra_event/2, handle_down/2, handle_queue_event/2, - soft_limit_exceeded/1, proto_version_tuple/1, - format_status/1]). + proto_version_tuple/1, throttle/3, format_status/1]). %% for testing purposes -export([get_vhost_username/1, get_vhost/3, get_vhost_from_user_mapping/2]). @@ -64,7 +63,8 @@ exchange :: option(rabbit_exchange:name()), %% Set if client has at least one subscription with QoS 1. queue_qos1 :: option(rabbit_amqqueue:name()), - has_published = false :: boolean(), + %% Did the client ever sent us a PUBLISH packet? + published = false :: boolean(), ssl_login_name :: none | binary(), retainer_pid :: option(pid()), auth_state :: option(#auth_state{}), @@ -75,7 +75,7 @@ info :: option(#info{}), delivery_flow :: flow | noflow, %% quorum queues and streams whose soft limit has been exceeded - soft_limit_exceeded = sets:new([{version, 2}]) :: sets:set(), + queues_soft_limit_exceeded = sets:new([{version, 2}]) :: sets:set(), qos0_messages_dropped = 0 :: non_neg_integer() }). @@ -1504,10 +1504,10 @@ handle_queue_actions(Actions, #state{} = State0) -> end end, U0, PktIds), S#state{unacked_client_pubs = U}; - ({block, QName}, S = #state{soft_limit_exceeded = SLE}) -> - S#state{soft_limit_exceeded = sets:add_element(QName, SLE)}; - ({unblock, QName}, S = #state{soft_limit_exceeded = SLE}) -> - S#state{soft_limit_exceeded = sets:del_element(QName, SLE)}; + ({block, QName}, S = #state{queues_soft_limit_exceeded = QSLE}) -> + S#state{queues_soft_limit_exceeded = sets:add_element(QName, QSLE)}; + ({unblock, QName}, S = #state{queues_soft_limit_exceeded = QSLE}) -> + S#state{queues_soft_limit_exceeded = sets:del_element(QName, QSLE)}; ({queue_down, QName}, S) -> handle_queue_down(QName, S) end, State0, Actions). @@ -1737,6 +1737,13 @@ is_socket_busy(Socket) -> false end. +-spec throttle(boolean(), boolean(), state()) -> boolean(). +throttle(Conserve, Connected, #state{published = Published, + queues_soft_limit_exceeded = QSLE}) -> + Conserve andalso (Published orelse not Connected) orelse + not sets:is_empty(QSLE) orelse + credit_flow:blocked(). + info(host, #state{info = #info{host = Val}}) -> Val; info(port, #state{info = #info{port = Val}}) -> Val; info(peer_host, #state{info = #info{peer_host = Val}}) -> Val; @@ -1798,8 +1805,9 @@ format_status(#state{queue_states = QState, register_state = RegisterState, conn_name = ConnName, info = Info, + queues_soft_limit_exceeded = QSLE, qos0_messages_dropped = Qos0MsgsDropped - } = State) -> + }) -> #{queue_states => rabbit_queue_type:format_status(QState), proto_ver => ProtoVersion, unacked_client_pubs => UnackClientPubs, @@ -1816,13 +1824,9 @@ format_status(#state{queue_states = QState, register_state => RegisterState, conn_name => ConnName, info => Info, - soft_limit_exceeded => soft_limit_exceeded(State), + queues_soft_limit_exceeded => sets:size(QSLE), qos0_messages_dropped => Qos0MsgsDropped}. --spec soft_limit_exceeded(state()) -> boolean(). -soft_limit_exceeded(#state{soft_limit_exceeded = SLE}) -> - not sets:is_empty(SLE). - proto_integer_to_atom(3) -> ?MQTT_PROTO_V3; proto_integer_to_atom(4) -> @@ -1834,14 +1838,14 @@ proto_version_tuple(#state{proto_ver = ?MQTT_PROTO_V3}) -> proto_version_tuple(#state{proto_ver = ?MQTT_PROTO_V4}) -> {3, 1, 1}. -maybe_increment_publisher(State = #state{has_published = false, +maybe_increment_publisher(State = #state{published = false, proto_ver = ProtoVer}) -> rabbit_global_counters:publisher_created(ProtoVer), - State#state{has_published = true}; + State#state{published = true}; maybe_increment_publisher(State) -> State. -maybe_decrement_publisher(#state{has_published = true, +maybe_decrement_publisher(#state{published = true, proto_ver = ProtoVer}) -> rabbit_global_counters:publisher_deleted(ProtoVer); maybe_decrement_publisher(_) -> diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index 1a96dc557426..f7db17f09c70 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -408,11 +408,11 @@ run_socket(State = #state{ socket = Sock }) -> control_throttle(State = #state{connection_state = ConnState, conserve = Conserve, - keepalive = KState, - proc_state = PState}) -> - Throttle = Conserve orelse - rabbit_mqtt_processor:soft_limit_exceeded(PState) orelse - credit_flow:blocked(), + received_connect_packet = Connected, + proc_state = PState, + keepalive = KState + }) -> + Throttle = rabbit_mqtt_processor:throttle(Conserve, Connected, PState), case {ConnState, Throttle} of {running, true} -> State#state{connection_state = blocked, diff --git a/deps/rabbitmq_mqtt/test/reader_SUITE.erl b/deps/rabbitmq_mqtt/test/reader_SUITE.erl index 32a9b239fa75..4c65e20e4790 100644 --- a/deps/rabbitmq_mqtt/test/reader_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/reader_SUITE.erl @@ -95,7 +95,7 @@ block_connack_timeout(Config) -> P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt), Ports0 = rpc(Config, erlang, ports, []), - ok = rpc(Config, vm_memory_monitor, set_vm_memory_high_watermark, [0.00000001]), + ok = rpc(Config, vm_memory_monitor, set_vm_memory_high_watermark, [0]), %% Let connection block. timer:sleep(100), diff --git a/deps/rabbitmq_mqtt/test/shared_SUITE.erl b/deps/rabbitmq_mqtt/test/shared_SUITE.erl index 6a4897f9e4fb..b58ed3a6de6f 100644 --- a/deps/rabbitmq_mqtt/test/shared_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/shared_SUITE.erl @@ -62,7 +62,8 @@ subgroups() -> ]}, {tests, [], [ - many_qos1_messages + block_only_publisher + ,many_qos1_messages ] ++ tests()} ]}, {cluster_size_3, [], @@ -1069,7 +1070,7 @@ block(Config) -> {ok, _, _} = emqtt:subscribe(C, Topic), {ok, _} = emqtt:publish(C, Topic, <<"Not blocked yet">>, [{qos, 1}]), - ok = rpc(Config, 0, vm_memory_monitor, set_vm_memory_high_watermark, [0.00000001]), + ok = rpc(Config, vm_memory_monitor, set_vm_memory_high_watermark, [0]), %% Let it block timer:sleep(100), @@ -1078,12 +1079,60 @@ block(Config) -> puback_timeout = publish_qos1_timeout(C, Topic, <<"Still blocked">>, 1000), %% Unblock - rpc(Config, 0, vm_memory_monitor, set_vm_memory_high_watermark, [0.4]), + rpc(Config, vm_memory_monitor, set_vm_memory_high_watermark, [0.4]), ok = expect_publishes(C, Topic, [<<"Not blocked yet">>, <<"Now blocked">>, <<"Still blocked">>]), ok = emqtt:disconnect(C). +block_only_publisher(Config) -> + Topic = atom_to_binary(?FUNCTION_NAME), + + Opts = [{ack_timeout, 1}], + Con = connect(<<"background-connection">>, Config, Opts), + Sub = connect(<<"subscriber-connection">>, Config, Opts), + Pub = connect(<<"publisher-connection">>, Config, Opts), + PubSub = connect(<<"publisher-and-subscriber-connection">>, Config, Opts), + + {ok, _, [1]} = emqtt:subscribe(Sub, Topic, qos1), + {ok, _, [1]} = emqtt:subscribe(PubSub, Topic, qos1), + {ok, _} = emqtt:publish(Pub, Topic, <<"from Pub">>, [{qos, 1}]), + {ok, _} = emqtt:publish(PubSub, Topic, <<"from PubSub">>, [{qos, 1}]), + ok = expect_publishes(Sub, Topic, [<<"from Pub">>, <<"from PubSub">>]), + ok = expect_publishes(PubSub, Topic, [<<"from Pub">>, <<"from PubSub">>]), + + ok = rpc(Config, vm_memory_monitor, set_vm_memory_high_watermark, [0]), + %% Let it block + timer:sleep(100), + + %% We expect that the publishing connections are blocked. + [?assertEqual({error, ack_timeout}, emqtt:ping(Pid)) || Pid <- [Pub, PubSub]], + %% We expect that the non-publishing connections are not blocked. + [?assertEqual(pong, emqtt:ping(Pid)) || Pid <- [Con, Sub]], + + %% While the memory alarm is on, let's turn a non-publishing connection + %% into a publishing connection. + {ok, _} = emqtt:publish(Con, Topic, <<"from Con 1">>, [{qos, 1}]), + %% The very first message still goes through. + ok = expect_publishes(Sub, Topic, [<<"from Con 1">>]), + %% But now the new publisher should be blocked as well. + ?assertEqual({error, ack_timeout}, emqtt:ping(Con)), + ?assertEqual(puback_timeout, publish_qos1_timeout(Con, Topic, <<"from Con 2">>, 500)), + ?assertEqual(pong, emqtt:ping(Sub)), + + rpc(Config, vm_memory_monitor, set_vm_memory_high_watermark, [0.4]), + %% Let it unblock + timer:sleep(100), + + %% All connections are unblocked. + [?assertEqual(pong, emqtt:ping(Pid)) || Pid <- [Con, Sub, Pub, PubSub]], + %% The publishing connections should be able to publish again. + {ok, _} = emqtt:publish(Con, Topic, <<"from Con 3">>, [{qos, 1}]), + ok = expect_publishes(Sub, Topic, [<<"from Con 2">>, <<"from Con 3">>]), + ok = expect_publishes(PubSub, Topic, [<<"from Con 1">>, <<"from Con 2">>, <<"from Con 3">>]), + + [ok = emqtt:disconnect(Pid) || Pid <- [Con, Sub, Pub, PubSub]]. + clean_session_disconnect_client(Config) -> C = connect(?FUNCTION_NAME, Config), {ok, _, _} = emqtt:subscribe(C, <<"topic0">>, qos0), diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index 9e8f492d2447..8e97e8221f19 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -314,11 +314,11 @@ handle_credits(State0) -> control_throttle(State = #state{connection_state = ConnState, conserve = Conserve, - keepalive = KState, - proc_state = PState}) -> - Throttle = Conserve orelse - rabbit_mqtt_processor:soft_limit_exceeded(PState) orelse - credit_flow:blocked(), + received_connect_packet = Connected, + proc_state = PState, + keepalive = KState + }) -> + Throttle = rabbit_mqtt_processor:throttle(Conserve, Connected, PState), case {ConnState, Throttle} of {running, true} -> State#state{connection_state = blocked, From 9283b4f4f61b18907bf9a86b7e1493fd2de9c097 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Thu, 5 Jan 2023 15:29:42 +0000 Subject: [PATCH 098/118] Add test AMQP 0.9.1 to MQTT with QoS 0 --- deps/rabbitmq_mqtt/test/shared_SUITE.erl | 24 +++++++++++++++++++----- 1 file changed, 19 insertions(+), 5 deletions(-) diff --git a/deps/rabbitmq_mqtt/test/shared_SUITE.erl b/deps/rabbitmq_mqtt/test/shared_SUITE.erl index b58ed3a6de6f..baad5ce9cfb6 100644 --- a/deps/rabbitmq_mqtt/test/shared_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/shared_SUITE.erl @@ -101,6 +101,7 @@ tests() -> ,subscribe_multiple ,large_message_mqtt_to_mqtt ,large_message_amqp_to_mqtt + ,amqp_to_mqtt_qos0 ,keepalive ,keepalive_turned_off ,duplicate_client_id @@ -279,7 +280,7 @@ will_without_disconnect(Config) -> ok = emqtt:disconnect(Sub). quorum_queue_rejects(Config) -> - {_Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0), + {_Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config), Name = atom_to_binary(?FUNCTION_NAME), ok = rabbit_ct_broker_helpers:set_policy( @@ -314,7 +315,7 @@ publish_to_all_queue_types_qos1(Config) -> publish_to_all_queue_types(Config, qos1). publish_to_all_queue_types(Config, QoS) -> - Ch = rabbit_ct_client_helpers:open_channel(Config, 0), + Ch = rabbit_ct_client_helpers:open_channel(Config), CQ = <<"classic-queue">>, CMQ = <<"classic-mirrored-queue">>, @@ -390,7 +391,7 @@ flow(Config, {App, Par, Val}, QueueType) Result = rpc_all(Config, application, set_env, [App, Par, Val]), ?assert(lists:all(fun(R) -> R =:= ok end, Result)), - Ch = rabbit_ct_client_helpers:open_channel(Config, 0), + Ch = rabbit_ct_client_helpers:open_channel(Config), QueueName = Topic = atom_to_binary(?FUNCTION_NAME), declare_queue(Ch, QueueName, [{<<"x-queue-type">>, longstr, QueueType}]), bind(Ch, QueueName, Topic), @@ -696,7 +697,7 @@ consuming_classic_queue_down(Config) -> ok. delete_create_queue(Config) -> - Ch = rabbit_ct_client_helpers:open_channel(Config, 0), + Ch = rabbit_ct_client_helpers:open_channel(Config), CQ1 = <<"classic-queue-1-delete-create">>, CQ2 = <<"classic-queue-2-delete-create">>, QQ = <<"quorum-queue-delete-create">>, @@ -857,7 +858,7 @@ large_message_amqp_to_mqtt(Config) -> C = connect(ClientId, Config), {ok, _, [1]} = emqtt:subscribe(C, {Topic, qos1}), - Ch = rabbit_ct_client_helpers:open_channel(Config, 0), + Ch = rabbit_ct_client_helpers:open_channel(Config), Payload0 = binary:copy(<<"x">>, 8_000_000), Payload = <>, amqp_channel:call(Ch, @@ -867,6 +868,19 @@ large_message_amqp_to_mqtt(Config) -> ok = expect_publishes(C, Topic, [Payload]), ok = emqtt:disconnect(C). +amqp_to_mqtt_qos0(Config) -> + Topic = ClientId = Payload = atom_to_binary(?FUNCTION_NAME), + C = connect(ClientId, Config), + {ok, _, [0]} = emqtt:subscribe(C, {Topic, qos0}), + + Ch = rabbit_ct_client_helpers:open_channel(Config), + amqp_channel:call(Ch, + #'basic.publish'{exchange = <<"amq.topic">>, + routing_key = Topic}, + #amqp_msg{payload = Payload}), + ok = expect_publishes(C, Topic, [Payload]), + ok = emqtt:disconnect(C). + %% Packet identifier is a non zero two byte integer. %% Test that the server wraps around the packet identifier. many_qos1_messages(Config) -> From a341912b75587768eab02238576ccf2d4d770f28 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Tue, 10 Jan 2023 09:20:19 +0000 Subject: [PATCH 099/118] Expand clean_session=false test --- deps/rabbitmq_mqtt/test/shared_SUITE.erl | 34 +++++++++++++++++------- 1 file changed, 25 insertions(+), 9 deletions(-) diff --git a/deps/rabbitmq_mqtt/test/shared_SUITE.erl b/deps/rabbitmq_mqtt/test/shared_SUITE.erl index baad5ce9cfb6..3288081ae1bf 100644 --- a/deps/rabbitmq_mqtt/test/shared_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/shared_SUITE.erl @@ -758,32 +758,48 @@ delete_create_queue(Config) -> ok = emqtt:disconnect(C). non_clean_sess_disconnect(Config) -> - C1 = connect(?FUNCTION_NAME, Config, [{clean_start, false}]), - Topic = <<"test-topic1">>, - {ok, _, [1]} = emqtt:subscribe(C1, Topic, qos1), + Pub = connect(<<"publisher">>, Config), + Topic = ClientId = atom_to_binary(?FUNCTION_NAME), ProtoVer = v4, + + C1 = connect(ClientId, Config, [{clean_start, false}]), + {ok, _, [1]} = emqtt:subscribe(C1, Topic, qos1), ?assertMatch(#{consumers := 1}, get_global_counters(Config, ProtoVer)), + ok = emqtt:publish(Pub, Topic, <<"msg-1-qos0">>, qos0), + {ok, _} = emqtt:publish(Pub, Topic, <<"msg-2-qos1">>, qos1), + ok = emqtt:disconnect(C1), ?assertMatch(#{consumers := 0}, get_global_counters(Config, ProtoVer)), - C2 = connect(?FUNCTION_NAME, Config, [{clean_start, false}]), + timer:sleep(50), + ok = emqtt:publish(Pub, Topic, <<"msg-3-qos0">>, qos0), + {ok, _} = emqtt:publish(Pub, Topic, <<"msg-4-qos1">>, qos1), + + C2 = connect(ClientId, Config, [{clean_start, false}]), ?assertMatch(#{consumers := 1}, get_global_counters(Config, ProtoVer)), + ok = emqtt:publish(Pub, Topic, <<"msg-5-qos0">>, qos0), + {ok, _} = emqtt:publish(Pub, Topic, <<"msg-6-qos1">>, qos1), + %% shouldn't receive message after unsubscribe {ok, _, _} = emqtt:unsubscribe(C2, Topic), ?assertMatch(#{consumers := 0}, get_global_counters(Config, ProtoVer)), - Msg = <<"msg">>, - {ok, _} = emqtt:publish(C2, Topic, Msg, qos1), - {publish_not_received, Msg} = expect_publishes(C2, Topic, [Msg]), - ok = emqtt:disconnect(C2), + {ok, _} = emqtt:publish(Pub, Topic, <<"msg-7-qos0">>, qos1), + ok = expect_publishes(C1, Topic, [<<"msg-1-qos0">>, <<"msg-2-qos1">>]), + ok = expect_publishes(C2, Topic, [<<"msg-3-qos0">>, <<"msg-4-qos1">>, + <<"msg-5-qos0">>, <<"msg-6-qos1">>]), + {publish_not_received, <<"msg-7-qos0">>} = expect_publishes(C2, Topic, [<<"msg-7-qos0">>]), + + ok = emqtt:disconnect(Pub), + ok = emqtt:disconnect(C2), %% connect with clean sess true to clean up - C3 = connect(?FUNCTION_NAME, Config, [{clean_start, true}]), + C3 = connect(ClientId, Config, [{clean_start, true}]), ok = emqtt:disconnect(C3). subscribe_same_topic_same_qos(Config) -> From 35afeffceb9044082bbb4f060922dc150d6d4250 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Tue, 10 Jan 2023 12:30:27 +0000 Subject: [PATCH 100/118] Eliminate bindings query CPU bottleneck Prior to this commit, there was a CPU bottleneck (not present in 3.11.x) when creating, deleting or disconnecting many MQTT subscribers. Example: Add 120 MQTT connections per second each creating a subscription. Starting at around 300k MQTT subscribers, all 45 CPUs on the server were maxed out spending time in `ets` according to msacc. When running a similar workload with only 30k MQTT subscribers on a local server with only 5 CPUs, all 5 CPUs were maxed out and the CPU flame graph showed that 86% of the CPU time is spent in function rabbit_mqtt_processor:topic_names/2. This commit uses the rabbit_reverse_route table to query MQTT subscriptions for a given client ID. CPU usage is now drastically lower. The configured source topic exchange is always the same in the MQTT plugin. There is however a high cardinality in the destination queues (MQTT client IDs) and routing keys (topics). --- deps/rabbit/src/rabbit_binding.erl | 65 ++++++++++--------- .../src/rabbit_mqtt_processor.erl | 8 ++- 2 files changed, 42 insertions(+), 31 deletions(-) diff --git a/deps/rabbit/src/rabbit_binding.erl b/deps/rabbit/src/rabbit_binding.erl index f998903a51ff..b1b9ba963129 100644 --- a/deps/rabbit/src/rabbit_binding.erl +++ b/deps/rabbit/src/rabbit_binding.erl @@ -11,8 +11,8 @@ -export([recover/0, recover/2, exists/1, add/2, add/3, remove/1, remove/2, remove/3, remove/4]). -export([list/1, list_for_source/1, list_for_destination/1, - list_for_source_and_destination/2, list_explicit/0, - list_between/2, has_any_between/2]). + list_for_source_and_destination/2, list_for_source_and_destination/3, + list_explicit/0, list_between/2, has_any_between/2]). -export([new_deletions/0, combine_deletions/2, add_deletion/3, process_deletions/2, binding_action/3]). -export([info_keys/0, info/1, info/2, info_all/1, info_all/2, info_all/4]). @@ -253,26 +253,18 @@ list(VHostPath) -> list_for_source(?DEFAULT_EXCHANGE(VHostPath)) -> implicit_bindings(VHostPath); list_for_source(SrcName) -> - mnesia:async_dirty( - fun() -> - Route = #route{binding = #binding{source = SrcName, _ = '_'}}, - [B || #route{binding = B} - <- mnesia:match_object(rabbit_route, Route, read)] - end). + Route = #route{binding = #binding{source = SrcName, _ = '_'}}, + Fun = list_for_route(Route, false), + mnesia:async_dirty(Fun). -spec list_for_destination (rabbit_types:binding_destination()) -> bindings(). list_for_destination(DstName = #resource{}) -> - ExplicitBindings = mnesia:async_dirty( - fun() -> - Route = #route{binding = #binding{destination = DstName, - _ = '_'}}, - [reverse_binding(B) || - #reverse_route{reverse_binding = B} <- - mnesia:match_object(rabbit_reverse_route, - reverse_route(Route), read)] - end), + Route = #route{binding = #binding{destination = DstName, + _ = '_'}}, + Fun = list_for_route(Route, true), + ExplicitBindings = mnesia:async_dirty(Fun), implicit_for_destination(DstName) ++ ExplicitBindings. -spec list_between( @@ -316,27 +308,40 @@ implicit_for_destination(DstQueue = #resource{kind = queue, implicit_for_destination(_) -> []. --spec list_for_source_and_destination - (rabbit_types:binding_source(), rabbit_types:binding_destination()) -> - bindings(). +-spec list_for_source_and_destination(rabbit_types:binding_source(), rabbit_types:binding_destination()) -> + bindings(). +list_for_source_and_destination(SrcName, DstName) -> + list_for_source_and_destination(SrcName, DstName, false). +-spec list_for_source_and_destination(rabbit_types:binding_source(), rabbit_types:binding_destination(), boolean()) -> + bindings(). list_for_source_and_destination(?DEFAULT_EXCHANGE(VHostPath), #resource{kind = queue, virtual_host = VHostPath, - name = QName} = DstQueue) -> + name = QName} = DstQueue, + _Reverse) -> [#binding{source = ?DEFAULT_EXCHANGE(VHostPath), destination = DstQueue, key = QName, args = []}]; -list_for_source_and_destination(SrcName, DstName) -> - mnesia:async_dirty( - fun() -> - Route = #route{binding = #binding{source = SrcName, - destination = DstName, - _ = '_'}}, - [B || #route{binding = B} <- mnesia:match_object(rabbit_route, - Route, read)] - end). +list_for_source_and_destination(SrcName, DstName, Reverse) -> + Route = #route{binding = #binding{source = SrcName, + destination = DstName, + _ = '_'}}, + Fun = list_for_route(Route, Reverse), + mnesia:async_dirty(Fun). + +list_for_route(Route, false) -> + fun() -> + [B || #route{binding = B} <- mnesia:match_object(rabbit_route, Route, read)] + end; +list_for_route(Route, true) -> + fun() -> + [reverse_binding(B) || + #reverse_route{reverse_binding = B} <- + mnesia:match_object(rabbit_reverse_route, + reverse_route(Route), read)] + end. -spec info_keys() -> rabbit_types:info_keys(). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index f4594d7fdf67..5647afdec07c 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -581,7 +581,13 @@ has_subs(State) -> topic_names(?QOS_1, State) =/= []. topic_names(QoS, #state{exchange = Exchange} = State) -> - Bindings = rabbit_binding:list_for_source_and_destination(Exchange, queue_name(QoS, State)), + Bindings = rabbit_binding:list_for_source_and_destination( + Exchange, + queue_name(QoS, State), + %% Querying table rabbit_reverse_route instead of rabbit_route provides + %% **much** better CPU performance because the source exchange is always the + %% same in the MQTT plugin while the destination queue will be different. + _Reverse = true), lists:map(fun(B) -> amqp_to_mqtt(B#binding.key) end, Bindings). %% "If a Server receives a SUBSCRIBE Packet containing a Topic Filter that is identical From 863b7ea16a3357061c94fb79a38d2fea71c16d99 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Thu, 12 Jan 2023 11:34:57 +0000 Subject: [PATCH 101/118] Include non-AMQP connections in connection count Prior to this commit: ``` rabbitmqctl status ... Totals Connection count: 0 Queue count: 64308 Virtual host count: 1 ... ``` only counted AMQP connections, but did not include MQTT or stream connections. Let's include the count of all connections in the output of `rabbitmqctl status`. --- deps/rabbit/src/rabbit.erl | 3 ++- deps/rabbitmq_mqtt/test/shared_SUITE.erl | 10 ++++++++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/deps/rabbit/src/rabbit.erl b/deps/rabbit/src/rabbit.erl index ab80ed0e695e..bbb7b8ed5001 100644 --- a/deps/rabbit/src/rabbit.erl +++ b/deps/rabbit/src/rabbit.erl @@ -727,7 +727,8 @@ status() -> true -> [{virtual_host_count, rabbit_vhost:count()}, {connection_count, - length(rabbit_networking:connections_local())}, + length(rabbit_networking:connections_local()) + + length(rabbit_networking:local_non_amqp_connections())}, {queue_count, total_queue_count()}]; false -> [] diff --git a/deps/rabbitmq_mqtt/test/shared_SUITE.erl b/deps/rabbitmq_mqtt/test/shared_SUITE.erl index 3288081ae1bf..fe72f077d051 100644 --- a/deps/rabbitmq_mqtt/test/shared_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/shared_SUITE.erl @@ -108,6 +108,7 @@ tests() -> ,block ,clean_session_disconnect_client ,clean_session_kill_node + ,rabbit_status_connection_count ]. suite() -> @@ -1208,6 +1209,15 @@ clean_session_kill_node(Config) -> %% state to be cleaned up on the server once the server comes back up. ?assertEqual(0, rpc(Config, ets, info, [rabbit_durable_queue, size])). +rabbit_status_connection_count(Config) -> + _ = rabbit_ct_client_helpers:open_connection(Config, 0), + C = connect(?FUNCTION_NAME, Config), + + {ok, String} = rabbit_ct_broker_helpers:rabbitmqctl(Config, 0, ["status"]), + ?assertNotEqual(nomatch, string:find(String, "Connection count: 2")), + + ok = emqtt:disconnect(C). + %% ------------------------------------------------------------------- %% Internal helpers %% ------------------------------------------------------------------- From f46f0541ea231d13c6f8c5dd131033ed8a7fcee5 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Fri, 13 Jan 2023 16:15:15 +0000 Subject: [PATCH 102/118] Fix "Clean Session" state for QoS 0 subscriptions MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Up to RabbitMQ 3.11, the following bug existed. The MQTT 3.1.1. protocol spec mandates: ``` The Session state in the Server consists of: * The Client’s subscriptions. * ... ``` However, because QoS 0 queues were auto-delete up to 3.11 (or exclusive prior to this commit), QoS 0 queues and therefore their bindings were deleted when a non-clean session terminated. When the same client re-connected, its QoS 0 subscription was lost. Note that storing **messages** for QoS 0 subscription is optional while the client is disconnected. However, storing the subscription itself (i.e. bindings in RabbitMQ terms) is not optional: The client must receive new messages for its QoS 0 subscriptions after it reconnects without having to send a SUBSCRIBE packet again. "After the disconnection of a Session that had CleanSession set to 0, the Server MUST store further QoS 1 and QoS 2 messages that match any subscriptions that the client had at the time of disconnection as part of the Session state [MQTT-3.1.2-5]. It MAY also store QoS 0 messages that meet the same criteria." This commit additionally implements the last sentence. --- deps/rabbitmq_mqtt/src/rabbit_mqtt.erl | 2 +- .../src/rabbit_mqtt_processor.erl | 98 +++++++++++-------- deps/rabbitmq_mqtt/test/shared_SUITE.erl | 69 ++++++++++++- 3 files changed, 125 insertions(+), 44 deletions(-) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl index 63e120a0c91d..818c161b2892 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl @@ -16,7 +16,7 @@ -export([emit_connection_info_all/4, emit_connection_info_local/3, close_local_client_connections/1, - %% exported for tests + %% Exported for tests, but could also be used for debugging. local_connection_pids/0]). start(normal, []) -> diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 5647afdec07c..31c2a9dfc6b3 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -338,7 +338,7 @@ process_connect(#mqtt_packet{ ?LOG_DEBUG("Received a CONNECT, client ID: ~s, username: ~s, " "clean session: ~s, protocol version: ~p, keepalive: ~p", [ClientId, Username, CleanSess, ProtoVersion, Keepalive]), - {ReturnCode, SessionPresent, State} = + {ReturnCode, SessPresent, State} = case rabbit_misc:pipeline([fun check_protocol_version/1, fun check_client_id/1, fun check_credentials/2, @@ -346,16 +346,17 @@ process_connect(#mqtt_packet{ fun register_client/2, fun start_keepalive/2, fun notify_connection_created/1, - fun handle_clean_session/2], + fun handle_clean_sess_qos0/2, + fun handle_clean_sess_qos1/2], PacketConnect, State0) of - {ok, SessionPresent0, State1} -> - {?CONNACK_ACCEPT, SessionPresent0, State1}; + {ok, SessPresent0, State1} -> + {?CONNACK_ACCEPT, SessPresent0, State1}; {error, ConnectionRefusedReturnCode, State1} -> {ConnectionRefusedReturnCode, false, State1} end, Response = #mqtt_packet{fixed = #mqtt_packet_fixed{type = ?CONNACK}, variable = #mqtt_packet_connack{ - session_present = SessionPresent, + session_present = SessPresent, return_code = ReturnCode}}, SendFun(Response, State), return_connack(ReturnCode, State). @@ -492,16 +493,51 @@ start_keepalive(#mqtt_packet_connect{keep_alive = Seconds}, #state{socket = Socket}) -> ok = rabbit_mqtt_keepalive:start(Seconds, Socket). -handle_clean_session(_, State0 = #state{clean_sess = false, - proto_ver = ProtoVer}) -> - case get_queue(?QOS_1, State0) of +handle_clean_sess_qos0(#mqtt_packet_connect{}, State) -> + handle_clean_sess(false, ?QOS_0, State). + +handle_clean_sess_qos1(QoS0SessPresent, State) -> + handle_clean_sess(QoS0SessPresent, ?QOS_1, State). + +handle_clean_sess(_, QoS, + State = #state{clean_sess = true, + auth_state = #auth_state{user = User, + username = Username, + authz_ctx = AuthzCtx}}) -> + %% "If the Server accepts a connection with CleanSession set to 1, the Server + %% MUST set Session Present to 0 in the CONNACK packet [MQTT-3.2.2-1]. + SessPresent = false, + case get_queue(QoS, State) of + {error, _} -> + {ok, SessPresent, State}; + {ok, Q0} -> + QName = amqqueue:get_name(Q0), + %% configure access to queue required for queue.delete + case check_resource_access(User, QName, configure, AuthzCtx) of + ok -> + delete_queue(QName, Username), + {ok, SessPresent, State}; + {error, access_refused} -> + {error, ?CONNACK_NOT_AUTHORIZED} + end + end; +handle_clean_sess(SessPresent, QoS, + State0 = #state{clean_sess = false, + proto_ver = ProtoVer}) -> + case get_queue(QoS, State0) of {error, _} -> %% Queue will be created later when client subscribes. - {ok, _SessionPresent = false, State0}; + {ok, SessPresent, State0}; {ok, Q} -> - case consume(Q, ?QOS_1, State0) of + case consume(Q, QoS, State0) of {ok, State} -> - rabbit_global_counters:consumer_created(ProtoVer), + case SessPresent of + false -> + rabbit_global_counters:consumer_created(ProtoVer); + true -> + %% We already incremented the consumer counter for QoS 0. + ok + end, {ok, _SessionPresent = true, State}; {error, access_refused} -> {error, ?CONNACK_NOT_AUTHORIZED}; @@ -509,24 +545,6 @@ handle_clean_session(_, State0 = #state{clean_sess = false, %% Let's use most generic error return code. {error, ?CONNACK_SERVER_UNAVAILABLE} end - end; -handle_clean_session(_, State = #state{clean_sess = true, - auth_state = #auth_state{user = User, - username = Username, - authz_ctx = AuthzCtx}}) -> - case get_queue(?QOS_1, State) of - {error, _} -> - {ok, _SessionPresent = false, State}; - {ok, Q0} -> - QName = amqqueue:get_name(Q0), - %% configure access to queue required for queue.delete - case check_resource_access(User, QName, configure, AuthzCtx) of - ok -> - delete_queue(QName, Username), - {ok, _SessionPresent = false, State}; - {error, access_refused} -> - {error, ?CONNACK_NOT_AUTHORIZED} - end end. -spec get_queue(qos(), state()) -> @@ -1020,7 +1038,7 @@ create_queue(QoS, #state{ self(), _Durable = true, _AutoDelete = false, - queue_owner(QoS, CleanSess), + queue_owner(CleanSess), QArgs, VHost, #{user => Username}, @@ -1045,30 +1063,30 @@ create_queue(QoS, #state{ E end. -queue_owner(QoS, CleanSess) - when QoS =:= ?QOS_0 orelse CleanSess -> +-spec queue_owner(CleanSession :: boolean()) -> + pid() | none. +queue_owner(true) -> %% Exclusive queues are auto-deleted after node restart while auto-delete queues are not. %% Therefore make durable queue exclusive. self(); -queue_owner(_, _) -> +queue_owner(false) -> none. -queue_args(QoS, CleanSess) - when QoS =:= ?QOS_0 orelse CleanSess -> - []; -queue_args(_, _) -> +queue_args(QoS, false) -> Args = case rabbit_mqtt_util:env(subscription_ttl) of Ms when is_integer(Ms) -> [{<<"x-expires">>, long, Ms}]; _ -> [] end, - case rabbit_mqtt_util:env(durable_queue_type) of - quorum -> + case {QoS, rabbit_mqtt_util:env(durable_queue_type)} of + {?QOS_1, quorum} -> [{<<"x-queue-type">>, longstr, <<"quorum">>} | Args]; _ -> Args - end. + end; +queue_args(_, _) -> + []. queue_type(?QOS_0, true, QArgs) -> case rabbit_feature_flags:is_enabled(?QUEUE_TYPE_QOS_0) of diff --git a/deps/rabbitmq_mqtt/test/shared_SUITE.erl b/deps/rabbitmq_mqtt/test/shared_SUITE.erl index fe72f077d051..51ca27f3b73c 100644 --- a/deps/rabbitmq_mqtt/test/shared_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/shared_SUITE.erl @@ -64,6 +64,7 @@ subgroups() -> [ block_only_publisher ,many_qos1_messages + ,subscription_ttl ] ++ tests()} ]}, {cluster_size_3, [], @@ -95,7 +96,9 @@ tests() -> ,publish_to_all_queue_types_qos1 ,events ,internal_event_handler - ,non_clean_sess_disconnect + ,non_clean_sess_reconnect_qos1 + ,non_clean_sess_reconnect_qos0 + ,non_clean_sess_reconnect_qos0_and_qos1 ,subscribe_same_topic_same_qos ,subscribe_same_topic_different_qos ,subscribe_multiple @@ -758,13 +761,37 @@ delete_create_queue(Config) -> delete_queue(Ch, [CQ1, CQ2, QQ]), ok = emqtt:disconnect(C). -non_clean_sess_disconnect(Config) -> +subscription_ttl(Config) -> + TTL = 1000, + App = rabbitmq_mqtt, + Par = ClientId = ?FUNCTION_NAME, + {ok, DefaultVal} = rpc(Config, application, get_env, [App, Par]), + ok = rpc(Config, application, set_env, [App, Par, TTL]), + + C = connect(ClientId, Config, [{clean_start, false}]), + {ok, _, [0, 1]} = emqtt:subscribe(C, [{<<"topic0">>, qos0}, + {<<"topic1">>, qos1}]), + ok = emqtt:disconnect(C), + + ?assertEqual(2, rpc(Config, rabbit_amqqueue, count, [])), + timer:sleep(TTL + 100), + ?assertEqual(0, rpc(Config, rabbit_amqqueue, count, [])), + + ok = rpc(Config, application, set_env, [App, Par, DefaultVal]). + +non_clean_sess_reconnect_qos1(Config) -> + non_clean_sess_reconnect(Config, qos1). + +non_clean_sess_reconnect_qos0(Config) -> + non_clean_sess_reconnect(Config, qos0). + +non_clean_sess_reconnect(Config, SubscriptionQoS) -> Pub = connect(<<"publisher">>, Config), Topic = ClientId = atom_to_binary(?FUNCTION_NAME), ProtoVer = v4, C1 = connect(ClientId, Config, [{clean_start, false}]), - {ok, _, [1]} = emqtt:subscribe(C1, Topic, qos1), + {ok, _, _} = emqtt:subscribe(C1, Topic, SubscriptionQoS), ?assertMatch(#{consumers := 1}, get_global_counters(Config, ProtoVer)), @@ -793,6 +820,11 @@ non_clean_sess_disconnect(Config) -> {ok, _} = emqtt:publish(Pub, Topic, <<"msg-7-qos0">>, qos1), ok = expect_publishes(C1, Topic, [<<"msg-1-qos0">>, <<"msg-2-qos1">>]), + %% "After the disconnection of a Session that had CleanSession set to 0, the Server MUST store + %% further QoS 1 and QoS 2 messages that match any subscriptions that the client had at the + %% time of disconnection as part of the Session state [MQTT-3.1.2-5]. + %% It MAY also store QoS 0 messages that meet the same criteria." + %% Starting with RabbitMQ v3.12 we store QoS 0 messages as well. ok = expect_publishes(C2, Topic, [<<"msg-3-qos0">>, <<"msg-4-qos1">>, <<"msg-5-qos0">>, <<"msg-6-qos1">>]), {publish_not_received, <<"msg-7-qos0">>} = expect_publishes(C2, Topic, [<<"msg-7-qos0">>]), @@ -803,6 +835,37 @@ non_clean_sess_disconnect(Config) -> C3 = connect(ClientId, Config, [{clean_start, true}]), ok = emqtt:disconnect(C3). +non_clean_sess_reconnect_qos0_and_qos1(Config) -> + Pub = connect(<<"publisher">>, Config), + ProtoVer = v4, + Topic0 = <<"t/0">>, + Topic1 = <<"t/1">>, + ClientId = ?FUNCTION_NAME, + + C1 = connect(ClientId, Config, [{clean_start, false}]), + {ok, _, [1, 0]} = emqtt:subscribe(C1, [{Topic1, qos1}, {Topic0, qos0}]), + ?assertMatch(#{consumers := 1}, + get_global_counters(Config, ProtoVer)), + + ok = emqtt:disconnect(C1), + ?assertMatch(#{consumers := 0}, + get_global_counters(Config, ProtoVer)), + + {ok, _} = emqtt:publish(Pub, Topic0, <<"msg-0">>, qos1), + {ok, _} = emqtt:publish(Pub, Topic1, <<"msg-1">>, qos1), + + C2 = connect(ClientId, Config, [{clean_start, false}]), + ?assertMatch(#{consumers := 1}, + get_global_counters(Config, ProtoVer)), + + ok = expect_publishes(C2, Topic0, [<<"msg-0">>]), + ok = expect_publishes(C2, Topic1, [<<"msg-1">>]), + + ok = emqtt:disconnect(Pub), + ok = emqtt:disconnect(C2), + C3 = connect(ClientId, Config, [{clean_start, true}]), + ok = emqtt:disconnect(C3). + subscribe_same_topic_same_qos(Config) -> C = connect(?FUNCTION_NAME, Config), Topic = <<"a/b">>, From b5febb8a08599f471a129627f57865f0f43d2c22 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Fri, 13 Jan 2023 19:52:08 +0000 Subject: [PATCH 103/118] Hold subsriptions in process state Although the first element (destination queue) of the compound key in rabbit_reverse_route is provided, scaling tests with million of subscribers have shown via timer:tc/1 that the mnesia:match_object/3 query often takes > 100 microseconds, sometimes even a few milliseconds. So, querying bindings is not super expensive, but moderately expensive when done many times concurrenlty. Espcecially when mass diconnecting millions of clients, `msacc` showed that all schedulers were 80% busy in `ets`. To put less pressure on the CPU, in this commit we rather decide to slightly increase memory usage. When first connecting a client, we only query bindings and cache them in the process state if a prior session for the same client is present. Thereafter, bindings are not queried again. --- .../src/rabbit_mqtt_processor.erl | 196 ++++++++---------- 1 file changed, 86 insertions(+), 110 deletions(-) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 31c2a9dfc6b3..8e0eac50d41e 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -61,6 +61,7 @@ clean_sess :: option(boolean()), will_msg :: option(mqtt_msg()), exchange :: option(rabbit_exchange:name()), + subscriptions = #{} :: #{Topic :: binary() => QoS :: ?QOS_0..?QOS_1}, %% Set if client has at least one subscription with QoS 1. queue_qos1 :: option(rabbit_amqqueue:name()), %% Did the client ever sent us a PUBLISH packet? @@ -218,12 +219,9 @@ process_request(?SUBSCRIBE, topic_table = Topics}, payload = undefined}, #state{send_fun = SendFun, - retainer_pid = RPid} = State0) -> + retainer_pid = RPid + } = State0) -> ?LOG_DEBUG("Received a SUBSCRIBE for topic(s) ~p", [Topics]), - TopicNamesQos0 = topic_names(?QOS_0, State0), - TopicNamesQos1 = topic_names(?QOS_1, State0), - HasSubsBefore = TopicNamesQos0 =/= [] orelse TopicNamesQos1 =/= [], - {QosResponse, State1} = lists:foldl( fun(_Topic, {[?SUBACK_FAILURE | _] = L, S}) -> @@ -232,26 +230,28 @@ process_request(?SUBSCRIBE, %% to close the client connection anyway. {[?SUBACK_FAILURE | L], S}; (#mqtt_topic{name = TopicName, - qos = Qos0} = Topic, + qos = TopicQos}, {L, S0}) -> - QoS = supported_sub_qos(Qos0), - case maybe_replace_old_sub(Topic, TopicNamesQos0, TopicNamesQos1, S0) of + QoS = supported_sub_qos(TopicQos), + case maybe_replace_old_sub(TopicName, QoS, S0) of {ok, S1} -> case ensure_queue(QoS, S1) of {ok, Q} -> QName = amqqueue:get_name(Q), case bind(QName, TopicName, S1) of - {ok, _Output, S2} -> + {ok, _Output, S2 = #state{subscriptions = Subs}} -> + S3 = S2#state{subscriptions = maps:put(TopicName, QoS, Subs)}, + maybe_increment_consumer(S2, S3), case self_consumes(Q) of false -> - case consume(Q, QoS, S2) of - {ok, S3} -> - {[QoS | L], S3}; + case consume(Q, QoS, S3) of + {ok, S4} -> + {[QoS | L], S4}; {error, _Reason} -> - {[?SUBACK_FAILURE | L], S2} + {[?SUBACK_FAILURE | L], S3} end; true -> - {[QoS | L], S2} + {[QoS | L], S3} end; {error, Reason, S2} -> ?LOG_ERROR("Failed to bind ~s with topic ~s: ~p", @@ -266,13 +266,13 @@ process_request(?SUBSCRIBE, end end, {[], State0}, Topics), - maybe_increment_consumer(HasSubsBefore, State1), SendFun( #mqtt_packet{fixed = #mqtt_packet_fixed{type = ?SUBACK}, variable = #mqtt_packet_suback{ packet_id = SubscribePktId, qos_table = QosResponse}}, State1), + case QosResponse of [?SUBACK_FAILURE | _] -> {error, subscribe_error, State1}; @@ -289,29 +289,30 @@ process_request(?UNSUBSCRIBE, payload = undefined}, State0 = #state{send_fun = SendFun}) -> ?LOG_DEBUG("Received an UNSUBSCRIBE for topic(s) ~p", [Topics]), - HasSubsBefore = has_subs(State0), State = lists:foldl( - fun(#mqtt_topic{name = TopicName}, #state{} = S0) -> - case find_queue_name(TopicName, S0) of - {ok, QName} -> + fun(#mqtt_topic{name = TopicName}, #state{subscriptions = Subs0} = S0) -> + case maps:take(TopicName, Subs0) of + {QoS, Subs} -> + QName = queue_name(QoS, S0), case unbind(QName, TopicName, S0) of - {ok, _, S} -> + {ok, _, S1} -> + S = S1#state{subscriptions = Subs}, + maybe_decrement_consumer(S1, S), S; {error, Reason, S} -> ?LOG_ERROR("Failed to unbind ~s with topic ~s: ~p", [rabbit_misc:rs(QName), TopicName, Reason]), S end; - {not_found, _} -> + error -> S0 end end, State0, Topics), SendFun( - #mqtt_packet{fixed = #mqtt_packet_fixed {type = ?UNSUBACK}, + #mqtt_packet{fixed = #mqtt_packet_fixed {type = ?UNSUBACK}, variable = #mqtt_packet_suback{packet_id = PacketId}}, State), - maybe_decrement_consumer(HasSubsBefore, State), {ok, State}; process_request(?PINGREQ, #mqtt_packet{}, State = #state{send_fun = SendFun, @@ -347,7 +348,8 @@ process_connect(#mqtt_packet{ fun start_keepalive/2, fun notify_connection_created/1, fun handle_clean_sess_qos0/2, - fun handle_clean_sess_qos1/2], + fun handle_clean_sess_qos1/2, + fun cache_subscriptions/2], PacketConnect, State0) of {ok, SessPresent0, State1} -> {?CONNACK_ACCEPT, SessPresent0, State1}; @@ -567,78 +569,60 @@ get_queue(QoS, State) -> Err end. +queue_name(?QOS_1, #state{queue_qos1 = #resource{kind = queue} = Name}) -> + Name; queue_name(QoS, #state{client_id = ClientId, auth_state = #auth_state{vhost = VHost}}) -> QNameBin = rabbit_mqtt_util:queue_name_bin(ClientId, QoS), rabbit_misc:r(VHost, queue, QNameBin). -find_queue_name(TopicName, #state{exchange = Exchange} = State) -> - RoutingKey = mqtt_to_amqp(TopicName), - QNameQoS0 = queue_name(?QOS_0, State), - case lookup_binding(Exchange, QNameQoS0, RoutingKey) of - true -> - {ok, QNameQoS0}; - false -> - QNameQoS1 = queue_name(?QOS_1, State), - case lookup_binding(Exchange, QNameQoS1, RoutingKey) of - true -> - {ok, QNameQoS1}; - false -> - {not_found, []} - end - end. - -lookup_binding(Exchange, QueueName, RoutingKey) -> - B = #binding{source = Exchange, - destination = QueueName, - key = RoutingKey}, - lists:member(B, rabbit_binding:list_for_source_and_destination(Exchange, QueueName)). - -has_subs(State) -> - topic_names(?QOS_0, State) =/= [] orelse - topic_names(?QOS_1, State) =/= []. +%% Query subscriptions from the database and hold them in process state +%% to avoid future mnesia:match_object/3 queries. +cache_subscriptions(_SessionPresent = _SubscriptionsPresent = true, State) -> + SubsQos0 = topic_names(?QOS_0, State), + SubsQos1 = topic_names(?QOS_1, State), + Subs = maps:merge(maps:from_keys(SubsQos0, ?QOS_0), + maps:from_keys(SubsQos1, ?QOS_1)), + {ok, State#state{subscriptions = Subs}}; +cache_subscriptions(_, _) -> + ok. topic_names(QoS, #state{exchange = Exchange} = State) -> - Bindings = rabbit_binding:list_for_source_and_destination( - Exchange, - queue_name(QoS, State), - %% Querying table rabbit_reverse_route instead of rabbit_route provides - %% **much** better CPU performance because the source exchange is always the - %% same in the MQTT plugin while the destination queue will be different. - _Reverse = true), + Bindings = + rabbit_binding:list_for_source_and_destination( + Exchange, + queue_name(QoS, State), + %% Querying table rabbit_route is catastrophic for CPU usage. + %% Querying table rabbit_reverse_route is acceptable because + %% the source exchange is always the same in the MQTT plugin whereas + %% the destination queue is different for each MQTT client and + %% rabbit_reverse_route is sorted by destination queue. + _Reverse = true), lists:map(fun(B) -> amqp_to_mqtt(B#binding.key) end, Bindings). %% "If a Server receives a SUBSCRIBE Packet containing a Topic Filter that is identical %% to an existing Subscription’s Topic Filter then it MUST completely replace that %% existing Subscription with a new Subscription. The Topic Filter in the new Subscription %% will be identical to that in the previous Subscription, although its maximum QoS value -%% could be different. [...]" [MQTT-3.8.4-3]. -%% -%% Therefore, if we receive a QoS 0 subscription for a topic that already has QoS 1, -%% we unbind QoS 1 (and vice versa). -maybe_replace_old_sub(#mqtt_topic{name = TopicName, qos = ?QOS_0}, - _, OldTopicNamesQos1, State) -> - QName = queue_name(?QOS_1, State), - maybe_unbind(TopicName, OldTopicNamesQos1, QName, State); -maybe_replace_old_sub(#mqtt_topic{name = TopicName, qos = QoS}, - OldTopicNamesQos0, _, State) - when QoS =:= ?QOS_1 orelse QoS =:= ?QOS_2 -> - QName = queue_name(?QOS_0, State), - maybe_unbind(TopicName, OldTopicNamesQos0, QName, State). - -maybe_unbind(TopicName, TopicNames, QName, State0) -> - case lists:member(TopicName, TopicNames) of - false -> - {ok, State0}; - true -> - case unbind(QName, TopicName, State0) of - {ok, _Output, State} -> - {ok, State}; - {error, Reason, _State} = Err -> - ?LOG_ERROR("Failed to unbind ~s with topic '~s': ~p", - [rabbit_misc:rs(QName), TopicName, Reason]), - Err - end +%% could be different." [MQTT-3.8.4-3]. +maybe_replace_old_sub(TopicName, QoS, State = #state{subscriptions = Subs}) -> + case Subs of + #{TopicName := OldQoS} + when OldQoS =/= QoS -> + replace_old_sub(OldQoS, TopicName, State); + _ -> + {ok, State} + end. + +replace_old_sub(QoS, TopicName, State0)-> + QName = queue_name(QoS, State0), + case unbind(QName, TopicName, State0) of + {ok, _Output, State} -> + {ok, State}; + {error, Reason, _State} = Err -> + ?LOG_ERROR("Failed to unbind ~s with topic '~s': ~p", + [rabbit_misc:rs(QName), TopicName, Reason]), + Err end. -spec hand_off_to_retainer(pid(), binary(), mqtt_msg()) -> ok. @@ -1875,38 +1859,30 @@ maybe_decrement_publisher(#state{published = true, maybe_decrement_publisher(_) -> ok. -%% multiple subscriptions from the same connection count as one consumer -maybe_increment_consumer(_WasConsumer = false, - #state{proto_ver = ProtoVer} = State) -> - case has_subs(State) of - true -> - rabbit_global_counters:consumer_created(ProtoVer); - false -> - ok - end; +%% Multiple subscriptions from the same connection count as one consumer. +maybe_increment_consumer(#state{subscriptions = OldSubs}, + #state{subscriptions = NewSubs, + proto_ver = ProtoVer}) + when map_size(OldSubs) =:= 0 andalso + map_size(NewSubs) > 0 -> + rabbit_global_counters:consumer_created(ProtoVer); maybe_increment_consumer(_, _) -> ok. -maybe_decrement_consumer(_WasConsumer = true, - #state{proto_ver = ProtoVer} = State) -> - case has_subs(State) of - false -> - rabbit_global_counters:consumer_deleted(ProtoVer); - true -> - ok - end; -maybe_decrement_consumer(_, _) -> +maybe_decrement_consumer(#state{subscriptions = Subs, + proto_ver = ProtoVer}) + when map_size(Subs) > 0 -> + rabbit_global_counters:consumer_deleted(ProtoVer); +maybe_decrement_consumer(_) -> ok. -maybe_decrement_consumer(#state{proto_ver = ProtoVer, - auth_state = #auth_state{}} = State) -> - case has_subs(State) of - true -> - rabbit_global_counters:consumer_deleted(ProtoVer); - false -> - ok - end; -maybe_decrement_consumer(_) -> +maybe_decrement_consumer(#state{subscriptions = OldSubs}, + #state{subscriptions = NewSubs, + proto_ver = ProtoVer}) + when map_size(OldSubs) > 0 andalso + map_size(NewSubs) =:= 0 -> + rabbit_global_counters:consumer_deleted(ProtoVer); +maybe_decrement_consumer(_, _) -> ok. message_acknowledged(QName, #state{proto_ver = ProtoVer, From dcb00f13240b0366490ff6c4687191408a2dddde Mon Sep 17 00:00:00 2001 From: David Ansari Date: Sat, 14 Jan 2023 11:09:25 +0000 Subject: [PATCH 104/118] Reduce test load Run test either on a RabbitMQ cluster of size 1 or size 3. Running a test on both cluster sizes does not result in higher test coverage. This puts less pressure on Buildbuddy and reduces overall test execution time. --- .../src/rabbit_mqtt_processor.erl | 2 +- deps/rabbitmq_mqtt/test/shared_SUITE.erl | 74 +++++++++---------- 2 files changed, 34 insertions(+), 42 deletions(-) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 8e0eac50d41e..f61aeda129ec 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -32,7 +32,7 @@ -define(MAX_PERMISSION_CACHE_SIZE, 12). -define(CONSUMER_TAG, <<"mqtt">>). --record(auth_state, {username :: binary(), +-record(auth_state, {username :: rabbit_types:username(), user :: #user{}, vhost :: rabbit_types:vhost(), authz_ctx :: #{binary() := binary()} diff --git a/deps/rabbitmq_mqtt/test/shared_SUITE.erl b/deps/rabbitmq_mqtt/test/shared_SUITE.erl index 51ca27f3b73c..045e273ad8a1 100644 --- a/deps/rabbitmq_mqtt/test/shared_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/shared_SUITE.erl @@ -65,7 +65,33 @@ subgroups() -> block_only_publisher ,many_qos1_messages ,subscription_ttl - ] ++ tests()} + ,management_plugin_connection + ,management_plugin_enable + ,disconnect + ,pubsub_shared_connection + ,pubsub_separate_connections + ,will_with_disconnect + ,will_without_disconnect + ,quorum_queue_rejects + ,events + ,internal_event_handler + ,non_clean_sess_reconnect_qos1 + ,non_clean_sess_reconnect_qos0 + ,non_clean_sess_reconnect_qos0_and_qos1 + ,subscribe_same_topic_same_qos + ,subscribe_same_topic_different_qos + ,subscribe_multiple + ,large_message_mqtt_to_mqtt + ,large_message_amqp_to_mqtt + ,keepalive + ,keepalive_turned_off + ,duplicate_client_id + ,block + ,amqp_to_mqtt_qos0 + ,clean_session_disconnect_client + ,clean_session_kill_node + ,rabbit_status_connection_count + ]} ]}, {cluster_size_3, [], [ @@ -77,41 +103,11 @@ subgroups() -> flow_stream, rabbit_mqtt_qos0_queue, cli_list_queues, - maintenance - ] ++ tests()} - ]. - -tests() -> - [ - management_plugin_connection - ,management_plugin_enable - ,disconnect - ,pubsub_shared_connection - ,pubsub_separate_connections - ,will_with_disconnect - ,will_without_disconnect - ,delete_create_queue - ,quorum_queue_rejects - ,publish_to_all_queue_types_qos0 - ,publish_to_all_queue_types_qos1 - ,events - ,internal_event_handler - ,non_clean_sess_reconnect_qos1 - ,non_clean_sess_reconnect_qos0 - ,non_clean_sess_reconnect_qos0_and_qos1 - ,subscribe_same_topic_same_qos - ,subscribe_same_topic_different_qos - ,subscribe_multiple - ,large_message_mqtt_to_mqtt - ,large_message_amqp_to_mqtt - ,amqp_to_mqtt_qos0 - ,keepalive - ,keepalive_turned_off - ,duplicate_client_id - ,block - ,clean_session_disconnect_client - ,clean_session_kill_node - ,rabbit_status_connection_count + maintenance, + delete_create_queue, + publish_to_all_queue_types_qos0, + publish_to_all_queue_types_qos1 + ]} ]. suite() -> @@ -795,14 +791,11 @@ non_clean_sess_reconnect(Config, SubscriptionQoS) -> ?assertMatch(#{consumers := 1}, get_global_counters(Config, ProtoVer)), - ok = emqtt:publish(Pub, Topic, <<"msg-1-qos0">>, qos0), - {ok, _} = emqtt:publish(Pub, Topic, <<"msg-2-qos1">>, qos1), - ok = emqtt:disconnect(C1), ?assertMatch(#{consumers := 0}, get_global_counters(Config, ProtoVer)), - timer:sleep(50), + timer:sleep(20), ok = emqtt:publish(Pub, Topic, <<"msg-3-qos0">>, qos0), {ok, _} = emqtt:publish(Pub, Topic, <<"msg-4-qos1">>, qos1), @@ -819,7 +812,6 @@ non_clean_sess_reconnect(Config, SubscriptionQoS) -> get_global_counters(Config, ProtoVer)), {ok, _} = emqtt:publish(Pub, Topic, <<"msg-7-qos0">>, qos1), - ok = expect_publishes(C1, Topic, [<<"msg-1-qos0">>, <<"msg-2-qos1">>]), %% "After the disconnection of a Session that had CleanSession set to 0, the Server MUST store %% further QoS 1 and QoS 2 messages that match any subscriptions that the client had at the %% time of disconnection as part of the Session state [MQTT-3.1.2-5]. From a4db85de0d10b1ce38231c3331d14358e61f727b Mon Sep 17 00:00:00 2001 From: David Ansari Date: Sat, 14 Jan 2023 12:23:07 +0000 Subject: [PATCH 105/118] Make pipeline fail when there are dialyzer warnings We want the build to fail if there are any dialyzer warnings in rabbitmq_mqtt or rabbitmq_web_mqtt. Otherwise we rely on people manually executing and checking the results of dialyzer. Also, we want any test to fail that is flaky. Flaky tests can indicate subtle errors in either test or program execution. Instead of marking them as flaky, we should understand and - if possible - fix the underlying root cause. Fix OTP 25.0 dialyzer warning Type gen_server:format_status() is known in OTP 25.2, but not in 25.0 --- deps/rabbit/test/direct_exchange_routing_v2_SUITE.erl | 2 +- deps/rabbitmq_mqtt/BUILD.bazel | 8 ++------ deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl | 2 +- deps/rabbitmq_mqtt/src/rabbit_mqtt_ff.erl | 2 +- deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl | 2 +- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 9 ++++++--- deps/rabbitmq_mqtt/test/config_SUITE.erl | 2 +- deps/rabbitmq_mqtt/test/event_recorder.erl | 2 +- deps/rabbitmq_mqtt/test/ff_SUITE.erl | 2 +- deps/rabbitmq_mqtt/test/shared_SUITE.erl | 2 +- deps/rabbitmq_web_mqtt/BUILD.bazel | 4 ++-- 11 files changed, 18 insertions(+), 19 deletions(-) diff --git a/deps/rabbit/test/direct_exchange_routing_v2_SUITE.erl b/deps/rabbit/test/direct_exchange_routing_v2_SUITE.erl index 3065c5394468..081461a41bf6 100644 --- a/deps/rabbit/test/direct_exchange_routing_v2_SUITE.erl +++ b/deps/rabbit/test/direct_exchange_routing_v2_SUITE.erl @@ -2,7 +2,7 @@ %% License, v. 2.0. If a copy of the MPL was not distributed with this %% file, You can obtain one at https://mozilla.org/MPL/2.0/. %% -%% Copyright (c) 2016-2022 VMware, Inc. or its affiliates. All rights reserved. +%% Copyright (c) 2016-2023 VMware, Inc. or its affiliates. All rights reserved. -module(direct_exchange_routing_v2_SUITE). diff --git a/deps/rabbitmq_mqtt/BUILD.bazel b/deps/rabbitmq_mqtt/BUILD.bazel index 7fc1732f1cb7..17b76482f728 100644 --- a/deps/rabbitmq_mqtt/BUILD.bazel +++ b/deps/rabbitmq_mqtt/BUILD.bazel @@ -3,7 +3,6 @@ load("@rules_erlang//:dialyze.bzl", "dialyze", "plt") load( "//:rabbitmq.bzl", "BROKER_VERSION_REQUIREMENTS_ANY", - "RABBITMQ_DIALYZER_OPTS", "assert_suites", "broker_for_integration_suites", "rabbitmq_app", @@ -78,9 +77,9 @@ plt( ) dialyze( - dialyzer_opts = RABBITMQ_DIALYZER_OPTS, + dialyzer_opts = ["-Werror_handling", "-Wno_undefined_callbacks"], plt = ":base_plt", - warnings_as_errors = False, + warnings_as_errors = True, ) broker_for_integration_suites( @@ -128,7 +127,6 @@ rabbitmq_integration_suite( additional_beam = [ ":rabbit_auth_backend_mqtt_mock", ], - flaky = True, runtime_deps = [ "@emqtt//:erlang_app", ], @@ -137,7 +135,6 @@ rabbitmq_integration_suite( rabbitmq_integration_suite( name = "cluster_SUITE", size = "large", - flaky = True, runtime_deps = [ "@emqtt//:erlang_app", ], @@ -167,7 +164,6 @@ rabbitmq_integration_suite( rabbitmq_integration_suite( name = "java_SUITE", - flaky = True, ) rabbitmq_suite( diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl index 36f2f12a1e39..a31c94bd16df 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl @@ -2,7 +2,7 @@ %% License, v. 2.0. If a copy of the MPL was not distributed with this %% file, You can obtain one at https://mozilla.org/MPL/2.0/. %% -%% Copyright (c) 2020-2022 VMware, Inc. or its affiliates. All rights reserved. +%% Copyright (c) 2020-2023 VMware, Inc. or its affiliates. All rights reserved. %% -module(rabbit_mqtt_confirms). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_ff.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_ff.erl index 2ebde6e4d1ef..b894cee42cc4 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_ff.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_ff.erl @@ -2,7 +2,7 @@ %% License, v. 2.0. If a copy of the MPL was not distributed with this %% file, You can obtain one at https://mozilla.org/MPL/2.0/. %% -%% Copyright (c) 2018-2022 VMware, Inc. or its affiliates. All rights reserved. +%% Copyright (c) 2018-2023 VMware, Inc. or its affiliates. All rights reserved. %% -module(rabbit_mqtt_ff). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl index e757fec4b60b..ea552e7fe27e 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl @@ -2,7 +2,7 @@ %% License, v. 2.0. If a copy of the MPL was not distributed with this %% file, You can obtain one at https://mozilla.org/MPL/2.0/. %% -%% Copyright (c) 2018-2022 VMware, Inc. or its affiliates. All rights reserved. +%% Copyright (c) 2018-2023 VMware, Inc. or its affiliates. All rights reserved. %% %% This module is a pseudo queue type. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index f7db17f09c70..c083ebd036af 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -403,7 +403,7 @@ run_socket(State = #state{ deferred_recv = Data }) when Data =/= undefined -> run_socket(State = #state{ await_recv = true }) -> State; run_socket(State = #state{ socket = Sock }) -> - rabbit_net:setopts(Sock, [{active, once}]), + ok = rabbit_net:setopts(Sock, [{active, once}]), State#state{ await_recv = true }. control_throttle(State = #state{connection_state = ConnState, @@ -506,8 +506,11 @@ i(protocol, #state{proc_state = ProcState}) -> i(Key, #state{proc_state = ProcState}) -> rabbit_mqtt_processor:info(Key, ProcState). --spec format_status(gen_server:format_status()) -> - gen_server:format_status(). +-spec format_status(Status) -> Status when + Status :: #{state => term(), + message => term(), + reason => term(), + log => [sys:system_event()]}. format_status(Status) -> maps:map( fun(state, State) -> diff --git a/deps/rabbitmq_mqtt/test/config_SUITE.erl b/deps/rabbitmq_mqtt/test/config_SUITE.erl index 0761d3e49a96..b2b04b52e2c6 100644 --- a/deps/rabbitmq_mqtt/test/config_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/config_SUITE.erl @@ -2,7 +2,7 @@ %% License, v. 2.0. If a copy of the MPL was not distributed with this %% file, You can obtain one at https://mozilla.org/MPL/2.0/. %% -%% Copyright (c) 2007-2022 VMware, Inc. or its affiliates. All rights reserved. +%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved. -module(config_SUITE). -compile([export_all, diff --git a/deps/rabbitmq_mqtt/test/event_recorder.erl b/deps/rabbitmq_mqtt/test/event_recorder.erl index c7175310bc1e..cd495f9427a5 100644 --- a/deps/rabbitmq_mqtt/test/event_recorder.erl +++ b/deps/rabbitmq_mqtt/test/event_recorder.erl @@ -2,7 +2,7 @@ %% License, v. 2.0. If a copy of the MPL was not distributed with this %% file, You can obtain one at https://mozilla.org/MPL/2.0/. %% -%% Copyright (c) 2007-2022 VMware, Inc. or its affiliates. All rights reserved. +%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved. %% -module(event_recorder). diff --git a/deps/rabbitmq_mqtt/test/ff_SUITE.erl b/deps/rabbitmq_mqtt/test/ff_SUITE.erl index 138a18fe59c5..a7c528c64076 100644 --- a/deps/rabbitmq_mqtt/test/ff_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/ff_SUITE.erl @@ -2,7 +2,7 @@ %% License, v. 2.0. If a copy of the MPL was not distributed with this %% file, You can obtain one at https://mozilla.org/MPL/2.0/. %% -%% Copyright (c) 2016-2022 VMware, Inc. or its affiliates. All rights reserved. +%% Copyright (c) 2016-2023 VMware, Inc. or its affiliates. All rights reserved. -module(ff_SUITE). diff --git a/deps/rabbitmq_mqtt/test/shared_SUITE.erl b/deps/rabbitmq_mqtt/test/shared_SUITE.erl index 045e273ad8a1..9c3f41b5c762 100644 --- a/deps/rabbitmq_mqtt/test/shared_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/shared_SUITE.erl @@ -2,7 +2,7 @@ %% License, v. 2.0. If a copy of the MPL was not distributed with this %% file, You can obtain one at https://mozilla.org/MPL/2.0/. %% -%% Copyright (c) 2007-2022 VMware, Inc. or its affiliates. All rights reserved. +%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved. %% Test suite shared between rabbitmq_mqtt and rabbitmq_web_mqtt. -module(shared_SUITE). diff --git a/deps/rabbitmq_web_mqtt/BUILD.bazel b/deps/rabbitmq_web_mqtt/BUILD.bazel index 6ac2d1251a5b..25a8504147d3 100644 --- a/deps/rabbitmq_web_mqtt/BUILD.bazel +++ b/deps/rabbitmq_web_mqtt/BUILD.bazel @@ -2,7 +2,6 @@ load("@rules_erlang//:xref2.bzl", "xref") load("@rules_erlang//:dialyze.bzl", "dialyze", "plt") load( "//:rabbitmq.bzl", - "RABBITMQ_DIALYZER_OPTS", "RABBITMQ_TEST_ERLC_OPTS", "assert_suites", "broker_for_integration_suites", @@ -60,8 +59,9 @@ plt( ) dialyze( - dialyzer_opts = RABBITMQ_DIALYZER_OPTS, + dialyzer_opts = ["-Werror_handling"], plt = ":base_plt", + warnings_as_errors = True, ) broker_for_integration_suites( From 63ccf3ea3b7df427bc723e10500c2124b71b9459 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Mon, 16 Jan 2023 15:34:11 +0000 Subject: [PATCH 106/118] Reduce inter-node traffic for MQTT QoS 0 queue type Use delegate. For large fan-outs with medium to large message size, this commit will reduce inter-node data traffic by multiple orders of magnitude preventing busy distribution ports. --- deps/rabbit/src/rabbit_queue_type.erl | 34 ++++++------ .../src/rabbit_mqtt_qos0_queue.erl | 53 ++++++++++--------- 2 files changed, 45 insertions(+), 42 deletions(-) diff --git a/deps/rabbit/src/rabbit_queue_type.erl b/deps/rabbit/src/rabbit_queue_type.erl index d675522f388f..9ab8b8a83ed2 100644 --- a/deps/rabbit/src/rabbit_queue_type.erl +++ b/deps/rabbit/src/rabbit_queue_type.erl @@ -511,23 +511,21 @@ deliver0(Qs, Delivery, stateless) -> deliver0(Qs, Delivery, #?STATE{} = State0) -> %% TODO: optimise single queue case? %% sort by queue type - then dispatch each group - {ByType, Actions0} = - lists:foldl( - fun (Q, {M, L}) -> - T = amqqueue:get_type(Q), - case T:is_stateful() of - true -> - Ctx = get_ctx(Q, State0), - {maps:update_with( - T, fun (A) -> - [{Q, Ctx#ctx.state} | A] - end, [{Q, Ctx#ctx.state}], M), - L}; - false -> - {[], DeliverActions} = T:deliver([{Q, stateless}], Delivery), - {M, DeliverActions ++ L} - end - end, {#{}, []}, Qs), + ByType = lists:foldl( + fun (Q, Acc) -> + T = amqqueue:get_type(Q), + QState = case T:is_stateful() of + true -> + #ctx{state = S} = get_ctx(Q, State0), + S; + false -> + stateless + end, + maps:update_with( + T, fun (A) -> + [{Q, QState} | A] + end, [{Q, QState}], Acc) + end, #{}, Qs), %%% dispatch each group to queue type interface? {Xs, Actions} = maps:fold(fun(Mod, QSs, {X0, A0}) -> {X, A} = Mod:deliver(QSs, Delivery), @@ -538,7 +536,7 @@ deliver0(Qs, Delivery, #?STATE{} = State0) -> Ctx = get_ctx_with(Q, Acc, S), set_ctx(qref(Q), Ctx#ctx{state = S}, Acc) end, State0, Xs), - {ok, State, Actions0 ++ Actions}. + {ok, State, Actions}. -spec settle(queue_name(), settle_op(), rabbit_types:ctag(), [non_neg_integer()], state()) -> diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl index ea552e7fe27e..4aa3d6e0192c 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl @@ -84,33 +84,38 @@ delete(Q, _IfUnused, _IfEmpty, ActingUser) -> -spec deliver([{amqqueue:amqqueue(), stateless}], Delivery :: term()) -> {[], rabbit_queue_type:actions()}. -deliver([{Q, stateless}], Delivery = #delivery{message = BasicMessage}) -> - Pid = amqqueue:get_pid(Q), +deliver(Qs, #delivery{message = BasicMessage, + confirm = Confirm, + msg_seq_no = SeqNo}) -> Msg = {queue_event, ?MODULE, - {?MODULE, Pid, _QMsgId = none, _Redelivered = false, BasicMessage}}, - gen_server:cast(Pid, Msg), - Actions = confirm(Delivery, Q), + {?MODULE, _QPid = none, _QMsgId = none, _Redelivered = false, BasicMessage}}, + {Pids, Actions} = + case Confirm of + false -> + Pids0 = lists:map(fun({Q, stateless}) -> amqqueue:get_pid(Q) end, Qs), + {Pids0, []}; + true -> + %% We confirm the message directly here in the queue client. + %% Alternatively, we could have the target MQTT connection process confirm the message. + %% However, given that this message might be lost anyway between target MQTT connection + %% process and MQTT subscriber, and we know that the MQTT subscriber wants to receive + %% this message at most once, we confirm here directly. + %% Benefits: + %% 1. We do not block sending the confirmation back to the publishing client just because a single + %% (at-most-once) target queue out of potentially many (e.g. million) queues might be unavailable. + %% 2. Memory usage in this (publishing) process is kept lower because the target queue name can be + %% directly removed from rabbit_mqtt_confirms and rabbit_confirms. + %% 3. Reduced network traffic across RabbitMQ nodes. + %% 4. Lower latency of sending publisher confirmation back to the publishing client. + SeqNos = [SeqNo], + lists:mapfoldl(fun({Q, stateless}, Actions) -> + {amqqueue:get_pid(Q), + [{settled, amqqueue:get_name(Q), SeqNos} | Actions]} + end, [], Qs) + end, + delegate:invoke_no_result(Pids, {gen_server, cast, [Msg]}), {[], Actions}. -confirm(#delivery{confirm = false}, _) -> - []; -confirm(#delivery{confirm = true, - msg_seq_no = SeqNo}, Q) -> - %% We confirm the message directly here in the queue client. - %% Alternatively, we could have the target MQTT connection process confirm the message. - %% However, given that this message might be lost anyway between target MQTT connection - %% process and MQTT subscriber, and we know that the MQTT subscriber wants to receive - %% this message at most once, we confirm here directly. - %% Benefits: - %% 1. We do not block sending the confirmation back to the publishing client just because a single - %% (at-most-once) target queue out of potentially many (e.g. million) queues might be unavailable. - %% 2. Memory usage in this (publishing) process is kept lower because the target queue name can be - %% directly removed from rabbit_mqtt_confirms and rabbit_confirms. - %% 3. Reduced network traffic across RabbitMQ nodes. - %% 4. Lower latency of sending publisher confirmation back to the publishing client. - QName = amqqueue:get_name(Q), - [{settled, QName, [SeqNo]}]. - -spec is_enabled() -> boolean(). is_enabled() -> From 437cbb73be20b6412ae3f4032bef7ecb4e6456eb Mon Sep 17 00:00:00 2001 From: David Ansari Date: Mon, 16 Jan 2023 16:43:32 +0000 Subject: [PATCH 107/118] Use delegate for stateless deliveries For example when at-most-once dead lettering does a fan out to many target classic queues this commit will reduce inter-node data traffic by using delegate. --- deps/rabbit/src/rabbit_queue_type.erl | 24 ++++++++++++++--------- deps/rabbitmq_mqtt/test/cluster_SUITE.erl | 2 +- 2 files changed, 16 insertions(+), 10 deletions(-) diff --git a/deps/rabbit/src/rabbit_queue_type.erl b/deps/rabbit/src/rabbit_queue_type.erl index 9ab8b8a83ed2..49ed61f3f718 100644 --- a/deps/rabbit/src/rabbit_queue_type.erl +++ b/deps/rabbit/src/rabbit_queue_type.erl @@ -503,18 +503,24 @@ deliver(Qs, Delivery, State) -> end. deliver0(Qs, Delivery, stateless) -> - lists:foreach(fun(Q) -> - Mod = amqqueue:get_type(Q), - _ = Mod:deliver([{Q, stateless}], Delivery) - end, Qs), + ByType = lists:foldl(fun(Q, Acc) -> + Mod = amqqueue:get_type(Q), + maps:update_with( + Mod, fun(A) -> + [{Q, stateless} | A] + end, [{Q, stateless}], Acc) + end, #{}, Qs), + maps:foreach(fun(Mod, QSs) -> + _ = Mod:deliver(QSs, Delivery) + end, ByType), {ok, stateless, []}; deliver0(Qs, Delivery, #?STATE{} = State0) -> %% TODO: optimise single queue case? %% sort by queue type - then dispatch each group ByType = lists:foldl( fun (Q, Acc) -> - T = amqqueue:get_type(Q), - QState = case T:is_stateful() of + Mod = amqqueue:get_type(Q), + QState = case Mod:is_stateful() of true -> #ctx{state = S} = get_ctx(Q, State0), S; @@ -522,9 +528,9 @@ deliver0(Qs, Delivery, #?STATE{} = State0) -> stateless end, maps:update_with( - T, fun (A) -> - [{Q, QState} | A] - end, [{Q, QState}], Acc) + Mod, fun (A) -> + [{Q, QState} | A] + end, [{Q, QState}], Acc) end, #{}, Qs), %%% dispatch each group to queue type interface? {Xs, Actions} = maps:fold(fun(Mod, QSs, {X0, A0}) -> diff --git a/deps/rabbitmq_mqtt/test/cluster_SUITE.erl b/deps/rabbitmq_mqtt/test/cluster_SUITE.erl index 7ec2240cc713..0ac2434ea211 100644 --- a/deps/rabbitmq_mqtt/test/cluster_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/cluster_SUITE.erl @@ -157,7 +157,7 @@ connection_id_tracking_with_decommissioned_node(Config) -> process_flag(trap_exit, true), {ok, _} = rabbitmqctl(Config, 0, ["decommission_mqtt_node", Server]), await_exit(C), - assert_connection_count(Config, 4, 2, 0), + assert_connection_count(Config, 5, 2, 0), ok end. From 3f85f8d90434f6269b4848e77c202b8ab8e84491 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Tue, 17 Jan 2023 14:23:57 +0000 Subject: [PATCH 108/118] Do not log message payload because it might contain large output. --- deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index f61aeda129ec..fa006cd731a8 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -1301,8 +1301,9 @@ serialise_and_send_to_client(Packet, #state{proto_ver = ProtoVer, socket = Sock} try rabbit_net:port_command(Sock, Data) catch error:Error -> ?LOG_ERROR("MQTT: a socket write failed: ~p", [Error]), - ?LOG_DEBUG("Failed to write to socket ~p, error: ~p, packet: ~p", - [Sock, Error, Packet]) + ?LOG_DEBUG("MQTT failed to write to socket ~p, error: ~p, " + "fixed packet header: ~p, variable packet header: ~p", + [Sock, Error, Packet#mqtt_packet.fixed, Packet#mqtt_packet.variable]) end. serialise(Packet, #state{proto_ver = ProtoVer}) -> From d4cfbddd359c78d6c6ca1d61b455d843389fc4f4 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Wed, 18 Jan 2023 18:03:47 +0000 Subject: [PATCH 109/118] Parse at most maximum packet length of 256MB "This allows applications to send Control Packets of size up to 268,435,455 (256 MB)." http://docs.oasis-open.org/mqtt/mqtt/v3.1.1/os/mqtt-v3.1.1-os.html#_Toc398718023 --- deps/rabbitmq_mqtt/src/rabbit_mqtt_packet.erl | 8 +++++--- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 9 +++++---- deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl | 12 ++++++++++-- .../src/rabbit_web_mqtt_handler.erl | 7 ++++--- 4 files changed, 24 insertions(+), 12 deletions(-) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_packet.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_packet.erl index b094e5b74289..9b63e7b45420 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_packet.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_packet.erl @@ -45,7 +45,8 @@ parse_remaining_len(<<0:1, Len:7, Rest/binary>>, Fixed, Multiplier, Value) -> parse_packet(Rest, Fixed, Value + Len * Multiplier). parse_packet(Bin, #mqtt_packet_fixed{ type = Type, - qos = Qos } = Fixed, Length) -> + qos = Qos } = Fixed, Length) + when Length =< ?MAX_LEN -> case {Type, Bin} of {?CONNECT, <>} -> {ProtoName, Rest1} = parse_utf(PacketBin), @@ -106,12 +107,13 @@ parse_packet(Bin, #mqtt_packet_fixed{ type = Type, when Minimal =:= ?DISCONNECT orelse Minimal =:= ?PINGREQ -> Length = 0, wrap(Fixed, Rest); - {_, TooShortBin} -> + {_, TooShortBin} + when byte_size(TooShortBin) < Length -> {more, fun(BinMore) -> parse_packet(<>, Fixed, Length) end} - end. + end. parse_topics(_, <<>>, Topics) -> Topics; diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index c083ebd036af..5f8ab5f54c70 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -358,10 +358,11 @@ process_received_bytes(Bytes, {stop, disconnect, ProcState1} -> {stop, normal, {_SendWill = false, pstate(State, ProcState1)}} end; - {error, {cannot_parse, Error, Stacktrace}} -> - ?LOG_ERROR("MQTT cannot parse a packet on connection '~ts', unparseable payload: ~tp, error: {~tp, ~tp} ", - [ConnName, Bytes, Error, Stacktrace]), - {stop, {shutdown, Error}, State}; + {error, {cannot_parse, Reason, Stacktrace}} -> + ?LOG_ERROR("MQTT cannot parse a packet on connection '~ts', reason: ~tp, " + "stacktrace: ~tp, payload (first 100 bytes): ~tp", + [ConnName, Reason, Stacktrace, rabbit_mqtt_util:truncate_binary(Bytes, 100)]), + {stop, {shutdown, Reason}, State}; {error, Error} -> ?LOG_ERROR("MQTT detected a framing error on connection ~ts: ~tp", [ConnName, Error]), {stop, {shutdown, Error}, State} diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl index db884212d96d..fa10d5a98487 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl @@ -22,11 +22,11 @@ remove_duplicate_clientid_connections/2, init_sparkplug/0, mqtt_to_amqp/1, - amqp_to_mqtt/1 + amqp_to_mqtt/1, + truncate_binary/2 ]). -define(MAX_TOPIC_TRANSLATION_CACHE_SIZE, 12). - -define(SPARKPLUG_MP_MQTT_TO_AMQP, sparkplug_mp_mqtt_to_amqp). -define(SPARKPLUG_MP_AMQP_TO_MQTT, sparkplug_mp_amqp_to_mqtt). @@ -197,3 +197,11 @@ remove_duplicate_clientid_connections(PgGroup, PidToKeep) -> %% MQTT supervision tree on this node not fully started ok end. + +-spec truncate_binary(binary(), non_neg_integer()) -> binary(). +truncate_binary(Bin, Size) + when is_binary(Bin) andalso byte_size(Bin) =< Size -> + Bin; +truncate_binary(Bin, Size) + when is_binary(Bin) -> + binary:part(Bin, 0, Size). diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index 8e97e8221f19..a987ea24db02 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -287,9 +287,10 @@ parse(Data, ParseState) -> try rabbit_mqtt_packet:parse(Data, ParseState) catch - _:Error:Stacktrace -> - ?LOG_ERROR("MQTT cannot parse a packet; payload: ~tp, error: {~tp, ~tp} ", - [Data, Error, Stacktrace]), + _:Reason:Stacktrace -> + ?LOG_ERROR("Web MQTT cannot parse a packet, reason: ~tp, stacktrace: ~tp, " + "payload (first 100 bytes): ~tp", + [Reason, Stacktrace, rabbit_mqtt_util:truncate_binary(Data, 100)]), {error, cannot_parse} end. From d86ce70fd611d196b96c01d20db48a9a3aee4f62 Mon Sep 17 00:00:00 2001 From: Chunyi Lyu Date: Thu, 19 Jan 2023 16:25:02 +0000 Subject: [PATCH 110/118] Add missing type definitions in mqtt records --- deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl | 4 ++-- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 14 ++++++++------ 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index fa006cd731a8..273134bf920d 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -46,7 +46,7 @@ connected_at :: pos_integer()}). -record(state, - {socket, + {socket :: rabbit_net:socket(), proto_ver :: option(mqtt310 | mqtt311), queue_states = rabbit_queue_type:init() :: rabbit_queue_type:state(), %% Packet IDs published to queues but not yet confirmed. @@ -71,7 +71,7 @@ auth_state :: option(#auth_state{}), peer_addr :: inet:ip_address(), send_fun :: fun((Packet :: tuple(), state()) -> term()), - register_state, + register_state :: option(registered | {pending, reference()}), conn_name :: option(binary()), info :: option(#info{}), delivery_flow :: flow | noflow, diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index 5f8ab5f54c70..b1cd16890bbd 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -24,19 +24,21 @@ -include("rabbit_mqtt.hrl"). +-type option(T) :: undefined | T. + -define(HIBERNATE_AFTER, 1000). -define(PROTO_FAMILY, 'MQTT'). -record(state, - {socket, - proxy_socket :: undefined | {rabbit_proxy_soket, any(), any()}, + {socket :: rabbit_net:socket(), + proxy_socket :: option({rabbit_proxy_socket, any(), any()}), await_recv :: boolean(), - deferred_recv :: undefined | binary(), - parse_state, + deferred_recv :: option(binary()), + parse_state :: atom(), proc_state :: rabbit_mqtt_processor:state(), connection_state :: running | blocked, conserve :: boolean(), - stats_timer, + stats_timer :: option(reference()), keepalive = rabbit_mqtt_keepalive:init() :: rabbit_mqtt_keepalive:state(), conn_name :: binary(), received_connect_packet :: boolean() @@ -388,7 +390,7 @@ network_error(closed, Args = [ConnName], case Connected of true -> ?LOG_INFO(Fmt, Args); - false -> ?LOG_DEBUG(Fmt, Args) + false -> ?LOG_DEBUG(Fmt, Args) end, {stop, {shutdown, conn_closed}, State}; From cd8962b5fdf534dcf22a9e9ae16c2502d14bb11a Mon Sep 17 00:00:00 2001 From: David Ansari Date: Thu, 19 Jan 2023 13:38:27 +0100 Subject: [PATCH 111/118] Remove optional rabbit_queue_type callbacks Instead of having optional rabbit_queue_type callbacks, add stub implementations to rabbit_mqtt_qos0_queue throwing an exception. The exception uses erlang:error/2 including stack trace and arguments of the unsupported functions to ease debugging in case these functions were ever to be called. Dialyzer suppressions are added for these functions such that dialyzer won't complain about: ``` rabbit_mqtt_qos0_queue.erl:244:1: Function init/1 only terminates with explicit exception ``` --- deps/rabbit/src/rabbit_queue_type.erl | 12 ----- .../src/rabbit_mqtt_qos0_queue.erl | 50 ++++++++++++++++++- 2 files changed, 49 insertions(+), 13 deletions(-) diff --git a/deps/rabbit/src/rabbit_queue_type.erl b/deps/rabbit/src/rabbit_queue_type.erl index 49ed61f3f718..b54e82ef638c 100644 --- a/deps/rabbit/src/rabbit_queue_type.erl +++ b/deps/rabbit/src/rabbit_queue_type.erl @@ -152,18 +152,6 @@ -callback is_stateful() -> boolean(). -%% stateful callbacks are optional --optional_callbacks([init/1, - close/1, - update/2, - consume/3, - cancel/5, - handle_event/2, - settle/4, - credit/4, - dequeue/4, - state_info/1]). - %% intitialise and return a queue type specific session context -callback init(amqqueue:amqqueue()) -> {ok, queue_state()} | {error, Reason :: term()}. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl index 4aa3d6e0192c..866b4ae20ccd 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl @@ -22,7 +22,7 @@ -include_lib("rabbit_common/include/rabbit.hrl"). -include_lib("rabbit/include/amqqueue.hrl"). -%% rabbit_queue_type callbacks +%% Stateless rabbit_queue_type callbacks. -export([ is_stateful/0, declare/2, @@ -40,6 +40,24 @@ notify_decorators/1 ]). +%% Stateful rabbit_queue_type callbacks are unsupported by this queue type. +-define(STATEFUL_CALLBACKS, + [ + init/1, + close/1, + update/2, + consume/3, + cancel/5, + handle_event/2, + settle/4, + credit/4, + dequeue/4, + state_info/1 + ]). +-export(?STATEFUL_CALLBACKS). +-dialyzer({nowarn_function, ?STATEFUL_CALLBACKS}). +-define(UNSUPPORTED(Args), erlang:error(unsupported, Args)). + -define(INFO_KEYS, [type, name, durable, auto_delete, arguments, pid, owner_pid, state, messages]). @@ -224,3 +242,33 @@ i(messages, Q) -> end; i(_, _) -> ''. + +init(A1) -> + ?UNSUPPORTED([A1]). + +close(A1) -> + ?UNSUPPORTED([A1]). + +update(A1,A2) -> + ?UNSUPPORTED([A1,A2]). + +consume(A1,A2,A3) -> + ?UNSUPPORTED([A1,A2,A3]). + +cancel(A1,A2,A3,A4,A5) -> + ?UNSUPPORTED([A1,A2,A3,A4,A5]). + +handle_event(A1,A2) -> + ?UNSUPPORTED([A1,A2]). + +settle(A1,A2,A3,A4) -> + ?UNSUPPORTED([A1,A2,A3,A4]). + +credit(A1,A2,A3,A4) -> + ?UNSUPPORTED([A1,A2,A3,A4]). + +dequeue(A1,A2,A3,A4) -> + ?UNSUPPORTED([A1,A2,A3,A4]). + +state_info(A1) -> + ?UNSUPPORTED([A1]). From 8a2a82e19bc07db9951c41f9fe141c9848dcf010 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Thu, 19 Jan 2023 15:33:28 +0100 Subject: [PATCH 112/118] Remove feature flag no_queue_name_in_classic_queue_client as it was unnecessary to introduce it in the first place. Remove the queue name from all queue type clients and pass the queue name to the queue type callbacks that need it. We have to leave feature flag classic_queue_type_delivery_support required because we removed the monitor registry https://github.com/rabbitmq/rabbitmq-server/blob/1fd4a6d353bd57793d38f7640bdfce1c6e6ada64/deps/rabbit/src/rabbit_queue_type.erl#L322-L325 Implements review from Karl: "rather than changing the message format we could amend the queue type callbacks involved with the stateful operation to also take the queue name record as an argument. This way we don't need to maintain the extra queue name (which uses memory for known but obscurely technical reasons with how maps work) in the queue type state (as it is used in the queue type state map as the key)" --- deps/rabbit/src/rabbit_classic_queue.erl | 83 ++---- deps/rabbit/src/rabbit_core_ff.erl | 7 - deps/rabbit/src/rabbit_fifo_client.erl | 119 ++++----- deps/rabbit/src/rabbit_queue_type.erl | 35 +-- deps/rabbit/src/rabbit_quorum_queue.erl | 51 ++-- deps/rabbit/src/rabbit_stream_queue.erl | 64 +++-- deps/rabbit/test/backing_queue_SUITE.erl | 4 - deps/rabbit/test/rabbit_fifo_int_SUITE.erl | 252 ++++++++---------- .../src/rabbit_mgmt_wm_connection.erl | 6 +- .../src/rabbit_mqtt_qos0_queue.erl | 24 +- 10 files changed, 285 insertions(+), 360 deletions(-) diff --git a/deps/rabbit/src/rabbit_classic_queue.erl b/deps/rabbit/src/rabbit_classic_queue.erl index fd566df1d9ba..a5b92bda94dc 100644 --- a/deps/rabbit/src/rabbit_classic_queue.erl +++ b/deps/rabbit/src/rabbit_classic_queue.erl @@ -13,8 +13,6 @@ -record(?STATE, { %% the current master pid pid :: undefined | pid(), - %% undefined if feature flag no_queue_name_in_classic_queue_client enabled - qref :: term(), unconfirmed = #{} :: #{non_neg_integer() => #msg_status{}}, monitored = #{} :: #{pid() => ok} }). @@ -39,11 +37,11 @@ update/2, consume/3, cancel/5, - handle_event/2, + handle_event/3, deliver/2, - settle/4, - credit/4, - dequeue/4, + settle/5, + credit/5, + dequeue/5, info/2, state_info/1, capabilities/0, @@ -163,14 +161,7 @@ stat(Q) -> -spec init(amqqueue:amqqueue()) -> {ok, state()}. init(Q) when ?amqqueue_is_classic(Q) -> - QRef = case rabbit_feature_flags:is_enabled(no_queue_name_in_classic_queue_client) of - true -> - undefined; - false -> - amqqueue:get_name(Q) - end, - {ok, #?STATE{pid = amqqueue:get_pid(Q), - qref = QRef}}. + {ok, #?STATE{pid = amqqueue:get_pid(Q)}}. -spec close(state()) -> ok. close(_State) -> @@ -223,61 +214,55 @@ cancel(Q, ConsumerTag, OkMsg, ActingUser, State) -> Err -> Err end. --spec settle(rabbit_queue_type:settle_op(), rabbit_types:ctag(), - [non_neg_integer()], state()) -> +-spec settle(rabbit_amqqueue:name(), rarabbit_queue_type:settle_op(), + rabbit_types:ctag(), [non_neg_integer()], state()) -> {state(), rabbit_queue_type:actions()}. -settle(complete, _CTag, MsgIds, State) -> +settle(_QName, complete, _CTag, MsgIds, State) -> Pid = State#?STATE.pid, delegate:invoke_no_result(Pid, {gen_server2, cast, [{ack, MsgIds, self()}]}), {State, []}; -settle(Op, _CTag, MsgIds, State) -> +settle(_QName, Op, _CTag, MsgIds, State) -> ChPid = self(), ok = delegate:invoke_no_result(State#?STATE.pid, {gen_server2, cast, [{reject, Op == requeue, MsgIds, ChPid}]}), {State, []}. -credit(CTag, Credit, Drain, State) -> +credit(_QName, CTag, Credit, Drain, State) -> ChPid = self(), delegate:invoke_no_result(State#?STATE.pid, {gen_server2, cast, [{credit, ChPid, CTag, Credit, Drain}]}), {State, []}. -handle_event({confirm, MsgSeqNos, Pid}, #?STATE{qref = QRef} = State) -> - %% backwards compatibility when feature flag no_queue_name_in_classic_queue_client disabled - handle_event({confirm, MsgSeqNos, Pid, QRef}, State); -handle_event({confirm, MsgSeqNos, Pid, QRef}, #?STATE{unconfirmed = U0} = State) -> +handle_event(QName, {confirm, MsgSeqNos, Pid}, #?STATE{unconfirmed = U0} = State) -> %% confirms should never result in rejections {Unconfirmed, ConfirmedSeqNos, []} = settle_seq_nos(MsgSeqNos, Pid, U0, confirm), - Actions = [{settled, QRef, ConfirmedSeqNos}], + Actions = [{settled, QName, ConfirmedSeqNos}], %% handle confirm event from queues %% in this case the classic queue should track each individual publish and %% the processes involved and only emit a settle action once they have all %% been received (or DOWN has been received). %% Hence this part of the confirm logic is queue specific. {ok, State#?STATE{unconfirmed = Unconfirmed}, Actions}; -handle_event({deliver, _, _, _} = Delivery, #?STATE{} = State) -> +handle_event(_QName, {deliver, _, _, _} = Delivery, #?STATE{} = State) -> {ok, State, [Delivery]}; -handle_event({reject_publish, SeqNo, QPid}, #?STATE{qref = QRef} = State) -> - %% backwards compatibility when feature flag no_queue_name_in_classic_queue_client disabled - handle_event({reject_publish, SeqNo, QPid, QRef}, State); -handle_event({reject_publish, SeqNo, _QPid, QRef}, +handle_event(QName, {reject_publish, SeqNo, _QPid}, #?STATE{unconfirmed = U0} = State) -> %% It does not matter which queue rejected the message, %% if any queue did, it should not be confirmed. {U, Rejected} = reject_seq_no(SeqNo, U0), - Actions = [{rejected, QRef, Rejected}], + Actions = [{rejected, QName, Rejected}], {ok, State#?STATE{unconfirmed = U}, Actions}; -handle_event({down, Pid, QRef, Info}, #?STATE{monitored = Monitored, - pid = MasterPid, - unconfirmed = U0} = State0) -> +handle_event(QName, {down, Pid, Info}, #?STATE{monitored = Monitored, + pid = MasterPid, + unconfirmed = U0} = State0) -> State = State0#?STATE{monitored = maps:remove(Pid, Monitored)}, Actions0 = case Pid =:= MasterPid of true -> - [{queue_down, QRef}]; + [{queue_down, QName}]; false -> [] end, @@ -294,8 +279,8 @@ handle_event({down, Pid, QRef, Info}, #?STATE{monitored = Monitored, {Unconfirmed, Settled, Rejected} = settle_seq_nos(MsgSeqNos, Pid, U0, down), Actions = settlement_action( - settled, QRef, Settled, - settlement_action(rejected, QRef, Rejected, Actions0)), + settled, QName, Settled, + settlement_action(rejected, QName, Rejected, Actions0)), {ok, State#?STATE{unconfirmed = Unconfirmed}, Actions}; true -> %% any abnormal exit should be considered a full reject of the @@ -312,11 +297,11 @@ handle_event({down, Pid, QRef, Info}, #?STATE{monitored = Monitored, end, [], U0), U = maps:without(MsgIds, U0), {ok, State#?STATE{unconfirmed = U}, - [{rejected, QRef, MsgIds} | Actions0]} + [{rejected, QName, MsgIds} | Actions0]} end; -handle_event({send_drained, _} = Action, State) -> +handle_event(_QName, {send_drained, _} = Action, State) -> {ok, State, [Action]}; -handle_event({send_credit_reply, _} = Action, State) -> +handle_event(_QName, {send_credit_reply, _} = Action, State) -> {ok, State, [Action]}. settlement_action(_Type, _QRef, [], Acc) -> @@ -354,11 +339,11 @@ deliver(Qs0, #delivery{flow = Flow, {Qs, []}. --spec dequeue(NoAck :: boolean(), LimiterPid :: pid(), - rabbit_types:ctag(), state()) -> +-spec dequeue(rabbit_amqqueue:name(), NoAck :: boolean(), + LimiterPid :: pid(), rabbit_types:ctag(), state()) -> {ok, Count :: non_neg_integer(), rabbit_amqqueue:qmsg(), state()} | {empty, state()}. -dequeue(NoAck, LimiterPid, _CTag, State) -> +dequeue(_QName, NoAck, LimiterPid, _CTag, State) -> QPid = State#?STATE.pid, case delegate:invoke(QPid, {gen_server2, call, [{basic_get, self(), NoAck, LimiterPid}, infinity]}) of @@ -554,21 +539,11 @@ ensure_monitor(Pid, QName, State = #?STATE{monitored = Monitored}) -> %% part of channel <-> queue api confirm_to_sender(Pid, QName, MsgSeqNos) -> - Msg = case rabbit_feature_flags:is_enabled(no_queue_name_in_classic_queue_client) of - true -> - {confirm, MsgSeqNos, self(), QName}; - false -> - {confirm, MsgSeqNos, self()} - end, + Msg = {confirm, MsgSeqNos, self()}, gen_server:cast(Pid, {queue_event, QName, Msg}). send_rejection(Pid, QName, MsgSeqNo) -> - Msg = case rabbit_feature_flags:is_enabled(no_queue_name_in_classic_queue_client) of - true -> - {reject_publish, MsgSeqNo, self(), QName}; - false -> - {reject_publish, MsgSeqNo, self()} - end, + Msg = {reject_publish, MsgSeqNo, self()}, gen_server:cast(Pid, {queue_event, QName, Msg}). deliver_to_consumer(Pid, QName, CTag, AckRequired, Message) -> diff --git a/deps/rabbit/src/rabbit_core_ff.erl b/deps/rabbit/src/rabbit_core_ff.erl index b3e43e053ac4..2be2bd23b354 100644 --- a/deps/rabbit/src/rabbit_core_ff.erl +++ b/deps/rabbit/src/rabbit_core_ff.erl @@ -125,13 +125,6 @@ depends_on => [stream_queue] }}). --rabbit_feature_flag( - {no_queue_name_in_classic_queue_client, - #{desc => "Remove queue name from classic queue type client to save memory", - stability => stable, - depends_on => [classic_queue_type_delivery_support] - }}). - %% ------------------------------------------------------------------- %% Direct exchange routing v2. %% ------------------------------------------------------------------- diff --git a/deps/rabbit/src/rabbit_fifo_client.erl b/deps/rabbit/src/rabbit_fifo_client.erl index 7b6c440be3cb..6403ff90f6bc 100644 --- a/deps/rabbit/src/rabbit_fifo_client.erl +++ b/deps/rabbit/src/rabbit_fifo_client.erl @@ -12,21 +12,20 @@ -module(rabbit_fifo_client). -export([ + init/1, init/2, - init/3, checkout/5, cancel_checkout/2, - enqueue/2, enqueue/3, - dequeue/3, + enqueue/4, + dequeue/4, settle/3, return/3, discard/3, credit/4, - handle_ra_event/3, + handle_ra_event/4, untracked_enqueue/2, purge/1, - queue_name/1, update_machine_state/2, pending_size/1, stat/1, @@ -47,17 +46,14 @@ rabbit_queue_type:action(). -type actions() :: [action()]. --type queue_name() :: rabbit_types:r(queue). - -record(consumer, {last_msg_id :: seq() | -1, ack = false :: boolean(), delivery_count = 0 :: non_neg_integer()}). --record(cfg, {queue_name :: queue_name(), - servers = [] :: [ra:server_id()], +-record(cfg, {servers = [] :: [ra:server_id()], soft_limit = ?SOFT_LIMIT :: non_neg_integer(), - timeout :: non_neg_integer(), - version = 0 :: non_neg_integer()}). + timeout :: non_neg_integer() + }). -record(state, {cfg :: #cfg{}, leader :: undefined | ra:server_id(), @@ -85,28 +81,26 @@ %% @doc Create the initial state for a new rabbit_fifo sessions. A state is needed %% to interact with a rabbit_fifo queue using @module. -%% @param QueueName the id of the cluster to interact with %% @param Servers The known servers of the queue. If the current leader is known %% ensure the leader node is at the head of the list. --spec init(queue_name(), [ra:server_id()]) -> state(). -init(QueueName, Servers) -> - init(QueueName, Servers, ?SOFT_LIMIT). +-spec init([ra:server_id()]) -> state(). +init(Servers) -> + init(Servers, ?SOFT_LIMIT). %% @doc Create the initial state for a new rabbit_fifo sessions. A state is needed %% to interact with a rabbit_fifo queue using @module. -%% @param QueueName the id of the cluster to interact with %% @param Servers The known servers of the queue. If the current leader is known %% ensure the leader node is at the head of the list. %% @param MaxPending size defining the max number of pending commands. --spec init(queue_name(), [ra:server_id()], non_neg_integer()) -> state(). -init(QueueName = #resource{}, Servers, SoftLimit) -> +-spec init([ra:server_id()], non_neg_integer()) -> state(). +init(Servers, SoftLimit) -> Timeout = application:get_env(kernel, net_ticktime, 60) + 5, - #state{cfg = #cfg{queue_name = QueueName, - servers = Servers, + #state{cfg = #cfg{servers = Servers, soft_limit = SoftLimit, timeout = Timeout * 1000}}. %% @doc Enqueues a message. +%% @param QueueName Name of the queue. %% @param Correlation an arbitrary erlang term used to correlate this %% command when it has been applied. %% @param Msg an arbitrary erlang term representing the message. @@ -116,9 +110,10 @@ init(QueueName = #resource{}, Servers, SoftLimit) -> %% {@module} assigns a sequence number to every raft command it issues. The %% SequenceNumber can be correlated to the applied sequence numbers returned %% by the {@link handle_ra_event/2. handle_ra_event/2} function. --spec enqueue(Correlation :: term(), Msg :: term(), State :: state()) -> +-spec enqueue(rabbit_amqqueue:name(), Correlation :: term(), + Msg :: term(), State :: state()) -> {ok, state(), actions()} | {reject_publish, state()}. -enqueue(Correlation, Msg, +enqueue(QName, Correlation, Msg, #state{queue_status = undefined, next_enqueue_seq = 1, cfg = #cfg{servers = Servers, @@ -128,7 +123,7 @@ enqueue(Correlation, Msg, case rpc:call(Node, ra_machine, version, [{machine, rabbit_fifo, #{}}]) of 0 -> %% the leader is running the old version - enqueue(Correlation, Msg, State0#state{queue_status = go}); + enqueue(QName, Correlation, Msg, State0#state{queue_status = go}); N when is_integer(N) -> %% were running the new version on the leader do sync initialisation %% of enqueuer session @@ -138,8 +133,8 @@ enqueue(Correlation, Msg, {reject_publish, State0#state{leader = Leader, queue_status = reject_publish}}; {ok, ok, Leader} -> - enqueue(Correlation, Msg, State0#state{leader = Leader, - queue_status = go}); + enqueue(QName, Correlation, Msg, State0#state{leader = Leader, + queue_status = go}); {error, {no_more_servers_to_try, _Errs}} -> %% if we are not able to process the register command %% it is safe to reject the message as we never attempted @@ -155,11 +150,11 @@ enqueue(Correlation, Msg, {badrpc, nodedown} -> {reject_publish, State0} end; -enqueue(_Correlation, _Msg, +enqueue(_QName, _Correlation, _Msg, #state{queue_status = reject_publish, cfg = #cfg{}} = State) -> {reject_publish, State}; -enqueue(Correlation, Msg, +enqueue(QName, Correlation, Msg, #state{slow = Slow, pending = Pending, queue_status = go, @@ -180,12 +175,13 @@ enqueue(Correlation, Msg, slow = Tag == slow}, case Tag of slow when not Slow -> - {ok, set_timer(State), [{block, cluster_name(State)}]}; + {ok, set_timer(QName, State), [{block, cluster_name(State)}]}; _ -> {ok, State, []} end. %% @doc Enqueues a message. +%% @param QueueName Name of the queue. %% @param Msg an arbitrary erlang term representing the message. %% @param State the current {@module} state. %% @returns @@ -194,29 +190,29 @@ enqueue(Correlation, Msg, %% SequenceNumber can be correlated to the applied sequence numbers returned %% by the {@link handle_ra_event/2. handle_ra_event/2} function. %% --spec enqueue(Msg :: term(), State :: state()) -> +-spec enqueue(rabbit_amqqueue:name(), Msg :: term(), State :: state()) -> {ok, state(), actions()} | {reject_publish, state()}. -enqueue(Msg, State) -> - enqueue(undefined, Msg, State). +enqueue(QName, Msg, State) -> + enqueue(QName, undefined, Msg, State). %% @doc Dequeue a message from the queue. %% %% This is a synchronous call. I.e. the call will block until the command %% has been accepted by the ra process or it times out. %% +%% @param QueueName Name of the queue. %% @param ConsumerTag a unique tag to identify this particular consumer. %% @param Settlement either `settled' or `unsettled'. When `settled' no %% further settlement needs to be done. %% @param State The {@module} state. %% %% @returns `{ok, IdMsg, State}' or `{error | timeout, term()}' --spec dequeue(rabbit_fifo:consumer_tag(), +-spec dequeue(rabbit_amqqueue:name(), rabbit_fifo:consumer_tag(), Settlement :: settled | unsettled, state()) -> {ok, non_neg_integer(), term(), non_neg_integer()} | {empty, state()} | {error | timeout, term()}. -dequeue(ConsumerTag, Settlement, - #state{cfg = #cfg{timeout = Timeout, - queue_name = QName}} = State0) -> +dequeue(QueueName, ConsumerTag, Settlement, + #state{cfg = #cfg{timeout = Timeout}} = State0) -> Node = pick_server(State0), ConsumerId = consumer_id(ConsumerTag), case ra:process_command(Node, @@ -234,7 +230,7 @@ dequeue(ConsumerTag, Settlement, IsDelivered = Count > 0, Msg = add_delivery_count_header(Msg0, Count), {ok, MsgsReady, - {QName, qref(Leader), MsgId, IsDelivered, Msg}, + {QueueName, qref(Leader), MsgId, IsDelivered, Msg}, State0#state{leader = Leader}}; {ok, {error, _} = Err, _Leader} -> Err; @@ -470,11 +466,6 @@ stat(Leader, Timeout) -> {timeout, _} = Error -> Error end. -%% @doc returns the cluster name --spec queue_name(state()) -> queue_name(). -queue_name(#state{cfg = #cfg{queue_name = QueueName}}) -> - QueueName. - update_machine_state(Server, Conf) -> case ra:process_command(Server, rabbit_fifo:make_update_config(Conf), ?COMMAND_TIMEOUT) of {ok, ok, _} -> @@ -505,6 +496,7 @@ update_machine_state(Server, Conf) -> %% end %% ''' %% +%% @param QName Name of the queue. %% @param From the {@link ra:server_id().} of the sending process. %% @param Event the body of the `ra_event'. %% @param State the current {@module} state. @@ -526,13 +518,12 @@ update_machine_state(Server, Conf) -> %%
  • `MsgId' is a consumer scoped monotonically incrementing id that can be %% used to {@link settle/3.} (roughly: AMQP 0.9.1 ack) message once finished %% with them.
  • --spec handle_ra_event(ra:server_id(), ra_server_proc:ra_event_body(), state()) -> +-spec handle_ra_event(rabbit_amqqueue:name(), ra:server_id(), + ra_server_proc:ra_event_body(), state()) -> {internal, Correlators :: [term()], actions(), state()} | {rabbit_fifo:client_msg(), state()} | {eol, actions()}. -handle_ra_event(From, {applied, Seqs}, - #state{cfg = #cfg{queue_name = QRef, - soft_limit = SftLmt - }} = State0) -> +handle_ra_event(QName, From, {applied, Seqs}, + #state{cfg = #cfg{soft_limit = SftLmt}} = State0) -> {Corrs, Actions0, State1} = lists:foldl(fun seq_applied/2, {[], [], State0#state{leader = From}}, @@ -546,7 +537,7 @@ handle_ra_event(From, {applied, Seqs}, %% The wrong order does not matter much because the channel sorts the %% sequence numbers before confirming to the client. But rabbit_fifo_client %% is sequence numer agnostic: it handles any correlation terms. - [{settled, QRef, Corrs} + [{settled, QName, Corrs} | lists:reverse(Actions0)] end, case maps:size(State1#state.pending) < SftLmt of @@ -578,13 +569,13 @@ handle_ra_event(From, {applied, Seqs}, _ -> {ok, State1, Actions} end; -handle_ra_event(From, {machine, {delivery, _ConsumerTag, _} = Del}, State0) -> - handle_delivery(From, Del, State0); -handle_ra_event(_, {machine, {queue_status, Status}}, +handle_ra_event(QName, From, {machine, {delivery, _ConsumerTag, _} = Del}, State0) -> + handle_delivery(QName, From, Del, State0); +handle_ra_event(_QName, _, {machine, {queue_status, Status}}, #state{} = State) -> %% just set the queue status {ok, State#state{queue_status = Status}, []}; -handle_ra_event(Leader, {machine, leader_change}, +handle_ra_event(_QName, Leader, {machine, leader_change}, #state{leader = OldLeader} = State0) -> %% we need to update leader %% and resend any pending commands @@ -592,28 +583,28 @@ handle_ra_event(Leader, {machine, leader_change}, [?MODULE, OldLeader, Leader]), State = resend_all_pending(State0#state{leader = Leader}), {ok, State, []}; -handle_ra_event(_From, {rejected, {not_leader, Leader, _Seq}}, +handle_ra_event(_QName, _From, {rejected, {not_leader, Leader, _Seq}}, #state{leader = Leader} = State) -> {ok, State, []}; -handle_ra_event(_From, {rejected, {not_leader, Leader, _Seq}}, +handle_ra_event(_QName, _From, {rejected, {not_leader, Leader, _Seq}}, #state{leader = OldLeader} = State0) -> rabbit_log:debug("~ts: Detected QQ leader change (rejection) from ~w to ~w", [?MODULE, OldLeader, Leader]), State = resend_all_pending(State0#state{leader = Leader}), {ok, cancel_timer(State), []}; -handle_ra_event(_From, {rejected, {not_leader, _UndefinedMaybe, _Seq}}, State0) -> +handle_ra_event(_QName, _From, {rejected, {not_leader, _UndefinedMaybe, _Seq}}, State0) -> % TODO: how should these be handled? re-sent on timer or try random {ok, State0, []}; -handle_ra_event(_, timeout, #state{cfg = #cfg{servers = Servers}} = State0) -> +handle_ra_event(QName, _, timeout, #state{cfg = #cfg{servers = Servers}} = State0) -> case find_leader(Servers) of undefined -> %% still no leader, set the timer again - {ok, set_timer(State0), []}; + {ok, set_timer(QName, State0), []}; Leader -> State = resend_all_pending(State0#state{leader = Leader}), {ok, State, []} end; -handle_ra_event(_Leader, {machine, eol}, State) -> +handle_ra_event(_QName, _Leader, {machine, eol}, State) -> {eol, [{unblock, cluster_name(State)}]}. %% @doc Attempts to enqueue a message using cast semantics. This provides no @@ -710,9 +701,8 @@ maybe_auto_ack(false, {deliver, Tag, _Ack, Msgs} = Deliver, State0) -> {State, Actions} = settle(Tag, MsgIds, State0), {ok, State, [Deliver] ++ Actions}. -handle_delivery(Leader, {delivery, Tag, [{FstId, _} | _] = IdMsgs}, - #state{cfg = #cfg{queue_name = QName}, - consumer_deliveries = CDels0} = State0) +handle_delivery(QName, Leader, {delivery, Tag, [{FstId, _} | _] = IdMsgs}, + #state{consumer_deliveries = CDels0} = State0) when is_map_key(Tag, CDels0) -> QRef = qref(Leader), {LastId, _} = lists:last(IdMsgs), @@ -755,7 +745,7 @@ handle_delivery(Leader, {delivery, Tag, [{FstId, _} | _] = IdMsgs}, [] -> {ok, State0, []}; IdMsgs2 -> - handle_delivery(Leader, {delivery, Tag, IdMsgs2}, State0) + handle_delivery(QName, Leader, {delivery, Tag, IdMsgs2}, State0) end; C when FstId =:= 0 -> % the very first delivery @@ -766,7 +756,7 @@ handle_delivery(Leader, {delivery, Tag, [{FstId, _} | _] = IdMsgs}, C#consumer{last_msg_id = LastId}, CDels0)}) end; -handle_delivery(_Leader, {delivery, Tag, [_ | _] = IdMsgs}, +handle_delivery(_QName, _Leader, {delivery, Tag, [_ | _] = IdMsgs}, #state{consumer_deliveries = CDels0} = State0) when not is_map_key(Tag, CDels0) -> %% Note: @@ -876,9 +866,8 @@ add_command(Cid, return, MsgIds, Acc) -> add_command(Cid, discard, MsgIds, Acc) -> [rabbit_fifo:make_discard(Cid, MsgIds) | Acc]. -set_timer(#state{leader = Leader0, - cfg = #cfg{servers = [Server | _], - queue_name = QName}} = State) -> +set_timer(QName, #state{leader = Leader0, + cfg = #cfg{servers = [Server | _]}} = State) -> Leader = case Leader0 of undefined -> Server; _ -> diff --git a/deps/rabbit/src/rabbit_queue_type.erl b/deps/rabbit/src/rabbit_queue_type.erl index b54e82ef638c..51ee4d74dbcc 100644 --- a/deps/rabbit/src/rabbit_queue_type.erl +++ b/deps/rabbit/src/rabbit_queue_type.erl @@ -49,7 +49,7 @@ notify_decorators/1 ]). --type queue_name() :: rabbit_types:r(queue). +-type queue_name() :: rabbit_amqqueue:name(). -type queue_state() :: term(). -type msg_tag() :: term(). -type arguments() :: queue_arguments | consumer_arguments. @@ -174,7 +174,8 @@ %% any async events returned from the queue system should be processed through %% this --callback handle_event(Event :: event(), +-callback handle_event(queue_name(), + Event :: event(), queue_state()) -> {ok, queue_state(), actions()} | {error, term()} | {eol, actions()} | {protocol_error, Type :: atom(), Reason :: string(), Args :: term()}. @@ -183,15 +184,16 @@ Delivery :: term()) -> {[{amqqueue:amqqueue(), queue_state()}], actions()}. --callback settle(settle_op(), rabbit_types:ctag(), [non_neg_integer()], queue_state()) -> +-callback settle(queue_name(), settle_op(), rabbit_types:ctag(), + [non_neg_integer()], queue_state()) -> {queue_state(), actions()} | {'protocol_error', Type :: atom(), Reason :: string(), Args :: term()}. --callback credit(rabbit_types:ctag(), +-callback credit(queue_name(), rabbit_types:ctag(), non_neg_integer(), Drain :: boolean(), queue_state()) -> {queue_state(), actions()}. --callback dequeue(NoAck :: boolean(), LimiterPid :: pid(), +-callback dequeue(queue_name(), NoAck :: boolean(), LimiterPid :: pid(), rabbit_types:ctag(), queue_state()) -> {ok, Count :: non_neg_integer(), rabbit_amqqueue:qmsg(), queue_state()} | {empty, queue_state()} | @@ -369,12 +371,11 @@ init() -> -spec close(state()) -> ok. close(#?STATE{ctxs = Contexts}) -> - _ = maps:map( - fun (_, #ctx{module = Mod, - state = S}) -> - ok = Mod:close(S) - end, Contexts), - ok. + maps:foreach( + fun (_, #ctx{module = Mod, + state = S}) -> + ok = Mod:close(S) + end, Contexts). -spec new(amqqueue:amqqueue(), state()) -> state(). new(Q, State) when ?is_amqqueue(Q) -> @@ -438,7 +439,7 @@ recover(VHost, Qs) -> -spec handle_down(pid(), queue_name(), term(), state()) -> {ok, state(), actions()} | {eol, state(), queue_name()} | {error, term()}. handle_down(Pid, QName, Info, State0) -> - case handle_event(QName, {down, Pid, QName, Info}, State0) of + case handle_event(QName, {down, Pid, Info}, State0) of {ok, State, Actions} -> {ok, State, Actions}; {eol, []} -> @@ -457,7 +458,7 @@ handle_event(QRef, Evt, Ctxs) -> case get_ctx(QRef, Ctxs, undefined) of #ctx{module = Mod, state = State0} = Ctx -> - case Mod:handle_event(Evt, State0) of + case Mod:handle_event(QRef, Evt, State0) of {ok, State, Actions} -> {ok, set_ctx(QRef, Ctx#ctx{state = State}, Ctxs), Actions}; Other -> @@ -544,7 +545,7 @@ settle(#resource{kind = queue} = QRef, Op, CTag, MsgIds, Ctxs) -> {ok, Ctxs, []}; #ctx{state = State0, module = Mod} = Ctx -> - case Mod:settle(Op, CTag, MsgIds, State0) of + case Mod:settle(QRef, Op, CTag, MsgIds, State0) of {State, Actions} -> {ok, set_ctx(QRef, Ctx#ctx{state = State}, Ctxs), Actions}; Err -> @@ -558,7 +559,8 @@ settle(#resource{kind = queue} = QRef, Op, CTag, MsgIds, Ctxs) -> credit(Q, CTag, Credit, Drain, Ctxs) -> #ctx{state = State0, module = Mod} = Ctx = get_ctx(Q, Ctxs), - {State, Actions} = Mod:credit(CTag, Credit, Drain, State0), + QName = amqqueue:get_name(Q), + {State, Actions} = Mod:credit(QName, CTag, Credit, Drain, State0), {ok, set_ctx(Q, Ctx#ctx{state = State}, Ctxs), Actions}. -spec dequeue(amqqueue:amqqueue(), boolean(), @@ -569,7 +571,8 @@ credit(Q, CTag, Credit, Drain, Ctxs) -> dequeue(Q, NoAck, LimiterPid, CTag, Ctxs) -> #ctx{state = State0} = Ctx = get_ctx(Q, Ctxs), Mod = amqqueue:get_type(Q), - case Mod:dequeue(NoAck, LimiterPid, CTag, State0) of + QName = amqqueue:get_name(Q), + case Mod:dequeue(QName, NoAck, LimiterPid, CTag, State0) of {ok, Num, Msg, State} -> {ok, Num, Msg, set_ctx(Q, Ctx#ctx{state = State}, Ctxs)}; {empty, State} -> diff --git a/deps/rabbit/src/rabbit_quorum_queue.erl b/deps/rabbit/src/rabbit_quorum_queue.erl index 97552a447062..7ede155cf5d0 100644 --- a/deps/rabbit/src/rabbit_quorum_queue.erl +++ b/deps/rabbit/src/rabbit_quorum_queue.erl @@ -12,7 +12,7 @@ -export([init/1, close/1, update/2, - handle_event/2]). + handle_event/3]). -export([is_recoverable/1, recover/2, stop/1, @@ -22,12 +22,11 @@ delete/4, delete_immediately/2]). -export([state_info/1, info/2, stat/1, infos/1]). --export([settle/4, dequeue/4, consume/3, cancel/5]). --export([credit/4]). +-export([settle/5, dequeue/5, consume/3, cancel/5]). +-export([credit/5]). -export([purge/1]). -export([stateless_deliver/2, deliver/2]). -export([dead_letter_publish/5]). --export([queue_name/1]). -export([cluster_state/1, status/2]). -export([update_consumer_handler/8, update_consumer/9]). -export([cancel_consumer_handler/2, cancel_consumer/3]). @@ -133,11 +132,10 @@ init(Q) when ?is_amqqueue(Q) -> %% know what to do if the queue has `disappeared`. Let it crash. {Name, _LeaderNode} = Leader = amqqueue:get_pid(Q), Nodes = get_nodes(Q), - QName = amqqueue:get_name(Q), %% Ensure the leader is listed first Servers0 = [{Name, N} || N <- Nodes], Servers = [Leader | lists:delete(Leader, Servers0)], - {ok, rabbit_fifo_client:init(QName, Servers, SoftLimit)}. + {ok, rabbit_fifo_client:init(Servers, SoftLimit)}. -spec close(rabbit_fifo_client:state()) -> ok. close(_State) -> @@ -149,13 +147,14 @@ update(Q, State) when ?amqqueue_is_quorum(Q) -> %% QQ state maintains it's own updates State. --spec handle_event({amqqueue:ra_server_id(), any()}, +-spec handle_event(rabbit_amqquue:name(), + {amqqueue:ra_server_id(), any()}, rabbit_fifo_client:state()) -> {ok, rabbit_fifo_client:state(), rabbit_queue_type:actions()} | {eol, rabbit_queue_type:actions()} | {protocol_error, Type :: atom(), Reason :: string(), Args :: term()}. -handle_event({From, Evt}, QState) -> - rabbit_fifo_client:handle_ra_event(From, Evt, QState). +handle_event(QName, {From, Evt}, QState) -> + rabbit_fifo_client:handle_ra_event(QName, From, Evt, QState). -spec declare(amqqueue:amqqueue(), node()) -> {new | existing, amqqueue:amqqueue()} | @@ -720,22 +719,22 @@ delete_immediately(Resource, {_Name, _} = QPid) -> rabbit_core_metrics:queue_deleted(Resource), ok. -settle(complete, CTag, MsgIds, QState) -> +settle(_QName, complete, CTag, MsgIds, QState) -> rabbit_fifo_client:settle(quorum_ctag(CTag), MsgIds, QState); -settle(requeue, CTag, MsgIds, QState) -> +settle(_QName, requeue, CTag, MsgIds, QState) -> rabbit_fifo_client:return(quorum_ctag(CTag), MsgIds, QState); -settle(discard, CTag, MsgIds, QState) -> +settle(_QName, discard, CTag, MsgIds, QState) -> rabbit_fifo_client:discard(quorum_ctag(CTag), MsgIds, QState). -credit(CTag, Credit, Drain, QState) -> +credit(_QName, CTag, Credit, Drain, QState) -> rabbit_fifo_client:credit(quorum_ctag(CTag), Credit, Drain, QState). --spec dequeue(NoAck :: boolean(), pid(), +-spec dequeue(rabbit_amqqueue:name(), NoAck :: boolean(), pid(), rabbit_types:ctag(), rabbit_fifo_client:state()) -> {empty, rabbit_fifo_client:state()} | {ok, QLen :: non_neg_integer(), qmsg(), rabbit_fifo_client:state()} | {error, term()}. -dequeue(NoAck, _LimiterPid, CTag0, QState0) -> +dequeue(QName, NoAck, _LimiterPid, CTag0, QState0) -> CTag = quorum_ctag(CTag0), Settlement = case NoAck of true -> @@ -743,7 +742,7 @@ dequeue(NoAck, _LimiterPid, CTag0, QState0) -> false -> unsettled end, - rabbit_fifo_client:dequeue(CTag, Settlement, QState0). + rabbit_fifo_client:dequeue(QName, CTag, Settlement, QState0). -spec consume(amqqueue:amqqueue(), rabbit_queue_type:consume_spec(), @@ -862,19 +861,20 @@ stateless_deliver(ServerId, Delivery) -> ok = rabbit_fifo_client:untracked_enqueue([ServerId], Delivery#delivery.message). --spec deliver(Confirm :: boolean(), rabbit_types:delivery(), - rabbit_fifo_client:state()) -> +-spec deliver(rabbit_amqqueue:name(), Confirm :: boolean(), + rabbit_types:delivery(), rabbit_fifo_client:state()) -> {ok, rabbit_fifo_client:state(), rabbit_queue_type:actions()} | {reject_publish, rabbit_fifo_client:state()}. -deliver(false, Delivery, QState0) -> - case rabbit_fifo_client:enqueue(Delivery#delivery.message, QState0) of +deliver(QName, false, Delivery, QState0) -> + case rabbit_fifo_client:enqueue(QName, Delivery#delivery.message, QState0) of {ok, _State, _Actions} = Res -> Res; {reject_publish, State} -> {ok, State, []} end; -deliver(true, Delivery, QState0) -> - rabbit_fifo_client:enqueue(Delivery#delivery.msg_seq_no, +deliver(QName, true, Delivery, QState0) -> + rabbit_fifo_client:enqueue(QName, + Delivery#delivery.msg_seq_no, Delivery#delivery.message, QState0). deliver(QSs, #delivery{message = #basic_message{content = Content0} = Msg, @@ -889,10 +889,10 @@ deliver(QSs, #delivery{message = #basic_message{content = Content0} = Msg, [QRef], Delivery#delivery.message), {Qs, Actions}; ({Q, S0}, {Qs, Actions}) -> - case deliver(Confirm, Delivery, S0) of + QName = amqqueue:get_name(Q), + case deliver(QName, Confirm, Delivery, S0) of {reject_publish, S} -> Seq = Delivery#delivery.msg_seq_no, - QName = rabbit_fifo_client:queue_name(S), {[{Q, S} | Qs], [{rejected, QName, [Seq]} | Actions]}; {ok, S, As} -> {[{Q, S} | Qs], As ++ Actions} @@ -1581,9 +1581,6 @@ quorum_ctag(Other) -> maybe_send_reply(_ChPid, undefined) -> ok; maybe_send_reply(ChPid, Msg) -> ok = rabbit_channel:send_command(ChPid, Msg). -queue_name(RaFifoState) -> - rabbit_fifo_client:queue_name(RaFifoState). - get_default_quorum_initial_group_size(Arguments) -> case rabbit_misc:table_lookup(Arguments, <<"x-quorum-initial-group-size">>) of undefined -> diff --git a/deps/rabbit/src/rabbit_stream_queue.erl b/deps/rabbit/src/rabbit_stream_queue.erl index 36166c3e9ee4..2672c0e3d4f5 100644 --- a/deps/rabbit/src/rabbit_stream_queue.erl +++ b/deps/rabbit/src/rabbit_stream_queue.erl @@ -19,11 +19,11 @@ is_recoverable/1, consume/3, cancel/5, - handle_event/2, + handle_event/3, deliver/2, - settle/4, - credit/4, - dequeue/4, + settle/5, + credit/5, + dequeue/5, info/2, queue_length/1, get_replicas/1, @@ -68,8 +68,7 @@ -type msg_id() :: non_neg_integer(). -type msg() :: term(). %% TODO: refine --record(stream, {name :: rabbit_types:r('queue'), - credit :: integer(), +-record(stream, {credit :: integer(), max :: non_neg_integer(), start_offset = 0 :: non_neg_integer(), listening_offset = 0 :: non_neg_integer(), @@ -247,7 +246,7 @@ consume(Q, Spec, QState0) when ?amqqueue_is_stream(Q) -> %% really it should be sent by the stream queue process like classic queues %% do maybe_send_reply(ChPid, OkMsg), - begin_stream(QState0, Q, ConsumerTag, OffsetSpec, ConsumerPrefetchCount) + begin_stream(QState0, ConsumerTag, OffsetSpec, ConsumerPrefetchCount) end; Err -> Err @@ -298,7 +297,7 @@ get_local_pid(#stream_client{stream_id = StreamId, end. begin_stream(#stream_client{name = QName, readers = Readers0} = State0, - Q, Tag, Offset, Max) -> + Tag, Offset, Max) -> {LocalPid, State} = get_local_pid(State0), case LocalPid of undefined -> @@ -316,8 +315,7 @@ begin_stream(#stream_client{name = QName, readers = Readers0} = State0, {timestamp, _} -> NextOffset; _ -> Offset end, - Str0 = #stream{name = amqqueue:get_name(Q), - credit = Max, + Str0 = #stream{credit = Max, start_offset = StartOffset, listening_offset = NextOffset, log = Seg0, @@ -343,13 +341,13 @@ cancel(_Q, ConsumerTag, OkMsg, ActingUser, #stream_client{readers = Readers0, {ok, State} end. -credit(CTag, Credit, Drain, #stream_client{readers = Readers0, +credit(QName, CTag, Credit, Drain, #stream_client{readers = Readers0, name = Name, local_pid = LocalPid} = State) -> {Readers1, Msgs} = case Readers0 of #{CTag := #stream{credit = Credit0} = Str0} -> Str1 = Str0#stream{credit = Credit0 + Credit}, - {Str, Msgs0} = stream_entries(Name, LocalPid, Str1), + {Str, Msgs0} = stream_entries(QName, Name, LocalPid, Str1), {Readers0#{CTag => Str}, Msgs0}; _ -> {Readers0, []} @@ -407,12 +405,12 @@ deliver(_Confirm, #delivery{message = Msg, msg_seq_no = MsgId}, correlation = Correlation, slow = Slow}, Actions}. --spec dequeue(_, _, _, client()) -> no_return(). -dequeue(_, _, _, #stream_client{name = Name}) -> +-spec dequeue(_, _, _, _, client()) -> no_return(). +dequeue(_, _, _, _, #stream_client{name = Name}) -> {protocol_error, not_implemented, "basic.get not supported by stream queues ~ts", [rabbit_misc:rs(Name)]}. -handle_event({osiris_written, From, _WriterId, Corrs}, +handle_event(_QName, {osiris_written, From, _WriterId, Corrs}, State = #stream_client{correlation = Correlation0, soft_limit = SftLmt, slow = Slow0, @@ -431,7 +429,7 @@ handle_event({osiris_written, From, _WriterId, Corrs}, end, {ok, State#stream_client{correlation = Correlation, slow = Slow}, [{settled, From, MsgIds} | Actions]}; -handle_event({osiris_offset, _From, _Offs}, +handle_event(QName, {osiris_offset, _From, _Offs}, State = #stream_client{local_pid = LocalPid, readers = Readers0, name = Name}) -> @@ -439,19 +437,19 @@ handle_event({osiris_offset, _From, _Offs}, %% current committed {Readers, TagMsgs} = maps:fold( fun (Tag, Str0, {Acc, TM}) -> - {Str, Msgs} = stream_entries(Name, LocalPid, Str0), + {Str, Msgs} = stream_entries(QName, Name, LocalPid, Str0), {Acc#{Tag => Str}, [{Tag, LocalPid, Msgs} | TM]} end, {#{}, []}, Readers0), Ack = true, Deliveries = [{deliver, Tag, Ack, OffsetMsg} || {Tag, _LeaderPid, OffsetMsg} <- TagMsgs], {ok, State#stream_client{readers = Readers}, Deliveries}; -handle_event({stream_leader_change, Pid}, State) -> +handle_event(_QName, {stream_leader_change, Pid}, State) -> {ok, update_leader_pid(Pid, State), []}; -handle_event({stream_local_member_change, Pid}, #stream_client{local_pid = P} = State) +handle_event(_QName, {stream_local_member_change, Pid}, #stream_client{local_pid = P} = State) when P == Pid -> {ok, State, []}; -handle_event({stream_local_member_change, Pid}, State = #stream_client{name = QName, +handle_event(_QName, {stream_local_member_change, Pid}, State = #stream_client{name = QName, readers = Readers0}) -> rabbit_log:debug("Local member change event for ~tp", [QName]), Readers1 = maps:fold(fun(T, #stream{log = Log0} = S0, Acc) -> @@ -469,7 +467,7 @@ handle_event({stream_local_member_change, Pid}, State = #stream_client{name = QN end, #{}, Readers0), {ok, State#stream_client{local_pid = Pid, readers = Readers1}, []}; -handle_event(eol, #stream_client{name = Name}) -> +handle_event(_QName, eol, #stream_client{name = Name}) -> {eol, [{unblock, Name}]}. is_recoverable(Q) -> @@ -485,20 +483,20 @@ recover(_VHost, Queues) -> {[Q | R0], F0} end, {[], []}, Queues). -settle(complete, CTag, MsgIds, #stream_client{readers = Readers0, +settle(QName, complete, CTag, MsgIds, #stream_client{readers = Readers0, local_pid = LocalPid, name = Name} = State) -> Credit = length(MsgIds), {Readers, Msgs} = case Readers0 of #{CTag := #stream{credit = Credit0} = Str0} -> Str1 = Str0#stream{credit = Credit0 + Credit}, - {Str, Msgs0} = stream_entries(Name, LocalPid, Str1), + {Str, Msgs0} = stream_entries(QName, Name, LocalPid, Str1), {Readers0#{CTag => Str}, Msgs0}; _ -> {Readers0, []} end, {State#stream_client{readers = Readers}, [{deliver, CTag, true, Msgs}]}; -settle(_, _, _, #stream_client{name = Name}) -> +settle(_, _, _, _, #stream_client{name = Name}) -> {protocol_error, not_implemented, "basic.nack and basic.reject not supported by stream queues ~ts", [rabbit_misc:rs(Name)]}. @@ -772,10 +770,9 @@ init(Q) when ?is_amqqueue(Q) -> end. close(#stream_client{readers = Readers}) -> - _ = maps:map(fun (_, #stream{log = Log}) -> + maps:foreach(fun (_, #stream{log = Log}) -> osiris_log:close(Log) - end, Readers), - ok. + end, Readers). update(Q, State) when ?is_amqqueue(Q) -> @@ -941,12 +938,11 @@ check_queue_exists_in_local_node(Q) -> maybe_send_reply(_ChPid, undefined) -> ok; maybe_send_reply(ChPid, Msg) -> ok = rabbit_channel:send_command(ChPid, Msg). -stream_entries(Name, LocalPid, Str) -> - stream_entries(Name, LocalPid, Str, []). +stream_entries(QName, Name, LocalPid, Str) -> + stream_entries(QName, Name, LocalPid, Str, []). -stream_entries(Name, LocalPid, - #stream{name = QName, - credit = Credit, +stream_entries(QName, Name, LocalPid, + #stream{credit = Credit, start_offset = StartOffs, listening_offset = LOffs, log = Seg0} = Str0, MsgIn) @@ -985,10 +981,10 @@ stream_entries(Name, LocalPid, false -> %% if there are fewer Msgs than Entries0 it means there were non-events %% in the log and we should recurse and try again - stream_entries(Name, LocalPid, Str, MsgIn ++ Msgs) + stream_entries(QName, Name, LocalPid, Str, MsgIn ++ Msgs) end end; -stream_entries(_Name, _LocalPid, Str, Msgs) -> +stream_entries(_QName, _Name, _LocalPid, Str, Msgs) -> {Str, Msgs}. binary_to_msg(#resource{virtual_host = VHost, diff --git a/deps/rabbit/test/backing_queue_SUITE.erl b/deps/rabbit/test/backing_queue_SUITE.erl index 65d7c8ffc2cc..d0eb1216bb73 100644 --- a/deps/rabbit/test/backing_queue_SUITE.erl +++ b/deps/rabbit/test/backing_queue_SUITE.erl @@ -1617,10 +1617,6 @@ wait_for_confirms(Unconfirmed) -> false -> receive {'$gen_cast', {queue_event, _QName, {confirm, Confirmed, _}}} -> - wait_for_confirms( - sets:subtract( - Unconfirmed, sets:from_list(Confirmed))); - {'$gen_cast', {queue_event, QName, {confirm, Confirmed, _, QName}}} -> wait_for_confirms( sets:subtract( Unconfirmed, sets:from_list(Confirmed))) diff --git a/deps/rabbit/test/rabbit_fifo_int_SUITE.erl b/deps/rabbit/test/rabbit_fifo_int_SUITE.erl index 756d7fc2ace6..39f4d52b01c0 100644 --- a/deps/rabbit/test/rabbit_fifo_int_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_int_SUITE.erl @@ -89,7 +89,7 @@ basics(Config) -> UId = ?config(uid, Config), ConsumerTag = UId, ok = start_cluster(ClusterName, [ServerId]), - FState0 = rabbit_fifo_client:init(ClusterName, [ServerId]), + FState0 = rabbit_fifo_client:init([ServerId]), {ok, FState1} = rabbit_fifo_client:checkout(ConsumerTag, 1, simple_prefetch, #{}, FState0), @@ -97,19 +97,19 @@ basics(Config) -> % create segment the segment will trigger a snapshot timer:sleep(1000), - {ok, FState2, []} = rabbit_fifo_client:enqueue(one, FState1), + {ok, FState2, []} = rabbit_fifo_client:enqueue(ClusterName, one, FState1), DeliverFun = fun DeliverFun(S0, F) -> receive {ra_event, From, Evt} -> ct:pal("ra_event ~p", [Evt]), - case rabbit_fifo_client:handle_ra_event(From, Evt, S0) of + case rabbit_fifo_client:handle_ra_event(ClusterName, From, Evt, S0) of {ok, S1, [{deliver, C, true, [{_Qname, _QRef, MsgId, _SomBool, _Msg}]}]} -> {S, _A} = rabbit_fifo_client:F(C, [MsgId], S1), %% settle applied event - process_ra_event(S, ?RA_EVENT_TIMEOUT); + process_ra_event(ClusterName, S, ?RA_EVENT_TIMEOUT); {ok, S, _} -> DeliverFun(S, F) end @@ -129,13 +129,13 @@ basics(Config) -> receive {ra_event, From, Evt} -> ct:pal("ra_event ~p", [Evt]), - {ok, F6, _} = rabbit_fifo_client:handle_ra_event(From, Evt, FState5), + {ok, F6, _} = rabbit_fifo_client:handle_ra_event(ClusterName, From, Evt, FState5), F6 after 5000 -> exit(leader_change_timeout) end, - {ok, FState6, []} = rabbit_fifo_client:enqueue(two, FState5b), + {ok, FState6, []} = rabbit_fifo_client:enqueue(ClusterName, two, FState5b), _FState8 = DeliverFun(FState6, return), rabbit_quorum_queue:stop_server(ServerId), @@ -146,11 +146,11 @@ return(Config) -> ServerId = ?config(node_id, Config), ok = start_cluster(ClusterName, [ServerId]), - F00 = rabbit_fifo_client:init(ClusterName, [ServerId]), - {ok, F0, []} = rabbit_fifo_client:enqueue(1, msg1, F00), - {ok, F1, []} = rabbit_fifo_client:enqueue(2, msg2, F0), - {_, _, F2} = process_ra_events(receive_ra_events(2, 0), F1), - {ok, _, {_, _, MsgId, _, _}, F} = rabbit_fifo_client:dequeue(<<"tag">>, unsettled, F2), + F00 = rabbit_fifo_client:init([ServerId]), + {ok, F0, []} = rabbit_fifo_client:enqueue(ClusterName, 1, msg1, F00), + {ok, F1, []} = rabbit_fifo_client:enqueue(ClusterName, 2, msg2, F0), + {_, _, F2} = process_ra_events(receive_ra_events(2, 0), ClusterName, F1), + {ok, _, {_, _, MsgId, _, _}, F} = rabbit_fifo_client:dequeue(ClusterName, <<"tag">>, unsettled, F2), _F2 = rabbit_fifo_client:return(<<"tag">>, [MsgId], F), rabbit_quorum_queue:stop_server(ServerId), @@ -160,11 +160,11 @@ rabbit_fifo_returns_correlation(Config) -> ClusterName = ?config(cluster_name, Config), ServerId = ?config(node_id, Config), ok = start_cluster(ClusterName, [ServerId]), - F0 = rabbit_fifo_client:init(ClusterName, [ServerId]), - {ok, F1, []} = rabbit_fifo_client:enqueue(corr1, msg1, F0), + F0 = rabbit_fifo_client:init([ServerId]), + {ok, F1, []} = rabbit_fifo_client:enqueue(ClusterName, corr1, msg1, F0), receive {ra_event, Frm, E} -> - case rabbit_fifo_client:handle_ra_event(Frm, E, F1) of + case rabbit_fifo_client:handle_ra_event(ClusterName, Frm, E, F1) of {ok, _F2, [{settled, _, _}]} -> ok; Del -> @@ -180,13 +180,13 @@ duplicate_delivery(Config) -> ClusterName = ?config(cluster_name, Config), ServerId = ?config(node_id, Config), ok = start_cluster(ClusterName, [ServerId]), - F0 = rabbit_fifo_client:init(ClusterName, [ServerId]), + F0 = rabbit_fifo_client:init([ServerId]), {ok, F1} = rabbit_fifo_client:checkout(<<"tag">>, 10, simple_prefetch, #{}, F0), - {ok, F2, []} = rabbit_fifo_client:enqueue(corr1, msg1, F1), + {ok, F2, []} = rabbit_fifo_client:enqueue(ClusterName, corr1, msg1, F1), Fun = fun Loop(S0) -> receive {ra_event, Frm, E} = Evt -> - case rabbit_fifo_client:handle_ra_event(Frm, E, S0) of + case rabbit_fifo_client:handle_ra_event(ClusterName, Frm, E, S0) of {ok, S1, [{settled, _, _}]} -> Loop(S1); {ok, S1, _} -> @@ -196,7 +196,7 @@ duplicate_delivery(Config) -> %% repeat or crash receive {ra_event, F, E1} -> - case rabbit_fifo_client:handle_ra_event( + case rabbit_fifo_client:handle_ra_event(ClusterName, F, E1, S1) of {ok, S2, _} -> S2 @@ -215,11 +215,11 @@ usage(Config) -> ClusterName = ?config(cluster_name, Config), ServerId = ?config(node_id, Config), ok = start_cluster(ClusterName, [ServerId]), - F0 = rabbit_fifo_client:init(ClusterName, [ServerId]), + F0 = rabbit_fifo_client:init([ServerId]), {ok, F1} = rabbit_fifo_client:checkout(<<"tag">>, 10, simple_prefetch, #{}, F0), - {ok, F2, []} = rabbit_fifo_client:enqueue(corr1, msg1, F1), - {ok, F3, []} = rabbit_fifo_client:enqueue(corr2, msg2, F2), - {_, _, _} = process_ra_events(receive_ra_events(2, 2), F3), + {ok, F2, []} = rabbit_fifo_client:enqueue(ClusterName, corr1, msg1, F1), + {ok, F3, []} = rabbit_fifo_client:enqueue(ClusterName, corr2, msg2, F2), + {_, _, _} = process_ra_events(receive_ra_events(2, 2), ClusterName, F3), % force tick and usage stats emission ServerId ! tick_timeout, timer:sleep(50), @@ -235,17 +235,17 @@ resends_lost_command(Config) -> ok = meck:new(ra, [passthrough]), - F0 = rabbit_fifo_client:init(ClusterName, [ServerId]), - {ok, F1, []} = rabbit_fifo_client:enqueue(msg1, F0), + F0 = rabbit_fifo_client:init([ServerId]), + {ok, F1, []} = rabbit_fifo_client:enqueue(ClusterName, msg1, F0), % lose the enqueue meck:expect(ra, pipeline_command, fun (_, _, _) -> ok end), - {ok, F2, []} = rabbit_fifo_client:enqueue(msg2, F1), + {ok, F2, []} = rabbit_fifo_client:enqueue(ClusterName, msg2, F1), meck:unload(ra), - {ok, F3, []} = rabbit_fifo_client:enqueue(msg3, F2), - {_, _, F4} = process_ra_events(receive_ra_events(2, 0), F3), - {ok, _, {_, _, _, _, msg1}, F5} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F4), - {ok, _, {_, _, _, _, msg2}, F6} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F5), - {ok, _, {_, _, _, _, msg3}, _F7} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F6), + {ok, F3, []} = rabbit_fifo_client:enqueue(ClusterName, msg3, F2), + {_, _, F4} = process_ra_events(receive_ra_events(2, 0), ClusterName, F3), + {ok, _, {_, _, _, _, msg1}, F5} = rabbit_fifo_client:dequeue(ClusterName, <<"tag">>, settled, F4), + {ok, _, {_, _, _, _, msg2}, F6} = rabbit_fifo_client:dequeue(ClusterName, <<"tag">>, settled, F5), + {ok, _, {_, _, _, _, msg3}, _F7} = rabbit_fifo_client:dequeue(ClusterName, <<"tag">>, settled, F6), rabbit_quorum_queue:stop_server(ServerId), ok. @@ -254,10 +254,10 @@ two_quick_enqueues(Config) -> ServerId = ?config(node_id, Config), ok = start_cluster(ClusterName, [ServerId]), - F0 = rabbit_fifo_client:init(ClusterName, [ServerId]), - F1 = element(2, rabbit_fifo_client:enqueue(msg1, F0)), - {ok, F2, []} = rabbit_fifo_client:enqueue(msg2, F1), - _ = process_ra_events(receive_ra_events(2, 0), F2), + F0 = rabbit_fifo_client:init([ServerId]), + F1 = element(2, rabbit_fifo_client:enqueue(ClusterName, msg1, F0)), + {ok, F2, []} = rabbit_fifo_client:enqueue(ClusterName, msg2, F1), + _ = process_ra_events(receive_ra_events(2, 0), ClusterName, F2), rabbit_quorum_queue:stop_server(ServerId), ok. @@ -266,12 +266,12 @@ detects_lost_delivery(Config) -> ServerId = ?config(node_id, Config), ok = start_cluster(ClusterName, [ServerId]), - F000 = rabbit_fifo_client:init(ClusterName, [ServerId]), - {ok, F00, []} = rabbit_fifo_client:enqueue(msg1, F000), - {_, _, F0} = process_ra_events(receive_ra_events(1, 0), F00), + F000 = rabbit_fifo_client:init([ServerId]), + {ok, F00, []} = rabbit_fifo_client:enqueue(ClusterName, msg1, F000), + {_, _, F0} = process_ra_events(receive_ra_events(1, 0), ClusterName, F00), {ok, F1} = rabbit_fifo_client:checkout(<<"tag">>, 10, simple_prefetch, #{}, F0), - {ok, F2, []} = rabbit_fifo_client:enqueue(msg2, F1), - {ok, F3, []} = rabbit_fifo_client:enqueue(msg3, F2), + {ok, F2, []} = rabbit_fifo_client:enqueue(ClusterName, msg2, F1), + {ok, F3, []} = rabbit_fifo_client:enqueue(ClusterName, msg3, F2), % lose first delivery receive {ra_event, _, {machine, {delivery, _, [{_, {_, msg1}}]}}} -> @@ -281,7 +281,7 @@ detects_lost_delivery(Config) -> end, % assert three deliveries were received - {[_, _, _], _, _} = process_ra_events(receive_ra_events(2, 2), F3), + {[_, _, _], _, _} = process_ra_events(receive_ra_events(2, 2), ClusterName, F3), rabbit_quorum_queue:stop_server(ServerId), ok. @@ -290,14 +290,14 @@ returns_after_down(Config) -> ServerId = ?config(node_id, Config), ok = start_cluster(ClusterName, [ServerId]), - F0 = rabbit_fifo_client:init(ClusterName, [ServerId]), - {ok, F1, []} = rabbit_fifo_client:enqueue(msg1, F0), - {_, _, F2} = process_ra_events(receive_ra_events(1, 0), F1), + F0 = rabbit_fifo_client:init([ServerId]), + {ok, F1, []} = rabbit_fifo_client:enqueue(ClusterName, msg1, F0), + {_, _, F2} = process_ra_events(receive_ra_events(1, 0), ClusterName, F1), % start a consumer in a separate processes % that exits after checkout Self = self(), _Pid = spawn(fun () -> - F = rabbit_fifo_client:init(ClusterName, [ServerId]), + F = rabbit_fifo_client:init([ServerId]), {ok, _} = rabbit_fifo_client:checkout(<<"tag">>, 10, simple_prefetch, #{}, F), @@ -306,7 +306,7 @@ returns_after_down(Config) -> receive checkout_done -> ok after 1000 -> exit(checkout_done_timeout) end, timer:sleep(1000), % message should be available for dequeue - {ok, _, {_, _, _, _, msg1}, _} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F2), + {ok, _, {_, _, _, _, msg1}, _} = rabbit_fifo_client:dequeue(ClusterName, <<"tag">>, settled, F2), rabbit_quorum_queue:stop_server(ServerId), ok. @@ -315,10 +315,10 @@ resends_after_lost_applied(Config) -> ServerId = ?config(node_id, Config), ok = start_cluster(ClusterName, [ServerId]), - F0 = rabbit_fifo_client:init(ClusterName, [ServerId]), - {ok, F1, []} = rabbit_fifo_client:enqueue(msg1, F0), - {_, _, F2} = process_ra_events(receive_ra_events(1, 0), F1), - {ok, F3, []} = rabbit_fifo_client:enqueue(msg2, F2), + F0 = rabbit_fifo_client:init([ServerId]), + {ok, F1, []} = rabbit_fifo_client:enqueue(ClusterName, msg1, F0), + {_, _, F2} = process_ra_events(receive_ra_events(1, 0), ClusterName, F1), + {ok, F3, []} = rabbit_fifo_client:enqueue(ClusterName, msg2, F2), % lose an applied event receive {ra_event, _, {applied, _}} -> @@ -327,11 +327,11 @@ resends_after_lost_applied(Config) -> exit(await_ra_event_timeout) end, % send another message - {ok, F4, []} = rabbit_fifo_client:enqueue(msg3, F3), - {_, _, F5} = process_ra_events(receive_ra_events(1, 0), F4), - {ok, _, {_, _, _, _, msg1}, F6} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F5), - {ok, _, {_, _, _, _, msg2}, F7} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F6), - {ok, _, {_, _, _, _, msg3}, _F8} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F7), + {ok, F4, []} = rabbit_fifo_client:enqueue(ClusterName, msg3, F3), + {_, _, F5} = process_ra_events(receive_ra_events(1, 0), ClusterName, F4), + {ok, _, {_, _, _, _, msg1}, F6} = rabbit_fifo_client:dequeue(ClusterName, <<"tag">>, settled, F5), + {ok, _, {_, _, _, _, msg2}, F7} = rabbit_fifo_client:dequeue(ClusterName, <<"tag">>, settled, F6), + {ok, _, {_, _, _, _, msg3}, _F8} = rabbit_fifo_client:dequeue(ClusterName, <<"tag">>, settled, F7), rabbit_quorum_queue:stop_server(ServerId), ok. @@ -349,13 +349,13 @@ handles_reject_notification(Config) -> {auto, 10, simple_prefetch}, #{})), % reverse order - should try the first node in the list first - F0 = rabbit_fifo_client:init(ClusterName, [ServerId2, ServerId1]), - {ok, F1, []} = rabbit_fifo_client:enqueue(one, F0), + F0 = rabbit_fifo_client:init([ServerId2, ServerId1]), + {ok, F1, []} = rabbit_fifo_client:enqueue(ClusterName, one, F0), timer:sleep(500), % the applied notification - _F2 = process_ra_events(receive_ra_events(1, 0), F1), + _F2 = process_ra_events(receive_ra_events(1, 0), ClusterName, F1), rabbit_quorum_queue:stop_server(ServerId1), rabbit_quorum_queue:stop_server(ServerId2), ok. @@ -378,12 +378,12 @@ discard(Config) -> ok = ra:trigger_election(ServerId), _ = ra:members(ServerId), - F0 = rabbit_fifo_client:init(ClusterName, [ServerId]), + F0 = rabbit_fifo_client:init([ServerId]), {ok, F1} = rabbit_fifo_client:checkout(<<"tag">>, 10, simple_prefetch, #{}, F0), - {ok, F2, []} = rabbit_fifo_client:enqueue(msg1, F1), - F3 = discard_next_delivery(F2, 5000), - {empty, _F4} = rabbit_fifo_client:dequeue(<<"tag1">>, settled, F3), + {ok, F2, []} = rabbit_fifo_client:enqueue(ClusterName, msg1, F1), + F3 = discard_next_delivery(ClusterName, F2, 5000), + {empty, _F4} = rabbit_fifo_client:dequeue(ClusterName, <<"tag1">>, settled, F3), receive {dead_letter, Reason, Letters} -> [msg1] = Letters, @@ -400,23 +400,23 @@ cancel_checkout(Config) -> ClusterName = ?config(cluster_name, Config), ServerId = ?config(node_id, Config), ok = start_cluster(ClusterName, [ServerId]), - F0 = rabbit_fifo_client:init(ClusterName, [ServerId], 4), - {ok, F1, []} = rabbit_fifo_client:enqueue(m1, F0), + F0 = rabbit_fifo_client:init([ServerId], 4), + {ok, F1, []} = rabbit_fifo_client:enqueue(ClusterName, m1, F0), {ok, F2} = rabbit_fifo_client:checkout(<<"tag">>, 10, simple_prefetch, #{}, F1), - {_, _, F3} = process_ra_events(receive_ra_events(1, 1), F2, [], [], fun (_, S) -> S end), + {_, _, F3} = process_ra_events(receive_ra_events(1, 1), ClusterName, F2, [], [], fun (_, S) -> S end), {ok, F4} = rabbit_fifo_client:cancel_checkout(<<"tag">>, F3), {F5, _} = rabbit_fifo_client:return(<<"tag">>, [0], F4), - {ok, _, {_, _, _, _, m1}, F5} = rabbit_fifo_client:dequeue(<<"d1">>, settled, F5), + {ok, _, {_, _, _, _, m1}, F5} = rabbit_fifo_client:dequeue(ClusterName, <<"d1">>, settled, F5), ok. lost_delivery(Config) -> ClusterName = ?config(cluster_name, Config), ServerId = ?config(node_id, Config), ok = start_cluster(ClusterName, [ServerId]), - F0 = rabbit_fifo_client:init(ClusterName, [ServerId], 4), - {ok, F1, []} = rabbit_fifo_client:enqueue(m1, F0), + F0 = rabbit_fifo_client:init([ServerId], 4), + {ok, F1, []} = rabbit_fifo_client:enqueue(ClusterName, m1, F0), {_, _, F2} = process_ra_events( - receive_ra_events(1, 0), F1, [], [], fun (_, S) -> S end), + receive_ra_events(1, 0), ClusterName, F1, [], [], fun (_, S) -> S end), {ok, F3} = rabbit_fifo_client:checkout(<<"tag">>, 10, simple_prefetch, #{}, F2), %% drop a delivery, simulating e.g. a full distribution buffer receive @@ -427,11 +427,11 @@ lost_delivery(Config) -> exit(await_ra_event_timeout) end, % send another message - {ok, F4, []} = rabbit_fifo_client:enqueue(m2, F3), + {ok, F4, []} = rabbit_fifo_client:enqueue(ClusterName, m2, F3), %% this hsould trigger the fifo client to fetch any missing messages %% from the server {_, _, _F5} = process_ra_events( - receive_ra_events(1, 1), F4, [], [], + receive_ra_events(1, 1), ClusterName, F4, [], [], fun ({deliver, _, _, Dels}, S) -> [{_, _, _, _, M1}, {_, _, _, _, M2}] = Dels, @@ -445,39 +445,39 @@ credit(Config) -> ClusterName = ?config(cluster_name, Config), ServerId = ?config(node_id, Config), ok = start_cluster(ClusterName, [ServerId]), - F0 = rabbit_fifo_client:init(ClusterName, [ServerId], 4), - {ok, F1, []} = rabbit_fifo_client:enqueue(m1, F0), - {ok, F2, []} = rabbit_fifo_client:enqueue(m2, F1), - {_, _, F3} = process_ra_events(receive_ra_events(2, 0), F2), + F0 = rabbit_fifo_client:init([ServerId], 4), + {ok, F1, []} = rabbit_fifo_client:enqueue(ClusterName, m1, F0), + {ok, F2, []} = rabbit_fifo_client:enqueue(ClusterName, m2, F1), + {_, _, F3} = process_ra_events(receive_ra_events(2, 0), ClusterName, F2), %% checkout with 0 prefetch {ok, F4} = rabbit_fifo_client:checkout(<<"tag">>, 0, credited, #{}, F3), %% assert no deliveries - {_, _, F5} = process_ra_events(receive_ra_events(), F4, [], [], + {_, _, F5} = process_ra_events(receive_ra_events(), ClusterName, F4, [], [], fun (D, _) -> error({unexpected_delivery, D}) end), %% provide some credit {F6, []} = rabbit_fifo_client:credit(<<"tag">>, 1, false, F5), {[{_, _, _, _, m1}], [{send_credit_reply, _}], F7} = - process_ra_events(receive_ra_events(1, 1), F6), + process_ra_events(receive_ra_events(1, 1), ClusterName, F6), %% credit and drain {F8, []} = rabbit_fifo_client:credit(<<"tag">>, 4, true, F7), {[{_, _, _, _, m2}], [{send_credit_reply, _}, {send_drained, _}], F9} = - process_ra_events(receive_ra_events(2, 1), F8), + process_ra_events(receive_ra_events(2, 1), ClusterName, F8), flush(), %% enqueue another message - at this point the consumer credit should be %% all used up due to the drain - {ok, F10, []} = rabbit_fifo_client:enqueue(m3, F9), + {ok, F10, []} = rabbit_fifo_client:enqueue(ClusterName, m3, F9), %% assert no deliveries - {_, _, F11} = process_ra_events(receive_ra_events(), F10, [], [], + {_, _, F11} = process_ra_events(receive_ra_events(), ClusterName, F10, [], [], fun (D, _) -> error({unexpected_delivery, D}) end), %% credit again and receive the last message {F12, []} = rabbit_fifo_client:credit(<<"tag">>, 10, false, F11), - {[{_, _, _, _, m3}], _, _} = process_ra_events(receive_ra_events(1, 1), F12), + {[{_, _, _, _, m3}], _, _} = process_ra_events(receive_ra_events(1, 1), ClusterName, F12), ok. untracked_enqueue(Config) -> @@ -487,8 +487,8 @@ untracked_enqueue(Config) -> ok = rabbit_fifo_client:untracked_enqueue([ServerId], msg1), timer:sleep(100), - F0 = rabbit_fifo_client:init(ClusterName, [ServerId]), - {ok, _, {_, _, _, _, msg1}, _F5} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F0), + F0 = rabbit_fifo_client:init([ServerId]), + {ok, _, {_, _, _, _, msg1}, _F5} = rabbit_fifo_client:dequeue(ClusterName, <<"tag">>, settled, F0), rabbit_quorum_queue:stop_server(ServerId), ok. @@ -497,14 +497,14 @@ flow(Config) -> ClusterName = ?config(cluster_name, Config), {Name, _Node} = ServerId = ?config(node_id, Config), ok = start_cluster(ClusterName, [ServerId]), - F0 = rabbit_fifo_client:init(ClusterName, [ServerId], 3), - {ok, F1, []} = rabbit_fifo_client:enqueue(m1, F0), - {ok, F2, []} = rabbit_fifo_client:enqueue(m2, F1), - {ok, F3, []} = rabbit_fifo_client:enqueue(m3, F2), - {ok, F4, [{block, Name}]} = rabbit_fifo_client:enqueue(m4, F3), - {_, Actions, F5} = process_ra_events(receive_ra_events(4, 0), F4), + F0 = rabbit_fifo_client:init([ServerId], 3), + {ok, F1, []} = rabbit_fifo_client:enqueue(ClusterName, m1, F0), + {ok, F2, []} = rabbit_fifo_client:enqueue(ClusterName, m2, F1), + {ok, F3, []} = rabbit_fifo_client:enqueue(ClusterName, m3, F2), + {ok, F4, [{block, Name}]} = rabbit_fifo_client:enqueue(ClusterName, m4, F3), + {_, Actions, F5} = process_ra_events(receive_ra_events(4, 0), ClusterName, F4), true = lists:member({unblock, Name}, Actions), - {ok, _, []} = rabbit_fifo_client:enqueue(m5, F5), + {ok, _, []} = rabbit_fifo_client:enqueue(ClusterName, m5, F5), rabbit_quorum_queue:stop_server(ServerId), ok. @@ -515,10 +515,10 @@ test_queries(Config) -> ok = start_cluster(ClusterName, [ServerId]), Self = self(), P = spawn(fun () -> - F0 = rabbit_fifo_client:init(ClusterName, [ServerId], 4), - {ok, F1, []} = rabbit_fifo_client:enqueue(m1, F0), - {ok, F2, []} = rabbit_fifo_client:enqueue(m2, F1), - process_ra_events(receive_ra_events(2, 0), F2), + F0 = rabbit_fifo_client:init([ServerId], 4), + {ok, F1, []} = rabbit_fifo_client:enqueue(ClusterName, m1, F0), + {ok, F2, []} = rabbit_fifo_client:enqueue(ClusterName, m2, F1), + process_ra_events(receive_ra_events(2, 0), ClusterName, F2), Self ! ready, receive stop -> ok end end), @@ -527,7 +527,7 @@ test_queries(Config) -> after 5000 -> exit(ready_timeout) end, - F0 = rabbit_fifo_client:init(ClusterName, [ServerId], 4), + F0 = rabbit_fifo_client:init([ServerId], 4), {ok, _} = rabbit_fifo_client:checkout(<<"tag">>, 1, simple_prefetch, #{}, F0), {ok, {_, Ready}, _} = ra:local_query(ServerId, fun rabbit_fifo:query_messages_ready/1), @@ -551,16 +551,16 @@ dequeue(Config) -> UId = ?config(uid, Config), Tag = UId, ok = start_cluster(ClusterName, [ServerId]), - F1 = rabbit_fifo_client:init(ClusterName, [ServerId]), - {empty, F1b} = rabbit_fifo_client:dequeue(Tag, settled, F1), - {ok, F2_, []} = rabbit_fifo_client:enqueue(msg1, F1b), - {_, _, F2} = process_ra_events(receive_ra_events(1, 0), F2_), - - % {ok, {{0, {_, msg1}}, _}, F3} = rabbit_fifo_client:dequeue(Tag, settled, F2), - {ok, _, {_, _, 0, _, msg1}, F3} = rabbit_fifo_client:dequeue(Tag, settled, F2), - {ok, F4_, []} = rabbit_fifo_client:enqueue(msg2, F3), - {_, _, F4} = process_ra_events(receive_ra_events(1, 0), F4_), - {ok, _, {_, _, MsgId, _, msg2}, F5} = rabbit_fifo_client:dequeue(Tag, unsettled, F4), + F1 = rabbit_fifo_client:init([ServerId]), + {empty, F1b} = rabbit_fifo_client:dequeue(ClusterName, Tag, settled, F1), + {ok, F2_, []} = rabbit_fifo_client:enqueue(ClusterName, msg1, F1b), + {_, _, F2} = process_ra_events(receive_ra_events(1, 0), ClusterName, F2_), + + % {ok, {{0, {_, msg1}}, _}, F3} = rabbit_fifo_client:dequeue(ClusterName, Tag, settled, F2), + {ok, _, {_, _, 0, _, msg1}, F3} = rabbit_fifo_client:dequeue(ClusterName, Tag, settled, F2), + {ok, F4_, []} = rabbit_fifo_client:enqueue(ClusterName, msg2, F3), + {_, _, F4} = process_ra_events(receive_ra_events(1, 0), ClusterName, F4_), + {ok, _, {_, _, MsgId, _, msg2}, F5} = rabbit_fifo_client:dequeue(ClusterName, Tag, unsettled, F4), {_F6, _A} = rabbit_fifo_client:settle(Tag, [MsgId], F5), rabbit_quorum_queue:stop_server(ServerId), ok. @@ -573,12 +573,12 @@ conf(ClusterName, UId, ServerId, _, Peers) -> initial_members => Peers, machine => {module, rabbit_fifo, #{}}}. -process_ra_event(State, Wait) -> +process_ra_event(ClusterName, State, Wait) -> receive {ra_event, From, Evt} -> ct:pal("Ra_event ~p", [Evt]), {ok, S, _Actions} = - rabbit_fifo_client:handle_ra_event(From, Evt, State), + rabbit_fifo_client:handle_ra_event(ClusterName, From, Evt, State), S after Wait -> flush(), @@ -616,19 +616,19 @@ receive_ra_events(Acc) -> Acc end. -process_ra_events(Events, State) -> +process_ra_events(Events, ClusterName, State) -> DeliveryFun = fun ({deliver, _, Tag, Msgs}, S) -> MsgIds = [element(1, M) || M <- Msgs], {S0, _} = rabbit_fifo_client:settle(Tag, MsgIds, S), S0 end, - process_ra_events(Events, State, [], [], DeliveryFun). + process_ra_events(Events, ClusterName, State, [], [], DeliveryFun). -process_ra_events([], State0, Acc, Actions0, _DeliveryFun) -> +process_ra_events([], _ClusterName, State0, Acc, Actions0, _DeliveryFun) -> {Acc, Actions0, State0}; -process_ra_events([{ra_event, From, Evt} | Events], State0, Acc, +process_ra_events([{ra_event, From, Evt} | Events], ClusterName, State0, Acc, Actions0, DeliveryFun) -> - case rabbit_fifo_client:handle_ra_event(From, Evt, State0) of + case rabbit_fifo_client:handle_ra_event(ClusterName, From, Evt, State0) of {ok, State1, Actions1} -> {Msgs, Actions, State} = lists:foldl( @@ -637,15 +637,15 @@ process_ra_events([{ra_event, From, Evt} | Events], State0, Acc, (Ac, {M, A, S}) -> {M, A ++ [Ac], S} end, {Acc, [], State1}, Actions1), - process_ra_events(Events, State, Msgs, Actions0 ++ Actions, DeliveryFun); + process_ra_events(Events, ClusterName, State, Msgs, Actions0 ++ Actions, DeliveryFun); eol -> eol end. -discard_next_delivery(State0, Wait) -> +discard_next_delivery(ClusterName, State0, Wait) -> receive {ra_event, _, {machine, {delivery, _, _}}} = Evt -> - element(3, process_ra_events([Evt], State0, [], [], + element(3, process_ra_events([Evt], ClusterName, State0, [], [], fun ({deliver, Tag, _, Msgs}, S) -> MsgIds = [element(3, M) || M <- Msgs], {S0, _} = rabbit_fifo_client:discard(Tag, MsgIds, S), @@ -655,30 +655,6 @@ discard_next_delivery(State0, Wait) -> State0 end. -return_next_delivery(State0, Wait) -> - receive - {ra_event, _, {machine, {delivery, _, _}}} = Evt -> - element(3, process_ra_events([Evt], State0, [], [], - fun ({deliver, Tag, _, Msgs}, S) -> - MsgIds = [element(3, M) || M <- Msgs], - {S0, _} = rabbit_fifo_client:return(Tag, MsgIds, S), - S0 - end)) - after Wait -> - State0 - end. - -validate_process_down(Name, 0) -> - exit({process_not_down, Name}); -validate_process_down(Name, Num) -> - case whereis(Name) of - undefined -> - ok; - _ -> - timer:sleep(100), - validate_process_down(Name, Num-1) - end. - start_cluster(ClusterName, ServerIds, RaFifoConfig) -> {ok, Started, _} = ra:start_cluster(?RA_SYSTEM, ClusterName#resource.name, diff --git a/deps/rabbitmq_management/src/rabbit_mgmt_wm_connection.erl b/deps/rabbitmq_management/src/rabbit_mgmt_wm_connection.erl index 820506400128..11d8fc02018a 100644 --- a/deps/rabbitmq_management/src/rabbit_mgmt_wm_connection.erl +++ b/deps/rabbitmq_management/src/rabbit_mgmt_wm_connection.erl @@ -98,6 +98,6 @@ force_close_connection(ReqData, Conn, Pid) -> _ -> %% Best effort will work for following plugins: %% rabbitmq_stream, rabbitmq_mqtt, rabbitmq_web_mqtt - Pid ! {shutdown, Reason} - end, - ok. + _ = Pid ! {shutdown, Reason}, + ok + end. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl index 866b4ae20ccd..deb0eff622de 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl @@ -48,10 +48,10 @@ update/2, consume/3, cancel/5, - handle_event/2, - settle/4, - credit/4, - dequeue/4, + handle_event/3, + settle/5, + credit/5, + dequeue/5, state_info/1 ]). -export(?STATEFUL_CALLBACKS). @@ -258,17 +258,17 @@ consume(A1,A2,A3) -> cancel(A1,A2,A3,A4,A5) -> ?UNSUPPORTED([A1,A2,A3,A4,A5]). -handle_event(A1,A2) -> - ?UNSUPPORTED([A1,A2]). +handle_event(A1,A2,A3) -> + ?UNSUPPORTED([A1,A2,A3]). -settle(A1,A2,A3,A4) -> - ?UNSUPPORTED([A1,A2,A3,A4]). +settle(A1,A2,A3,A4,A5) -> + ?UNSUPPORTED([A1,A2,A3,A4,A5]). -credit(A1,A2,A3,A4) -> - ?UNSUPPORTED([A1,A2,A3,A4]). +credit(A1,A2,A3,A4,A5) -> + ?UNSUPPORTED([A1,A2,A3,A4,A5]). -dequeue(A1,A2,A3,A4) -> - ?UNSUPPORTED([A1,A2,A3,A4]). +dequeue(A1,A2,A3,A4,A5) -> + ?UNSUPPORTED([A1,A2,A3,A4,A5]). state_info(A1) -> ?UNSUPPORTED([A1]). From 9db8626abf536859a6f4801b4ed70584faf952e7 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Mon, 23 Jan 2023 13:10:54 +0000 Subject: [PATCH 113/118] Re-enable dialyzer option Wunmatched_returns --- deps/rabbitmq_mqtt/BUILD.bazel | 6 +++--- ....CLI.Ctl.Commands.ListMqttConnectionsCommand.erl | 3 ++- deps/rabbitmq_mqtt/src/mqtt_node.erl | 13 ++++++------- deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl | 2 +- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 4 ++-- .../src/rabbit_mqtt_retained_msg_store_ets.erl | 4 ++-- deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl | 4 ++-- deps/rabbitmq_web_mqtt/BUILD.bazel | 4 ++-- deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_app.erl | 2 +- .../src/rabbit_web_mqtt_handler.erl | 4 ++-- 10 files changed, 23 insertions(+), 23 deletions(-) diff --git a/deps/rabbitmq_mqtt/BUILD.bazel b/deps/rabbitmq_mqtt/BUILD.bazel index 17b76482f728..be73e3e9a862 100644 --- a/deps/rabbitmq_mqtt/BUILD.bazel +++ b/deps/rabbitmq_mqtt/BUILD.bazel @@ -2,6 +2,7 @@ load("@rules_erlang//:xref2.bzl", "xref") load("@rules_erlang//:dialyze.bzl", "dialyze", "plt") load( "//:rabbitmq.bzl", + "RABBITMQ_DIALYZER_OPTS", "BROKER_VERSION_REQUIREMENTS_ANY", "assert_suites", "broker_for_integration_suites", @@ -73,13 +74,12 @@ xref( plt( name = "base_plt", libs = ["//deps/rabbitmq_cli:elixir"], - deps = ["//deps/rabbitmq_cli:elixir"] + BUILD_DEPS + DEPS + RUNTIME_DEPS, + deps = ["//deps/rabbitmq_cli:elixir"] + BUILD_DEPS + DEPS, ) dialyze( - dialyzer_opts = ["-Werror_handling", "-Wno_undefined_callbacks"], + dialyzer_opts = RABBITMQ_DIALYZER_OPTS, plt = ":base_plt", - warnings_as_errors = True, ) broker_for_integration_suites( diff --git a/deps/rabbitmq_mqtt/src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListMqttConnectionsCommand.erl b/deps/rabbitmq_mqtt/src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListMqttConnectionsCommand.erl index f5534f37b428..07265b56e109 100644 --- a/deps/rabbitmq_mqtt/src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListMqttConnectionsCommand.erl +++ b/deps/rabbitmq_mqtt/src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListMqttConnectionsCommand.erl @@ -36,8 +36,9 @@ help_section() -> {plugin, mqtt}. validate(Args, _) -> + InfoItems = lists:map(fun atom_to_list/1, ?INFO_ITEMS), case 'Elixir.RabbitMQ.CLI.Ctl.InfoKeys':validate_info_keys(Args, - ?INFO_ITEMS) of + InfoItems) of {ok, _} -> ok; Error -> Error end. diff --git a/deps/rabbitmq_mqtt/src/mqtt_node.erl b/deps/rabbitmq_mqtt/src/mqtt_node.erl index e8baa149b642..a6442fa85b90 100644 --- a/deps/rabbitmq_mqtt/src/mqtt_node.erl +++ b/deps/rabbitmq_mqtt/src/mqtt_node.erl @@ -34,7 +34,7 @@ start() -> start(300, Repetitions). start(_Delay, AttemptsLeft) when AttemptsLeft =< 0 -> - start_server(), + ok = start_server(), trigger_election(); start(Delay, AttemptsLeft) -> NodeId = server_id(), @@ -60,16 +60,15 @@ start(Delay, AttemptsLeft) -> %% This is required when we start a node for the first time. %% Using default timeout because it supposed to reply fast. rabbit_log:info("MQTT: discovered ~tp cluster peers that support client ID tracking", [length(Peers)]), - start_server(), - join_peers(NodeId, Peers), + ok = start_server(), + _ = join_peers(NodeId, Peers), ra:trigger_election(NodeId, ?RA_OPERATION_TIMEOUT) end; _ -> - join_peers(NodeId, Nodes), - ra:restart_server(?RA_SYSTEM, NodeId), + _ = join_peers(NodeId, Nodes), + ok = ra:restart_server(?RA_SYSTEM, NodeId), ra:trigger_election(NodeId, ?RA_OPERATION_TIMEOUT) - end, - ok. + end. compatible_peer_servers() -> all_node_ids() -- [(node_id())]. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 273134bf920d..bfe633e89e00 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -1333,7 +1333,7 @@ maybe_send_will(true, ConnStr, [Topic, ConnStr]), case check_topic_access(Topic, write, State) of ok -> - publish_to_queues(WillMsg, State), + _ = publish_to_queues(WillMsg, State), case Retain of false -> ok; diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index b1cd16890bbd..4eaee6607903 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -80,7 +80,7 @@ init(Ref) -> {ok, ConnStr} -> ConnName = rabbit_data_coercion:to_binary(ConnStr), ?LOG_DEBUG("MQTT accepting TCP connection ~tp (~ts)", [self(), ConnName]), - rabbit_alarm:register(self(), {?MODULE, conserve_resources, []}), + _ = rabbit_alarm:register(self(), {?MODULE, conserve_resources, []}), LoginTimeout = application:get_env(?APP_NAME, login_timeout, 10_000), erlang:send_after(LoginTimeout, self(), login_timeout), ProcessorState = rabbit_mqtt_processor:initial_state(RealSocket, ConnName), @@ -258,7 +258,7 @@ terminate(Reason, {SendWill, State = #state{conn_name = ConnName, proc_state = PState}}) -> KState = rabbit_mqtt_keepalive:cancel_timer(KState0), maybe_emit_stats(State#state{keepalive = KState}), - rabbit_mqtt_processor:terminate(SendWill, ConnName, ?PROTO_FAMILY, PState), + _ = rabbit_mqtt_processor:terminate(SendWill, ConnName, ?PROTO_FAMILY, PState), log_terminate(Reason, State). log_terminate({network_error, {ssl_upgrade_error, closed}, ConnName}, _State) -> diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_ets.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_ets.erl index 88a384d74e07..e2f5831de49e 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_ets.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_ets.erl @@ -23,14 +23,14 @@ new(Dir, VHost) -> Path = rabbit_mqtt_util:path_for(Dir, VHost), TableName = rabbit_mqtt_util:vhost_name_to_table_name(VHost), - file:delete(Path), + _ = file:delete(Path), Tid = ets:new(TableName, [set, public, {keypos, #retained_message.topic}]), #store_state{table = Tid, filename = Path}. recover(Dir, VHost) -> Path = rabbit_mqtt_util:path_for(Dir, VHost), case ets:file2tab(Path) of - {ok, Tid} -> file:delete(Path), + {ok, Tid} -> _ = file:delete(Path), {ok, #store_state{table = Tid, filename = Path}}; {error, _} -> {error, uninitialized} end. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl index a7cd43865251..5d1a49b7b136 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl @@ -71,8 +71,8 @@ init([{Listeners, SslListeners0}]) -> ]}}. stop_listeners() -> - rabbit_networking:stop_ranch_listener_of_protocol(?TCP_PROTOCOL), - rabbit_networking:stop_ranch_listener_of_protocol(?TLS_PROTOCOL), + _ = rabbit_networking:stop_ranch_listener_of_protocol(?TCP_PROTOCOL), + _ = rabbit_networking:stop_ranch_listener_of_protocol(?TLS_PROTOCOL), ok. %% diff --git a/deps/rabbitmq_web_mqtt/BUILD.bazel b/deps/rabbitmq_web_mqtt/BUILD.bazel index 25a8504147d3..6ac2d1251a5b 100644 --- a/deps/rabbitmq_web_mqtt/BUILD.bazel +++ b/deps/rabbitmq_web_mqtt/BUILD.bazel @@ -2,6 +2,7 @@ load("@rules_erlang//:xref2.bzl", "xref") load("@rules_erlang//:dialyze.bzl", "dialyze", "plt") load( "//:rabbitmq.bzl", + "RABBITMQ_DIALYZER_OPTS", "RABBITMQ_TEST_ERLC_OPTS", "assert_suites", "broker_for_integration_suites", @@ -59,9 +60,8 @@ plt( ) dialyze( - dialyzer_opts = ["-Werror_handling"], + dialyzer_opts = RABBITMQ_DIALYZER_OPTS, plt = ":base_plt", - warnings_as_errors = True, ) broker_for_integration_suites( diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_app.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_app.erl index b93242801a67..707012820519 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_app.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_app.erl @@ -116,7 +116,7 @@ start_tcp_listener(TCPConf0, CowboyOpts) -> [IpStr, Port]). start_tls_listener(TLSConf0, CowboyOpts) -> - rabbit_networking:ensure_ssl(), + _ = rabbit_networking:ensure_ssl(), {TLSConf, TLSIpStr, TLSPort} = get_tls_conf(TLSConf0), RanchRef = rabbit_networking:ranch_ref(TLSConf), RanchTransportOpts = diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index a987ea24db02..39977a66218d 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -101,7 +101,7 @@ websocket_init({State0 = #state{socket = Sock}, PeerAddr}) -> {ok, ConnStr} -> ConnName = rabbit_data_coercion:to_binary(ConnStr), ?LOG_INFO("Accepting Web MQTT connection ~s", [ConnName]), - rabbit_alarm:register(self(), {?MODULE, conserve_resources, []}), + _ = rabbit_alarm:register(self(), {?MODULE, conserve_resources, []}), PState = rabbit_mqtt_processor:initial_state( rabbit_net:unwrap_socket(Sock), ConnName, @@ -231,7 +231,7 @@ terminate(_Reason, _Request, keepalive = KState} = State}) -> ?LOG_INFO("Web MQTT closing connection ~ts", [ConnName]), maybe_emit_stats(State), - rabbit_mqtt_keepalive:cancel_timer(KState), + _ = rabbit_mqtt_keepalive:cancel_timer(KState), ok = file_handle_cache:release(), rabbit_mqtt_processor:terminate(SendWill, ConnName, ?PROTO_FAMILY, PState). From 6cc65ecceb2269bf4b53c9f5f3d9ab6bb4f05767 Mon Sep 17 00:00:00 2001 From: Chunyi Lyu Date: Tue, 24 Jan 2023 15:54:46 +0000 Subject: [PATCH 114/118] Export opaque types for event and mqtt_packet state --- deps/rabbit_common/src/rabbit_event.erl | 3 ++- deps/rabbitmq_mqtt/src/rabbit_mqtt_packet.erl | 10 +++++++--- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 4 ++-- 3 files changed, 11 insertions(+), 6 deletions(-) diff --git a/deps/rabbit_common/src/rabbit_event.erl b/deps/rabbit_common/src/rabbit_event.erl index 4bf9b0f52a5c..a6760b238858 100644 --- a/deps/rabbit_common/src/rabbit_event.erl +++ b/deps/rabbit_common/src/rabbit_event.erl @@ -25,7 +25,7 @@ %%---------------------------------------------------------------------------- --export_type([event_type/0, event_props/0, event_timestamp/0, event/0]). +-export_type([event_type/0, event_props/0, event_timestamp/0, event/0, state/0]). -type event_type() :: atom(). -type event_props() :: term(). @@ -41,6 +41,7 @@ -type timer_fun() :: fun (() -> 'ok'). -type container() :: tuple(). -type pos() :: non_neg_integer(). +-opaque state() :: #state{}. -spec start_link() -> rabbit_types:ok_pid_or_error(). -spec init_stats_timer(container(), pos()) -> container(). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_packet.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_packet.erl index 9b63e7b45420..abaf9cccd8b6 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_packet.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_packet.erl @@ -7,19 +7,23 @@ -module(rabbit_mqtt_packet). --export([parse/2, initial_state/0]). --export([serialise/2]). - -include("rabbit_mqtt_packet.hrl"). -include("rabbit_mqtt.hrl"). +-export([parse/2, initial_state/0, serialise/2]). +-export_type([state/0]). + +-opaque state() :: 'none' | {more, any()}. + -define(RESERVED, 0). -define(MAX_LEN, 16#fffffff). -define(HIGHBIT, 2#10000000). -define(LOWBITS, 2#01111111). +-spec initial_state() -> state(). initial_state() -> none. +-spec parse(binary(), state()) -> {more, state()} | {ok, any(), any()} | {error, any()}. parse(<<>>, none) -> {more, fun(Bin) -> parse(Bin, none) end}; parse(<>, none) -> diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index 4eaee6607903..508a785198b5 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -34,11 +34,11 @@ proxy_socket :: option({rabbit_proxy_socket, any(), any()}), await_recv :: boolean(), deferred_recv :: option(binary()), - parse_state :: atom(), + parse_state :: rabbit_mqtt_packet:state(), proc_state :: rabbit_mqtt_processor:state(), connection_state :: running | blocked, conserve :: boolean(), - stats_timer :: option(reference()), + stats_timer :: option(rabbit_event:state()), keepalive = rabbit_mqtt_keepalive:init() :: rabbit_mqtt_keepalive:state(), conn_name :: binary(), received_connect_packet :: boolean() From 9c2f5975ea87d8c5afddcde47584cd14e0a8a815 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Tue, 24 Jan 2023 16:35:00 +0000 Subject: [PATCH 115/118] Support tracing in Native MQTT --- deps/rabbit/src/rabbit_trace.erl | 44 ++++++++++--- .../src/rabbit_mqtt_processor.erl | 63 ++++++++++++++---- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 5 ++ deps/rabbitmq_mqtt/test/shared_SUITE.erl | 64 +++++++++++++++++++ .../src/rabbit_stream_reader.erl | 5 +- .../src/rabbit_web_mqtt_handler.erl | 6 ++ 6 files changed, 163 insertions(+), 24 deletions(-) diff --git a/deps/rabbit/src/rabbit_trace.erl b/deps/rabbit/src/rabbit_trace.erl index 02f78a26fe12..9fa25a0c04ab 100644 --- a/deps/rabbit/src/rabbit_trace.erl +++ b/deps/rabbit/src/rabbit_trace.erl @@ -7,13 +7,16 @@ -module(rabbit_trace). --export([init/1, enabled/1, tap_in/6, tap_out/5, start/1, stop/1]). +-export([init/1, enabled/1, tap_in/5, tap_in/6, + tap_out/4, tap_out/5, start/1, stop/1]). -include_lib("rabbit_common/include/rabbit.hrl"). -include_lib("rabbit_common/include/rabbit_framing.hrl"). -define(TRACE_VHOSTS, trace_vhosts). -define(XNAME, <<"amq.rabbitmq.trace">>). +%% "The channel number is 0 for all frames which are global to the connection" [AMQP 0-9-1 spec] +-define(CONNECTION_GLOBAL_CHANNEL_NUM, 0). %%---------------------------------------------------------------------------- @@ -23,7 +26,8 @@ -spec init(rabbit_types:vhost()) -> state(). -init(VHost) -> +init(VHost) + when is_binary(VHost) -> case enabled(VHost) of false -> none; true -> {ok, X} = rabbit_exchange:lookup( @@ -31,11 +35,21 @@ init(VHost) -> X end. --spec enabled(rabbit_types:vhost()) -> boolean(). +-spec enabled(rabbit_types:vhost() | state()) -> boolean(). -enabled(VHost) -> +enabled(VHost) + when is_binary(VHost) -> {ok, VHosts} = application:get_env(rabbit, ?TRACE_VHOSTS), - lists:member(VHost, VHosts). + lists:member(VHost, VHosts); +enabled(none) -> + false; +enabled(#exchange{}) -> + true. + +-spec tap_in(rabbit_types:basic_message(), [rabbit_amqqueue:name()], + binary(), rabbit_types:username(), state()) -> 'ok'. +tap_in(Msg, QNames, ConnName, Username, State) -> + tap_in(Msg, QNames, ConnName, ?CONNECTION_GLOBAL_CHANNEL_NUM, Username, State). -spec tap_in(rabbit_types:basic_message(), [rabbit_amqqueue:name()], binary(), rabbit_channel:channel_number(), @@ -53,6 +67,11 @@ tap_in(Msg = #basic_message{exchange_name = #resource{name = XName, {<<"routed_queues">>, array, [{longstr, QName#resource.name} || QName <- QNames]}]). +-spec tap_out(rabbit_amqqueue:qmsg(), binary(), + rabbit_types:username(), state()) -> 'ok'. +tap_out(Msg, ConnName, Username, State) -> + tap_out(Msg, ConnName, ?CONNECTION_GLOBAL_CHANNEL_NUM, Username, State). + -spec tap_out(rabbit_amqqueue:qmsg(), binary(), rabbit_channel:channel_number(), rabbit_types:username(), state()) -> 'ok'. @@ -73,7 +92,8 @@ tap_out({#resource{name = QName, virtual_host = VHost}, -spec start(rabbit_types:vhost()) -> 'ok'. -start(VHost) -> +start(VHost) + when is_binary(VHost) -> case lists:member(VHost, vhosts_with_tracing_enabled()) of true -> rabbit_log:info("Tracing is already enabled for vhost '~ts'", [VHost]), @@ -87,7 +107,8 @@ start(VHost) -> -spec stop(rabbit_types:vhost()) -> 'ok'. -stop(VHost) -> +stop(VHost) + when is_binary(VHost) -> case lists:member(VHost, vhosts_with_tracing_enabled()) of true -> rabbit_log:info("Disabling tracing for vhost '~ts'", [VHost]), @@ -101,11 +122,14 @@ update_config(Fun) -> VHosts0 = vhosts_with_tracing_enabled(), VHosts = Fun(VHosts0), application:set_env(rabbit, ?TRACE_VHOSTS, VHosts), - rabbit_log:debug("Will now refresh channel state after virtual host tracing changes"), + NonAmqpPids = rabbit_networking:local_non_amqp_connections(), + rabbit_log:debug("Will now refresh state of channels and of ~b non AMQP 0.9.1 " + "connections after virtual host tracing changes", + [length(NonAmqpPids)]), + lists:foreach(fun(Pid) -> gen_server:cast(Pid, refresh_config) end, NonAmqpPids), {Time, _} = timer:tc(fun rabbit_channel:refresh_config_local/0), - rabbit_log:debug("Refreshed channel state in ~fs", [Time/1000000]), - + rabbit_log:debug("Refreshed channel state in ~fs", [Time/1_000_000]), ok. vhosts_with_tracing_enabled() -> diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index bfe633e89e00..7b914e5518af 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -12,7 +12,8 @@ process_packet/2, serialise/2, terminate/4, handle_pre_hibernate/0, handle_ra_event/2, handle_down/2, handle_queue_event/2, - proto_version_tuple/1, throttle/3, format_status/1]). + proto_version_tuple/1, throttle/3, format_status/1, + update_trace/2]). %% for testing purposes -export([get_vhost_username/1, get_vhost/3, get_vhost_from_user_mapping/2]). @@ -77,7 +78,8 @@ delivery_flow :: flow | noflow, %% quorum queues and streams whose soft limit has been exceeded queues_soft_limit_exceeded = sets:new([{version, 2}]) :: sets:set(), - qos0_messages_dropped = 0 :: non_neg_integer() + qos0_messages_dropped = 0 :: non_neg_integer(), + trace_state }). -opaque state() :: #state{}. @@ -347,9 +349,11 @@ process_connect(#mqtt_packet{ fun register_client/2, fun start_keepalive/2, fun notify_connection_created/1, + fun init_trace/2, fun handle_clean_sess_qos0/2, fun handle_clean_sess_qos1/2, - fun cache_subscriptions/2], + fun cache_subscriptions/2 + ], PacketConnect, State0) of {ok, SessPresent0, State1} -> {?CONNACK_ACCEPT, SessPresent0, State1}; @@ -467,6 +471,21 @@ notify_connection_created(#mqtt_packet_connect{}) -> self() ! connection_created, ok. +init_trace(#mqtt_packet_connect{}, State = #state{conn_name = ConnName}) -> + {ok, update_trace(ConnName, State)}. + +-spec update_trace(binary(), state()) -> state(). +update_trace(ConnName0, State = #state{auth_state = #auth_state{vhost = VHost}}) -> + ConnName = case rabbit_trace:enabled(VHost) of + true -> + ConnName0; + false -> + %% We won't need conn_name. Use less memmory by setting to undefined. + undefined + end, + State#state{trace_state = rabbit_trace:init(VHost), + conn_name = ConnName}. + return_connack(?CONNACK_ACCEPT, S) -> {ok, S}; return_connack(?CONNACK_BAD_CREDENTIALS, S) -> @@ -764,7 +783,7 @@ check_user_login(#{vhost := VHost, username_bin := UsernameBin, pass_bin := PassBin, client_id := ClientId - } = In, State0) -> + } = In, State) -> AuthProps = case PassBin of none -> %% SSL user name provided. @@ -778,11 +797,7 @@ check_user_login(#{vhost := VHost, case rabbit_access_control:check_user_login( UsernameBin, AuthProps) of {ok, User = #user{username = Username}} -> - notify_auth_result(user_authentication_success, Username, State0), - State = State0#state{ - %% We won't need conn_name anymore. - %% Use less memmory by setting to undefined. - conn_name = undefined}, + notify_auth_result(user_authentication_success, Username, State), {ok, maps:put(user, User, In), State}; {refused, Username, Msg, Args} -> ?LOG_ERROR( @@ -790,7 +805,7 @@ check_user_login(#{vhost := VHost, "access refused for user '~s' in vhost '~s' " ++ Msg, [self(), Username, VHost] ++ Args), - notify_auth_result(user_authentication_failure, Username, State0), + notify_auth_result(user_authentication_failure, Username, State), {error, ?CONNACK_BAD_CREDENTIALS} end. @@ -1187,8 +1202,11 @@ publish_to_queues( dup = Dup, packet_id = PacketId, payload = Payload}, - #state{exchange = ExchangeName, - delivery_flow = Flow} = State) -> + #state{exchange = ExchangeName, + delivery_flow = Flow, + conn_name = ConnName, + auth_state = #auth_state{username = Username}, + trace_state = TraceState} = State) -> RoutingKey = mqtt_to_amqp(Topic), Confirm = Qos > ?QOS_0, Headers = [{<<"x-mqtt-publish-qos">>, byte, Qos}, @@ -1222,6 +1240,7 @@ publish_to_queues( case rabbit_exchange:lookup(ExchangeName) of {ok, Exchange} -> QNames = rabbit_exchange:route(Exchange, Delivery), + rabbit_trace:tap_in(BasicMessage, QNames, ConnName, Username, TraceState), deliver_to_queues(Delivery, QNames, State); {error, not_found} -> ?LOG_ERROR("~s not found", [rabbit_misc:rs(ExchangeName)]), @@ -1585,7 +1604,7 @@ maybe_publish_to_client( {QNameOrType, _QPid, QMsgId, Redelivered, #basic_message{ routing_keys = [RoutingKey | _CcRoutes], - content = #content{payload_fragments_rev = FragmentsRev}}}, + content = #content{payload_fragments_rev = FragmentsRev}}} = Msg, QoS, State0 = #state{send_fun = SendFun}) -> {PacketId, State} = msg_id_to_packet_id(QMsgId, QoS, State0), Packet = @@ -1605,6 +1624,7 @@ maybe_publish_to_client( topic_name = amqp_to_mqtt(RoutingKey)}, payload = lists:reverse(FragmentsRev)}, SendFun(Packet, State), + trace_tap_out(Msg, State), message_delivered(QNameOrType, Redelivered, QoS, State), State. @@ -1640,6 +1660,23 @@ maybe_notify_sent(QName, QPid, #state{queue_states = QStates}) -> ok end. +trace_tap_out(Msg = {#resource{}, _, _, _, _}, + #state{conn_name = ConnName, + trace_state = TraceState, + auth_state = #auth_state{username = Username}}) -> + rabbit_trace:tap_out(Msg, ConnName, Username, TraceState); +trace_tap_out(Msg0 = {?QUEUE_TYPE_QOS_0, _, _, _, _}, + State = #state{trace_state = TraceState}) -> + case rabbit_trace:enabled(TraceState) of + false -> + ok; + true -> + %% Pay penalty of creating queue name only if tracing is enabled. + QName = queue_name(?QOS_0, State), + Msg = setelement(1, Msg0, QName), + trace_tap_out(Msg, State) + end. + publish_to_queues_with_checks( TopicName, PublishFun, #state{exchange = Exchange, diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index 508a785198b5..99bbd9998371 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -148,6 +148,11 @@ handle_cast({force_event_refresh, Ref}, State0) -> State = rabbit_event:init_stats_timer(State0, #state.stats_timer), {noreply, State, ?HIBERNATE_AFTER}; +handle_cast(refresh_config, State = #state{proc_state = PState0, + conn_name = ConnName}) -> + PState = rabbit_mqtt_processor:update_trace(ConnName, PState0), + {noreply, pstate(State, PState), ?HIBERNATE_AFTER}; + handle_cast(Msg, State) -> {stop, {mqtt_unexpected_cast, Msg}, State}. diff --git a/deps/rabbitmq_mqtt/test/shared_SUITE.erl b/deps/rabbitmq_mqtt/test/shared_SUITE.erl index 9c3f41b5c762..71183d5f9cca 100644 --- a/deps/rabbitmq_mqtt/test/shared_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/shared_SUITE.erl @@ -91,6 +91,7 @@ subgroups() -> ,clean_session_disconnect_client ,clean_session_kill_node ,rabbit_status_connection_count + ,trace ]} ]}, {cluster_size_3, [], @@ -1273,6 +1274,69 @@ rabbit_status_connection_count(Config) -> ok = emqtt:disconnect(C). +trace(Config) -> + Server = atom_to_binary(get_node_config(Config, 0, nodename)), + Topic = Payload = TraceQ = atom_to_binary(?FUNCTION_NAME), + Ch = rabbit_ct_client_helpers:open_channel(Config), + declare_queue(Ch, TraceQ, []), + #'queue.bind_ok'{} = amqp_channel:call( + Ch, #'queue.bind'{queue = TraceQ, + exchange = <<"amq.rabbitmq.trace">>, + routing_key = <<"#">>}), + + %% We expect traced messages for connections created before and connections + %% created after tracing is enabled. + Pub = connect(<<(atom_to_binary(?FUNCTION_NAME))/binary, "_publisher">>, Config), + {ok, _} = rabbit_ct_broker_helpers:rabbitmqctl(Config, 0, ["trace_on"]), + Sub = connect(<<(atom_to_binary(?FUNCTION_NAME))/binary, "_subscriber">>, Config), + + {ok, _, [0]} = emqtt:subscribe(Sub, Topic, qos0), + {ok, _} = emqtt:publish(Pub, Topic, Payload, qos1), + ok = expect_publishes(Sub, Topic, [Payload]), + + {#'basic.get_ok'{routing_key = <<"publish.amq.topic">>}, + #amqp_msg{props = #'P_basic'{headers = PublishHeaders}, + payload = Payload}} = + amqp_channel:call(Ch, #'basic.get'{queue = TraceQ, no_ack = false}), + ?assertMatch(#{<<"exchange_name">> := <<"amq.topic">>, + <<"routing_keys">> := [Topic], + <<"connection">> := <<"127.0.0.1:", _/binary>>, + <<"node">> := Server, + <<"vhost">> := <<"/">>, + <<"channel">> := 0, + <<"user">> := <<"guest">>, + <<"properties">> := #{<<"delivery_mode">> := 2, + <<"headers">> := #{<<"x-mqtt-publish-qos">> := 1, + <<"x-mqtt-dup">> := false}}, + <<"routed_queues">> := [<<"mqtt-subscription-trace_subscriberqos0">>]}, + rabbit_misc:amqp_table(PublishHeaders)), + + {#'basic.get_ok'{routing_key = <<"deliver.mqtt-subscription-trace_subscriberqos0">>}, + #amqp_msg{props = #'P_basic'{headers = DeliverHeaders}, + payload = Payload}} = + amqp_channel:call(Ch, #'basic.get'{queue = TraceQ, no_ack = false}), + ?assertMatch(#{<<"exchange_name">> := <<"amq.topic">>, + <<"routing_keys">> := [Topic], + <<"connection">> := <<"127.0.0.1:", _/binary>>, + <<"node">> := Server, + <<"vhost">> := <<"/">>, + <<"channel">> := 0, + <<"user">> := <<"guest">>, + <<"properties">> := #{<<"delivery_mode">> := 2, + <<"headers">> := #{<<"x-mqtt-publish-qos">> := 1, + <<"x-mqtt-dup">> := false}}, + <<"redelivered">> := 0}, + rabbit_misc:amqp_table(DeliverHeaders)), + + {ok, _} = rabbit_ct_broker_helpers:rabbitmqctl(Config, 0, ["trace_off"]), + {ok, _} = emqtt:publish(Pub, Topic, Payload, qos1), + ok = expect_publishes(Sub, Topic, [Payload]), + ?assertMatch(#'basic.get_empty'{}, + amqp_channel:call(Ch, #'basic.get'{queue = TraceQ, no_ack = false})), + + delete_queue(Ch, TraceQ), + [ok = emqtt:disconnect(C) || C <- [Pub, Sub]]. + %% ------------------------------------------------------------------- %% Internal helpers %% ------------------------------------------------------------------- diff --git a/deps/rabbitmq_stream/src/rabbit_stream_reader.erl b/deps/rabbitmq_stream/src/rabbit_stream_reader.erl index 16f947b1f663..1f7ea4183800 100644 --- a/deps/rabbitmq_stream/src/rabbit_stream_reader.erl +++ b/deps/rabbitmq_stream/src/rabbit_stream_reader.erl @@ -1140,7 +1140,10 @@ open(cast, {force_event_refresh, Ref}, rabbit_event:init_stats_timer(Connection, #stream_connection.stats_timer), Connection2 = ensure_stats_timer(Connection1), - {keep_state, StatemData#statem_data{connection = Connection2}}. + {keep_state, StatemData#statem_data{connection = Connection2}}; +open(cast, refresh_config, _StatemData) -> + %% tracing not supported + keep_state_and_data. close_sent(enter, _OldState, #statem_data{config = diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index 39977a66218d..7c7bcb5cac19 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -177,6 +177,12 @@ websocket_info({'$gen_cast', {force_event_refresh, Ref}}, State0) -> rabbit_event:notify(connection_created, Infos, Ref), State = rabbit_event:init_stats_timer(State0, #state.stats_timer), {[], State, hibernate}; +websocket_info({'$gen_cast', refresh_config}, + State0 = #state{proc_state = PState0, + conn_name = ConnName}) -> + PState = rabbit_mqtt_processor:update_trace(ConnName, PState0), + State = State0#state{proc_state = PState}, + {[], State, hibernate}; websocket_info({keepalive, Req}, State = #state{keepalive = KState0, conn_name = ConnName}) -> case rabbit_mqtt_keepalive:handle(Req, KState0) of From ec137bc7832b686e0dfc76e56423ef77b123d4d5 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Wed, 25 Jan 2023 15:55:35 +0000 Subject: [PATCH 116/118] Add nested config record for rarely changing fields When a single field in a record is updated, all remaining fields' pointers are copied. Hence, if the record is large, a lot will be copied. Therefore, put static or rarely changing fields into their own record. The same was done for the state in rabbit_channel or rabbit_fifo for example. Also, merge #info{} record into the new #cfg{} record. --- .../src/rabbit_mqtt_processor.erl | 479 ++++++++++-------- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 23 +- 2 files changed, 268 insertions(+), 234 deletions(-) diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 7b914e5518af..c57ed634a7af 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -33,22 +33,41 @@ -define(MAX_PERMISSION_CACHE_SIZE, 12). -define(CONSUMER_TAG, <<"mqtt">>). --record(auth_state, {username :: rabbit_types:username(), - user :: #user{}, - vhost :: rabbit_types:vhost(), - authz_ctx :: #{binary() := binary()} - }). - --record(info, {prefetch :: non_neg_integer(), - host :: inet:ip_address(), - port :: inet:port_number(), - peer_host :: inet:ip_address(), - peer_port :: inet:port_number(), - connected_at :: pos_integer()}). +-record(auth_state, + {username :: rabbit_types:username(), + user :: #user{}, + vhost :: rabbit_types:vhost(), + authz_ctx :: #{binary() := binary()} + }). --record(state, +-record(cfg, {socket :: rabbit_net:socket(), proto_ver :: option(mqtt310 | mqtt311), + clean_sess :: option(boolean()), + will_msg :: option(mqtt_msg()), + exchange :: option(rabbit_exchange:name()), + %% Set if client has at least one subscription with QoS 1. + queue_qos1 :: option(rabbit_amqqueue:name()), + %% Did the client ever sent us a PUBLISH packet? + published = false :: boolean(), + ssl_login_name :: none | binary(), + retainer_pid :: option(pid()), + delivery_flow :: flow | noflow, + trace_state, + prefetch :: non_neg_integer(), + client_id :: option(binary()), + conn_name :: option(binary()), + peer_addr :: inet:ip_address(), + host :: inet:ip_address(), + port :: inet:port_number(), + peer_host :: inet:ip_address(), + peer_port :: inet:port_number(), + connected_at :: pos_integer(), + send_fun :: fun((Packet :: tuple(), state()) -> term()) + }). + +-record(state, + {cfg :: #cfg{}, queue_states = rabbit_queue_type:init() :: rabbit_queue_type:state(), %% Packet IDs published to queues but not yet confirmed. unacked_client_pubs = rabbit_mqtt_confirms:init() :: rabbit_mqtt_confirms:state(), @@ -58,28 +77,12 @@ %% (Not to be confused with packet IDs sent from client to server which can be the %% same IDs because client and server assign IDs independently of each other.) packet_id = 1 :: packet_id(), - client_id :: option(binary()), - clean_sess :: option(boolean()), - will_msg :: option(mqtt_msg()), - exchange :: option(rabbit_exchange:name()), subscriptions = #{} :: #{Topic :: binary() => QoS :: ?QOS_0..?QOS_1}, - %% Set if client has at least one subscription with QoS 1. - queue_qos1 :: option(rabbit_amqqueue:name()), - %% Did the client ever sent us a PUBLISH packet? - published = false :: boolean(), - ssl_login_name :: none | binary(), - retainer_pid :: option(pid()), auth_state :: option(#auth_state{}), - peer_addr :: inet:ip_address(), - send_fun :: fun((Packet :: tuple(), state()) -> term()), register_state :: option(registered | {pending, reference()}), - conn_name :: option(binary()), - info :: option(#info{}), - delivery_flow :: flow | noflow, %% quorum queues and streams whose soft limit has been exceeded queues_soft_limit_exceeded = sets:new([{version, 2}]) :: sets:set(), - qos0_messages_dropped = 0 :: non_neg_integer(), - trace_state + qos0_messages_dropped = 0 :: non_neg_integer() }). -opaque state() :: #state{}. @@ -103,12 +106,19 @@ initial_state(Socket, ConnectionName, SendFun, PeerAddr) -> true -> flow; false -> noflow end, - #state{socket = Socket, - conn_name = ConnectionName, - ssl_login_name = ssl_login_name(Socket), - peer_addr = PeerAddr, - send_fun = SendFun, - delivery_flow = Flow}. + {ok, {PeerHost, PeerPort, Host, Port}} = rabbit_net:socket_ends(Socket, inbound), + #state{cfg = #cfg{socket = Socket, + conn_name = ConnectionName, + ssl_login_name = ssl_login_name(Socket), + send_fun = SendFun, + prefetch = rabbit_mqtt_util:env(prefetch), + delivery_flow = Flow, + connected_at = os:system_time(milli_seconds), + peer_addr = PeerAddr, + peer_host = PeerHost, + peer_port = PeerPort, + host = Host, + port = Port}}. -spec process_packet(mqtt_packet(), state()) -> {ok, state()} | @@ -125,7 +135,7 @@ process_packet(Packet = #mqtt_packet{fixed = #mqtt_packet_fixed{type = Type}}, S {ok, state()} | {stop, disconnect, state()} | {error, Reason :: term(), state()}. -process_request(?CONNECT, Packet, State = #state{socket = Socket}) -> +process_request(?CONNECT, Packet, State = #state{cfg = #cfg{socket = Socket}}) -> %% Check whether peer closed the connection. %% For example, this can happen when connection was blocked because of resource %% alarm and client therefore disconnected due to client side CONNACK timeout. @@ -137,11 +147,10 @@ process_request(?CONNECT, Packet, State = #state{socket = Socket}) -> end; process_request(?PUBACK, - #mqtt_packet{ - variable = #mqtt_packet_publish{packet_id = PacketId}}, + #mqtt_packet{variable = #mqtt_packet_publish{packet_id = PacketId}}, #state{unacked_server_pubs = U0, queue_states = QStates0, - queue_qos1 = QName} = State) -> + cfg = #cfg{queue_qos1 = QName}} = State) -> case maps:take(PacketId, U0) of {QMsgId, U} -> case rabbit_queue_type:settle(QName, complete, ?CONSUMER_TAG, [QMsgId], QStates0) of @@ -173,9 +182,9 @@ process_request(?PUBLISH, variable = #mqtt_packet_publish{topic_name = Topic, packet_id = PacketId }, payload = Payload}, - State0 = #state{retainer_pid = RPid, - unacked_client_pubs = U, - proto_ver = ProtoVer}) -> + State0 = #state{unacked_client_pubs = U, + cfg = #cfg{retainer_pid = RPid, + proto_ver = ProtoVer}}) -> rabbit_global_counters:messages_received(ProtoVer, 1), State = maybe_increment_publisher(State0), Publish = fun() -> @@ -220,9 +229,8 @@ process_request(?SUBSCRIBE, packet_id = SubscribePktId, topic_table = Topics}, payload = undefined}, - #state{send_fun = SendFun, - retainer_pid = RPid - } = State0) -> + #state{cfg = #cfg{send_fun = SendFun, + retainer_pid = RPid}} = State0) -> ?LOG_DEBUG("Received a SUBSCRIBE for topic(s) ~p", [Topics]), {QosResponse, State1} = lists:foldl( @@ -289,7 +297,7 @@ process_request(?UNSUBSCRIBE, #mqtt_packet{variable = #mqtt_packet_subscribe{packet_id = PacketId, topic_table = Topics}, payload = undefined}, - State0 = #state{send_fun = SendFun}) -> + State0 = #state{cfg = #cfg{send_fun = SendFun}}) -> ?LOG_DEBUG("Received an UNSUBSCRIBE for topic(s) ~p", [Topics]), State = lists:foldl( fun(#mqtt_topic{name = TopicName}, #state{subscriptions = Subs0} = S0) -> @@ -317,8 +325,9 @@ process_request(?UNSUBSCRIBE, {ok, State}; -process_request(?PINGREQ, #mqtt_packet{}, State = #state{send_fun = SendFun, - client_id = ClientId}) -> +process_request(?PINGREQ, #mqtt_packet{}, + State = #state{cfg = #cfg{send_fun = SendFun, + client_id = ClientId}}) -> ?LOG_DEBUG("Received a PINGREQ, client ID: ~s", [ClientId]), SendFun( #mqtt_packet{fixed = #mqtt_packet_fixed{type = ?PINGRESP}}, @@ -337,7 +346,7 @@ process_connect(#mqtt_packet{ clean_sess = CleanSess, client_id = ClientId, keep_alive = Keepalive} = PacketConnect}, - State0 = #state{send_fun = SendFun}) -> + State0 = #state{cfg = #cfg{send_fun = SendFun}}) -> ?LOG_DEBUG("Received a CONNECT, client ID: ~s, username: ~s, " "clean session: ~s, protocol version: ~p, keepalive: ~p", [ClientId, Username, CleanSess, ProtoVersion, Keepalive]), @@ -383,8 +392,8 @@ check_client_id(_) -> check_credentials(Packet = #mqtt_packet_connect{username = Username, password = Password}, - State = #state{ssl_login_name = SslLoginName, - peer_addr = PeerAddr}) -> + State = #state{cfg = #cfg{ssl_login_name = SslLoginName, + peer_addr = PeerAddr}}) -> Ip = list_to_binary(inet:ntoa(PeerAddr)), case creds(Username, Password, SslLoginName) of nocreds -> @@ -406,12 +415,13 @@ check_credentials(Packet = #mqtt_packet_connect{username = Username, login({UserBin, PassBin, Packet = #mqtt_packet_connect{client_id = ClientId0, clean_sess = CleanSess}}, - State0) -> + State0 = #state{cfg = Cfg0}) -> ClientId = ensure_client_id(ClientId0), case process_login(UserBin, PassBin, ClientId, State0) of {ok, State} -> - {ok, Packet, State#state{clean_sess = CleanSess, - client_id = ClientId}}; + Cfg = Cfg0#cfg{client_id = ClientId, + clean_sess = CleanSess}, + {ok, Packet, State#state{cfg = Cfg}}; {error, _ConnectionRefusedReturnCode, _State} = Err -> Err end. @@ -426,30 +436,20 @@ ensure_client_id(ClientId) ClientId. register_client(Packet = #mqtt_packet_connect{proto_ver = ProtoVersion}, - State = #state{client_id = ClientId, - socket = Socket, - auth_state = #auth_state{vhost = VHost}}) -> + State = #state{auth_state = #auth_state{vhost = VHost}, + cfg = Cfg0 = #cfg{client_id = ClientId}}) -> NewProcState = fun(RegisterState) -> rabbit_mqtt_util:register_clientid(VHost, ClientId), RetainerPid = rabbit_mqtt_retainer_sup:child_for_vhost(VHost), - Prefetch = rabbit_mqtt_util:env(prefetch), - {ok, {PeerHost, PeerPort, Host, Port}} = rabbit_net:socket_ends(Socket, inbound), ExchangeBin = rabbit_mqtt_util:env(exchange), ExchangeName = rabbit_misc:r(VHost, exchange, ExchangeBin), - State#state{ - exchange = ExchangeName, - will_msg = make_will_msg(Packet), - retainer_pid = RetainerPid, - register_state = RegisterState, - proto_ver = proto_integer_to_atom(ProtoVersion), - info = #info{prefetch = Prefetch, - peer_host = PeerHost, - peer_port = PeerPort, - host = Host, - port = Port, - connected_at = os:system_time(milli_seconds) - }} + Cfg = Cfg0#cfg{exchange = ExchangeName, + will_msg = make_will_msg(Packet), + retainer_pid = RetainerPid, + proto_ver = proto_integer_to_atom(ProtoVersion)}, + State#state{cfg = Cfg, + register_state = RegisterState} end, case rabbit_mqtt_ff:track_client_id_in_ra() of true -> @@ -471,11 +471,12 @@ notify_connection_created(#mqtt_packet_connect{}) -> self() ! connection_created, ok. -init_trace(#mqtt_packet_connect{}, State = #state{conn_name = ConnName}) -> +init_trace(#mqtt_packet_connect{}, State = #state{cfg = #cfg{conn_name = ConnName}}) -> {ok, update_trace(ConnName, State)}. -spec update_trace(binary(), state()) -> state(). -update_trace(ConnName0, State = #state{auth_state = #auth_state{vhost = VHost}}) -> +update_trace(ConnName0, State = #state{cfg = Cfg0, + auth_state = #auth_state{vhost = VHost}}) -> ConnName = case rabbit_trace:enabled(VHost) of true -> ConnName0; @@ -483,8 +484,9 @@ update_trace(ConnName0, State = #state{auth_state = #auth_state{vhost = VHost}}) %% We won't need conn_name. Use less memmory by setting to undefined. undefined end, - State#state{trace_state = rabbit_trace:init(VHost), - conn_name = ConnName}. + Cfg = Cfg0#cfg{conn_name = ConnName, + trace_state = rabbit_trace:init(VHost)}, + State#state{cfg = Cfg}. return_connack(?CONNACK_ACCEPT, S) -> {ok, S}; @@ -511,7 +513,7 @@ self_consumes(Queue) -> end. start_keepalive(#mqtt_packet_connect{keep_alive = Seconds}, - #state{socket = Socket}) -> + #state{cfg = #cfg{socket = Socket}}) -> ok = rabbit_mqtt_keepalive:start(Seconds, Socket). handle_clean_sess_qos0(#mqtt_packet_connect{}, State) -> @@ -521,7 +523,7 @@ handle_clean_sess_qos1(QoS0SessPresent, State) -> handle_clean_sess(QoS0SessPresent, ?QOS_1, State). handle_clean_sess(_, QoS, - State = #state{clean_sess = true, + State = #state{cfg = #cfg{clean_sess = true}, auth_state = #auth_state{user = User, username = Username, authz_ctx = AuthzCtx}}) -> @@ -543,8 +545,8 @@ handle_clean_sess(_, QoS, end end; handle_clean_sess(SessPresent, QoS, - State0 = #state{clean_sess = false, - proto_ver = ProtoVer}) -> + State0 = #state{cfg = #cfg{clean_sess = false, + proto_ver = ProtoVer}}) -> case get_queue(QoS, State0) of {error, _} -> %% Queue will be created later when client subscribes. @@ -588,10 +590,10 @@ get_queue(QoS, State) -> Err end. -queue_name(?QOS_1, #state{queue_qos1 = #resource{kind = queue} = Name}) -> +queue_name(?QOS_1, #state{cfg = #cfg{queue_qos1 = #resource{kind = queue} = Name}}) -> Name; -queue_name(QoS, #state{client_id = ClientId, - auth_state = #auth_state{vhost = VHost}}) -> +queue_name(QoS, #state{auth_state = #auth_state{vhost = VHost}, + cfg = #cfg{client_id = ClientId}}) -> QNameBin = rabbit_mqtt_util:queue_name_bin(ClientId, QoS), rabbit_misc:r(VHost, queue, QNameBin). @@ -606,7 +608,7 @@ cache_subscriptions(_SessionPresent = _SubscriptionsPresent = true, State) -> cache_subscriptions(_, _) -> ok. -topic_names(QoS, #state{exchange = Exchange} = State) -> +topic_names(QoS, State = #state{cfg = #cfg{exchange = Exchange}}) -> Bindings = rabbit_binding:list_for_source_and_destination( Exchange, @@ -655,8 +657,8 @@ hand_off_to_retainer(RetainerPid, Topic0, Msg) -> ok. maybe_send_retained_message(RPid, #mqtt_topic{name = Topic0, qos = SubscribeQos}, - #state{packet_id = PacketId0, - send_fun = SendFun} = State0) -> + State0 = #state{packet_id = PacketId0, + cfg = #cfg{send_fun = SendFun}}) -> Topic1 = amqp_to_mqtt(Topic0), case rabbit_mqtt_retainer:fetch(RPid, Topic1) of undefined -> @@ -697,7 +699,7 @@ make_will_msg(#mqtt_packet_connect{will_retain = Retain, payload = Msg}. process_login(_UserBin, _PassBin, ClientId, - #state{peer_addr = Addr, + #state{cfg = #cfg{peer_addr = Addr}, auth_state = #auth_state{username = Username, user = User, vhost = VHost @@ -709,10 +711,11 @@ process_login(_UserBin, _PassBin, ClientId, [ClientId, Username, VHost]), {error, ?CONNACK_ID_REJECTED, State}; process_login(UserBin, PassBin, ClientId, - #state{socket = Sock, - ssl_login_name = SslLoginName, - peer_addr = Addr, - auth_state = undefined} = State0) -> + #state{auth_state = undefined, + cfg = #cfg{socket = Sock, + ssl_login_name = SslLoginName, + peer_addr = Addr + }} = State0) -> {ok, {_PeerHost, _PeerPort, _Host, Port}} = rabbit_net:socket_ends(Sock, inbound), {VHostPickedUsing, {VHost, UsernameBin}} = get_vhost(UserBin, SslLoginName, Port), ?LOG_DEBUG("MQTT vhost picked using ~s", @@ -809,7 +812,7 @@ check_user_login(#{vhost := VHost, {error, ?CONNACK_BAD_CREDENTIALS} end. -notify_auth_result(AuthResult, Username, #state{conn_name = ConnName}) -> +notify_auth_result(AuthResult, Username, #state{cfg = #cfg{conn_name = ConnName}}) -> rabbit_event:notify( AuthResult, [{name, Username}, @@ -834,7 +837,7 @@ check_vhost_access(#{vhost := VHost, client_id := ClientId, user := User = #user{username = Username} } = In, - #state{peer_addr = PeerAddr} = State) -> + #state{cfg = #cfg{peer_addr = PeerAddr}} = State) -> AuthzCtx = #{<<"client_id">> => ClientId}, try rabbit_access_control:check_vhost_access( User, @@ -856,7 +859,7 @@ check_user_loopback(#{vhost := VHost, user := User, authz_ctx := AuthzCtx }, - #state{peer_addr = PeerAddr} = State) -> + #state{cfg = #cfg{peer_addr = PeerAddr}} = State) -> case rabbit_access_control:check_user_loopback(UsernameBin, PeerAddr) of ok -> AuthState = #auth_state{user = User, @@ -1016,14 +1019,15 @@ ensure_queue(QoS, State = #state{auth_state = #auth_state{user = #user{username create_queue(QoS, State) end. -create_queue(QoS, #state{ - client_id = ClientId, - clean_sess = CleanSess, - auth_state = #auth_state{ - vhost = VHost, - user = User = #user{username = Username}, - authz_ctx = AuthzCtx} - }) -> +create_queue( + QoS, #state{cfg = #cfg{ + client_id = ClientId, + clean_sess = CleanSess}, + auth_state = #auth_state{ + vhost = VHost, + user = User = #user{username = Username}, + authz_ctx = AuthzCtx} + }) -> QNameBin = rabbit_mqtt_util:queue_name_bin(ClientId, QoS), QName = rabbit_misc:r(VHost, queue, QNameBin), %% configure access to queue required for queue.declare @@ -1099,10 +1103,10 @@ queue_type(_, _, QArgs) -> consume(Q, QoS, #state{ queue_states = QStates0, + cfg = #cfg{prefetch = Prefetch}, auth_state = #auth_state{ - user = User = #user{username = Username}, - authz_ctx = AuthzCtx}, - info = #info{prefetch = Prefetch} + authz_ctx = AuthzCtx, + user = User = #user{username = Username}} } = State0) -> QName = amqqueue:get_name(Q), %% read access to queue required for basic.consume @@ -1145,8 +1149,8 @@ consume(Q, QoS, #state{ %% To save memory, we only store the queue_qos1 value in process state if there is a QoS 1 subscription. %% We store it in the process state such that we don't have to build the binary on every PUBACK we receive. -maybe_set_queue_qos1(?QOS_1, State = #state{queue_qos1 = undefined}) -> - State#state{queue_qos1 = queue_name(?QOS_1, State)}; +maybe_set_queue_qos1(?QOS_1, State = #state{cfg = Cfg = #cfg{queue_qos1 = undefined}}) -> + State#state{cfg = Cfg#cfg{queue_qos1 = queue_name(?QOS_1, State)}}; maybe_set_queue_qos1(_, State) -> State. @@ -1174,7 +1178,7 @@ check_queue_write_access( check_resource_access(User, QueueName, write, AuthzCtx). check_exchange_read_access( - _, #state{exchange = ExchangeName, + _, #state{cfg = #cfg{exchange = ExchangeName}, auth_state = #auth_state{ user = User, authz_ctx = AuthzCtx}}) -> @@ -1186,10 +1190,8 @@ check_topic_access({_, TopicName, _}, State) -> binding_action( {QueueName, TopicName, BindingFun}, - #state{exchange = ExchangeName, - auth_state = #auth_state{ - user = #user{username = Username}} - }) -> + #state{cfg = #cfg{exchange = ExchangeName}, + auth_state = #auth_state{user = #user{username = Username}}}) -> RoutingKey = mqtt_to_amqp(TopicName), Binding = #binding{source = ExchangeName, destination = QueueName, @@ -1202,11 +1204,12 @@ publish_to_queues( dup = Dup, packet_id = PacketId, payload = Payload}, - #state{exchange = ExchangeName, - delivery_flow = Flow, - conn_name = ConnName, - auth_state = #auth_state{username = Username}, - trace_state = TraceState} = State) -> + #state{cfg = #cfg{exchange = ExchangeName, + delivery_flow = Flow, + conn_name = ConnName, + trace_state = TraceState}, + auth_state = #auth_state{username = Username} + } = State) -> RoutingKey = mqtt_to_amqp(Topic), Confirm = Qos > ?QOS_0, Headers = [{<<"x-mqtt-publish-qos">>, byte, Qos}, @@ -1250,7 +1253,7 @@ publish_to_queues( deliver_to_queues(Delivery, RoutedToQNames, State0 = #state{queue_states = QStates0, - proto_ver = ProtoVer}) -> + cfg = #cfg{proto_ver = ProtoVer}}) -> Qs0 = rabbit_amqqueue:lookup(RoutedToQNames), Qs = rabbit_amqqueue:prepend_extra_bcc(Qs0), case rabbit_queue_type:deliver(Qs, Delivery, QStates0) of @@ -1268,18 +1271,18 @@ deliver_to_queues(Delivery, end. process_routing_confirm(#delivery{confirm = false}, - [], State = #state{proto_ver = ProtoVer}) -> + [], State = #state{cfg = #cfg{proto_ver = ProtoVer}}) -> rabbit_global_counters:messages_unroutable_dropped(ProtoVer, 1), State; process_routing_confirm(#delivery{confirm = true, msg_seq_no = undefined}, - [], State = #state{proto_ver = ProtoVer}) -> + [], State = #state{cfg = #cfg{proto_ver = ProtoVer}}) -> %% unroutable will message with QoS > 0 rabbit_global_counters:messages_unroutable_dropped(ProtoVer, 1), State; process_routing_confirm(#delivery{confirm = true, msg_seq_no = PktId}, - [], State = #state{proto_ver = ProtoVer}) -> + [], State = #state{cfg = #cfg{proto_ver = ProtoVer}}) -> rabbit_global_counters:messages_unroutable_returned(ProtoVer, 1), %% MQTT 5 spec: %% If the Server knows that there are no matching subscribers, it MAY use @@ -1307,15 +1310,16 @@ send_puback(PktIds0, State) lists:foreach(fun(Id) -> send_puback(Id, State) end, PktIds); -send_puback(PktId, State = #state{send_fun = SendFun, - proto_ver = ProtoVer}) -> +send_puback(PktId, State = #state{cfg = #cfg{send_fun = SendFun, + proto_ver = ProtoVer}}) -> rabbit_global_counters:messages_confirmed(ProtoVer, 1), SendFun( #mqtt_packet{fixed = #mqtt_packet_fixed{type = ?PUBACK}, variable = #mqtt_packet_publish{packet_id = PktId}}, State). -serialise_and_send_to_client(Packet, #state{proto_ver = ProtoVer, socket = Sock}) -> +serialise_and_send_to_client(Packet, #state{cfg = #cfg{proto_ver = ProtoVer, + socket = Sock}}) -> Data = rabbit_mqtt_packet:serialise(Packet, ProtoVer), try rabbit_net:port_command(Sock, Data) catch error:Error -> @@ -1325,7 +1329,7 @@ serialise_and_send_to_client(Packet, #state{proto_ver = ProtoVer, socket = Sock} [Sock, Error, Packet#mqtt_packet.fixed, Packet#mqtt_packet.variable]) end. -serialise(Packet, #state{proto_ver = ProtoVer}) -> +serialise(Packet, #state{cfg = #cfg{proto_ver = ProtoVer}}) -> rabbit_mqtt_packet:serialise(Packet, ProtoVer). terminate(SendWill, ConnName, ProtoFamily, State) -> @@ -1343,11 +1347,12 @@ terminate(SendWill, ConnName, ProtoFamily, State) -> maybe_decrement_publisher(State), maybe_delete_mqtt_qos0_queue(State). -maybe_send_will(true, ConnStr, - #state{will_msg = WillMsg = #mqtt_msg{retain = Retain, - topic = Topic}, - retainer_pid = RPid - } = State) -> +maybe_send_will( + true, ConnStr, + #state{cfg = #cfg{retainer_pid = RPid, + will_msg = WillMsg = #mqtt_msg{retain = Retain, + topic = Topic}} + } = State) -> ?LOG_DEBUG("sending MQTT will message to topic ~s on connection ~s", [Topic, ConnStr]), case check_topic_access(Topic, write, State) of @@ -1375,7 +1380,7 @@ additional_connection_closed_info( additional_connection_closed_info(_, _) -> []. -maybe_unregister_client(#state{client_id = ClientId}) +maybe_unregister_client(#state{cfg = #cfg{client_id = ClientId}}) when ClientId =/= undefined -> case rabbit_mqtt_ff:track_client_id_in_ra() of true -> @@ -1387,8 +1392,9 @@ maybe_unregister_client(#state{client_id = ClientId}) maybe_unregister_client(_) -> ok. -maybe_delete_mqtt_qos0_queue(State = #state{clean_sess = true, - auth_state = #auth_state{username = Username}}) -> +maybe_delete_mqtt_qos0_queue( + State = #state{cfg = #cfg{clean_sess = true}, + auth_state = #auth_state{username = Username}}) -> case get_queue(?QOS_0, State) of {ok, Q} -> %% double check we delete the right queue @@ -1432,7 +1438,7 @@ handle_ra_event({applied, [{Corr, ok}]}, State#state{register_state = registered}; handle_ra_event({not_leader, Leader, Corr}, State = #state{register_state = {pending, Corr}, - client_id = ClientId}) -> + cfg = #cfg{client_id = ClientId}}) -> case rabbit_mqtt_ff:track_client_id_in_ra() of true -> %% retry command against actual leader @@ -1443,7 +1449,7 @@ handle_ra_event({not_leader, Leader, Corr}, end; handle_ra_event(register_timeout, State = #state{register_state = {pending, _Corr}, - client_id = ClientId}) -> + cfg = #cfg{client_id = ClientId}}) -> case rabbit_mqtt_ff:track_client_id_in_ra() of true -> {ok, NewCorr} = rabbit_mqtt_collector:register(ClientId, self()), @@ -1540,7 +1546,7 @@ handle_queue_actions(Actions, #state{} = State0) -> handle_queue_down(QName, S) end, State0, Actions). -handle_queue_down(QName, State0 = #state{client_id = ClientId}) -> +handle_queue_down(QName, State0 = #state{cfg = #cfg{client_id = ClientId}}) -> %% Classic queue is down. case rabbit_amqqueue:lookup(QName) of {ok, Q} -> @@ -1605,7 +1611,7 @@ maybe_publish_to_client( #basic_message{ routing_keys = [RoutingKey | _CcRoutes], content = #content{payload_fragments_rev = FragmentsRev}}} = Msg, - QoS, State0 = #state{send_fun = SendFun}) -> + QoS, State0 = #state{cfg = #cfg{send_fun = SendFun}}) -> {PacketId, State} = msg_id_to_packet_id(QMsgId, QoS, State0), Packet = #mqtt_packet{ @@ -1661,12 +1667,12 @@ maybe_notify_sent(QName, QPid, #state{queue_states = QStates}) -> end. trace_tap_out(Msg = {#resource{}, _, _, _, _}, - #state{conn_name = ConnName, - trace_state = TraceState, - auth_state = #auth_state{username = Username}}) -> + #state{auth_state = #auth_state{username = Username}, + cfg = #cfg{conn_name = ConnName, + trace_state = TraceState}}) -> rabbit_trace:tap_out(Msg, ConnName, Username, TraceState); trace_tap_out(Msg0 = {?QUEUE_TYPE_QOS_0, _, _, _, _}, - State = #state{trace_state = TraceState}) -> + State = #state{cfg = #cfg{trace_state = TraceState}}) -> case rabbit_trace:enabled(TraceState) of false -> ok; @@ -1679,9 +1685,10 @@ trace_tap_out(Msg0 = {?QUEUE_TYPE_QOS_0, _, _, _, _}, publish_to_queues_with_checks( TopicName, PublishFun, - #state{exchange = Exchange, + #state{cfg = #cfg{exchange = Exchange}, auth_state = #auth_state{user = User, - authz_ctx = AuthzCtx}} = State) -> + authz_ctx = AuthzCtx} + } = State) -> case check_resource_access(User, Exchange, write, AuthzCtx) of ok -> case check_topic_access(TopicName, write, State) of @@ -1717,14 +1724,13 @@ check_resource_access(User, Resource, Perm, Context) -> end end. -check_topic_access(TopicName, Access, - #state{ - auth_state = #auth_state{user = User = #user{username = Username}, - vhost = VHost, - authz_ctx = AuthzCtx}, - exchange = #resource{name = ExchangeBin}, - client_id = ClientId - }) -> +check_topic_access( + TopicName, Access, + #state{auth_state = #auth_state{user = User = #user{username = Username}, + vhost = VHost, + authz_ctx = AuthzCtx}, + cfg = #cfg{client_id = ClientId, + exchange = #resource{name = ExchangeBin}}}) -> Cache = case get(topic_permission_cache) of undefined -> []; Other -> Other @@ -1757,7 +1763,7 @@ check_topic_access(TopicName, Access, boolean(). drop_qos0_message(State) -> mailbox_soft_limit_exceeded() andalso - is_socket_busy(State#state.socket). + is_socket_busy(State#state.cfg#cfg.socket). -spec mailbox_soft_limit_exceeded() -> boolean(). @@ -1784,35 +1790,35 @@ is_socket_busy(Socket) -> end. -spec throttle(boolean(), boolean(), state()) -> boolean(). -throttle(Conserve, Connected, #state{published = Published, - queues_soft_limit_exceeded = QSLE}) -> +throttle(Conserve, Connected, #state{queues_soft_limit_exceeded = QSLE, + cfg = #cfg{published = Published}}) -> Conserve andalso (Published orelse not Connected) orelse not sets:is_empty(QSLE) orelse credit_flow:blocked(). -info(host, #state{info = #info{host = Val}}) -> Val; -info(port, #state{info = #info{port = Val}}) -> Val; -info(peer_host, #state{info = #info{peer_host = Val}}) -> Val; -info(peer_port, #state{info = #info{peer_port = Val}}) -> Val; -info(connected_at, #state{info = #info{connected_at = Val}}) -> Val; -info(ssl_login_name, #state{ssl_login_name = Val}) -> Val; +info(host, #state{cfg = #cfg{host = Val}}) -> Val; +info(port, #state{cfg = #cfg{port = Val}}) -> Val; +info(peer_host, #state{cfg = #cfg{peer_host = Val}}) -> Val; +info(peer_port, #state{cfg = #cfg{peer_port = Val}}) -> Val; +info(connected_at, #state{cfg = #cfg{connected_at = Val}}) -> Val; +info(ssl_login_name, #state{cfg = #cfg{ssl_login_name = Val}}) -> Val; info(vhost, #state{auth_state = #auth_state{vhost = Val}}) -> Val; info(user_who_performed_action, S) -> info(user, S); info(user, #state{auth_state = #auth_state{username = Val}}) -> Val; -info(clean_sess, #state{clean_sess = Val}) -> Val; -info(will_msg, #state{will_msg = Val}) -> Val; -info(retainer_pid, #state{retainer_pid = Val}) -> Val; -info(exchange, #state{exchange = #resource{name = Val}}) -> Val; -info(prefetch, #state{info = #info{prefetch = Val}}) -> Val; +info(clean_sess, #state{cfg = #cfg{clean_sess = Val}}) -> Val; +info(will_msg, #state{cfg = #cfg{will_msg = Val}}) -> Val; +info(retainer_pid, #state{cfg = #cfg{retainer_pid = Val}}) -> Val; +info(exchange, #state{cfg = #cfg{exchange = #resource{name = Val}}}) -> Val; +info(prefetch, #state{cfg = #cfg{prefetch = Val}}) -> Val; info(messages_unconfirmed, #state{unacked_client_pubs = Val}) -> rabbit_mqtt_confirms:size(Val); info(messages_unacknowledged, #state{unacked_server_pubs = Val}) -> maps:size(Val); info(node, _) -> node(); -info(client_id, #state{client_id = Val}) -> Val; +info(client_id, #state{cfg = #cfg{client_id = Val}}) -> Val; %% for rabbitmq_management/priv/www/js/tmpl/connection.ejs -info(client_properties, #state{client_id = Val}) -> +info(client_properties, #state{cfg = #cfg{client_id = Val}}) -> [{client_id, longstr, Val}]; info(channel_max, _) -> 0; %% Maximum packet size supported only in MQTT 5.0. @@ -1834,65 +1840,26 @@ ssl_login_name(Sock) -> nossl -> none end. --spec format_status(state()) -> map(). -format_status(#state{queue_states = QState, - proto_ver = ProtoVersion, - unacked_client_pubs = UnackClientPubs, - unacked_server_pubs = UnackSerPubs, - packet_id = PackID, - client_id = ClientID, - clean_sess = CleanSess, - will_msg = WillMsg, - exchange = Exchange, - ssl_login_name = SSLLoginName, - retainer_pid = RetainerPid, - auth_state = AuthState, - peer_addr = PeerAddr, - register_state = RegisterState, - conn_name = ConnName, - info = Info, - queues_soft_limit_exceeded = QSLE, - qos0_messages_dropped = Qos0MsgsDropped - }) -> - #{queue_states => rabbit_queue_type:format_status(QState), - proto_ver => ProtoVersion, - unacked_client_pubs => UnackClientPubs, - unacked_server_pubs => UnackSerPubs, - packet_id => PackID, - client_id => ClientID, - clean_sess => CleanSess, - will_msg_defined => WillMsg =/= undefined, - exchange => Exchange, - ssl_login_name => SSLLoginName, - retainer_pid => RetainerPid, - auth_state => AuthState, - peer_addr => PeerAddr, - register_state => RegisterState, - conn_name => ConnName, - info => Info, - queues_soft_limit_exceeded => sets:size(QSLE), - qos0_messages_dropped => Qos0MsgsDropped}. - proto_integer_to_atom(3) -> ?MQTT_PROTO_V3; proto_integer_to_atom(4) -> ?MQTT_PROTO_V4. -spec proto_version_tuple(state()) -> tuple(). -proto_version_tuple(#state{proto_ver = ?MQTT_PROTO_V3}) -> +proto_version_tuple(#state{cfg = #cfg{proto_ver = ?MQTT_PROTO_V3}}) -> {3, 1, 0}; -proto_version_tuple(#state{proto_ver = ?MQTT_PROTO_V4}) -> +proto_version_tuple(#state{cfg = #cfg{proto_ver = ?MQTT_PROTO_V4}}) -> {3, 1, 1}. -maybe_increment_publisher(State = #state{published = false, - proto_ver = ProtoVer}) -> +maybe_increment_publisher(State = #state{cfg = Cfg = #cfg{published = false, + proto_ver = ProtoVer}}) -> rabbit_global_counters:publisher_created(ProtoVer), - State#state{published = true}; + State#state{cfg = Cfg#cfg{published = true}}; maybe_increment_publisher(State) -> State. -maybe_decrement_publisher(#state{published = true, - proto_ver = ProtoVer}) -> +maybe_decrement_publisher(#state{cfg = #cfg{published = true, + proto_ver = ProtoVer}}) -> rabbit_global_counters:publisher_deleted(ProtoVer); maybe_decrement_publisher(_) -> ok. @@ -1900,7 +1867,7 @@ maybe_decrement_publisher(_) -> %% Multiple subscriptions from the same connection count as one consumer. maybe_increment_consumer(#state{subscriptions = OldSubs}, #state{subscriptions = NewSubs, - proto_ver = ProtoVer}) + cfg = #cfg{proto_ver = ProtoVer}}) when map_size(OldSubs) =:= 0 andalso map_size(NewSubs) > 0 -> rabbit_global_counters:consumer_created(ProtoVer); @@ -1908,7 +1875,7 @@ maybe_increment_consumer(_, _) -> ok. maybe_decrement_consumer(#state{subscriptions = Subs, - proto_ver = ProtoVer}) + cfg = #cfg{proto_ver = ProtoVer}}) when map_size(Subs) > 0 -> rabbit_global_counters:consumer_deleted(ProtoVer); maybe_decrement_consumer(_) -> @@ -1916,15 +1883,15 @@ maybe_decrement_consumer(_) -> maybe_decrement_consumer(#state{subscriptions = OldSubs}, #state{subscriptions = NewSubs, - proto_ver = ProtoVer}) + cfg = #cfg{proto_ver = ProtoVer}}) when map_size(OldSubs) > 0 andalso map_size(NewSubs) =:= 0 -> rabbit_global_counters:consumer_deleted(ProtoVer); maybe_decrement_consumer(_, _) -> ok. -message_acknowledged(QName, #state{proto_ver = ProtoVer, - queue_states = QStates}) -> +message_acknowledged(QName, #state{queue_states = QStates, + cfg = #cfg{proto_ver = ProtoVer}}) -> case rabbit_queue_type:module(QName, QStates) of {ok, QType} -> rabbit_global_counters:messages_acknowledged(ProtoVer, QType, 1); @@ -1932,16 +1899,17 @@ message_acknowledged(QName, #state{proto_ver = ProtoVer, ok end. -message_delivered(?QUEUE_TYPE_QOS_0, false, ?QOS_0, #state{proto_ver = ProtoVer}) -> +message_delivered(?QUEUE_TYPE_QOS_0, false, ?QOS_0, + #state{cfg = #cfg{proto_ver = ProtoVer}}) -> rabbit_global_counters:messages_delivered(ProtoVer, ?QUEUE_TYPE_QOS_0, 1), %% Technically, the message is not acked to a queue at all. %% However, from a user perspective it is still auto acked because: %% "In automatic acknowledgement mode, a message is considered to be successfully %% delivered immediately after it is sent." rabbit_global_counters:messages_delivered_consume_auto_ack(ProtoVer, ?QUEUE_TYPE_QOS_0, 1); -message_delivered(QName, Redelivered, QoS, #state{proto_ver = ProtoVer, - queue_states = QStates - }) -> +message_delivered(QName, Redelivered, QoS, + #state{queue_states = QStates, + cfg = #cfg{proto_ver = ProtoVer}}) -> case rabbit_queue_type:module(QName, QStates) of {ok, QType} -> rabbit_global_counters:messages_delivered(ProtoVer, QType, 1), @@ -1960,3 +1928,68 @@ message_redelivered(true, ProtoVer, QType) -> rabbit_global_counters:messages_redelivered(ProtoVer, QType, 1); message_redelivered(_, _, _) -> ok. + +-spec format_status(state()) -> map(). +format_status( + #state{queue_states = QState, + unacked_client_pubs = UnackClientPubs, + unacked_server_pubs = UnackSerPubs, + packet_id = PackID, + subscriptions = Subscriptions, + auth_state = AuthState, + register_state = RegisterState, + queues_soft_limit_exceeded = QSLE, + qos0_messages_dropped = Qos0MsgsDropped, + cfg = #cfg{ + socket = Socket, + proto_ver = ProtoVersion, + clean_sess = CleanSess, + will_msg = WillMsg, + exchange = Exchange, + queue_qos1 = _, + published = Published, + ssl_login_name = SSLLoginName, + retainer_pid = RetainerPid, + delivery_flow = DeliveryFlow, + trace_state = TraceState, + prefetch = Prefetch, + client_id = ClientID, + conn_name = ConnName, + peer_addr = PeerAddr, + host = Host, + port = Port, + peer_host = PeerHost, + peer_port = PeerPort, + connected_at = ConnectedAt, + send_fun = _ + }}) -> + Cfg = #{socket => Socket, + proto_ver => ProtoVersion, + clean_sess => CleanSess, + will_msg_defined => WillMsg =/= undefined, + exchange => Exchange, + published => Published, + ssl_login_name => SSLLoginName, + retainer_pid => RetainerPid, + + delivery_flow => DeliveryFlow, + trace_state => TraceState, + prefetch => Prefetch, + client_id => ClientID, + conn_name => ConnName, + peer_addr => PeerAddr, + host => Host, + port => Port, + peer_host => PeerHost, + peer_port => PeerPort, + connected_at => ConnectedAt}, + #{cfg => Cfg, + queue_states => rabbit_queue_type:format_status(QState), + unacked_client_pubs => UnackClientPubs, + unacked_server_pubs => UnackSerPubs, + packet_id => PackID, + subscriptions => Subscriptions, + auth_state => AuthState, + register_state => RegisterState, + queues_soft_limit_exceeded => QSLE, + qos0_messages_dropped => Qos0MsgsDropped}. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index 99bbd9998371..d2fb4c42f128 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -528,26 +528,27 @@ format_status(Status) -> end, Status). -spec format_state(state()) -> map(). -format_state(#state{proc_state = PState, - socket = Socket, +format_state(#state{socket = Socket, proxy_socket = ProxySock, - conn_name = ConnName, await_recv = AwaitRecv, deferred_recv = DeferredRecv, - received_connect_packet = ReceivedConnectPacket, + parse_state = _, + proc_state = PState, connection_state = ConnectionState, conserve = Conserve, stats_timer = StatsTimer, keepalive = Keepalive, - parse_state = _}) -> - #{proc_state => rabbit_mqtt_processor:format_status(PState), - socket => Socket, + conn_name = ConnName, + received_connect_packet = ReceivedConnectPacket + }) -> + #{socket => Socket, proxy_socket => ProxySock, - conn_name => ConnName, await_recv => AwaitRecv, - deferred_recv => DeferredRecv, - received_connect_packet => ReceivedConnectPacket, + deferred_recv => DeferredRecv =/= undefined, + proc_state => rabbit_mqtt_processor:format_status(PState), connection_state => ConnectionState, conserve => Conserve, stats_timer => StatsTimer, - keepalive => Keepalive}. + keepalive => Keepalive, + conn_name => ConnName, + received_connect_packet => ReceivedConnectPacket}. From 1f106fcd989f7e5e6a09ac6bc290455dfc8e46df Mon Sep 17 00:00:00 2001 From: David Ansari Date: Wed, 25 Jan 2023 17:13:54 +0000 Subject: [PATCH 117/118] Fix wrong and add missing type specs --- deps/rabbit/src/rabbit_channel.erl | 4 +--- deps/rabbit/src/rabbit_trace.erl | 3 ++- deps/rabbitmq_mqtt/src/rabbit_mqtt_packet.erl | 7 +++++-- deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl | 2 +- deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl | 3 +-- deps/rabbitmq_stream/src/rabbit_stream_reader.erl | 2 +- .../src/rabbit_web_mqtt_handler.erl | 14 ++++++-------- 7 files changed, 17 insertions(+), 18 deletions(-) diff --git a/deps/rabbit/src/rabbit_channel.erl b/deps/rabbit/src/rabbit_channel.erl index e276efa273b1..2d5ccdcad422 100644 --- a/deps/rabbit/src/rabbit_channel.erl +++ b/deps/rabbit/src/rabbit_channel.erl @@ -104,9 +104,7 @@ %% same as capabilities in the reader capabilities, - %% tracing exchange resource if tracing is enabled, - %% 'none' otherwise - trace_state, + trace_state :: rabbit_trace:state(), consumer_prefetch, %% Message content size limit max_message_size, diff --git a/deps/rabbit/src/rabbit_trace.erl b/deps/rabbit/src/rabbit_trace.erl index 9fa25a0c04ab..6c556301373b 100644 --- a/deps/rabbit/src/rabbit_trace.erl +++ b/deps/rabbit/src/rabbit_trace.erl @@ -20,7 +20,8 @@ %%---------------------------------------------------------------------------- --type state() :: rabbit_types:exchange() | 'none'. +-opaque state() :: rabbit_types:exchange() | 'none'. +-export_type([state/0]). %%---------------------------------------------------------------------------- diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_packet.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_packet.erl index abaf9cccd8b6..ededed8c5b01 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_packet.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_packet.erl @@ -13,7 +13,7 @@ -export([parse/2, initial_state/0, serialise/2]). -export_type([state/0]). --opaque state() :: 'none' | {more, any()}. +-opaque state() :: none | fun(). -define(RESERVED, 0). -define(MAX_LEN, 16#fffffff). @@ -23,7 +23,10 @@ -spec initial_state() -> state(). initial_state() -> none. --spec parse(binary(), state()) -> {more, state()} | {ok, any(), any()} | {error, any()}. +-spec parse(binary(), state()) -> + {more, state()} | + {ok, mqtt_packet(), binary()} | + {error, any()}. parse(<<>>, none) -> {more, fun(Bin) -> parse(Bin, none) end}; parse(<>, none) -> diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index c57ed634a7af..6f5900aa79ff 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -53,7 +53,7 @@ ssl_login_name :: none | binary(), retainer_pid :: option(pid()), delivery_flow :: flow | noflow, - trace_state, + trace_state :: option(rabbit_trace:state()), prefetch :: non_neg_integer(), client_id :: option(binary()), conn_name :: option(binary()), diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl index d2fb4c42f128..39499892306d 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl @@ -334,10 +334,9 @@ process_received_bytes(Bytes, {ok, Packet, Rest} -> case rabbit_mqtt_processor:process_packet(Packet, ProcState) of {ok, ProcState1} -> - PS = rabbit_mqtt_packet:initial_state(), process_received_bytes( Rest, - State #state{parse_state = PS, + State #state{parse_state = rabbit_mqtt_packet:initial_state(), proc_state = ProcState1}); %% PUBLISH and more {error, unauthorized = Reason, ProcState1} -> diff --git a/deps/rabbitmq_stream/src/rabbit_stream_reader.erl b/deps/rabbitmq_stream/src/rabbit_stream_reader.erl index 1f7ea4183800..39ab2f0d12d6 100644 --- a/deps/rabbitmq_stream/src/rabbit_stream_reader.erl +++ b/deps/rabbitmq_stream/src/rabbit_stream_reader.erl @@ -82,7 +82,7 @@ heartbeater :: any(), client_properties = #{} :: #{binary() => binary()}, monitors = #{} :: #{reference() => stream()}, - stats_timer :: undefined | reference(), + stats_timer :: undefined | rabbit_event:state(), resource_alarm :: boolean(), send_file_oct :: atomics:atomics_ref(), % number of bytes sent with send_file (for metrics) diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index 7c7bcb5cac19..756382b918bb 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -30,13 +30,13 @@ -record(state, { socket, - parse_state, - proc_state, + parse_state = rabbit_mqtt_packet:initial_state() :: rabbit_mqtt_packet:state(), + proc_state :: undefined | rabbit_mqtt_processor:state(), connection_state = running :: running | blocked, conserve = false :: boolean(), - stats_timer, + stats_timer :: undefined | rabbit_event:state(), keepalive = rabbit_mqtt_keepalive:init() :: rabbit_mqtt_keepalive:state(), - conn_name, + conn_name :: undefined | binary(), received_connect_packet = false :: boolean() }). @@ -84,8 +84,7 @@ init(Req, Opts) -> true -> {?MODULE, cowboy_req:set_resp_header(<<"sec-websocket-protocol">>, <<"mqtt">>, Req), - {#state{parse_state = rabbit_mqtt_packet:initial_state(), - socket = maps:get(proxy_header, Req, undefined)}, + {#state{socket = maps:get(proxy_header, Req, undefined)}, PeerAddr}, WsOpts} end @@ -275,10 +274,9 @@ handle_data1(Data, State = #state{ parse_state = ParseState, {ok, Packet, Rest} -> case rabbit_mqtt_processor:process_packet(Packet, ProcState) of {ok, ProcState1} -> - PS = rabbit_mqtt_packet:initial_state(), handle_data1( Rest, - State#state{parse_state = PS, + State#state{parse_state = rabbit_mqtt_packet:initial_state(), proc_state = ProcState1}); {error, Reason, _} -> stop_mqtt_protocol_error(State, Reason, ConnName); From 50e25778bb52e6e30d7193a14bf47f284ec39f7e Mon Sep 17 00:00:00 2001 From: Chunyi Lyu Date: Wed, 25 Jan 2023 18:18:42 +0000 Subject: [PATCH 118/118] Adding missing function specs --- deps/rabbitmq_mqtt/src/mqtt_machine.erl | 3 +++ deps/rabbitmq_mqtt/src/mqtt_machine_v0.erl | 3 +++ deps/rabbitmq_mqtt/src/rabbit_mqtt.erl | 2 ++ deps/rabbitmq_mqtt/src/rabbit_mqtt_collector.erl | 3 +++ deps/rabbitmq_mqtt/src/rabbit_mqtt_ff.erl | 1 + deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl | 9 +++++++++ .../src/rabbit_mqtt_retained_msg_store_dets.erl | 8 ++++++++ .../src/rabbit_mqtt_retained_msg_store_ets.erl | 8 ++++++++ .../src/rabbit_mqtt_retainer_sup.erl | 1 + deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl | 1 + deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl | 6 +++++- .../src/rabbit_web_mqtt_handler.erl | 15 ++++++--------- 12 files changed, 50 insertions(+), 10 deletions(-) diff --git a/deps/rabbitmq_mqtt/src/mqtt_machine.erl b/deps/rabbitmq_mqtt/src/mqtt_machine.erl index 13e6ce7b2562..20e69a7a2a8f 100644 --- a/deps/rabbitmq_mqtt/src/mqtt_machine.erl +++ b/deps/rabbitmq_mqtt/src/mqtt_machine.erl @@ -172,6 +172,8 @@ apply(_Meta, Unknown, State) -> logger:error("MQTT Raft state machine v1 received unknown command ~tp", [Unknown]), {State, {error, {unknown_command, Unknown}}, []}. +-spec state_enter(ra_server:ra_state() | eol, state()) -> + ra_machine:effects(). state_enter(leader, State) -> %% re-request monitors for all known pids, this would clean up %% records for all connections are no longer around, e.g. right after node restart @@ -188,6 +190,7 @@ overview(#machine_state{client_ids = ClientIds, %% ========================== %% Avoids blocking the Raft leader. +-spec notify_connection(pid(), duplicate_id | decommission_node) -> pid(). notify_connection(Pid, Reason) -> spawn(fun() -> gen_server2:cast(Pid, Reason) end). diff --git a/deps/rabbitmq_mqtt/src/mqtt_machine_v0.erl b/deps/rabbitmq_mqtt/src/mqtt_machine_v0.erl index c76c268c177d..4b32ac88dd98 100644 --- a/deps/rabbitmq_mqtt/src/mqtt_machine_v0.erl +++ b/deps/rabbitmq_mqtt/src/mqtt_machine_v0.erl @@ -113,6 +113,8 @@ apply(_Meta, Unknown, State) -> logger:error("MQTT Raft state machine received an unknown command ~tp", [Unknown]), {State, {error, {unknown_command, Unknown}}, []}. +-spec state_enter(ra_server:ra_state(), state()) -> + ra_machine:effects(). state_enter(leader, State) -> %% re-request monitors for all known pids, this would clean up %% records for all connections are no longer around, e.g. right after node restart @@ -123,6 +125,7 @@ state_enter(_, _) -> %% ========================== %% Avoids blocking the Raft leader. +-spec notify_connection(pid(), duplicate_id | decommission_node) -> pid(). notify_connection(Pid, Reason) -> spawn(fun() -> gen_server2:cast(Pid, Reason) end). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl index 818c161b2892..5fb18612553d 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl @@ -41,6 +41,7 @@ start(normal, []) -> stop(_) -> rabbit_mqtt_sup:stop_listeners(). +-spec emit_connection_info_all([node()], rabbit_types:info_keys(), reference(), pid()) -> term(). emit_connection_info_all(Nodes, Items, Ref, AggregatorPid) -> case rabbit_mqtt_ff:track_client_id_in_ra() of true -> @@ -57,6 +58,7 @@ emit_connection_info_all(Nodes, Items, Ref, AggregatorPid) -> rabbit_control_misc:await_emitters_termination(Pids) end. +-spec emit_connection_info_local(rabbit_types:info_keys(), reference(), pid()) -> ok. emit_connection_info_local(Items, Ref, AggregatorPid) -> LocalPids = local_connection_pids(), emit_connection_info(Items, Ref, AggregatorPid, LocalPids). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_collector.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_collector.erl index 28c0506ef70b..5b5050d64efd 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_collector.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_collector.erl @@ -36,6 +36,7 @@ register(ServerId, ClientId, Pid) -> erlang:send_after(5000, self(), {ra_event, undefined, register_timeout}), {ok, Corr}. +-spec unregister(binary(), pid()) -> ok. unregister(ClientId, Pid) -> {ClusterName, _} = mqtt_node:server_id(), case ra_leaderboard:lookup_leader(ClusterName) of @@ -49,6 +50,7 @@ unregister(ClientId, Pid) -> list_pids() -> list(fun(#machine_state{pids = Pids}) -> maps:keys(Pids) end). +-spec list() -> term(). list() -> list(fun(#machine_state{client_ids = Ids}) -> maps:to_list(Ids) end). @@ -76,6 +78,7 @@ list(QF) -> end end. +-spec leave(binary()) -> ok | timeout | nodedown. leave(NodeBin) -> Node = binary_to_atom(NodeBin, utf8), ServerId = mqtt_node:server_id(), diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_ff.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_ff.erl index b894cee42cc4..2432cc2ac5d0 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_ff.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_ff.erl @@ -24,5 +24,6 @@ callbacks => #{enable => {mqtt_node, delete}} }}). +-spec track_client_id_in_ra() -> boolean(). track_client_id_in_ra() -> not rabbit_feature_flags:is_enabled(delete_ra_cluster_mqtt_node). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 6f5900aa79ff..58bccaa2f5fd 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -1329,9 +1329,13 @@ serialise_and_send_to_client(Packet, #state{cfg = #cfg{proto_ver = ProtoVer, [Sock, Error, Packet#mqtt_packet.fixed, Packet#mqtt_packet.variable]) end. +-spec serialise(#mqtt_packet{}, state()) -> + iodata(). serialise(Packet, #state{cfg = #cfg{proto_ver = ProtoVer}}) -> rabbit_mqtt_packet:serialise(Packet, ProtoVer). +-spec terminate(boolean(), binary(), atom(), state()) -> + ok. terminate(SendWill, ConnName, ProtoFamily, State) -> maybe_send_will(SendWill, ConnName, State), Infos = [{name, ConnName}, @@ -1427,11 +1431,15 @@ delete_queue(QName, Username) -> ok end). +-spec handle_pre_hibernate() -> ok. handle_pre_hibernate() -> erase(permission_cache), erase(topic_permission_cache), ok. +-spec handle_ra_event(register_timeout +| {applied, [{reference(), ok}]} +| {not_leader, term(), reference()}, state()) -> state(). handle_ra_event({applied, [{Corr, ok}]}, State = #state{register_state = {pending, Corr}}) -> %% success case - command was applied transition into registered state @@ -1796,6 +1804,7 @@ throttle(Conserve, Connected, #state{queues_soft_limit_exceeded = QSLE, not sets:is_empty(QSLE) orelse credit_flow:blocked(). +-spec info(rabbit_types:info_key(), state()) -> any(). info(host, #state{cfg = #cfg{host = Val}}) -> Val; info(port, #state{cfg = #cfg{port = Val}}) -> Val; info(peer_host, #state{cfg = #cfg{peer_host = Val}}) -> Val; diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_dets.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_dets.erl index b057e499ad50..8dbd06a44021 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_dets.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_dets.erl @@ -17,29 +17,37 @@ table }). +-type store_state() :: #store_state{}. +-spec new(file:name_all(), rabbit_types:vhost()) -> store_state(). new(Dir, VHost) -> Tid = open_table(Dir, VHost), #store_state{table = Tid}. +-spec recover(file:name_all(), rabbit_types:vhost()) -> + {error, uninitialized} | {ok, store_state()}. recover(Dir, VHost) -> case open_table(Dir, VHost) of {error, _} -> {error, uninitialized}; {ok, Tid} -> {ok, #store_state{table = Tid}} end. +-spec insert(binary(), mqtt_msg(), store_state()) -> ok. insert(Topic, Msg, #store_state{table = T}) -> ok = dets:insert(T, #retained_message{topic = Topic, mqtt_msg = Msg}). +-spec lookup(binary(), store_state()) -> retained_message() | not_found. lookup(Topic, #store_state{table = T}) -> case dets:lookup(T, Topic) of [] -> not_found; [Entry] -> Entry end. +-spec delete(binary(), store_state()) -> ok. delete(Topic, #store_state{table = T}) -> ok = dets:delete(T, Topic). +-spec terminate(store_state()) -> ok. terminate(#store_state{table = T}) -> ok = dets:close(T). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_ets.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_ets.erl index e2f5831de49e..3a0a7384dbb5 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_ets.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_ets.erl @@ -19,7 +19,9 @@ filename }). +-type store_state() :: #store_state{}. +-spec new(file:name_all(), rabbit_types:vhost()) -> store_state(). new(Dir, VHost) -> Path = rabbit_mqtt_util:path_for(Dir, VHost), TableName = rabbit_mqtt_util:vhost_name_to_table_name(VHost), @@ -27,6 +29,8 @@ new(Dir, VHost) -> Tid = ets:new(TableName, [set, public, {keypos, #retained_message.topic}]), #store_state{table = Tid, filename = Path}. +-spec recover(file:name_all(), rabbit_types:vhost()) -> + {error, uninitialized} | {ok, store_state()}. recover(Dir, VHost) -> Path = rabbit_mqtt_util:path_for(Dir, VHost), case ets:file2tab(Path) of @@ -35,19 +39,23 @@ recover(Dir, VHost) -> {error, _} -> {error, uninitialized} end. +-spec insert(binary(), mqtt_msg(), store_state()) -> ok. insert(Topic, Msg, #store_state{table = T}) -> true = ets:insert(T, #retained_message{topic = Topic, mqtt_msg = Msg}), ok. +-spec lookup(binary(), store_state()) -> retained_message() | not_found. lookup(Topic, #store_state{table = T}) -> case ets:lookup(T, Topic) of [] -> not_found; [Entry] -> Entry end. +-spec delete(binary(), store_state()) -> ok. delete(Topic, #store_state{table = T}) -> true = ets:delete(T, Topic), ok. +-spec terminate(store_state()) -> ok. terminate(#store_state{table = T, filename = Path}) -> ok = ets:tab2file(T, Path, [{extended_info, [object_count]}]). diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer_sup.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer_sup.erl index 9e8c983a9b5b..eadaab7ca272 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer_sup.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer_sup.erl @@ -17,6 +17,7 @@ start_link(SupName) -> supervisor:start_link(SupName, ?MODULE, []). +-spec child_for_vhost(rabbit_types:vhost()) -> pid(). child_for_vhost(VHost) when is_binary(VHost) -> case rabbit_mqtt_retainer_sup:start_child(VHost) of {ok, Pid} -> Pid; diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl index 5d1a49b7b136..a04ef73cb91f 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl @@ -70,6 +70,7 @@ init([{Listeners, SslListeners0}]) -> ) ]}}. +-spec stop_listeners() -> ok. stop_listeners() -> _ = rabbit_networking:stop_ranch_listener_of_protocol(?TCP_PROTOCOL), _ = rabbit_networking:stop_ranch_listener_of_protocol(?TLS_PROTOCOL), diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl index fa10d5a98487..d098d3ff931a 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl @@ -133,6 +133,7 @@ to_mqtt(T0) -> T2 = string:replace(T1, ".", "/", all), erlang:iolist_to_binary(T2). +-spec env(atom()) -> any(). env(Key) -> case application:get_env(?APP_NAME, Key) of {ok, Val} -> coerce_env_value(Key, Val); @@ -145,6 +146,8 @@ coerce_env_value(exchange, Val) -> rabbit_data_coercion:to_binary(Val); coerce_env_value(vhost, Val) -> rabbit_data_coercion:to_binary(Val); coerce_env_value(_, Val) -> Val. +-spec table_lookup(rabbit_framing:amqp_table() | undefined, binary()) -> + tuple() | undefined. table_lookup(undefined, _Key) -> undefined; table_lookup(Table, Key) -> @@ -156,13 +159,14 @@ vhost_name_to_dir_name(VHost, Suffix) -> <> = erlang:md5(VHost), "mqtt_retained_" ++ rabbit_misc:format("~36.16.0b", [Num]) ++ Suffix. +-spec path_for(file:name_all(), rabbit_types:vhost()) -> file:filename_all(). path_for(Dir, VHost) -> filename:join(Dir, vhost_name_to_dir_name(VHost)). +-spec path_for(file:name_all(), rabbit_types:vhost(), string()) -> file:filename_all(). path_for(Dir, VHost, Suffix) -> filename:join(Dir, vhost_name_to_dir_name(VHost, Suffix)). - -spec vhost_name_to_table_name(rabbit_types:vhost()) -> atom(). vhost_name_to_table_name(VHost) -> diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl index 756382b918bb..72f52b3977af 100644 --- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl +++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl @@ -28,17 +28,19 @@ upgrade/5, takeover/7]). +-type option(T) :: undefined | T. + -record(state, { - socket, + socket :: {rabbit_proxy_socket, any(), any()} | rabbit_net:socket(), parse_state = rabbit_mqtt_packet:initial_state() :: rabbit_mqtt_packet:state(), proc_state :: undefined | rabbit_mqtt_processor:state(), connection_state = running :: running | blocked, conserve = false :: boolean(), - stats_timer :: undefined | rabbit_event:state(), + stats_timer :: option(rabbit_event:state()), keepalive = rabbit_mqtt_keepalive:init() :: rabbit_mqtt_keepalive:state(), - conn_name :: undefined | binary(), + conn_name :: option(binary()), received_connect_packet = false :: boolean() - }). + }). -type state() :: #state{}. @@ -66,10 +68,6 @@ takeover(Parent, Ref, Socket, Transport, Opts, Buffer, {Handler, {HandlerState, {Handler, {HandlerState#state{socket = Sock}, PeerAddr}}). %% cowboy_websocket --spec init(Req, any()) -> - {ok | module(), Req, any()} | - {module(), Req, any(), any()} - when Req::cowboy_req:req(). init(Req, Opts) -> case cowboy_req:parse_header(<<"sec-websocket-protocol">>, Req) of undefined -> @@ -225,7 +223,6 @@ websocket_info(Msg, State) -> ?LOG_WARNING("Web MQTT: unexpected message ~tp", [Msg]), {[], State, hibernate}. --spec terminate(any(), cowboy_req:req(), any()) -> ok. terminate(_Reason, _Req, #state{proc_state = undefined}) -> ok; terminate(Reason, Request, #state{} = State) ->