[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14263956#comment-14263956 ] Stephen Haberman commented on SPARK-3633: - Sorry for the latest reply on this, but our skew was a join with a key of 15k rows one side, and 165k rows on the other, which resulted in .join trying to create a list with ~2.5 billion entries when joined together (this was before the recent use an iterator fix to .join). So, yes, this pounded the GC which then showed up as fetch failures to the rest of our nodes. Apologies for the false negative. It is odd that somehow 0.9.2 powered through this, which is why I thought the job failing in 1.1/1.2 was a regression. We were probably just getting lucky with having barely enough memory/something (even though using the same machine sizes). Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Priority: Blocker Fix For: 1.1.1, 1.2.0 Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14235999#comment-14235999 ] Josh Rosen commented on SPARK-3633: --- [~stephen], Do you know if the hosts that failed to ACK experienced long GC pauses? Did bumping up the connection timeout help? I'd like to try to figure out whether this is still an issue. Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Priority: Blocker Fix For: 1.1.1, 1.2.0 Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14236163#comment-14236163 ] Stephen Haberman commented on SPARK-3633: - Hi Josh, Yes, it was GC issues; however, we're still kind of tracking down. Turns out the job that is failing is a real PITA, and is a month-to-date report, and started failing even on 0.9.2 at the end of last month. We were able to get it to run on 0.9.2 with 8 i2.2xlarge machines (with 1.6gb SSDs) because we have a stage with ~1700 tasks, where the average shuffle write is ~400mb, but two of the tasks have huge schew, and shuffle writes of 15gb and 125gb (!). Since we got the rerun to work on the i2.2xlarges on 0.9.2, we have not been able to try the same data/job on 1.1.1/1.2, but it's on my list of things to do. (Obviously we're going to want to solve this underlying schew issue as well, but, in theory :-), it should just lead to really slow jobs, vs. OOMEs/etc.) Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Priority: Blocker Fix For: 1.1.1, 1.2.0 Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14223645#comment-14223645 ] Stephen Haberman commented on SPARK-3633: - I just tried a job on 1.1.1-rc2 and am still getting this issue: Could not get block(s) from ConnectionManagerId(ip-10-13-192-156.ec2.internal,47193) java.io.IOException: sendMessageReliably failed because ack was not received within 60 sec The cluster was still running; when I logged into that host (the -156.ec2.internal that did not respond to the ack), it was just fine, and *also* getting failed because ack not received messages, at basically the same time (within ~2 seconds of the other host). I checked another host, and same thing; AFAICT it looks like ~3 hosts of out the 20 all had problems responding to acks within the 60s timeout, for ~1-2 minutes. I'm going to up the ack connection timeout to 600, per an earlier comment, and try again. Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Priority: Blocker Fix For: 1.1.1, 1.2.0 Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14222501#comment-14222501 ] Stephen Haberman commented on SPARK-3633: - Is there a Spark 1.1.1/1.2 snapshot/RC in a Maven repo somewhere to try out the fixes for this? (SPARK-3493 and Spark-4480)? Or is 1.1.1 close enough to release that I should just sit tight? Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Priority: Blocker Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14222545#comment-14222545 ] Matei Zaharia commented on SPARK-3633: -- [~stephen] you can try the 1.1.1 RC in http://apache-spark-developers-list.1001551.n3.nabble.com/VOTE-Release-Apache-Spark-1-1-1-RC2-td9439.html, which includes a Maven staging repo that you can just add as a repo in a build. Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Priority: Blocker Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14222611#comment-14222611 ] Stephen Haberman commented on SPARK-3633: - Hi Matei; cool, thanks for the 1.1.1 link. In my head that had happened before these bugs got fixed, but I maybe I was thinking of rc1. Thanks! Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Priority: Blocker Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14220728#comment-14220728 ] Nishkam Ravi commented on SPARK-3633: - Fixed by PR for SPARK-4480. Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Priority: Blocker Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14221699#comment-14221699 ] Patrick Wendell commented on SPARK-3633: [~nravi] resolved this because his original issue was solved. However, subsequent comments have identified other issues, so if there are still users facing out-standing issues related to this, please feel free to open new JIRA's and link back to this one. Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Priority: Blocker Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14219765#comment-14219765 ] Hector Yee commented on SPARK-3633: --- I'm still seeing a similar error in spark 1.2 rc2 14/11/20 18:41:12 WARN TaskSetManager: Lost task 4.1 in stage 1.0 (TID 907, i-8cb72661.inst.aws.airbnb.com): FetchFailed(null, shuffleId=1, mapId=-1, reduceId=4, message= org.apache.spark.shuffle.MetadataFetchFailedException: Missing an output location for shuffle 1 at org.apache.spark.MapOutputTracker$$anonfun$org$apache$spark$MapOutputTracker$$convertMapStatuses$1.apply(MapOutputTracker.scala:386) at org.apache.spark.MapOutputTracker$$anonfun$org$apache$spark$MapOutputTracker$$convertMapStatuses$1.apply(MapOutputTracker.scala:383) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) at scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33) at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:108) at scala.collection.TraversableLike$class.map(TraversableLike.scala:244) at scala.collection.mutable.ArrayOps$ofRef.map(ArrayOps.scala:108) at org.apache.spark.MapOutputTracker$.org$apache$spark$MapOutputTracker$$convertMapStatuses(MapOutputTracker.scala:382) at org.apache.spark.MapOutputTracker.getServerStatuses(MapOutputTracker.scala:178) at org.apache.spark.shuffle.hash.BlockStoreShuffleFetcher$.fetch(BlockStoreShuffleFetcher.scala:42) at org.apache.spark.shuffle.hash.HashShuffleReader.read(HashShuffleReader.scala:40) at org.apache.spark.rdd.ShuffledRDD.compute(ShuffledRDD.scala:92) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:263) at org.apache.spark.rdd.RDD.iterator(RDD.scala:230) Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Priority: Blocker Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14219807#comment-14219807 ] Hector Yee commented on SPARK-3633: --- I think it may be a different bug.. looked at the failed executor and it looks like something is closing the connection causing fetches to fail 14/11/20 18:53:43 INFO TransportClientFactory: Found inactive connection to i-974cd879.inst.aws.airbnb.com/10.154.228.43:57773, closing it. 14/11/20 18:53:43 ERROR RetryingBlockFetcher: Exception while beginning fetch of 1 outstanding blocks java.io.IOException: Failed to connect to i-974cd879.inst.aws.airbnb.com/10.154.228.43:57773 at org.apache.spark.network.client.TransportClientFactory.createClient(TransportClientFactory.java:141) at org.apache.spark.network.netty.NettyBlockTransferService$$anon$1.createAndStart(NettyBlockTransferService.scala:78) at org.apache.spark.network.shuffle.RetryingBlockFetcher.fetchAllOutstanding(RetryingBlockFetcher.java:140) at org.apache.spark.network.shuffle.RetryingBlockFetcher.start(RetryingBlockFetcher.java:120) at org.apache.spark.network.netty.NettyBlockTransferService.fetchBlocks(NettyBlockTransferService.scala:87) at org.apache.spark.storage.ShuffleBlockFetcherIterator.sendRequest(ShuffleBlockFetcherIterator.scala:148) at org.apache.spark.storage.ShuffleBlockFetcherIterator.next(ShuffleBlockFetcherIterator.scala:288) at org.apache.spark.storage.ShuffleBlockFetcherIterator.next(ShuffleBlockFetcherIterator.scala:52) at scala.collection.Iterator$$anon$13.hasNext(Iterator.scala:371) at org.apache.spark.util.CompletionIterator.hasNext(CompletionIterator.scala:32) at org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:39) at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:327) at scala.collection.Iterator$$anon$13.hasNext(Iterator.scala:371) at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:327) at scala.collection.Iterator$class.foreach(Iterator.scala:727) at scala.collection.AbstractIterator.foreach(Iterator.scala:1157) at com.airbnb.common.ml.training.LinearRankerTrainer$$anonfun$7.apply(LinearRankerTrainer.scala:246) at com.airbnb.common.ml.training.LinearRankerTrainer$$anonfun$7.apply(LinearRankerTrainer.scala:235) at org.apache.spark.rdd.RDD$$anonfun$13.apply(RDD.scala:601) at org.apache.spark.rdd.RDD$$anonfun$13.apply(RDD.scala:601) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:35) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:263) at org.apache.spark.rdd.RDD.iterator(RDD.scala:230) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) at org.apache.spark.scheduler.Task.run(Task.scala:56) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:196) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:745) Caused by: java.net.ConnectException: Connection refused: i-974cd879.inst.aws.airbnb.com/10.154.228.43:57773 at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method) at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:739) at io.netty.channel.socket.nio.NioSocketChannel.doFinishConnect(NioSocketChannel.java:208) at io.netty.channel.nio.AbstractNioChannel$AbstractNioUnsafe.finishConnect(AbstractNioChannel.java:287) at io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:528) 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) ... 1 more Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Priority: Blocker Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com):
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14217300#comment-14217300 ] Andrew Or commented on SPARK-3633: -- I have filed SPARK-4480 as an immediate fix. The longer term solution is described in SPARK-4452 and will likely be targeted for 1.3, though I imagine that the fix for SPARK-4480 is sufficient for most cases. Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Priority: Blocker Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14215417#comment-14215417 ] Arun Ahuja commented on SPARK-3633: --- [~andrewor14] We were using Hash-Based shuffle when we were running into this due to the Snappy+Kryo issue with sort based shuffle Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Priority: Blocker Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14215403#comment-14215403 ] Andrew Or commented on SPARK-3633: -- Hey [~nravi] [~arahuja] were you using sort-based or hash-based shuffle when you hit this? Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Priority: Blocker Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14210100#comment-14210100 ] Stephen Haberman commented on SPARK-3633: - FWIW we've seen this issue several times on jobs that have moved to Spark 1.1. Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Priority: Critical Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14211624#comment-14211624 ] Josh Rosen commented on SPARK-3633: --- I've opened SPARK-4393 to address the memory leak in the ConnectionManager timers and submitted a pull request. Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Priority: Blocker Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14208028#comment-14208028 ] Cristian Opris commented on SPARK-3633: --- FWIW I get this as well, with a very straightforward job and setup. Spark 1.1.0, executors configured to 2GB, storage.fraction=0.2, shuffle.spill=true 50GB dataset on ext4, spread over 7000 files, hence the coalescing below The jobs is only doing: input.coalesce(72, false).groupBy(key).count The groupBy is successful then I get the dreaded fetch error on count stage (oddly enough), but it seems to me that's when it does the actual shuffling for groupBy ? Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Priority: Critical Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14208318#comment-14208318 ] Cristian Opris commented on SPARK-3633: --- This looks like a memory leak in ConnectionManager where responses (BufferMessage) are retained by the TimerTask waiting for ACK even after the Future completes with Success, please see the Possibly related to https://github.com/apache/spark/commit/76fa0eaf515fd6771cdd69422b1259485debcae5 +--+--+--+-+ |Class | Objects| Shallow Size | Retained Size | +--+--+--+-+ | java.util.TaskQueue |10 % | 240 % | 885,048,168 100 % | | java.util.TimerTask[] |10 % | 2,0640 % | 885,048,144 99 % | | org.apache.spark.network.ConnectionManager$$anon$5 | 2865 % | 13,7280 % | ~ 885,046,080 99 % | | org.apache.spark.network.BufferMessage | 572 10 % | 36,6080 % | ~ 885,018,624 99 % | | scala.concurrent.impl.Promise$DefaultPromise| 2865 % | 4,5760 % | ~ 884,968,288 99 % | | scala.util.Success | 2865 % | 4,5760 % | ~ 884,963,712 99 % | | scala.collection.mutable.ArrayBuffer| 572 10 % | 13,7280 % | ~ 884,915,768 99 % | | java.lang.Object[] | 572 10 % | 45,7600 % | ~ 884,902,040 99 % | | java.nio.HeapByteBuffer | 2865 % | 13,7280 % | ~ 884,856,280 99 % | | byte[] | 2865 % | 884,842,552 99 % | ~ 884,842,552 99 % | | java.net.InetSocketAddress | 572 10 % | 9,1520 % | ~ 66,2480 % | | java.net.InetSocketAddress$InetSocketAddressHolder | 572 10 % | 13,7280 % | ~ 57,0960 % | | java.net.Inet4Address | 2865 % | 6,8640 % | ~ 43,3680 % | | java.net.InetAddress$InetAddressHolder | 2865 % | 6,8640 % | ~ 36,5040 % | | java.lang.String| 2855 % | 6,8400 % | ~ 29,6400 % | | char[] | 2855 % | 22,8000 % | ~ 22,8000 % | | java.lang.Object| 2865 % | 4,5760 % |~ 4,5760 % | +--+--+--+-+ Generated by YourKit Java Profiler 2014 build 14110 12-Nov-2014 17:44:32 Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Priority: Critical Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145)
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14208378#comment-14208378 ] Cristian Opris commented on SPARK-3633: --- At first sight (haven't tested this) the problem is in the code below. The TimerTask is cancelled on Success but this doesn't actually remove it from the Timer TaskQueue since the TimerThread doesn't actually remove cancelled tasks until they're actually scheduled to run, which in this case is by default 60 secs ack timeout. A quick fix would be to call Timer.purge() after task cancel below, or better yet change to a better Timer like the HashedWheel one from Netty {code:title=|borderStyle=solid} val status = new MessageStatus(message, connectionManagerId, s = { timeoutTask.cancel() s.ackMessage match { case None = // Indicates a failure where we either never sent or never got ACK'd promise.failure(new IOException(sendMessageReliably failed without being ACK'd)) case Some(ackMessage) = if (ackMessage.hasError) { promise.failure( new IOException(sendMessageReliably failed with ACK that signalled a remote error)) } else { promise.success(ackMessage) } } }) {code} Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Priority: Critical Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14176747#comment-14176747 ] Saisai Shao commented on SPARK-3633: From my test, I think this problem might be existed for a while, previously we do not have timeout ack mechanism, so we might ignore this problem, after this commit (https://github.com/apache/spark/commit/bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2) is merged, the timeout checking mechanism will raise the exception as mentioned before. My previous assumption is that there might be some thread contention or lock issue in NioBlockTransferService after refactoring, but after I roll back to branch 1.1 without refactoring connection module, the problem still exists. So I guess there might be a problem in connection module, we neglected it until this patch is merged. Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Priority: Critical Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14163859#comment-14163859 ] Chen Song commented on SPARK-3633: -- Looks like we have addressed fetch failure caused by Too many files open. Anyone has more insight on the timeout thing? The timeout happened during the transfer of BufferAckMessage between the sender and receiver. To shed more light on this issue, I turned on DEBUG level logging and it kind of give the trace of life cycle of this event. * On sender host, sending of the message seems healthy. {noformat} 4/09/25 19:59:48 DEBUG ConnectionManager: Before Sending [BufferAckMessage(aid = 582, id = 1503, size = 9601)] to [ConnectionManagerId(receiver_host,52315)] connectionid: sender_host_60072_260 14/09/25 19:59:48 DEBUG ConnectionManager: Sending [BufferAckMessage(aid = 582, id = 1503, size = 9601)] to [ConnectionManagerId(receiver_host,52315)] 14/09/25 19:59:48 DEBUG SendingConnection: Added [BufferAckMessage(aid = 582, id = 1503, size = 9601)] to outbox for sending to [ConnectionManagerId(receiver_host,52315)] 14/09/25 19:59:48 DEBUG SendingConnection: Starting to send [BufferAckMessage(aid = 582, id = 1503, size = 9601)] to [ConnectionManagerId(receiver_host,52315)] 14/09/25 19:59:48 TRACE SendingConnection: Sending chunk from [BufferAckMessage(aid = 582, id = 1503, size = 9601)] to [ConnectionManagerId(receiver_host,52315)] 14/09/25 19:59:48 TRACE SendingConnection: Sending chunk from [BufferAckMessage(aid = 582, id = 1503, size = 9601)] to [ConnectionManagerId(receiver_host,52315)] 14/09/25 19:59:48 TRACE SendingConnection: Sending chunk from [BufferAckMessage(aid = 582, id = 1503, size = 9601)] to [ConnectionManagerId(receiver_host,52315)] 14/09/25 19:59:48 TRACE SendingConnection: Sending chunk from [BufferAckMessage(aid = 582, id = 1503, size = 9601)] to [ConnectionManagerId(receiver_host,52315)] 14/09/25 19:59:48 TRACE SendingConnection: Sending chunk from [BufferAckMessage(aid = 582, id = 1503, size = 9601)] to [ConnectionManagerId(receiver_host,52315)] 14/09/25 19:59:48 TRACE SendingConnection: Sending chunk from [BufferAckMessage(aid = 582, id = 1503, size = 9601)] to [ConnectionManagerId(receiver_host,52315)] 14/09/25 19:59:48 TRACE SendingConnection: Sending chunk from [BufferAckMessage(aid = 582, id = 1503, size = 9601)] to [ConnectionManagerId(receiver_host,52315)] 14/09/25 19:59:48 TRACE SendingConnection: Sending chunk from [BufferAckMessage(aid = 582, id = 1503, size = 9601)] to [ConnectionManagerId(receiver_host,52315)] 14/09/25 19:59:48 TRACE SendingConnection: Sending chunk from [BufferAckMessage(aid = 582, id = 1503, size = 9601)] to [ConnectionManagerId(receiver_host,52315)] 14/09/25 19:59:48 TRACE SendingConnection: Sending chunk from [BufferAckMessage(aid = 582, id = 1503, size = 9601)] to [ConnectionManagerId(receiver_host,52315)] 14/09/25 19:59:48 TRACE SendingConnection: Sending chunk from [BufferAckMessage(aid = 582, id = 1503, size = 9601)] to [ConnectionManagerId(receiver_host,52315)] 14/09/25 19:59:48 TRACE SendingConnection: Sending chunk from [BufferAckMessage(aid = 582, id = 1503, size = 9601)] to [ConnectionManagerId(receiver_host,52315)] 14/09/25 19:59:48 TRACE SendingConnection: Sending chunk from [BufferAckMessage(aid = 582, id = 1503, size = 9601)] to [ConnectionManagerId(receiver_host,52315)] 14/09/25 19:59:48 TRACE SendingConnection: Sending chunk from [BufferAckMessage(aid = 582, id = 1503, size = 9601)] to [ConnectionManagerId(receiver_host,52315)] 14/09/25 19:59:48 TRACE SendingConnection: Sending chunk from [BufferAckMessage(aid = 582, id = 1503, size = 9601)] to [ConnectionManagerId(receiver_host,52315)] 14/09/25 19:59:48 TRACE SendingConnection: Sending chunk from [BufferAckMessage(aid = 582, id = 1503, size = 9601)] to [ConnectionManagerId(receiver_host,52315)] 14/09/25 19:59:48 TRACE SendingConnection: Sending chunk from [BufferAckMessage(aid = 582, id = 1503, size = 9601)] to [ConnectionManagerId(receiver_host,52315)] 14/09/25 19:59:48 TRACE SendingConnection: Sending chunk from [BufferAckMessage(aid = 582, id = 1503, size = 9601)] to [ConnectionManagerId(receiver_host,52315)] 14/09/25 19:59:48 TRACE SendingConnection: Sending chunk from [BufferAckMessage(aid = 582, id = 1503, size = 9601)] to [ConnectionManagerId(receiver_host,52315)] 14/09/25 19:59:48 TRACE SendingConnection: Sending chunk from [BufferAckMessage(aid = 582, id = 1503, size = 9601)] to [ConnectionManagerId(receiver_host,52315)] 14/09/25 19:59:48 TRACE SendingConnection: Sending chunk from [BufferAckMessage(aid = 582, id = 1503, size = 9601)] to [ConnectionManagerId(receiver_host,52315)] 14/09/25 19:59:48 TRACE SendingConnection: Sending chunk from [BufferAckMessage(aid = 582, id = 1503, size = 9601)] to [ConnectionManagerId(receiver_host,52315)] 14/09/25 19:59:48 TRACE SendingConnection: Sending
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14161117#comment-14161117 ] Matei Zaharia commented on SPARK-3633: -- I'm curious, why do you think this is caused by SPARK-2711? That will cause fewer files to be created for intermediate spills, if anything. It seems this might be due to changes in the shuffle or communication code. Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Priority: Critical Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14161120#comment-14161120 ] Marcelo Vanzin commented on SPARK-3633: --- [~matei], as Nishkam mentions above, we reverted that change in our internal, 1.1-based branch and all the observed issues (exceptions during fetch and performance) were fixed. Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Priority: Critical Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14161142#comment-14161142 ] Matei Zaharia commented on SPARK-3633: -- In that case though, the problem might be that these maps are allocating more memory without that patch, and exceeding the spark.shuffle.memoryFraction, which would lead to other bugs. That is also consistent with it running faster. It would be good to investigate why exactly this happened (e.g. was this job just at the edge of exceeding its ulimit). It sounds like PageRank has exactly the problem described in SPARK-2711 (you have reduce tasks that are also writing map output files for the next stage, so they contain two ExternalAppendOnlyMaps, so the old version of ExternalAppendOnlyMap does not properly account for memory). That's going to lead to trouble in jobs that are also doing caching. Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Priority: Critical Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14159059#comment-14159059 ] Nishkam Ravi commented on SPARK-3633: - Quick update: reverting 4fde28c gets rid of the fetch failures and recovers significant perf regressions in two workloads. Thanks [~vanzin]. Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14157351#comment-14157351 ] Marcelo Vanzin commented on SPARK-3633: --- Hey [~pwendell] [~matei], is anyone activelly looking at this issue? Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14155415#comment-14155415 ] Chen Song commented on SPARK-3633: -- I tried increasing timeout for this property spark.core.connection.ack.wait.timeout. I saw less fetch failures due to ack timeout but they still exist. I also tried relaxing the following properties but none of them seems to help. spark.core.connection.handler.threads.* spark.core.connection.io.threads.* spark.core.connection.connect.threads.* Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14156074#comment-14156074 ] Nishkam Ravi commented on SPARK-3633: - For a different workload (variant of TeraSort), I see fetch failures in the standalone mode but not with YARN (with identical ulimit and timeout values). Wondering why this might be. Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14147918#comment-14147918 ] Arun Ahuja commented on SPARK-3633: --- Which timeout values were increased to work around this? We have been seeing many more errors with FetchFailed(BlockManagerId(21, And I also see a java.io.IOException: Failed to list files for dir: /data/09/mapred/local/yarn/nm/usercache/ahujaa01/appcache/application_1403901413406_1926/spark-local-20140925115858-c4a7 at org.apache.spark.util.Utils$.listFilesSafely(Utils.scala:673) at org.apache.spark.util.Utils$.deleteRecursively(Utils.scala:685) exception with that failure Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14147939#comment-14147939 ] Zhan Zhang commented on SPARK-3633: --- Increasing timeout does not help my case either. I still keep getting fetch error. Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14148004#comment-14148004 ] Arun Ahuja commented on SPARK-3633: --- Also, which timeout setting was useful: spark.akka.timeout or spark.core.connection.ack.wait.timeout. Using GC logging I see this both when there are many Full GC and even on smaller datasets when there are not. It much more frequent on the former. Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14148026#comment-14148026 ] Nishkam Ravi commented on SPARK-3633: - Increasing the value of spark.core.connection.ack.wait.timeout (600) worked in my case Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14144490#comment-14144490 ] Patrick Wendell commented on SPARK-3633: [~nravi] if you are trying to debug this, try and look at what file handles are open by the executor process using lsof. This can usually help narrow down bugs of this nature. Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14145676#comment-14145676 ] Nishkam Ravi commented on SPARK-3633: - [~pwendell] I see a large number of open files of the form shuffle* (e.g., shuffle_3_101_23) as expected. Trying higher values of ulimit to see if/when this issue goes away (that would only help understand the extent of the problem). Since we have narrowed the problem down to a specific commit + know what the issue is, could someone more familiar with that code try and see where the bug is? Unless creation of a large number of temp shuffle files is a known side-effect of the commit. In which case, it may need to be rethought at a design level? Here is the latest ulimit settings on the cluster (rebooted): core file size (blocks, -c) 0 data seg size (kbytes, -d) unlimited scheduling priority (-e) 0 file size (blocks, -f) unlimited pending signals (-i) 385910 max locked memory (kbytes, -l) 64 max memory size (kbytes, -m) unlimited open files (-n) 100 pipe size(512 bytes, -p) 8 POSIX message queues (bytes, -q) 819200 real-time priority (-r) 0 stack size (kbytes, -s) 10240 cpu time (seconds, -t) unlimited max user processes (-u) 32768 virtual memory (kbytes, -v) unlimited file locks (-x) unlimited Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14145686#comment-14145686 ] Andrew Ash commented on SPARK-3633: --- Hi [~nravi] you can also check the limits on an individual process (sometimes they're different from system limits) on linux with {{cat /proc/$PID/limits}} Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14145713#comment-14145713 ] Nishkam Ravi commented on SPARK-3633: - Looks ok (still waiting on the run to finish/fail): Limit Soft Limit Hard Limit Units Max cpu time unlimitedunlimitedseconds Max file size unlimitedunlimitedbytes Max data size unlimitedunlimitedbytes Max stack size10485760 unlimitedbytes Max core file size0unlimitedbytes Max resident set unlimitedunlimitedbytes Max processes 6553665536processes Max open files100 100 files Max locked memory unlimitedunlimitedbytes Max address space unlimitedunlimitedbytes Max file locksunlimitedunlimitedlocks Max pending signals 385911 385911 signals Max msgqueue size 819200 819200 bytes Max nice priority 00 Max realtime priority 00 Max realtime timeout unlimitedunlimitedus Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14145749#comment-14145749 ] Nishkam Ravi commented on SPARK-3633: - Unsurprisingly, the run goes through. It would be good to at least document this if we don't have bandwidth to fully fix the issue. Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14145754#comment-14145754 ] Andrew Ash commented on SPARK-3633: --- You're getting exceptions but they're not fatal to the application? Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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
[jira] [Commented] (SPARK-3633) Fetches failure observed after SPARK-2711
[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14145763#comment-14145763 ] Nishkam Ravi commented on SPARK-3633: - With higher values of ulimit and timeout values, the exceptions go away (both fetch failures and too-many-files-open) and the app runs to completion. There is a 15% perf regression though. I see 15% perf regression in another benchmark after switching to 1.1. Fetches failure observed after SPARK-2711 - Key: SPARK-3633 URL: https://issues.apache.org/jira/browse/SPARK-3633 Project: Spark Issue Type: Bug Components: Block Manager Affects Versions: 1.1.0 Reporter: Nishkam Ravi Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. Recently upgraded to Spark 1.1. The workload fails with the following error message(s): {code} 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120) 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages {code} In order to identify the problem, I carried out change set analysis. As I go back in time, the error message changes to: {code} 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, c1706.halxg.cloudera.com): java.io.FileNotFoundException: /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034 (Too many open files) java.io.FileOutputStream.open(Native Method) java.io.FileOutputStream.init(FileOutputStream.java:221) org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117) org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185) org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197) org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145) org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58) org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68) org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) {code} All the way until Aug 4th. Turns out the problem changeset is 4fde28c. -- 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