Skip to content
This repository has been archived by the owner on Nov 18, 2020. It is now read-only.

heartbeat=0 is negotiated but not actually respected #112

Closed
dch opened this issue Nov 19, 2018 · 5 comments
Closed

heartbeat=0 is negotiated but not actually respected #112

dch opened this issue Nov 19, 2018 · 5 comments

Comments

@dch
Copy link

dch commented Nov 19, 2018

issue

An erlang client (or elixir that depends on this module) cannot be configured to run with disabled amqp-level heartbeats, and always accepts the default sent by the server - e.g. as recommended by cloudamqp. NB whether that is a good choice is not the intended purpose of this issue ;-)

In addition, I speculate that this same bug in the erlang client, causes the heartbeat sender on the server side to be disabled (as the client requested), but the erlang gen_server on the client side is incorrectly waiting the negotiated 2xHB duration for heartbeats inside rabbitmq-common, that cascades a timeout & subsequent socket termination, as the awaited heartbeat never comes.

rabbitmq docs

The client must be configured to request heartbeats. In RabbitMQ versions 3.0 and higher, the broker will attempt to negotiate heartbeats by default (although the client can still veto them). The timeout is in seconds, and default value is 60.

expected behaviour

Connecting to amqp://..?heartbeat=37 does show expected behaviour - client & server agree on the heartbeat frequency - as seen via tcpdump & validated in client amqp_connection:info(Conn, [heartbeat]). for client, and server in connection web ui.

Connecting to amqp://..?heartbeat=0 should end up with heartbeat=0 on both client & in rabbitmq web ui.

unexpected behaviour

Actually, amqp://..?heartbeat=0 does not end up with a configured HB=0, but only inherits whatever the server has recommended.

In addition, despite the wire protocol negotiation being correct, the actual heartbeats are not sent at the expected 1/2 heartbeat time, (presumably as the requested 0 HB setting is propagated into rabbitmq-common's heartbeat timer on the server side, even though this doesn't match what has been agreed on the wire).

Thus, from the client's perspective, after the server has not sent any heartbeats, the gen_server hits a timeout, and the erlang client then cleans up, requesting to close the TCP connection and the server responds in kind.

possible bug

https://github.com/rabbitmq/rabbitmq-erlang-client/blob/master/src/amqp_network_connection.erl#L236-L243 looks like it makes incorrect comparisons in this specific case:

    [ChannelMax, Heartbeat, FrameMax] =
        lists:zipwith(fun (Client, Server) when Client =:= 0; Server =:= 0 ->
                              lists:max([Client, Server]);
                          (Client, Server) ->
                              lists:min([Client, Server])
                      end,
                      [ClientChannelMax, ClientHeartbeat, ClientFrameMax],
[ServerChannelMax, ServerHeartbeat, ServerFrameMax]),
3> [ClientChannelMax, ClientHeartbeat, ClientFrameMax] = [0,0,0].
[0,0,0]
4> [ServerChannelMax, ServerHeartbeat, ServerFrameMax] = [0,10,0].
[0,10,0]
5> lists:zipwith(fun (Client, Server) when Client =:= 0; Server =:= 0 ->
5>                               lists:max([Client, Server]);
5>                           (Client, Server) ->
5>                               lists:min([Client, Server])
5>                       end,
5>                       [ClientChannelMax, ClientHeartbeat, ClientFrameMax],
5> [ServerChannelMax, ServerHeartbeat, ServerFrameMax]).
[0,10,0]

I think we should see [0,0,0] here, for example.

speculation

As mentioned above, I think this is therefore wrong on both server and client for RabbitMQ as the library is shared in both systems. In addition, I think this incorrect setting propagates further, into rabbit_common:start_timer, causing the negotiated AMQP heartbeats (which we didn't request!) not to be send from the server (as the amqp params has heartbeat=0 which doesn't match what is agreed on the wire, as confirmed by tcpdump), but the client is still expecting to receive a heartbeat, and when it doesn't receive one, the TCP connection is reset, which causes the entire connection to fail. I think this hits codepath here https://github.com/rabbitmq/rabbitmq-common/blob/master/src/rabbit_heartbeat.erl#L109-L113 with the 0 pattern match.

This ultimately causes the client to send a FIN/ACK requesting to close the channel as the connection has not received the expected heartbeats from the server -- this is the "matching bug" on the server end causing problems, I think.

Anyway, I have a tcpdump that I can share privately if needed, email/irc is fine, but it's easy enough to reproduce as follows:

test setup

  • git clone this repo and gmake deps app shell
application:ensure_all_started(lager).
application:ensure_all_started(amqp_client).
{ok, Params} = amqp_uri:parse("amqp://localhost?heartbeat=0").
{ok, Conn} = amqp_connection:start(Params, <<"hb_0">>).
{ok, Chan} = amqp_connection:open_channel(Conn).
amqp_connection:info(Conn,  [heartbeat]).

tcpdump of succesful tune negotiation

Advanced Message Queueing Protocol
    Type: Method (1)
    Channel: 0
    Length: 12
    Class: Connection (10)
    Method: Tune-Ok (31)
    Arguments
        Channel-Max: 200
        Frame-Max: 16384
        Heartbeat: 120

tcpdump of final RST conversation at end of stream

client->server

Frame 16: 66 bytes on wire (528 bits), 66 bytes captured (528 bits)
Ethernet II, Src: SuperMic_67:e1:38 (ac:1f:6b:67:e1:38), Dst: LannerEl_4d:67:99 (00:90:0b:4d:67:99)
Internet Protocol Version 4, Src: 172.16.1.14 (172.16.1.14), Dst: 35.205.27.21 (35.205.27.21)
Transmission Control Protocol, Src Port: 36664, Dst Port: 5672, Seq: 519, Ack: 539, Len: 0
    Source Port: 36664
    Destination Port: 5672
    <Source or Destination Port: 36664>
    <Source or Destination Port: 5672>
    [Stream index: 0]
    [TCP Segment Len: 0]
    Sequence number: 519    (relative sequence number)
    [Next sequence number: 519    (relative sequence number)]
    Acknowledgment number: 539    (relative ack number)
    1000 .... = Header Length: 32 bytes (8)
    Flags: 0x011 (FIN, ACK)
        000. .... .... = Reserved: Not set
        ...0 .... .... = Nonce: Not set
        .... 0... .... = Congestion Window Reduced (CWR): Not set
        .... .0.. .... = ECN-Echo: Not set
        .... ..0. .... = Urgent: Not set
        .... ...1 .... = Acknowledgment: Set
        .... .... 0... = Push: Not set
        .... .... .0.. = Reset: Not set
        .... .... ..0. = Syn: Not set
        .... .... ...1 = Fin: Set
        [TCP Flags: ·······A···F]
    Window size value: 8192
    [Calculated window size: 4194304]
    [Window size scaling factor: 512]
    Checksum: 0xec26 [unverified]
    [Checksum Status: Unverified]
    Urgent pointer: 0
    Options: (12 bytes), No-Operation (NOP), No-Operation (NOP), Timestamps
        TCP Option - No-Operation (NOP)
            Kind: No-Operation (1)
        TCP Option - No-Operation (NOP)
            Kind: No-Operation (1)
        TCP Option - Timestamps: TSval 103686017, TSecr 998118205
            Kind: Time Stamp Option (8)
            Length: 10
            Timestamp value: 103686017
            Timestamp echo reply: 998118205
    [Timestamps]
        [Time since first frame in this TCP stream: 60.228733000 seconds]
        [Time since previous frame in this TCP stream: 59.915805000 seconds]

server->client

Frame 17: 66 bytes on wire (528 bits), 66 bytes captured (528 bits)
Ethernet II, Src: LannerEl_4d:67:99 (00:90:0b:4d:67:99), Dst: SuperMic_67:e1:38 (ac:1f:6b:67:e1:38)
Internet Protocol Version 4, Src: 35.205.27.21 (35.205.27.21), Dst: 172.16.1.14 (172.16.1.14)
Transmission Control Protocol, Src Port: 5672, Dst Port: 36664, Seq: 539, Ack: 520, Len: 0
    Source Port: 5672
    Destination Port: 36664
    <Source or Destination Port: 5672>
    <Source or Destination Port: 36664>
    [Stream index: 0]
    [TCP Segment Len: 0]
    Sequence number: 539    (relative sequence number)
    [Next sequence number: 539    (relative sequence number)]
    Acknowledgment number: 520    (relative ack number)
    1000 .... = Header Length: 32 bytes (8)
    Flags: 0x014 (RST, ACK)
        000. .... .... = Reserved: Not set
        ...0 .... .... = Nonce: Not set
        .... 0... .... = Congestion Window Reduced (CWR): Not set
        .... .0.. .... = ECN-Echo: Not set
        .... ..0. .... = Urgent: Not set
        .... ...1 .... = Acknowledgment: Set
        .... .... 0... = Push: Not set
        .... .... .1.. = Reset: Set
        .... .... ..0. = Syn: Not set
        .... .... ...0 = Fin: Not set
        [TCP Flags: ·······A·R··]
    Window size value: 2816
    [Calculated window size: 2816]
    [Window size scaling factor: 1]
    Checksum: 0xc02a [unverified]
    [Checksum Status: Unverified]
    Urgent pointer: 0
    Options: (12 bytes), No-Operation (NOP), No-Operation (NOP), Timestamps
        TCP Option - No-Operation (NOP)
            Kind: No-Operation (1)
        TCP Option - No-Operation (NOP)
            Kind: No-Operation (1)
        TCP Option - Timestamps: TSval 998133217, TSecr 103686017
            Kind: Time Stamp Option (8)
            Length: 10
            Timestamp value: 998133217
            Timestamp echo reply: 103686017
    [SEQ/ACK analysis]
    [Timestamps]
        [Time since first frame in this TCP stream: 60.261940000 seconds]
        [Time since previous frame in this TCP stream: 0.033207000 seconds]

erlang shell session showing bug and timeout

These are logs from client side of reproduced issue (with connection loss after 60s as cloudamqp has a 120s heartbeat default).

rabbitmq-erlang-client> gmake deps app shell
gmake[1]: Entering directory '/repos/rabbitmq-erlang-client/deps/rabbit_common'
gmake[2]: Entering directory '/repos/rabbitmq-erlang-client/deps/rabbitmq_codegen'
gmake[2]: Leaving directory '/repos/rabbitmq-erlang-client/deps/rabbitmq_codegen'
gmake[2]: Entering directory '/repos/rabbitmq-erlang-client/deps/lager'
gmake[3]: Entering directory '/repos/rabbitmq-erlang-client/deps/goldrush'
gmake[3]: Leaving directory '/repos/rabbitmq-erlang-client/deps/goldrush'
 DEPEND lager.d
gmake[2]: Leaving directory '/repos/rabbitmq-erlang-client/deps/lager'
gmake[2]: Entering directory '/repos/rabbitmq-erlang-client/deps/jsx'
gmake[2]: Leaving directory '/repos/rabbitmq-erlang-client/deps/jsx'
gmake[2]: Entering directory '/repos/rabbitmq-erlang-client/deps/ranch'
gmake[2]: Leaving directory '/repos/rabbitmq-erlang-client/deps/ranch'
gmake[2]: Entering directory '/repos/rabbitmq-erlang-client/deps/ranch_proxy_protocol'
gmake[2]: Leaving directory '/repos/rabbitmq-erlang-client/deps/ranch_proxy_protocol'
gmake[2]: Entering directory '/repos/rabbitmq-erlang-client/deps/recon'
gmake[2]: Leaving directory '/repos/rabbitmq-erlang-client/deps/recon'
gmake[1]: Leaving directory '/repos/rabbitmq-erlang-client/deps/rabbit_common'
 DEPEND amqp_client.d
 GEN    shell
Erlang/OTP 21 [erts-10.1.1] [source] [64-bit] [smp:8:8] [ds:8:8:10] [async-threads:1] [hipe] [dtrace]
Eshell V10.1.1  (abort with ^G)
1> application:ensure_all_started(lager), application:ensure_all_started(amqp_client), f(), {ok, Params} = amqp_uri:parse("amqp://...?heartbeat=0"), logger:warning("STARTING NOW"), {ok, Conn} = amqp_connection:start(Params, <<"hb_0">>), {ok, Chan} = amqp_connection:open_channel(Conn), amqp_connection:info(Conn,  [heartbeat]).

=WARNING REPORT==== 19-Nov-2018::00:29:12.837544 ===
STARTING NOW
[{heartbeat,120}]
2> =ERROR REPORT==== 19-Nov-2018::00:30:13.054977 ===
** Generic server <0.154.0> terminating 
** Last message in was {inet_async,#Port<0.6>,5,{error,timeout}}
** When Server state == {state,#Port<0.6>,<0.148.0>,<0.152.0>,
                               {method,rabbit_framing_amqp_0_9_1},
                               {expecting_header,<<>>}}
** Reason for termination == 
** {socket_error,timeout}

=ERROR REPORT==== 19-Nov-2018::00:30:13.055066 ===
** Generic server <0.148.0> terminating 
** Last message in was {socket_error,timeout}
** When Server state == {state,amqp_network_connection,
                            {state,#Port<0.6>,
                                <<"client 172.16.1.14:37471 -> 35.205.27.21:5672">>,
                                120,<0.153.0>,16384,<0.147.0>,undefined,false},
                            <0.152.0>,
                            {amqp_params_network,<<"...">>,
                                <<"...">>,
                                <<"...">>,"....cloudamqp.com",5672,
                                2047,0,0,60000,none,
                                [#Fun<amqp_uri.12.131604370>,
                                 #Fun<amqp_uri.12.131604370>],
                                [{<<"connection_name">>,longstr,<<"hb_0">>}],
                                []},
                            200,
                            [{<<"capabilities">>,table,
                              [{<<"publisher_confirms">>,bool,true},
                               {<<"exchange_exchange_bindings">>,bool,true},
                               {<<"basic.nack">>,bool,true},
                               {<<"consumer_cancel_notify">>,bool,true},
                               {<<"connection.blocked">>,bool,true},
                               {<<"consumer_priorities">>,bool,true},
                               {<<"authentication_failure_close">>,bool,true},
                               {<<"per_consumer_qos">>,bool,true},
                               {<<"direct_reply_to">>,bool,true}]},
                             {<<"cluster_name">>,longstr,<<"...">>},
                             {<<"copyright">>,longstr,
                              <<"Copyright (C) 2007-2018 Pivotal Software, Inc.">>},
                             {<<"information">>,longstr,
                              <<"Licensed under the MPL.  See http://www.rabbitmq.com/">>},
                             {<<"platform">>,longstr,
                              <<"Erlang/OTP 20.3.8.7">>},
                             {<<"product">>,longstr,<<"RabbitMQ">>},
                             {<<"version">>,longstr,<<"3.6.16">>}],
                            none,false}
** Reason for termination == 
** {socket_error,timeout}

=CRASH REPORT==== 19-Nov-2018::00:30:13.055693 ===
  crasher:
    initial call: amqp_main_reader:init/1
    pid: <0.154.0>
    registered_name: []
    exception exit: {socket_error,timeout}
      in function  gen_server:handle_common_reply/8 (gen_server.erl, line 751)
    ancestors: [<0.147.0>,<0.146.0>,amqp_sup,<0.144.0>] 
    message_queue_len: 0
    messages: []
    links: [<0.147.0>]
    dictionary: [{process_name,
                      {amqp_main_reader,
                          <<"client 172.16.1.14:37471 -> 35.205.27.21:5672">>}},
                  {gen_server_call_timeout,60000}]  ***********************************************************************
    trap_exit: false
    status: running
    heap_size: 1598
    stack_size: 27
    reductions: 10297
  neighbours:

=CRASH REPORT==== 19-Nov-2018::00:30:13.059347 ===
  crasher:
    initial call: amqp_gen_connection:init/1
    pid: <0.148.0>
    registered_name: []
    exception exit: {socket_error,timeout}
      in function  gen_server:handle_common_reply/8 (gen_server.erl, line 751)
    ancestors: [<0.146.0>,amqp_sup,<0.144.0>]
    message_queue_len: 0
    messages: []
    links: [<0.146.0>,#Port<0.6>]
    dictionary: [{process_name,
                      {amqp_gen_connection,
                          <<"client 172.16.1.14:37471 -> 35.205.27.21:5672">>}},
                  {gen_server_call_timeout,60000}]
    trap_exit: true
    status: running
    heap_size: 6772
    stack_size: 27
    reductions: 211709
  neighbours:

00:30:13.060 [error] gen_server <0.154.0> terminated with reason: {socket_error,timeout}
00:30:13.060 [error] gen_server <0.148.0> terminated with reason: {socket_error,timeout}
00:30:13.060 [error] CRASH REPORT Process <0.154.0> with 0 neighbours exited with reason: {socket_error,timeout} in gen_server:handle_common_reply/8 line 751
00:30:13.060 [error] CRASH REPORT Process <0.148.0> with 0 neighbours exited with reason: {socket_error,timeout} in gen_server:handle_common_reply/8 line 751
00:30:13.065 [error] Supervisor {<0.147.0>,amqp_connection_type_sup} had child main_reader started with amqp_main_reader:start_link(#Port<0.6>, <0.148.0>, <0.152.0>, {method,rabbit_framing_amqp_0_9_1}, <<"client 172.16.1.14:37471 -> 35.205.27.21:5672">>) at <0.154.0> exit with reason {socket_error,timeout} in context child_terminated
00:30:13.065 [error] Supervisor {<0.147.0>,amqp_connection_type_sup} had child main_reader started with amqp_main_reader:start_link(#Port<0.6>, <0.148.0>, <0.152.0>, {method,rabbit_framing_amqp_0_9_1}, <<"client 172.16.1.14:37471 -> 35.205.27.21:5672">>) at <0.154.0> exit with reason reached_max_restart_intensity in context shutdown
00:30:13.066 [error] Supervisor {<0.146.0>,amqp_connection_sup} had child connection started with amqp_gen_connection:start_link(<0.147.0>, {amqp_params_network,<<"hvvnfues">>,<<"722qGIKqIvvTfGa--Zj23Lq1_fpy50w7">>,<<"hvvnfues">>,"sheep...",...}) at <0.148.0> exit with reason {socket_error,timeout} in context child_terminated
00:30:13.067 [error] Supervisor {<0.146.0>,amqp_connection_sup} had child connection started with amqp_gen_connection:start_link(<0.147.0>, {amqp_params_network,<<"hvvnfues">>,<<"722qGIKqIvvTfGa--Zj23Lq1_fpy50w7">>,<<"hvvnfues">>,"sheep...",...}) at <0.148.0> exit with reason reached_max_restart_intensity in context shutdown
@michaelklishin
Copy link
Member

michaelklishin commented Nov 19, 2018

Thank you for the detailed analysis. You haven't specified what version of RabbitMQ was used but my guess is that it was 3.7.8 at most. One part of your findings — the server not always using an interval computed from the negotiated value — seems to be addressed by rabbitmq/rabbitmq-common#278, shipped in 3.7.9 two days ago.

The other part in this client matches https://github.com/rabbitmq/rabbitmq-java-client/blame/master/src/main/java/com/rabbitmq/client/impl/AMQConnection.java#L579, which has been around for 11 years and we are not going to change.

In most environments (CloudAMQP is an exception), users who set heartbeats to 0 do not do so intentionally or do not necessarily understand the implications. If you use TCP keepalives then you have two options:

  • Configure the server to negotiate with the value of 0.
  • Use a very high value (e.g. 30 minutes) which effectively disables heartbeats.

@michaelklishin
Copy link
Member

I am updating the docs to clarify the negotiation process and how heartbeats can be disabled. This helped me go through the negotiation code one more time and I agree that the client code never worked as the docs (as of 3.0) claim it to be. That sucks but breaking client changes are a painful process to go through since there are so many of them and RabbitMQ users have come to expect any reasonably recent client version to work with any reasonably recent server version. So our compatibility story with clients is both a blessing and a curse :(

Given that most users do not want heartbeats disabled, I still lean towards "we won't change the negotiation process". It's just not a commonly enough reported issue.

If you use a dedicated plan CloudAMQP support likely can change the server config for you.

I'll ask other members of our team to take a look and share their opinion.

michaelklishin added a commit to rabbitmq/rabbitmq-website that referenced this issue Nov 19, 2018
And explain further how to disable heartbeats.

See rabbitmq/rabbitmq-erlang-client#112 for background.
@dch
Copy link
Author

dch commented Nov 19, 2018

thanks Michael. For reference the client here is the erlang one and I’ve tested this against both 3.6.12, .14 and my 3.7.8 install. I agree that changing the server side is not an option and I’m fine with heartbeats in erlang and other clients that can support it. They’re there for a good reason.

@michaelklishin
Copy link
Member

michaelklishin commented Nov 19, 2018

To clarify: the server doesn't seem to be the problematic bit to me. It's the client logic that we tried to unify across all clients (the Java client is used here because it is the reference implementation for us) that negotiates to a max or min value, and never 0 unless both values are 0.

As far as the protocol goes, in AMQP 0-9-1 it's the client that has the final call.

@lukebakken
Copy link
Contributor

never 0 unless both values are 0

👍 to this

Sign up for free to subscribe to this conversation on GitHub. Already have an account? Sign in.
Projects
None yet
Development

No branches or pull requests

3 participants