[ https://issues.apache.org/jira/browse/SPARK-3633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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): 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