Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
fix potential blocking of NettyTransport connect and disconnect methods
Currently, in NettyTransport the locks for connecting and disconnecting channels are stored in a ConcurrentMap that has 500 entries. A tread can acquire a lock for an id and the lock returned is chosen on a hash function computed from the id. Unfortunately, a collision of two ids can cause a deadlock as follows: Scenario: one master (no data), one datanode (only data node) DiscoveryNode id of master is X DiscoveryNode id of datanode is Y Both X and Y cause the same lock to be returned by NettyTransport#connectLock() Both are up and running, all is fine until master stops. Thread 1: The master fault detection of the datanode is notified (onNodeDisconnected()), which in turn leads the node to try and reconnect to master via the callstack titled "Thread 1" below. -> connectToNode() is called and lock for X is acquired. The method waits for 45s for the cannels to reconnect. Furthermore, Thread 1 holds the NettyTransport#masterNodeMutex. Thread 2: The connection fails with an exception (connection refused, see callstack below), because the master shut down already. The exception is handled in NettyTransport#exceptionCaught which calls NettyTransport#disconnectFromNodeChannel. This method acquires the lock for Y (see Thread 2 below). Now, if Y and X have two different locks, this would get the lock, disconnect the channels and notify thread 1. But since X and Y have the same locks, thread 2 is deadlocked with thread 1 which waits for 45s. In this time, no thread can acquire the masterNodeMutex (held by thread 1), so the node can, for example, not stop. This commit introduces a mechanism that assures unique locks for unique ids. This lock is not reentrant and therfore assures that threads can not end up in an infinite recursion (see Thread 3 below for an example on how a thread can aquire a lock twice). While this is not a problem right now, it is potentially dangerous to have it that way, because the callstacks are complex as is and slight changes might cause unecpected recursions. Thread 1 ---- owns: Object (id=114) owns: Object (id=118) waiting for: DefaultChannelFuture (id=140) Object.wait(long) line: not available [native method] DefaultChannelFuture(Object).wait(long, int) line: 461 DefaultChannelFuture.await0(long, boolean) line: 311 DefaultChannelFuture.awaitUninterruptibly(long) line: 285 NettyTransport.connectToChannels(NettyTransport$NodeChannels, DiscoveryNode) line: 672 NettyTransport.connectToNode(DiscoveryNode, boolean) line: 609 NettyTransport.connectToNode(DiscoveryNode) line: 579 TransportService.connectToNode(DiscoveryNode) line: 129 MasterFaultDetection.handleTransportDisconnect(DiscoveryNode) line: 195 MasterFaultDetection.access$0(MasterFaultDetection, DiscoveryNode) line: 188 MasterFaultDetection$FDConnectionListener.onNodeDisconnected(DiscoveryNode) line: 245 TransportService$Adapter$2.run() line: 298 EsThreadPoolExecutor(ThreadPoolExecutor).runWorker(ThreadPoolExecutor$Worker) line: 1145 ThreadPoolExecutor$Worker.run() line: 615 Thread.run() line: 724 Thread 2 ------- waiting for: Object (id=114) NettyTransport.disconnectFromNodeChannel(Channel, Throwable) line: 790 NettyTransport.exceptionCaught(ChannelHandlerContext, ExceptionEvent) line: 495 MessageChannelHandler.exceptionCaught(ChannelHandlerContext, ExceptionEvent) line: 228 MessageChannelHandler(SimpleChannelUpstreamHandler).handleUpstream(ChannelHandlerContext, ChannelEvent) line: 112 DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline$DefaultChannelHandlerContext, ChannelEvent) line: 564 DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(ChannelEvent) line: 791 SizeHeaderFrameDecoder(FrameDecoder).exceptionCaught(ChannelHandlerContext, ExceptionEvent) line: 377 SizeHeaderFrameDecoder(SimpleChannelUpstreamHandler).handleUpstream(ChannelHandlerContext, ChannelEvent) line: 112 DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline$DefaultChannelHandlerContext, ChannelEvent) line: 564 DefaultChannelPipeline.sendUpstream(ChannelEvent) line: 559 Channels.fireExceptionCaught(Channel, Throwable) line: 525 NioClientBoss.processSelectedKeys(Set<SelectionKey>) line: 110 NioClientBoss.process(Selector) line: 79 NioClientBoss(AbstractNioSelector).run() line: 312 NioClientBoss.run() line: 42 ThreadRenamingRunnable.run() line: 108 DeadLockProofWorker$1.run() line: 42 ThreadPoolExecutor.runWorker(ThreadPoolExecutor$Worker) line: 1145 ThreadPoolExecutor$Worker.run() line: 615 Thread.run() line: 724 Thread 3 --------- org.elasticsearch.transport.netty.NettyTransport.disconnectFromNode(NettyTransport.java:772) org.elasticsearch.transport.netty.NettyTransport.access$1200(NettyTransport.java:92) org.elasticsearch.transport.netty.NettyTransport$ChannelCloseListener.operationComplete(NettyTransport.java:830) org.jboss.netty.channel.DefaultChannelFuture.notifyListener(DefaultChannelFuture.java:427) org.jboss.netty.channel.DefaultChannelFuture.notifyListeners(DefaultChannelFuture.java:418) org.jboss.netty.channel.DefaultChannelFuture.setSuccess(DefaultChannelFuture.java:362) org.jboss.netty.channel.AbstractChannel$ChannelCloseFuture.setClosed(AbstractChannel.java:355) org.jboss.netty.channel.AbstractChannel.setClosed(AbstractChannel.java:185) org.jboss.netty.channel.socket.nio.AbstractNioChannel.setClosed(AbstractNioChannel.java:197) org.jboss.netty.channel.socket.nio.NioSocketChannel.setClosed(NioSocketChannel.java:84) org.jboss.netty.channel.socket.nio.AbstractNioWorker.close(AbstractNioWorker.java:357) org.jboss.netty.channel.socket.nio.NioClientSocketPipelineSink.eventSunk(NioClientSocketPipelineSink.java:58) org.jboss.netty.channel.Channels.close(Channels.java:812) org.jboss.netty.channel.AbstractChannel.close(AbstractChannel.java:197) org.elasticsearch.transport.netty.NettyTransport$NodeChannels.closeChannelsAndWait(NettyTransport.java:892) org.elasticsearch.transport.netty.NettyTransport$NodeChannels.close(NettyTransport.java:879) org.elasticsearch.transport.netty.NettyTransport.disconnectFromNode(NettyTransport.java:778) org.elasticsearch.transport.netty.NettyTransport.access$1200(NettyTransport.java:92) org.elasticsearch.transport.netty.NettyTransport$ChannelCloseListener.operationComplete(NettyTransport.java:830) org.jboss.netty.channel.DefaultChannelFuture.notifyListener(DefaultChannelFuture.java:427) org.jboss.netty.channel.DefaultChannelFuture.notifyListeners(DefaultChannelFuture.java:418) org.jboss.netty.channel.DefaultChannelFuture.setSuccess(DefaultChannelFuture.java:362) org.jboss.netty.channel.AbstractChannel$ChannelCloseFuture.setClosed(AbstractChannel.java:355) org.jboss.netty.channel.AbstractChannel.setClosed(AbstractChannel.java:185) org.jboss.netty.channel.socket.nio.AbstractNioChannel.setClosed(AbstractNioChannel.java:197) org.jboss.netty.channel.socket.nio.NioSocketChannel.setClosed(NioSocketChannel.java:84) org.jboss.netty.channel.socket.nio.AbstractNioWorker.close(AbstractNioWorker.java:357) org.jboss.netty.channel.socket.nio.NioWorker.read(NioWorker.java:93) org.jboss.netty.channel.socket.nio.AbstractNioWorker.process(AbstractNioWorker.java:109) org.jboss.netty.channel.socket.nio.AbstractNioSelector.run(AbstractNioSelector.java:312) org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:90) org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:178) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:724)
- Loading branch information