Uploaded image for project: 'Apache Avro'
  1. Apache Avro
  2. AVRO-1293

NettyTransceiver: Deadlock can occur when different threads call getChannel() and close() concurrently

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Major
    • Resolution: Fixed
    • 1.7.4
    • 1.7.5
    • java
    • None

    Description

      While testing patches for AVRO-1292 I stumbled upon a deadlock in NettyTransceiver that I've never seen before. It happened when close() was called at roughly the same time that another thread was trying to invoke an RPC. Here are the stack traces for the two threads that were involved in the deadlock:

      "Thread 1: Writer":
              at org.apache.avro.ipc.NettyTransceiver.getChannel(NettyTransceiver.java:267)
              - waiting to lock <0x000000067b1a7bc8> (a java.lang.Object)
              at org.apache.avro.ipc.NettyTransceiver.getRemoteName(NettyTransceiver.java:391)
              at org.apache.avro.ipc.Requestor.writeHandshake(Requestor.java:202)
              at org.apache.avro.ipc.Requestor.access$3(Requestor.java:198)
              at org.apache.avro.ipc.Requestor$Request.getBytes(Requestor.java:478)
              at org.apache.avro.ipc.Requestor.request(Requestor.java:147)
              at org.apache.avro.ipc.Requestor.request(Requestor.java:101)
              at org.apache.avro.ipc.specific.SpecificRequestor.invoke(SpecificRequestor.java:88)
      
      "Thread 2: Closer":
              at sun.misc.Unsafe.park(Native Method)
              - parking to wait for  <0x000000067aedea90> (a java.util.concurrent.locks.ReentrantReadWriteLock$NonfairSync)
              at java.util.concurrent.locks.LockSupport.park(LockSupport.java:156)
              at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:811)
              at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:842)
              at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1178)
              at java.util.concurrent.locks.ReentrantReadWriteLock$WriteLock.lock(ReentrantReadWriteLock.java:807)
              at org.apache.avro.ipc.NettyTransceiver.disconnect(NettyTransceiver.java:307)
              at org.apache.avro.ipc.NettyTransceiver.access$2(NettyTransceiver.java:293)
              at org.apache.avro.ipc.NettyTransceiver$NettyClientAvroHandler.handleUpstream(NettyTransceiver.java:542)
              at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564)
              at org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:783)
              at org.jboss.netty.handler.codec.frame.FrameDecoder.cleanup(FrameDecoder.java:348)
              at org.jboss.netty.handler.codec.frame.FrameDecoder.channelClosed(FrameDecoder.java:232)
              at org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:98)
              at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564)
              at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:559)
              at org.jboss.netty.channel.Channels.fireChannelClosed(Channels.java:404)
              at org.jboss.netty.channel.socket.nio.NioWorker.close(NioWorker.java:602)
              at org.jboss.netty.channel.socket.nio.NioClientSocketPipelineSink.eventSunk(NioClientSocketPipelineSink.java:101)
              at org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendDownstream(DefaultChannelPipeline.java:771)
              at org.jboss.netty.handler.codec.oneone.OneToOneEncoder.handleDownstream(OneToOneEncoder.java:60)
              at org.jboss.netty.channel.DefaultChannelPipeline.sendDownstream(DefaultChannelPipeline.java:591)
              at org.jboss.netty.channel.DefaultChannelPipeline.sendDownstream(DefaultChannelPipeline.java:582)
              at org.jboss.netty.channel.Channels.close(Channels.java:720)
              at org.jboss.netty.channel.AbstractChannel.close(AbstractChannel.java:200)
              at org.jboss.netty.channel.ChannelFutureListener$2.operationComplete(ChannelFutureListener.java:57)
              at org.jboss.netty.channel.DefaultChannelFuture.notifyListener(DefaultChannelFuture.java:381)
              at org.jboss.netty.channel.DefaultChannelFuture.notifyListeners(DefaultChannelFuture.java:367)
              at org.jboss.netty.channel.DefaultChannelFuture.cancel(DefaultChannelFuture.java:356)
              at org.apache.avro.ipc.NettyTransceiver.disconnect(NettyTransceiver.java:301)
              - locked <0x000000067b1a7bc8> (a java.lang.Object)
              at org.apache.avro.ipc.NettyTransceiver.close(NettyTransceiver.java:380)
              at java.lang.Thread.run(Thread.java:662)
      

      Both of these methods acquire two different locks, the stateLock write lock and the monitor of channelFutureLock. The problem is that, under certain circumstances, these methods will acquire the locks in different orders which results in the deadlock. The sequence of events is something like this:

      1. Thread 2 calls close() -> disconnect(true, true, null)
      2. Inside a synchronized(channelFutureLock) block the disconnect method calls channelFuture.cancel(). Normally this would trigger an asynchronous event which would fire in a separate thread, but in this case Netty fires the event in the same thread, and NettyClientAvroHandler#handleUpstream(...) is invoked.
      3. Thread 1 calls getChannel() and obtains the write lock on stateLock. It then tries to synchronize on channelFutureLock but blocks because Thread 2 has already locked its monitor.
      4. Thread 2 calls the disconnect method from the handleUpstream method but blocks while attempting to acquire the stateLock write lock because Thread 1 has already locked it.

      There are a couple of fairly simple solutions to this problem. The first is that the disconnect method should call channelFuture.cancel() outside of the synchronized(channelFutureLock) block. Another solution would be to use a ExecutorService to guarantee that channelFuture.cancel() is always called in a separate thread. I think I prefer the first solution because it's simpler and does not require introducing a thread pool. I'll work on a patch for that solution.

      Attachments

        1. AVRO-1293.patch
          0.9 kB
          James Baldassari

        Activity

          People

            jbaldassari James Baldassari
            jbaldassari James Baldassari
            Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: