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

Saisai Shao commented on SPARK-15828:
-------------------------------------

I see, but I don't clearly understand your scenario, are the NMs decommissioned 
automatically when there's no container running on it? From my understanding it 
should be triggered manually and occasionally.

This may not be a Spark-only problem, I think there should be some changes in 
the YARN side. Basically I think your usage of YARN may have some conflicts 
with dynamic allocation and it requires mutual operation between two sides.

> YARN is not aware of Spark's External Shuffle Service
> -----------------------------------------------------
>
>                 Key: SPARK-15828
>                 URL: https://issues.apache.org/jira/browse/SPARK-15828
>             Project: Spark
>          Issue Type: Bug
>    Affects Versions: 1.6.1
>         Environment: EMR
>            Reporter: Miles Crawford
>
> When using Spark with dynamic allocation, it is common for all containers on a
> particular YARN node to be released.  This is generally okay because of the
> external shuffle service.
> The problem arises when YARN is attempting to downsize the cluster - once all
> containers on the node are gone, YARN will decommission the node, regardless 
> of
> whether the external shuffle service is still required!
> The once the node is shut down, jobs begin failing with messages such as:
> {code}
> 2016-06-07 18:56:40,016 ERROR o.a.s.n.shuffle.RetryingBlockFetcher: Exception 
> while beginning fetch of 13 outstanding blocks
> java.io.IOException: Failed to connect to 
> ip-10-12-32-67.us-west-2.compute.internal/10.12.32.67:7337
>         at 
> org.apache.spark.network.client.TransportClientFactory.createClient(TransportClientFactory.java:216)
>  
> ~[d58092b50d2880a1c259cb51c6ed83955f97e34a4b75cedaa8ab00f89a09df50-spark-network-common_2.11-1.6.1.jar:1.6.1]
>         at 
> org.apache.spark.network.client.TransportClientFactory.createClient(TransportClientFactory.java:167)
>  
> ~[d58092b50d2880a1c259cb51c6ed83955f97e34a4b75cedaa8ab00f89a09df50-spark-network-common_2.11-1.6.1.jar:1.6.1]
>         at 
> org.apache.spark.network.shuffle.ExternalShuffleClient$1.createAndStart(ExternalShuffleClient.java:105)
>  
> ~[2d5c6a1b64d0070faea2e852616885c0110121f4f5c3206cbde88946abce11c3-spark-network-shuffle_2.11-1.6.1.jar:1.6.1]
>         at 
> org.apache.spark.network.shuffle.RetryingBlockFetcher.fetchAllOutstanding(RetryingBlockFetcher.java:140)
>  
> [2d5c6a1b64d0070faea2e852616885c0110121f4f5c3206cbde88946abce11c3-spark-network-shuffle_2.11-1.6.1.jar:1.6.1]
>         at 
> org.apache.spark.network.shuffle.RetryingBlockFetcher.start(RetryingBlockFetcher.java:120)
>  
> [2d5c6a1b64d0070faea2e852616885c0110121f4f5c3206cbde88946abce11c3-spark-network-shuffle_2.11-1.6.1.jar:1.6.1]
>         at 
> org.apache.spark.network.shuffle.ExternalShuffleClient.fetchBlocks(ExternalShuffleClient.java:114)
>  
> [2d5c6a1b64d0070faea2e852616885c0110121f4f5c3206cbde88946abce11c3-spark-network-shuffle_2.11-1.6.1.jar:1.6.1]
>         at 
> org.apache.spark.storage.ShuffleBlockFetcherIterator.sendRequest(ShuffleBlockFetcherIterator.scala:152)
>  
> [d56f3336b4a0fcc71fe8beb90052dbafd0e88a749bdb4bbb15d37894cf443364-spark-core_2.11-1.6.1.jar:1.6.1]
>         at 
> org.apache.spark.storage.ShuffleBlockFetcherIterator.fetchUpToMaxBytes(ShuffleBlockFetcherIterator.scala:316)
>  
> [d56f3336b4a0fcc71fe8beb90052dbafd0e88a749bdb4bbb15d37894cf443364-spark-core_2.11-1.6.1.jar:1.6.1]
>         at 
> org.apache.spark.storage.ShuffleBlockFetcherIterator.initialize(ShuffleBlockFetcherIterator.scala:263)
>  
> [d56f3336b4a0fcc71fe8beb90052dbafd0e88a749bdb4bbb15d37894cf443364-spark-core_2.11-1.6.1.jar:1.6.1]
>         at 
> org.apache.spark.storage.ShuffleBlockFetcherIterator.<init>(ShuffleBlockFetcherIterator.scala:112)
>  
> [d56f3336b4a0fcc71fe8beb90052dbafd0e88a749bdb4bbb15d37894cf443364-spark-core_2.11-1.6.1.jar:1.6.1]
>         at 
> org.apache.spark.shuffle.BlockStoreShuffleReader.read(BlockStoreShuffleReader.scala:43)
>  
> [d56f3336b4a0fcc71fe8beb90052dbafd0e88a749bdb4bbb15d37894cf443364-spark-core_2.11-1.6.1.jar:1.6.1]
>         at org.apache.spark.rdd.ShuffledRDD.compute(ShuffledRDD.scala:98) 
> [d56f3336b4a0fcc71fe8beb90052dbafd0e88a749bdb4bbb15d37894cf443364-spark-core_2.11-1.6.1.jar:1.6.1]
>         at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306) 
> [d56f3336b4a0fcc71fe8beb90052dbafd0e88a749bdb4bbb15d37894cf443364-spark-core_2.11-1.6.1.jar:1.6.1]
>         at org.apache.spark.rdd.RDD.iterator(RDD.scala:270) 
> [d56f3336b4a0fcc71fe8beb90052dbafd0e88a749bdb4bbb15d37894cf443364-spark-core_2.11-1.6.1.jar:1.6.1]
>         at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) 
> [d56f3336b4a0fcc71fe8beb90052dbafd0e88a749bdb4bbb15d37894cf443364-spark-core_2.11-1.6.1.jar:1.6.1]
>         at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306) 
> [d56f3336b4a0fcc71fe8beb90052dbafd0e88a749bdb4bbb15d37894cf443364-spark-core_2.11-1.6.1.jar:1.6.1]
>         at org.apache.spark.rdd.RDD.iterator(RDD.scala:270) 
> [d56f3336b4a0fcc71fe8beb90052dbafd0e88a749bdb4bbb15d37894cf443364-spark-core_2.11-1.6.1.jar:1.6.1]
>         at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) 
> [d56f3336b4a0fcc71fe8beb90052dbafd0e88a749bdb4bbb15d37894cf443364-spark-core_2.11-1.6.1.jar:1.6.1]
>         at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306) 
> [d56f3336b4a0fcc71fe8beb90052dbafd0e88a749bdb4bbb15d37894cf443364-spark-core_2.11-1.6.1.jar:1.6.1]
>         at org.apache.spark.rdd.RDD.iterator(RDD.scala:270) 
> [d56f3336b4a0fcc71fe8beb90052dbafd0e88a749bdb4bbb15d37894cf443364-spark-core_2.11-1.6.1.jar:1.6.1]
>         at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66) 
> [d56f3336b4a0fcc71fe8beb90052dbafd0e88a749bdb4bbb15d37894cf443364-spark-core_2.11-1.6.1.jar:1.6.1]
>         at org.apache.spark.scheduler.Task.run(Task.scala:89) 
> [d56f3336b4a0fcc71fe8beb90052dbafd0e88a749bdb4bbb15d37894cf443364-spark-core_2.11-1.6.1.jar:1.6.1]
>         at 
> org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214) 
> [d56f3336b4a0fcc71fe8beb90052dbafd0e88a749bdb4bbb15d37894cf443364-spark-core_2.11-1.6.1.jar:1.6.1]
>         at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>  [na:1.8.0_91]
>         at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  [na:1.8.0_91]
>         at java.lang.Thread.run(Thread.java:745) [na:1.8.0_91]
> Caused by: java.net.ConnectException: Connection refused: 
> ip-10-12-32-67.us-west-2.compute.internal/10.12.32.67:7337
>         at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method) 
> ~[na:1.8.0_91]
>         at 
> sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717) 
> ~[na:1.8.0_91]
>         at 
> io.netty.channel.socket.nio.NioSocketChannel.doFinishConnect(NioSocketChannel.java:224)
>  
> ~[9cb53ee0c581954e1f791bedcaea3a54e8e31b2f093da96e943faa342baad529-netty-all-4.0.29.Final.jar:4.0.29.Final]
>         at 
> io.netty.channel.nio.AbstractNioChannel$AbstractNioUnsafe.finishConnect(AbstractNioChannel.java:289)
>  
> ~[9cb53ee0c581954e1f791bedcaea3a54e8e31b2f093da96e943faa342baad529-netty-all-4.0.29.Final.jar:4.0.29.Final]
>         at 
> io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:528) 
> ~[9cb53ee0c581954e1f791bedcaea3a54e8e31b2f093da96e943faa342baad529-netty-all-4.0.29.Final.jar:4.0.29.Final]
>         at 
> io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468)
>  
> ~[9cb53ee0c581954e1f791bedcaea3a54e8e31b2f093da96e943faa342baad529-netty-all-4.0.29.Final.jar:4.0.29.Final]
>         at 
> io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382) 
> ~[9cb53ee0c581954e1f791bedcaea3a54e8e31b2f093da96e943faa342baad529-netty-all-4.0.29.Final.jar:4.0.29.Final]
>         at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354) 
> ~[9cb53ee0c581954e1f791bedcaea3a54e8e31b2f093da96e943faa342baad529-netty-all-4.0.29.Final.jar:4.0.29.Final]
>         at 
> io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:111)
>  
> ~[9cb53ee0c581954e1f791bedcaea3a54e8e31b2f093da96e943faa342baad529-netty-all-4.0.29.Final.jar:4.0.29.Final]
>         ... 1 common frames omitted
> {code}
> Because of this issue dynamic allocation and auto-scaling cannot be used 
> together.



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