[ https://issues.apache.org/jira/browse/SPARK-5085?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14265194#comment-14265194 ]
Stephen Haberman commented on SPARK-5085: ----------------------------------------- I think I've found a good clue: {code} [ 2527.610744] xen_netfront: xennet: skb rides the rocket: 19 slots [ 2555.073922] xen_netfront: xennet: skb rides the rocket: 19 slots [ 2606.652438] xen_netfront: xennet: skb rides the rocket: 19 slots [ 2615.427676] xen_netfront: xennet: skb rides the rocket: 19 slots [ 2626.008450] xen_netfront: xennet: skb rides the rocket: 19 slots [ 2742.996355] xen_netfront: xennet: skb rides the rocket: 19 slots [ 2744.434263] xen_netfront: xennet: skb rides the rocket: 19 slots [ 2744.623440] xen_netfront: xennet: skb rides the rocket: 19 slots [ 2745.204023] xen_netfront: xennet: skb rides the rocket: 19 slots [ 2745.470360] xen_netfront: xennet: skb rides the rocket: 19 slots [ 2745.517182] xen_netfront: xennet: skb rides the rocket: 19 slots [ 2745.616516] xen_netfront: xennet: skb rides the rocket: 19 slots [ 2818.547464] xen_netfront: xennet: skb rides the rocket: 19 slots [ 2824.525844] xen_netfront: xennet: skb rides the rocket: 19 slots [ 2831.868035] xen_netfront: xennet: skb rides the rocket: 19 slots [ 2833.644154] xen_netfront: xennet: skb rides the rocket: 19 slots [ 2895.396963] xen_netfront: xennet: skb rides the rocket: 19 slots [ 2896.939451] xen_netfront: xennet: skb rides the rocket: 19 slots [ 2901.464337] xen_netfront: xennet: skb rides the rocket: 19 slots [ 2902.461459] xen_netfront: xennet: skb rides the rocket: 19 slots [ 2904.840728] xen_netfront: xennet: skb rides the rocket: 19 slots [ 2908.156252] xen_netfront: xennet: skb rides the rocket: 19 slots [ 2908.925033] xen_netfront: xennet: skb rides the rocket: 19 slots [ 2933.240541] xen_netfront: xennet: skb rides the rocket: 19 slots [ 2975.218843] xen_netfront: xennet: skb rides the rocket: 19 slots [ 2975.333279] xen_netfront: xennet: skb rides the rocket: 19 slots [ 2980.533872] xen_netfront: xennet: skb rides the rocket: 19 slots [ 2984.017055] xen_netfront: xennet: skb rides the rocket: 19 slots [ 2984.107575] xen_netfront: xennet: skb rides the rocket: 19 slots [ 2991.252054] xen_netfront: xennet: skb rides the rocket: 19 slots [ 2993.965474] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3000.521793] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3057.080236] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3067.674541] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3077.984465] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3139.590085] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3140.145975] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3217.729824] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3249.614154] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3252.775976] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3261.234940] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3302.538848] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3325.811720] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3332.873067] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3340.724759] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3349.646235] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3354.857573] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3361.728122] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3373.623622] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3384.000029] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3394.701554] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3402.048682] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3408.972487] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3415.697781] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3415.746289] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3428.234060] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3438.317541] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3467.061761] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3592.827300] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3598.320551] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3601.487113] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3636.656200] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3670.347676] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3672.573875] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3720.392902] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3748.385374] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3763.997229] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3776.472560] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3783.343165] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3819.665983] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3829.223240] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3834.077136] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3835.979225] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3849.486165] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3863.844525] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3911.054876] xen_netfront: xennet: skb rides the rocket: 19 slots [ 3931.210743] xen_netfront: xennet: skb rides the rocket: 19 slots [ 4021.144539] xen_netfront: xennet: skb rides the rocket: 19 slots [ 4074.403245] xen_netfront: xennet: skb rides the rocket: 19 slots [ 4152.990938] xen_netfront: xennet: skb rides the rocket: 19 slots [ 4167.123956] xen_netfront: xennet: skb rides the rocket: 19 slots [ 4208.860334] xen_netfront: xennet: skb rides the rocket: 19 slots [ 4213.026821] xen_netfront: xennet: skb rides the rocket: 19 slots [ 4213.896904] xen_netfront: xennet: skb rides the rocket: 19 slots [ 4240.374372] xen_netfront: xennet: skb rides the rocket: 19 slots [ 4242.704989] xen_netfront: xennet: skb rides the rocket: 19 slots [ 4251.108215] xen_netfront: xennet: skb rides the rocket: 19 slots [ 4273.694682] xen_netfront: xennet: skb rides the rocket: 19 slots [ 4274.996669] xen_netfront: xennet: skb rides the rocket: 19 slots [ 4292.412638] xen_netfront: xennet: skb rides the rocket: 19 slots [ 4324.607161] xen_netfront: xennet: skb rides the rocket: 19 slots [ 4353.229704] xen_netfront: xennet: skb rides the rocket: 19 slots [ 4354.939760] xen_netfront: xennet: skb rides the rocket: 19 slots [ 4378.316735] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5290.740395] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5291.876587] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5292.643901] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5292.859787] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5310.823667] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5311.479128] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5311.499906] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5311.754147] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5312.202389] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5312.294099] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5312.294472] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5312.294891] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5312.295154] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5312.295505] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5316.500087] net_ratelimit: 1063 callbacks suppressed [ 5316.500096] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5316.648114] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5316.900077] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5317.096083] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5317.127139] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5317.135752] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5317.492085] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5317.700087] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5318.524080] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5319.304087] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5322.508077] net_ratelimit: 3 callbacks suppressed [ 5322.508087] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5324.940073] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5326.268099] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5326.716080] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5327.116077] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5328.924075] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5334.315679] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5337.756080] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5339.100110] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5339.548095] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5339.932076] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5341.756092] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5363.420099] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5364.764121] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5365.212080] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5365.596079] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5367.388076] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5414.748074] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5416.156105] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5416.540079] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5416.924097] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5418.716082] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5517.276077] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5518.812117] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5519.324089] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5519.580091] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5521.372086] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5637.596077] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5639.132114] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5639.644057] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5639.644082] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5641.692080] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5757.916074] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5759.452118] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5759.964097] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5759.964115] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5762.012080] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5878.236126] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5879.772110] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5880.284085] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5880.284095] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5882.332088] xen_netfront: xennet: skb rides the rocket: 19 slots [ 5998.556084] xen_netfront: xennet: skb rides the rocket: 19 slots [ 6000.092108] xen_netfront: xennet: skb rides the rocket: 19 slots [ 6000.604086] xen_netfront: xennet: skb rides the rocket: 19 slots [ 6000.604102] xen_netfront: xennet: skb rides the rocket: 19 slots [ 6002.652080] xen_netfront: xennet: skb rides the rocket: 19 slots [ 6118.876085] xen_netfront: xennet: skb rides the rocket: 19 slots [ 6120.412107] xen_netfront: xennet: skb rides the rocket: 19 slots [ 6120.924079] xen_netfront: xennet: skb rides the rocket: 19 slots [ 6120.924093] xen_netfront: xennet: skb rides the rocket: 19 slots [ 6122.972077] xen_netfront: xennet: skb rides the rocket: 19 slots {code} IANAE, but sounds like the kernel is dropping packets. Still googling around about possible work arounds. > 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 log output from one executor (I turned on trace output for the network > package and ShuffleBlockFetcherIterator; all executors in the cluster have > basically the same pattern of ~15m of silence then timeouts): > {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 > // note 15m of silence here... > 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