[GitHub] spark pull request #14887: [SPARK-17321][YARN] YARN shuffle service should u...
Github user zhaoyunjiong closed the pull request at: https://github.com/apache/spark/pull/14887 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #14887: [SPARK-17321][YARN] YARN shuffle service should use good...
Github user zhaoyunjiong commented on the issue: https://github.com/apache/spark/pull/14887 @jerryshao, In the original patch I made a mistake, it should return _recoveryPath in the last instead of return null. After that fix, I don't think it will introduce a situation where recovery data existed in multiple directories. I think your idea is great if yarn didn't set _recoveryPath, skip keep records in db. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #14887: [SPARK-17321][YARN] YARN shuffle service should u...
GitHub user zhaoyunjiong reopened a pull request: https://github.com/apache/spark/pull/14887 [SPARK-17321][YARN] YARN shuffle service should use good disk from yarn.nodemanager.local-dirs ## What changes were proposed in this pull request? Use good disk from yarn.nodemanager.local-dirs if first one was broken. ## How was this patch tested? Manual tests similar code in branch 1.6.2 on one of the node which have first disk broken. You can merge this pull request into a Git repository by running: $ git pull https://github.com/zhaoyunjiong/spark SPARK-17321 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/spark/pull/14887.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #14887 commit b27b5bc1e9c6edad8739ff9748e764e273075369 Author: John Zhao Date: 2016-08-30T22:52:10Z SPARK-17321 YARN shuffle service should use good disk from yarn.nodemanager.local-dirs commit ef06b62d68cbff9c4768ab05009793cde361b54f Author: John Zhao Date: 2016-09-09T00:06:19Z return _recoveryPath if it was set before --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #14887: [SPARK-17321][YARN] YARN shuffle service should u...
Github user zhaoyunjiong closed the pull request at: https://github.com/apache/spark/pull/14887 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #14887: [SPARK-17321][YARN] YARN shuffle service should u...
Github user zhaoyunjiong commented on a diff in the pull request: https://github.com/apache/spark/pull/14887#discussion_r77383971 --- Diff: common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java --- @@ -25,6 +25,8 @@ import com.google.common.collect.Lists; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.util.DiskChecker; --- End diff -- In our case, we have about 200 nodes which have the first disk broken, and spark application is not minority. We can't accept lost so much nodes just because the first disk was broken, also we can't accept let spark application fail randomly (I already patched 1.6 use same logic in our cluster so we can keep enable spark dynamic allocation). If no one see any value of the improvement, that's fine, I'll close this PR. Thank you for your time. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #14887: [SPARK-17321][YARN] YARN shuffle service should u...
Github user zhaoyunjiong commented on a diff in the pull request: https://github.com/apache/spark/pull/14887#discussion_r77246075 --- Diff: common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java --- @@ -25,6 +25,8 @@ import com.google.common.collect.Lists; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.util.DiskChecker; --- End diff -- The problem is when using default settings(recovery and spark.yarn.shuffle.stopOnFailure were disabled), if localDirs[0] was broken, then NodeManager can continue work, but spark tasks running on that NodeManager will fail due to can't connected to port 7337. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #14887: [SPARK-17321][YARN] YARN shuffle service should u...
Github user zhaoyunjiong commented on a diff in the pull request: https://github.com/apache/spark/pull/14887#discussion_r77217138 --- Diff: common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java --- @@ -25,6 +25,8 @@ import com.google.common.collect.Lists; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.util.DiskChecker; --- End diff -- Yes, my log is from spark 1.6, and in that branch YarnShuffleService will put registeredExecutors.ldb under localDirs[0]. By default yarn.nodemanager.recovery.enabled was set to false and spark.yarn.shuffle.stopOnFailure was set to false, lots of user will use those default setting, so I believe that users will continue have issue when localDirs[0] broken. `if (_recoveryPath == null) { _recoveryPath = new Path(localDirs[0]); }` At least set spark.yarn.shuffle.stopOnFailure's default value to true will help user. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #14887: [SPARK-17321][YARN] YARN shuffle service should use good...
Github user zhaoyunjiong commented on the issue: https://github.com/apache/spark/pull/14887 @SaintBacchus Please check below logs: 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)
[GitHub] spark issue #14887: [SPARK-17321][YARN] YARN shuffle service should use good...
Github user zhaoyunjiong commented on the issue: https://github.com/apache/spark/pull/14887 Thanks @vanzin for review. Update PR title, let me know if it's still not properly. If I can't use DiskChecker, I'll try to find something similar in Spark. If there are no similar code in spark, can I just copy DiskChecker to Spark? --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #14887: [SPARK-17321] [YARN]
GitHub user zhaoyunjiong opened a pull request: https://github.com/apache/spark/pull/14887 [SPARK-17321] [YARN] ## What changes were proposed in this pull request? Use good disk from yarn.nodemanager.local-dirs if first one was broken. ## How was this patch tested? Manual tests similar code in branch 1.6.2 on one of the node which have first disk broken. You can merge this pull request into a Git repository by running: $ git pull https://github.com/zhaoyunjiong/spark SPARK-17321 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/spark/pull/14887.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #14887 commit b27b5bc1e9c6edad8739ff9748e764e273075369 Author: John Zhao Date: 2016-08-30T22:52:10Z SPARK-17321 YARN shuffle service should use good disk from yarn.nodemanager.local-dirs --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org