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

Jon Chase commented on SPARK-6962:
----------------------------------

Here's the stderr from the executors at the time of the lock up (there are 3 
executors).

18:26:00 is when the lockup happened, and after 20+ minutes, these are still 
the most recent logs in executor 1:

15/04/17 18:26:00 INFO executor.CoarseGrainedExecutorBackend: Got assigned task 
1132
15/04/17 18:26:00 INFO executor.Executor: Running task 110.0 in stage 15.0 (TID 
1132)
15/04/17 18:26:00 INFO storage.ShuffleBlockFetcherIterator: Getting 1008 
non-empty blocks out of 1008 blocks
15/04/17 18:26:00 INFO storage.ShuffleBlockFetcherIterator: Started 2 remote 
fetches in 3 ms
15/04/17 18:26:00 INFO executor.Executor: Finished task 107.0 in stage 15.0 
(TID 1129). 8325 bytes result sent to driver
15/04/17 18:26:00 INFO executor.CoarseGrainedExecutorBackend: Got assigned task 
1133
15/04/17 18:26:00 INFO executor.Executor: Running task 111.0 in stage 15.0 (TID 
1133)
15/04/17 18:26:00 INFO storage.ShuffleBlockFetcherIterator: Getting 1008 
non-empty blocks out of 1008 blocks
15/04/17 18:26:00 INFO storage.ShuffleBlockFetcherIterator: Started 2 remote 
fetches in 2 ms




Here's executor 2, it doesn't have any activity for about 20 minutes (again, 
the lockup happened at ~18:26:00):

15/04/17 18:25:48 INFO storage.ShuffleBlockFetcherIterator: Getting 1008 
non-empty blocks out of 1008 blocks
15/04/17 18:25:48 INFO storage.ShuffleBlockFetcherIterator: Started 2 remote 
fetches in 11 ms
15/04/17 18:25:49 INFO executor.Executor: Finished task 13.0 in stage 15.0 (TID 
1035). 12013 bytes result sent to driver
15/04/17 18:25:49 INFO executor.CoarseGrainedExecutorBackend: Got assigned task 
1068
15/04/17 18:25:49 INFO executor.Executor: Running task 46.0 in stage 15.0 (TID 
1068)
15/04/17 18:25:49 INFO storage.ShuffleBlockFetcherIterator: Getting 1008 
non-empty blocks out of 1008 blocks
15/04/17 18:25:49 INFO storage.ShuffleBlockFetcherIterator: Started 2 remote 
fetches in 16 ms
15/04/17 18:41:19 WARN server.TransportChannelHandler: Exception in connection 
from /10.106.144.109:49697
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:380)
        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/04/17 18:41:27 WARN server.TransportChannelHandler: Exception in connection 
from /10.106.145.10:38473
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:380)
        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)



Same with executor 3:

15/04/17 18:25:52 INFO executor.CoarseGrainedExecutorBackend: Got assigned task 
1092
15/04/17 18:25:52 INFO executor.Executor: Running task 70.0 in stage 15.0 (TID 
1092)
15/04/17 18:25:52 INFO storage.ShuffleBlockFetcherIterator: Getting 1008 
non-empty blocks out of 1008 blocks
15/04/17 18:25:52 INFO storage.ShuffleBlockFetcherIterator: Started 2 remote 
fetches in 4 ms
15/04/17 18:25:52 INFO executor.Executor: Finished task 56.0 in stage 15.0 (TID 
1078). 9791 bytes result sent to driver
15/04/17 18:25:52 INFO executor.CoarseGrainedExecutorBackend: Got assigned task 
1093
15/04/17 18:25:52 INFO executor.Executor: Running task 71.0 in stage 15.0 (TID 
1093)
15/04/17 18:25:52 INFO storage.ShuffleBlockFetcherIterator: Getting 1008 
non-empty blocks out of 1008 blocks
15/04/17 18:25:52 INFO storage.ShuffleBlockFetcherIterator: Started 2 remote 
fetches in 3 ms
15/04/17 18:41:14 WARN server.TransportChannelHandler: Exception in connection 
from /10.106.143.39:46278
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:380)
        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)

> Netty BlockTransferService hangs in the middle of SQL query
> -----------------------------------------------------------
>
>                 Key: SPARK-6962
>                 URL: https://issues.apache.org/jira/browse/SPARK-6962
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 1.2.0, 1.2.1, 1.3.0
>            Reporter: Jon Chase
>         Attachments: jstacks.txt
>
>
> Spark SQL queries (though this seems to be a Spark Core issue - I'm just 
> using queries in the REPL to surface this, so I mention Spark SQL) hang 
> indefinitely under certain (not totally understood) circumstances.  
> This is resolved by setting spark.shuffle.blockTransferService=nio, which 
> seems to point to netty as the issue.  Netty was set as the default for the 
> block transport layer in 1.2.0, which is when this issue started.  Setting 
> the service to nio allows queries to complete normally.
> I do not see this problem when running queries over smaller (~20 5MB files) 
> datasets.  When I increase the scope to include more data (several hundred 
> ~5MB files), the queries will get through several steps but eventuall hang  
> indefinitely.
> Here's the email chain regarding this issue, including stack traces:
> http://mail-archives.apache.org/mod_mbox/spark-user/201503.mbox/<cae61spfqt2y7d5vqzomzz2dmr-jx2c2zggcyky40npkjjx4...@mail.gmail.com>
> For context, here's the announcement regarding the block transfer service 
> change: 
> http://mail-archives.apache.org/mod_mbox/spark-dev/201411.mbox/<cabpqxssl04q+rbltp-d8w+z3atn+g-um6gmdgdnh-hzcvd-...@mail.gmail.com>



--
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

Reply via email to