[ https://issues.apache.org/jira/browse/SPARK-5085?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14264196#comment-14264196 ]
Stephen Haberman commented on SPARK-5085: ----------------------------------------- There is no interesting output in the driver logs...everything was fine, until 5:33, tasks just stop getting completed across the entire cluster: {code} // lots of logs... 15/01/03 05:33:37 INFO [sparkDriver-akka.actor.default-dispatcher-15] scheduler.TaskSetManager (Logging.scala:logInfo(59)) - Starting task 231.0 in stage 5.0 (TID 7310, ip-10-169-175-179.ec2.internal, PROCESS_LOCAL, 2004 bytes) 15/01/03 05:33:37 INFO [task-result-getter-2] scheduler.TaskSetManager (Logging.scala:logInfo(59)) - Finished task 209.0 in stage 5.0 (TID 7288) in 26069 ms on ip-10-169-175-179.ec2.internal (152/2639) {code} There are several stages before this stage 5.0 that involve shuffles as well, so this is not the first shuffle done by the job. The executor thread dumps don't show anything interesting...all of the task threads are blocked in ShuffleBlockFetcherIterator, where it's trying to take blocks from the LinkedBlockingQueue. Here are a few netstats from before, right before, after the timeouts: {code} {code} $ cat netstat-0531.txt Active Internet connections (w/o servers) Proto Recv-Q Send-Q Local Address Foreign Address State tcp 0 0 ip-10-181-127-41.ec2.:51718 ip-10-169-82-122:cslistener ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.in:ssh ip-10-169-82-122.ec2.:39880 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:49745 ip-10-71-184-33.ec2.i:53275 ESTABLISHED tcp 0 134 ip-10-181-127-41.ec2.:36031 s3-1-w.amazonaws.com:https ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:44171 ip-10-181-127-41.ec2.:39134 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:35301 ip-10-63-193-69.ec2.i:49596 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:60981 ip-10-63-193-69.ec2.i:35531 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:36594 ip-10-169-82-122.ec2.:36365 ESTABLISHED tcp 38 0 ip-10-181-127-41.ec2.:48338 s3-1-w.amazonaws.com:https CLOSE_WAIT tcp 0 0 ip-10-181-127-41.ec2.:38358 ip-10-169-82-122.ec2.:swa-1 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:35301 ip-10-13-165-77.ec2.i:44291 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:36030 s3-1-w.amazonaws.com:https ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:36028 s3-1-w.amazonaws.com:https ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:35301 ip-10-169-175-179.ec2:39157 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:35941 ip-10-169-247-194.ec2:55602 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:44389 ip-10-31-61-144.ec2.i:59883 ESTABLISHED tcp 38 0 ip-10-181-127-41.ec2.:48342 s3-1-w.amazonaws.com:https CLOSE_WAIT tcp 0 0 ip-10-181-127-41.ec2.:39794 ip-10-169-82-122.ec2.:52518 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:35301 ip-10-69-85-11.ec2.in:41533 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:37274 ip-10-13-165-77.ec2.i:56072 ESTABLISHED tcp 0 134 ip-10-181-127-41.ec2.:36032 s3-1-w.amazonaws.com:https ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:36029 s3-1-w.amazonaws.com:https ESTABLISHED tcp 38 0 ip-10-181-127-41.ec2.:48339 s3-1-w.amazonaws.com:https CLOSE_WAIT tcp 0 0 ip-10-181-127-41.ec2.:35301 ip-10-71-184-33.ec2.i:56943 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:56283 ip-10-169-82-12:pcsync-http TIME_WAIT tcp 0 0 ip-10-181-127-41.ec2.:35301 ip-10-33-166-218.ec2.:46591 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:39778 ip-10-69-85-11.ec2.in:34222 ESTABLISHED tcp 38 0 ip-10-181-127-41.ec2.:48340 s3-1-w.amazonaws.com:https CLOSE_WAIT tcp 0 0 ip-10-181-127-41.ec2.:35301 ip-10-181-61-165.ec2.:41291 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:48805 ip-10-181-61-165.ec2.:57986 ESTABLISHED tcp 2255316 0 ip-10-181-127-41.ec2.:36024 s3-1-w.amazonaws.com:https ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:39134 ip-10-181-127-41.ec2.:44171 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:36023 s3-1-w.amazonaws.com:https ESTABLISHED tcp 367803 0 ip-10-181-127-41.ec2.:36026 s3-1-w.amazonaws.com:https ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:45560 ip-10-33-166-218.ec2.:42942 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:35301 ip-10-169-247-194.ec2:42871 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:35301 ip-10-31-61-144.ec2.i:41293 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:36581 ip-10-169-82-122.ec2.i:7077 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:38684 ip-10-169-175-179.ec2:45582 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:36025 s3-1-w.amazonaws.com:https ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:36021 s3-1-w.amazonaws.com:https ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:52347 ip-10-169-82-122:cslistener TIME_WAIT tcp 1251469 0 ip-10-181-127-41.ec2.:36027 s3-1-w.amazonaws.com:https ESTABLISHED $ cat netstat-0535.txt Active Internet connections (w/o servers) Proto Recv-Q Send-Q Local Address Foreign Address State tcp 0 0 ip-10-181-127-41.ec2.:51718 ip-10-169-82-122:cslistener ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.in:ssh ip-10-169-82-122.ec2.:39880 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:49745 ip-10-71-184-33.ec2.i:53275 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:44171 ip-10-181-127-41.ec2.:39134 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:35301 ip-10-63-193-69.ec2.i:49596 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:60981 ip-10-63-193-69.ec2.i:35531 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:36594 ip-10-169-82-122.ec2.:36365 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:38358 ip-10-169-82-122.ec2.:swa-1 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:35301 ip-10-13-165-77.ec2.i:44291 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:52360 ip-10-169-82-122:cslistener TIME_WAIT tcp 0 0 ip-10-181-127-41.ec2.:35301 ip-10-169-175-179.ec2:39157 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:35941 ip-10-169-247-194.ec2:55602 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:44389 ip-10-31-61-144.ec2.i:59883 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:39794 ip-10-169-82-122.ec2.:52518 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:35301 ip-10-69-85-11.ec2.in:41533 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:37274 ip-10-13-165-77.ec2.i:56072 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:35301 ip-10-71-184-33.ec2.i:56943 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:35301 ip-10-33-166-218.ec2.:46591 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:39778 ip-10-69-85-11.ec2.in:34222 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:50657 ip-10-169-82-12:pcsync-http TIME_WAIT tcp 0 0 ip-10-181-127-41.ec2.:35301 ip-10-181-61-165.ec2.:41291 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:48805 ip-10-181-61-165.ec2.:57986 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:39134 ip-10-181-127-41.ec2.:44171 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:45560 ip-10-33-166-218.ec2.:42942 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:35301 ip-10-169-247-194.ec2:42871 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:35301 ip-10-31-61-144.ec2.i:41293 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:36581 ip-10-169-82-122.ec2.i:7077 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:38684 ip-10-169-175-179.ec2:45582 ESTABLISHED $ cat netstat-0549.txt Active Internet connections (w/o servers) Proto Recv-Q Send-Q Local Address Foreign Address State tcp 0 0 ip-10-181-127-41.ec2.:51718 ip-10-169-82-122:cslistener ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.in:ssh ip-10-169-82-122.ec2.:40390 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.in:ssh ip-10-169-82-122.ec2.:39880 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:49745 ip-10-71-184-33.ec2.i:53275 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:44171 ip-10-181-127-41.ec2.:39134 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:35301 ip-10-63-193-69.ec2.i:49596 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:60981 ip-10-63-193-69.ec2.i:35531 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:36594 ip-10-169-82-122.ec2.:36365 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:52384 ip-10-169-82-122:cslistener TIME_WAIT tcp 0 0 ip-10-181-127-41.ec2.:38358 ip-10-169-82-122.ec2.:swa-1 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:35301 ip-10-13-165-77.ec2.i:44291 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:35301 ip-10-169-175-179.ec2:39157 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:35941 ip-10-169-247-194.ec2:55602 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:44389 ip-10-31-61-144.ec2.i:59883 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:39794 ip-10-169-82-122.ec2.:52518 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:35301 ip-10-69-85-11.ec2.in:41533 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:37274 ip-10-13-165-77.ec2.i:56072 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:35301 ip-10-71-184-33.ec2.i:56943 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:35301 ip-10-33-166-218.ec2.:46591 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:39778 ip-10-69-85-11.ec2.in:34222 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:35301 ip-10-181-61-165.ec2.:41291 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:48805 ip-10-181-61-165.ec2.:57986 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:39134 ip-10-181-127-41.ec2.:44171 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:45560 ip-10-33-166-218.ec2.:42942 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:35301 ip-10-169-247-194.ec2:42871 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:35301 ip-10-31-61-144.ec2.i:41293 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:36581 ip-10-169-82-122.ec2.i:7077 ESTABLISHED tcp 0 0 ip-10-181-127-41.ec2.:38684 ip-10-169-175-179.ec2:45582 ESTABLISHED {code} The send/receive buffers are all empty. I don't have any good ideas. I've started reading the netty backend code, which is interesting/mildly fun, but I don't really have any good ideas about where to look to even begin debugging this. Any ideas? > netty shuffle service causing connection timeouts > ------------------------------------------------- > > Key: SPARK-5085 > URL: https://issues.apache.org/jira/browse/SPARK-5085 > Project: Spark > Issue Type: Bug > Components: Shuffle > Affects Versions: 1.2.0 > Environment: EMR, transient cluster of 10 m3.2xlarges, spark 1.2.0 > Here's our spark-defaults: > {code} > spark.master spark://$MASTER_IP:7077 > spark.eventLog.enabled true > spark.eventLog.dir /mnt/spark/work/history > spark.serializer org.apache.spark.serializer.KryoSerializer > spark.executor.memory ${EXECUTOR_MEM}m > spark.core.connection.ack.wait.timeout 600 > spark.storage.blockManagerSlaveTimeoutMs 60000 > spark.shuffle.consolidateFiles true > spark.shuffle.service.enabled false > spark.shuffle.blockTransferService nio # works with nio, fails with netty > # Use snappy because LZF uses ~100-300k buffer per block > spark.io.compression.codec org.apache.spark.io.SnappyCompressionCodec > spark.shuffle.file.buffer.kb 10 > spark.executor.extraJavaOptions -XX:+PrintGCDetails > -XX:+HeapDumpOnOutOfMemoryError -Xss2m -XX:+UseConcMarkSweepGC > -XX:CMSInitiatingOccupancyFraction=70 -XX:MaxHeapFreeRati... > spark.akka.logLifecycleEvents true > spark.akka.timeout 360 > spark.akka.askTimeout 120 > spark.akka.lookupTimeout 120 > spark.akka.frameSize 100 > spark.files.userClassPathFirst true > spark.shuffle.memoryFraction 0.5 > spark.storage.memoryFraction 0.2 > {code} > Reporter: Stephen Haberman > > In Spark 1.2.0, the netty backend is causing our report's cluster to lock up > with connection timeouts, ~75% of the way through the job. > It happens with both the external shuffle server and the > non-external/executor-hosted shuffle server, but if I change the shuffle > service from netty to nio, it immediately works. > Here's the main log output (I turned on trace output for the network package > and ShuffleBlockFetcherIterator): > {code} > // lots of log output, doing fine... > 15/01/03 05:33:39 TRACE [shuffle-server-0] protocol.MessageDecoder > (MessageDecoder.java:decode(42)) - Received message ChunkFetchRequest: > ChunkFetchRequest{streamChunkId=StreamChunkId{streamId=1465867812750, > chunkIndex=170}} > 15/01/03 05:33:39 TRACE [shuffle-server-0] server.TransportRequestHandler > (TransportRequestHandler.java:processFetchRequest(107)) - Received req from > /10.169.175.179:57056 to fetch block StreamChunkId{streamId=1465867812750, > chunkIndex=170} > 15/01/03 05:33:39 TRACE [shuffle-server-0] server.OneForOneStreamManager > (OneForOneStreamManager.java:getChunk(75)) - Removing stream id 1465867812750 > 15/01/03 05:33:39 TRACE [shuffle-server-0] server.TransportRequestHandler > (TransportRequestHandler.java:operationComplete(152)) - Sent result > ChunkFetchSuccess{streamChunkId=StreamChunkId{streamId=1465867812750, > chunkIndex=170}, > buffer=FileSegmentManagedBuffer{file=/mnt1/spark/local/spark-local-20150103040327-c554/28/shuffle_4_1723_0.data, > offset=4574685, length=20939}} to client /10.169.175.179:57056 > 15/01/03 05:48:13 WARN [shuffle-server-7] server.TransportChannelHandler > (TransportChannelHandler.java:exceptionCaught(66)) - Exception in connection > from /10.33.166.218:42780 > java.io.IOException: Connection timed out > at sun.nio.ch.FileDispatcherImpl.read0(Native Method) > at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:39) > at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223) > at sun.nio.ch.IOUtil.read(IOUtil.java:192) > at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:379) > at > io.netty.buffer.PooledUnsafeDirectByteBuf.setBytes(PooledUnsafeDirectByteBuf.java:311) > at io.netty.buffer.AbstractByteBuf.writeBytes(AbstractByteBuf.java:881) > at > io.netty.channel.socket.nio.NioSocketChannel.doReadBytes(NioSocketChannel.java:225) > at > io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:119) > at > io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511) > at > io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468) > at > io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382) > at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354) > at > io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:116) > at java.lang.Thread.run(Thread.java:745) > 15/01/03 05:48:13 ERROR [shuffle-server-7] server.TransportRequestHandler > (TransportRequestHandler.java:operationComplete(154)) - Error sending result > ChunkFetchSuccess{streamChunkId=StreamChunkId{streamId=1465867812408, > chunkIndex=52}, > buffer=FileSegmentManagedBuffer{file=/mnt1/spark/local/spark-local-20150103040327-c554/2d/shuffle_4_520_0.data, > offset=2214139, length=20607}} to /10.33.166.218:42780; closing connection > java.nio.channels.ClosedChannelException > 15/01/03 05:48:13 ERROR [shuffle-server-7] server.TransportRequestHandler > (TransportRequestHandler.java:operationComplete(154)) - Error sending result > ChunkFetchSuccess{streamChunkId=StreamChunkId{streamId=1465867812408, > chunkIndex=53}, > buffer=FileSegmentManagedBuffer{file=/mnt1/spark/local/spark-local-20150103040327-c554/10/shuffle_4_524_0.data, > offset=2215548, length=23998}} to /10.33.166.218:42780; closing connection > java.nio.channels.ClosedChannelException > 15/01/03 05:48:13 ERROR [shuffle-server-7] server.TransportRequestHandler > (TransportRequestHandler.java:operationComplete(154)) - Error sending result > ChunkFetchSuccess{streamChunkId=StreamChunkId{streamId=1465867812408, > chunkIndex=54}, > buffer=FileSegmentManagedBuffer{file=/mnt/spark/local/spark-local-20150103040327-4f92/32/shuffle_4_532_0.data, > offset=2248230, length=20580}} to /10.33.166.218:42780; closing connection > java.nio.channels.ClosedChannelException > // lots more of these... > {code} > Note how, up through 5:33, everything was fine, then after ~15 minutes of > silence, at 5:48, the shuffle-server connection times out, and all of that > server-7's requests fail. > Here is shuffle-server-1 from the same stdout (with 1 last > ClosedChannelException from shuffle-server-7): > {code} > 15/01/03 05:48:13 ERROR [shuffle-server-7] server.TransportRequestHandler > (TransportRequestHandler.java:operationComplete(154)) - Error sending result > RpcResponse{requestId=6670015694248454083, response=[B@6653587b} to > /10.33.166.218:42780; closing connection > java.nio.channels.ClosedChannelException > 15/01/03 05:48:45 WARN [shuffle-server-1] server.TransportChannelHandler > (TransportChannelHandler.java:exceptionCaught(66)) - Exception in connection > from /10.13.165.77:41845 > java.io.IOException: Connection timed out > at sun.nio.ch.FileDispatcherImpl.read0(Native Method) > at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:39) > at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223) > at sun.nio.ch.IOUtil.read(IOUtil.java:192) > at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:379) > at > io.netty.buffer.PooledUnsafeDirectByteBuf.setBytes(PooledUnsafeDirectByteBuf.java:311) > at io.netty.buffer.AbstractByteBuf.writeBytes(AbstractByteBuf.java:881) > at > io.netty.channel.socket.nio.NioSocketChannel.doReadBytes(NioSocketChannel.java:225) > at > io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:119) > at > io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511) > at > io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468) > at > io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382) > at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354) > at > io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:116) > at java.lang.Thread.run(Thread.java:745) > 15/01/03 05:48:45 ERROR [shuffle-server-1] server.TransportRequestHandler > (TransportRequestHandler.java:operationComplete(154)) - Error sending result > ChunkFetchSuccess{streamChunkId=StreamChunkId{streamId=1465867812684, > chunkIndex=50}, > buffer=FileSegmentManagedBuffer{file=/mnt1/spark/local/spark-local-20150103040327-c554/20/shuffle_4_513_0.data, > offset=4029264, length=19717}} to /10.13.165.77:41845; closing connection > java.nio.channels.ClosedChannelException > 15/01/03 05:48:45 ERROR [shuffle-server-1] server.TransportRequestHandler > (TransportRequestHandler.java:operationComplete(154)) - Error sending result > ChunkFetchSuccess{streamChunkId=StreamChunkId{streamId=1465867812684, > chunkIndex=51}, > buffer=FileSegmentManagedBuffer{file=/mnt/spark/local/spark-local-20150103040327-4f92/30/shuffle_4_514_0.data, > offset=3976999, length=18459}} to /10.13.165.77:41845; closing connection > java.nio.channels.ClosedChannelException > 15/01/03 05:48:45 ERROR [shuffle-server-1] server.TransportRequestHandler > (TransportRequestHandler.java:operationComplete(154)) - Error sending result > ChunkFetchSuccess{streamChunkId=StreamChunkId{streamId=1465867812684, > chunkIndex=52}, > buffer=FileSegmentManagedBuffer{file=/mnt1/spark/local/spark-local-20150103040327-c554/2d/shuffle_4_520_0.data, > offset=4177037, length=18689}} to /10.13.165.77:41845; closing connection > java.nio.channels.ClosedChannelException > {code} > The last line from shuffle-server-7 from before it died was earlier, at > 5:33:06: > {code} > 15/01/03 05:33:06 TRACE [shuffle-server-7] netty.NettyBlockRpcServer > (Logging.scala:logTrace(67)) - Registered streamId 1465867812639 with 171 > buffers > {code} > And here's the last few lines-ish from shuffle-server-1 before the timeout: > {code} > 15/01/03 05:33:25 TRACE [shuffle-server-1] netty.NettyBlockRpcServer > (Logging.scala:logTrace(67)) - Registered streamId 1465867812719 with 230 > buffers > 15/01/03 05:33:25 TRACE [shuffle-server-3] server.TransportRequestHandler > (TransportRequestHandler.java:operationComplete(152)) - Sent result > ChunkFetchSuccess{streamChunkId=StreamChunkId{streamId=1465867812717, > chunkIndex=119}, > buffer=FileSegmentManagedBuffer{file=/mnt1/spark/local/spark-local-20150103040327-c554/0e/shuffle_4_1192_0.data, > offset=4788234, length=26721}} to client /10.181.127.41:60981 > 15/01/03 05:33:25 TRACE [shuffle-server-1] protocol.MessageDecoder > (MessageDecoder.java:decode(42)) - Received message RpcRequest: > RpcRequest{requestId=7443158706773552834, message=[B@73157c0d} > 15/01/03 05:33:25 TRACE [shuffle-server-1] netty.NettyBlockRpcServer > (Logging.scala:logTrace(67)) - Received request: > OpenBlocks{appId=app-20150103040324-0000, execId=0, > blockIds=[shuffle_4_16_222, shuffle_4_18_222, shuffle_4_39_222, > shuffle_4_49_222, shuffle_4_50_222, shuffle_4_53_222, shuffle_4_64_222, > shuffle_4_86_222, shuffle_4_87_222, shuffle_4_88_222, shuffle_4_99_222, > shuffle_4_100_222, shuffle_4_126_222, shuffle_4_134_222, shuffle_4_153_222, > shuffle_4_162_222, shuffle_4_163_222, shuffle_4_169_222, shuffle_4_171_222, > shuffle_4_173_222, shuffle_4_177_222, shuffle_4_191_222, shuffle_4_200_222, > shuffle_4_248_222, shuffle_4_266_222, shuffle_4_269_222, shuffle_4_280_222, > shuffle_4_283_222, shuffle_4_286_222, shuffle_4_288_222, shuffle_4_289_222, > shuffle_4_331_222, shuffle_4_341_222, shuffle_4_360_222, shuffle_4_362_222, > shuffle_4_367_222, shuffle_4_369_222, shuffle_4_374_222, shuffle_4_376_222, > shuffle_4_408_222, shuffle_4_418_222, shuffle_4_426_222, shuffle_4_428_222, > shuffle_4_435_222, shuffle_4_436_222, shuffle_4_442_222, shuffle_4_443_222, > shuffle_4_491_222, shuffle_4_497_222, shuffle_4_498_222, shuffle_4_513_222, > shuffle_4_514_222, shuffle_4_520_222, shuffle_4_524_222, shuffle_4_532_222, > shuffle_4_572_222, shuffle_4_573_222, shuffle_4_576_222, shuffle_4_594_222, > shuffle_4_596_222, shuffle_4_604_222, shuffle_4_621_222, shuffle_4_627_222, > shuffle_4_653_222, shuffle_4_654_222, shuffle_4_669_222, shuffle_4_678_222, > shuffle_4_679_222, shuffle_4_685_222, shuffle_4_701_222, shuffle_4_708_222, > shuffle_4_730_222, shuffle_4_734_222, shuffle_4_740_222, shuffle_4_763_222, > shuffle_4_766_222, shuffle_4_769_222, shuffle_4_779_222, shuffle_4_784_222, > shuffle_4_794_222, shuffle_4_816_222, shuffle_4_823_222, shuffle_4_847_222, > shuffle_4_851_222, shuffle_4_856_222, shuffle_4_859_222, shuffle_4_860_222, > shuffle_4_877_222, shuffle_4_904_222, shuffle_4_911_222, shuffle_4_917_222, > shuffle_4_926_222, shuffle_4_929_222, shuffle_4_930_222, shuffle_4_933_222, > shuffle_4_953_222, shuffle_4_956_222, shuffle_4_965_222, shuffle_4_969_222, > shuffle_4_1000_222, shuffle_4_1001_222, shuffle_4_1003_222, > shuffle_4_1005_222, shuffle_4_1029_222, shuffle_4_1030_222, > shuffle_4_1032_222, shuffle_4_1070_222, shuffle_4_1083_222, > shuffle_4_1091_222, shuffle_4_1093_222, shuffle_4_1094_222, > shuffle_4_1110_222, shuffle_4_1120_222, shuffle_4_1121_222, > shuffle_4_1141_222, shuffle_4_1157_222, shuffle_4_1170_222, > shuffle_4_1172_222, shuffle_4_1178_222, shuffle_4_1192_222, > shuffle_4_1201_222, shuffle_4_1205_222, shuffle_4_1229_222, > shuffle_4_1235_222, shuffle_4_1249_222, shuffle_4_1250_222, > shuffle_4_1256_222, shuffle_4_1286_222, shuffle_4_1297_222, > shuffle_4_1299_222, shuffle_4_1313_222, shuffle_4_1318_222, > shuffle_4_1328_222, shuffle_4_1331_222, shuffle_4_1334_222, > shuffle_4_1371_222, shuffle_4_1381_222, shuffle_4_1387_222, > shuffle_4_1397_222, shuffle_4_1400_222, shuffle_4_1408_222, > shuffle_4_1409_222, shuffle_4_1413_222, shuffle_4_1456_222, > shuffle_4_1458_222, shuffle_4_1470_222, shuffle_4_1478_222, > shuffle_4_1480_222, shuffle_4_1486_222, shuffle_4_1490_222, > shuffle_4_1493_222, shuffle_4_1544_222, shuffle_4_1545_222, > shuffle_4_1552_222, shuffle_4_1560_222, shuffle_4_1565_222, > shuffle_4_1569_222, shuffle_4_1570_222, shuffle_4_1577_222, > shuffle_4_1620_222, shuffle_4_1623_222, shuffle_4_1633_222, > shuffle_4_1640_222, shuffle_4_1651_222, shuffle_4_1652_222, > shuffle_4_1656_222, shuffle_4_1665_222, shuffle_4_1710_222, > shuffle_4_1717_222, shuffle_4_1718_222, shuffle_4_1723_222]} > 15/01/03 05:33:25 TRACE [shuffle-server-3] protocol.MessageDecoder > (MessageDecoder.java:decode(42)) - Received message ChunkFetchRequest: > ChunkFetchRequest{streamChunkId=StreamChunkId{streamId=1465867812717, > chunkIndex=120}} > 15/01/03 05:33:25 TRACE [shuffle-server-3] server.TransportRequestHandler > (TransportRequestHandler.java:processFetchRequest(107)) - Received req from > /10.181.127.41:60981 to fetch block StreamChunkId{streamId=1465867812717, > chunkIndex=120} > 15/01/03 05:33:25 TRACE [shuffle-server-3] server.TransportRequestHandler > (TransportRequestHandler.java:operationComplete(152)) - Sent result > ChunkFetchSuccess{streamChunkId=StreamChunkId{streamId=1465867812717, > chunkIndex=120}, > buffer=FileSegmentManagedBuffer{file=/mnt/spark/local/spark-local-20150103040327-4f92/16/shuffle_4_1201_0.data, > offset=4406836, length=18644}} to client /10.181.127.41:60981 > 15/01/03 05:33:25 TRACE [shuffle-server-3] protocol.MessageDecoder > (MessageDecoder.java:decode(42)) - Received message ChunkFetchRequest: > ChunkFetchRequest{streamChunkId=StreamChunkId{streamId=1465867812717, > chunkIndex=121}} > 15/01/03 05:33:25 TRACE [shuffle-server-3] server.TransportRequestHandler > (TransportRequestHandler.java:processFetchRequest(107)) - Received req from > /10.181.127.41:60981 to fetch block StreamChunkId{streamId=1465867812717, > chunkIndex=121} > 15/01/03 05:33:25 TRACE [shuffle-server-3] server.TransportRequestHandler > (TransportRequestHandler.java:operationComplete(152)) - Sent result > ChunkFetchSuccess{streamChunkId=StreamChunkId{streamId=1465867812717, > chunkIndex=121}, > buffer=FileSegmentManagedBuffer{file=/mnt/spark/local/spark-local-20150103040327-4f92/28/shuffle_4_1205_0.data, > offset=4285664, length=18251}} to client /10.181.127.41:60981 > 15/01/03 05:33:25 TRACE [shuffle-server-1] netty.NettyBlockRpcServer > (Logging.scala:logTrace(67)) - Registered streamId 1465867812720 with 171 > buffers > {code} > What's odd is that the entire cluster seems to all hit these connect timeouts > at once. Running dstat during this period, nothing is happening--no cpu, > disk, or network usage. -- This message was sent by Atlassian JIRA (v6.3.4#6332) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org