Skip to content

Under certain condition client connection can block for a long period #236

Closed
@dimas

Description

@dimas

When you have a low heartbeat interval you normally expect RabbitMQ client to quickly recognise connection loss. However, there is a scenario when it is not the case. Pre-requisites for that are:

  • AMQP over SSL. Won't happen on a plain socket
  • some weird network condition causing certain packet to get lost
  • unfortunate timing. Client should be sending message around that "network event". To increase probability and reproduce it more reliably I just made client send a message every second.

So the client just periodically publishes a message. Then happens that mysterious network even that by the looks of it blocks packets passing in one direction, from client to the server, while allowing packets in opposite direction. (There can be more than that as some active network equipment is definitely involved and I blame it for the root cause)

Still, you would expect, a few missing heartbeats (which I set for 30 seconds) and connection will be killed. Not exactly.

  1. The RabbitMQ server indeed very quickly detect missing heartbeats and closes the connection
  2. The client basically hangs for 15 minutes (!) and only after that time it declares connection dead.

There are three threads involved. So my interpretation of what happens is that

  • Application thread. It tries to publish message and goes all the way down to SSL socket write+flush. Socket's write/flush blocks because client cannot get confirmation from the remote TCP stack.
"pinger-1-Thread-1" daemon prio=10 tid=0x9f743688 nid=0x527 runnable [0x9f0fe000]
   java.lang.Thread.State: RUNNABLE
    at java.net.SocketOutputStream.socketWrite0(Native Method)
    at java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:109)
    at java.net.SocketOutputStream.write(SocketOutputStream.java:153)
    at sun.security.ssl.OutputRecord.writeBuffer(OutputRecord.java:375)
    at sun.security.ssl.OutputRecord.write(OutputRecord.java:363)
    at sun.security.ssl.SSLSocketImpl.writeRecordInternal(SSLSocketImpl.java:801)
    at sun.security.ssl.SSLSocketImpl.writeRecord(SSLSocketImpl.java:773)
    at sun.security.ssl.AppOutputStream.write(AppOutputStream.java:119)
    - locked <0xa5870a10> (a sun.security.ssl.AppOutputStream)
    at java.io.BufferedOutputStream.flushBuffer(BufferedOutputStream.java:82)
    at java.io.BufferedOutputStream.flush(BufferedOutputStream.java:140)
    - locked <0xa5872a38> (a java.io.BufferedOutputStream)
    at java.io.DataOutputStream.flush(DataOutputStream.java:123)
    at com.rabbitmq.client.impl.SocketFrameHandler.flush(SocketFrameHandler.java:149)
    at com.rabbitmq.client.impl.AMQConnection.flush(AMQConnection.java:521)
    at com.rabbitmq.client.impl.AMQCommand.transmit(AMQCommand.java:124)
    at com.rabbitmq.client.impl.AMQChannel.quiescingTransmit(AMQChannel.java:333)
    - locked <0xa06a4370> (a java.lang.Object)
    at com.rabbitmq.client.impl.AMQChannel.transmit(AMQChannel.java:309)
    - locked <0xa06a4370> (a java.lang.Object)
    at com.rabbitmq.client.impl.ChannelN.basicPublish(ChannelN.java:656)
    at com.rabbitmq.client.impl.ChannelN.basicPublish(ChannelN.java:639)
    at com.rabbitmq.client.impl.ChannelN.basicPublish(ChannelN.java:630)
...

Note that write() timeout unlike read() timeout is not controlled by setSoTimeout() but chosen by TCP stack based on calculated round-trip-time and number of retries.

  • HeartbeatSender thread is here only for completeness. It does not play any role in this race condition as it just tries to write to the socket and gets completely blocked because lock to output stream is currently held by application thread:
"pool-1-thread-2" prio=10 tid=0x9f74fb48 nid=0x525 waiting for monitor entry [0x9f3fe000]
   java.lang.Thread.State: BLOCKED (on object monitor)
    at java.io.BufferedOutputStream.write(BufferedOutputStream.java:94)
    - waiting to lock <0xa5872a38> (a java.io.BufferedOutputStream)
    at java.io.DataOutputStream.writeByte(DataOutputStream.java:153)
    at com.rabbitmq.client.impl.Frame.writeTo(Frame.java:188)
    at com.rabbitmq.client.impl.SocketFrameHandler.writeFrame(SocketFrameHandler.java:144)
    - locked <0xa5872a50> (a java.io.DataOutputStream)
    at com.rabbitmq.client.impl.HeartbeatSender$HeartbeatRunnable.run(HeartbeatSender.java:138)
    at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
    at java.util.concurrent.FutureTask$Sync.innerRunAndReset(FutureTask.java:351)
    at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:178)
    at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:165)
    at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:267)
    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1146)
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
    at java.lang.Thread.run(Thread.java:701)
  • finally, there is connection main loop thread thread that is supposed to detect missing heartbeats from the remote side and kill the connection. Unfortunately, as it reads from the socket, it receives graceful connection closure notification from the remote SSL stack (when server closes the connection because of missing heartbeat) so it starts closing connection on the local side too. But closing it apparently involves notifying the remote side of the action. And this makes the main thread to attempt writing to the same socket. And, just to remind, all the appropriate locks are held by the application thread
"AMQP Connection 10.19.36.12:5671" prio=10 tid=0x9f70c578 nid=0x524 waiting on condition [0x9f57e000]
   java.lang.Thread.State: WAITING (parking)
    at sun.misc.Unsafe.park(Native Method)
    - parking to wait for  <0xa5884f88> (a java.util.concurrent.locks.ReentrantLock$NonfairSync)
    at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186)
    at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:838)
    at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:871)
    at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1201)
    at java.util.concurrent.locks.ReentrantLock$NonfairSync.lock(ReentrantLock.java:214)
    at java.util.concurrent.locks.ReentrantLock.lock(ReentrantLock.java:290)
    at sun.security.ssl.SSLSocketImpl.writeRecord(SSLSocketImpl.java:771)
    at sun.security.ssl.SSLSocketImpl.writeRecord(SSLSocketImpl.java:644)
    at sun.security.ssl.SSLSocketImpl.sendAlert(SSLSocketImpl.java:1819)
    at sun.security.ssl.SSLSocketImpl.warning(SSLSocketImpl.java:1668)
    at sun.security.ssl.SSLSocketImpl.closeInternal(SSLSocketImpl.java:1470)
    - locked <0xa5870948> (a sun.security.ssl.SSLSocketImpl)
    at sun.security.ssl.SSLSocketImpl.recvAlert(SSLSocketImpl.java:1763)
    at sun.security.ssl.SSLSocketImpl.readRecord(SSLSocketImpl.java:1024)
    - locked <0xa5870948> (a sun.security.ssl.SSLSocketImpl)
    - locked <0xa58709d8> (a java.lang.Object)
    at sun.security.ssl.SSLSocketImpl.readDataRecord(SSLSocketImpl.java:840)
    at sun.security.ssl.AppInputStream.read(AppInputStream.java:94)
    - locked <0xa587d330> (a sun.security.ssl.AppInputStream)
    at java.io.BufferedInputStream.fill(BufferedInputStream.java:235)
    at java.io.BufferedInputStream.read(BufferedInputStream.java:254)
    - locked <0xa5885000> (a java.io.BufferedInputStream)
    at java.io.DataInputStream.readUnsignedByte(DataInputStream.java:288)
    at com.rabbitmq.client.impl.Frame.readFrom(Frame.java:94)
    at com.rabbitmq.client.impl.SocketFrameHandler.readFrame(SocketFrameHandler.java:138)
    - locked <0xa5881498> (a java.io.DataInputStream)
    at com.rabbitmq.client.impl.AMQConnection$MainLoop.run(AMQConnection.java:541)
    at java.lang.Thread.run(Thread.java:701)

15 minutes later application thread received a SocketTimeout exception from java.net.SocketOutputStream.socketWrite0() and everything finally aborts.

It does look as a bug in SSLSocket in the first place - the moment they start calling write() from read() method, they cannot guarantee that timeout configured on the socket will work. (Again, my assumption is that write timeout is not configurable). And this can affect connections of any type not just AMQP.
However I am not overly optimistic that issue like that could be easily fixed. After all, it was in the standard library from the day one.

So the question is more like - can something be done at rabbitmq client level instead?

PS: line numbers in stack traces are for JRE 1.6.0_27 (but from the looks of it, Java 8 library is not much different)

Metadata

Metadata

Assignees

No one assigned

    Labels

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions