You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@avro.apache.org by "Doug Cutting (JIRA)" <ji...@apache.org> on 2013/05/16 23:35:16 UTC

[jira] [Commented] (AVRO-1293) NettyTransceiver: Deadlock can occur when different threads call getChannel() and close() concurrently

    [ https://issues.apache.org/jira/browse/AVRO-1293?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13660026#comment-13660026 ] 

Doug Cutting commented on AVRO-1293:
------------------------------------

James, do you think this is ready to commit?  Is it possible to create a test case for this?

                
> NettyTransceiver: Deadlock can occur when different threads call getChannel() and close() concurrently
> ------------------------------------------------------------------------------------------------------
>
>                 Key: AVRO-1293
>                 URL: https://issues.apache.org/jira/browse/AVRO-1293
>             Project: Avro
>          Issue Type: Bug
>          Components: java
>    Affects Versions: 1.7.4
>            Reporter: James Baldassari
>            Assignee: James Baldassari
>         Attachments: AVRO-1293.patch
>
>
> 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:
> {code}
> "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)
>         at $Proxy10.onMemberJoin(Unknown Source)
>         at dataxu.common.cluster.coordinator.HazelcastClusterCoordinator$MembersMapListener.entryAdded(HazelcastClusterCoordinator.java:409)
>         at com.hazelcast.impl.ListenerManager.callListener(ListenerManager.java:392)
>         at com.hazelcast.impl.ListenerManager.callListeners(ListenerManager.java:334)
>         at com.hazelcast.impl.BaseManager$2.run(BaseManager.java:1382)
>         at com.hazelcast.impl.executor.ParallelExecutorService$ParallelExecutorImpl$ExecutionSegment.run(ParallelExecutorService.java:212)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>         at java.lang.Thread.run(Thread.java:662)
>         at com.hazelcast.impl.ExecutorThreadFactory$1.run(ExecutorThreadFactory.java:38)
> {code}
> {code}
> "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 dataxu.common.cluster.coordinator.ClusterCoordinator$ClusterListener.close(ClusterCoordinator.java:1178)
>         at dataxu.common.cluster.coordinator.ClusterCoordinator$ClusterListenerRemover.call(ClusterCoordinator.java:952)
>         at dataxu.common.cluster.coordinator.ClusterCoordinator$ClusterListenerRemover.call(ClusterCoordinator.java:1)
>         at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
>         at java.util.concurrent.FutureTask.run(FutureTask.java:138)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>         at java.lang.Thread.run(Thread.java:662)
> {code}
> 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.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira