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

yunjiong zhao commented on SPARK-17321:
---------------------------------------

Below logs shows that NodeManager already detect the disk failure, but 
ExternalShuffleBlockResolver still use the failure disk.

2016-08-30 10:16:24,982 INFO 
org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService: Disk(s) 
failed. 3/12 local-dirs turned bad: 
/hadoop/1/scratch/local,/hadoop/7/scratch/local,/hadoop/10/scratch/local;3/12 
log-dirs turned bad: 
/hadoop/1/scratch/logs,/hadoop/7/scratch/logs,/hadoop/10/scratch/logs
...
2016-08-30 10:16:38,008 INFO org.apache.spark.network.yarn.YarnShuffleService: 
Initializing YARN shuffle service for Spark
2016-08-30 10:16:38,008 INFO 
org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServices: Adding 
auxiliary service spark_shuffle, "spark_shuffle"
2016-08-30 10:16:38,260 ERROR 
org.apache.spark.network.shuffle.ExternalShuffleBlockResolver: error opening 
leveldb file /hadoop/1/scratch/local/registeredExecutors.ldb. Creating new 
file, will not be able to recover state for existing applications
org.fusesource.leveldbjni.internal.NativeDB$DBException: IO error: 
/hadoop/1/scratch/local/registeredExecutors.ldb/LOCK: No such file or directory
at org.fusesource.leveldbjni.internal.NativeDB.checkStatus(NativeDB.java:200)
at org.fusesource.leveldbjni.internal.NativeDB.open(NativeDB.java:218)
at org.fusesource.leveldbjni.JniDBFactory.open(JniDBFactory.java:168)
at 
org.apache.spark.network.shuffle.ExternalShuffleBlockResolver.(ExternalShuffleBlockResolver.java:100)
at 
org.apache.spark.network.shuffle.ExternalShuffleBlockResolver.(ExternalShuffleBlockResolver.java:81)
at 
org.apache.spark.network.shuffle.ExternalShuffleBlockHandler.(ExternalShuffleBlockHandler.java:56)
at 
org.apache.spark.network.yarn.YarnShuffleService.serviceInit(YarnShuffleService.java:129)
at org.apache.hadoop.service.AbstractService.init(AbstractService.java:163)
at 
org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServices.serviceInit(AuxServices.java:122)
at org.apache.hadoop.service.AbstractService.init(AbstractService.java:163)
at 
org.apache.hadoop.service.CompositeService.serviceInit(CompositeService.java:107)
at 
org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl.serviceInit(ContainerManagerImpl.java:220)
at org.apache.hadoop.service.AbstractService.init(AbstractService.java:163)
at 
org.apache.hadoop.service.CompositeService.serviceInit(CompositeService.java:107)
at 
org.apache.hadoop.yarn.server.nodemanager.NodeManager.serviceInit(NodeManager.java:186)
at org.apache.hadoop.service.AbstractService.init(AbstractService.java:163)
at 
org.apache.hadoop.yarn.server.nodemanager.NodeManager.initAndStartNodeManager(NodeManager.java:357)
at 
org.apache.hadoop.yarn.server.nodemanager.NodeManager.main(NodeManager.java:404)
2016-08-30 10:16:38,262 WARN 
org.apache.spark.network.shuffle.ExternalShuffleBlockResolver: error deleting 
/hadoop/1/scratch/local/registeredExecutors.ldb
2016-08-30 10:16:38,262 ERROR org.apache.spark.network.yarn.YarnShuffleService: 
Failed to initialize external shuffle service
java.io.IOException: Unable to create state store
at 
org.apache.spark.network.shuffle.ExternalShuffleBlockResolver.(ExternalShuffleBlockResolver.java:129)
at 
org.apache.spark.network.shuffle.ExternalShuffleBlockResolver.(ExternalShuffleBlockResolver.java:81)
at 
org.apache.spark.network.shuffle.ExternalShuffleBlockHandler.(ExternalShuffleBlockHandler.java:56)
at 
org.apache.spark.network.yarn.YarnShuffleService.serviceInit(YarnShuffleService.java:129)
at org.apache.hadoop.service.AbstractService.init(AbstractService.java:163)
at 
org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServices.serviceInit(AuxServices.java:122)
at org.apache.hadoop.service.AbstractService.init(AbstractService.java:163)
at 
org.apache.hadoop.service.CompositeService.serviceInit(CompositeService.java:107)
at 
org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl.serviceInit(ContainerManagerImpl.java:220)
at org.apache.hadoop.service.AbstractService.init(AbstractService.java:163)
at 
org.apache.hadoop.service.CompositeService.serviceInit(CompositeService.java:107)
at 
org.apache.hadoop.yarn.server.nodemanager.NodeManager.serviceInit(NodeManager.java:186)
at org.apache.hadoop.service.AbstractService.init(AbstractService.java:163)
at 
org.apache.hadoop.yarn.server.nodemanager.NodeManager.initAndStartNodeManager(NodeManager.java:357)
at 
org.apache.hadoop.yarn.server.nodemanager.NodeManager.main(NodeManager.java:404)
Caused by: org.fusesource.leveldbjni.internal.NativeDB$DBException: IO error: 
/hadoop/1/scratch/local/registeredExecutors.ldb/LOCK: No such file or directory
at org.fusesource.leveldbjni.internal.NativeDB.checkStatus(NativeDB.java:200)
at org.fusesource.leveldbjni.internal.NativeDB.open(NativeDB.java:218)
at org.fusesource.leveldbjni.JniDBFactory.open(JniDBFactory.java:168)
at 
org.apache.spark.network.shuffle.ExternalShuffleBlockResolver.(ExternalShuffleBlockResolver.java:127)
... 14 more
2016-08-30 10:16:38,456 INFO org.apache.spark.network.yarn.YarnShuffleService: 
Started YARN shuffle service for Spark on port 7337. Authentication is not 
enabled. Registered executor file is 
/hadoop/1/scratch/local/registeredExecutors.ldb

> YARN shuffle service should use good disk from yarn.nodemanager.local-dirs
> --------------------------------------------------------------------------
>
>                 Key: SPARK-17321
>                 URL: https://issues.apache.org/jira/browse/SPARK-17321
>             Project: Spark
>          Issue Type: Bug
>          Components: YARN
>    Affects Versions: 1.6.2, 2.0.0
>            Reporter: yunjiong zhao
>
> We run spark on yarn, after enabled spark dynamic allocation, we notice some 
> spark application failed randomly due to YarnShuffleService.
> From log I found
> {quote}
> 2016-08-29 11:33:03,450 ERROR org.apache.spark.network.TransportContext: 
> Error while initializing Netty pipeline
> java.lang.NullPointerException
>         at 
> org.apache.spark.network.server.TransportRequestHandler.<init>(TransportRequestHandler.java:77)
>         at 
> org.apache.spark.network.TransportContext.createChannelHandler(TransportContext.java:159)
>         at 
> org.apache.spark.network.TransportContext.initializePipeline(TransportContext.java:135)
>         at 
> org.apache.spark.network.server.TransportServer$1.initChannel(TransportServer.java:123)
>         at 
> org.apache.spark.network.server.TransportServer$1.initChannel(TransportServer.java:116)
>         at 
> io.netty.channel.ChannelInitializer.channelRegistered(ChannelInitializer.java:69)
>         at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRegistered(AbstractChannelHandlerContext.java:133)
>         at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRegistered(AbstractChannelHandlerContext.java:119)
>         at 
> io.netty.channel.DefaultChannelPipeline.fireChannelRegistered(DefaultChannelPipeline.java:733)
>         at 
> io.netty.channel.AbstractChannel$AbstractUnsafe.register0(AbstractChannel.java:450)
>         at 
> io.netty.channel.AbstractChannel$AbstractUnsafe.access$100(AbstractChannel.java:378)
>         at 
> io.netty.channel.AbstractChannel$AbstractUnsafe$1.run(AbstractChannel.java:424)
>         at 
> io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:357)
>         at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:357)
>         at 
> io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:111)
>         at java.lang.Thread.run(Thread.java:745)
> {quote} 
> Which caused by the first disk in yarn.nodemanager.local-dirs was broken.
> If we enabled spark.yarn.shuffle.stopOnFailure(SPARK-16505) we might lost 
> hundred nodes which is unacceptable.
> We have 12 disks in yarn.nodemanager.local-dirs, so why not use other good 
> disks if the first one is broken?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to