You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Apache Spark (JIRA)" <ji...@apache.org> on 2018/04/05 21:29:00 UTC

[jira] [Assigned] (SPARK-23816) FetchFailedException when killing speculative task

     [ https://issues.apache.org/jira/browse/SPARK-23816?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Apache Spark reassigned SPARK-23816:
------------------------------------

    Assignee:     (was: Apache Spark)

> FetchFailedException when killing speculative task
> --------------------------------------------------
>
>                 Key: SPARK-23816
>                 URL: https://issues.apache.org/jira/browse/SPARK-23816
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 2.2.0
>            Reporter: chen xiao
>            Priority: Major
>
> When spark trying to kill speculative tasks because of another attempt has already success, sometimes the task throws "org.apache.spark.shuffle.FetchFailedException: Error in opening FileSegmentManagedBuffer" and the whole stage will fail.
> Other active stages will also fail with error "org.apache.spark.shuffle.MetadataFetchFailedException: Missing an output location for shuffle" Then I checked the log in failed executor, there is not error like "MetadataFetchFailedException". So they just failed with no error.
> {code:java}
> 18/03/26 23:12:09 INFO Executor: Executor is trying to kill task 2879.1 in stage 4.0 (TID 13023), reason: another attempt succeeded
> 18/03/26 23:12:09 ERROR ShuffleBlockFetcherIterator: Failed to create input stream from local block
> java.io.IOException: Error in opening FileSegmentManagedBuffer{file=/hadoop02/yarn/local/usercache/pp_risk_grs_datamart_batch/appcache/application_1521504416249_116088/blockmgr-754a22fd-e8d6-4478-bcf8-f1d95f07f4a2/0c/shuffle_24_10_0.data, offset=263687568, length=87231}
> 	at org.apache.spark.network.buffer.FileSegmentManagedBuffer.createInputStream(FileSegmentManagedBuffer.java:114)
> 	at org.apache.spark.storage.ShuffleBlockFetcherIterator.next(ShuffleBlockFetcherIterator.scala:401)
> 	at org.apache.spark.storage.ShuffleBlockFetcherIterator.next(ShuffleBlockFetcherIterator.scala:61)
> 	at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434)
> 	at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:440)
> 	at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408)
> 	at org.apache.spark.util.CompletionIterator.hasNext(CompletionIterator.scala:32)
> 	at org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:37)
> 	at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408)
> 	at org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec$$anonfun$doExecute$1$$anonfun$2.apply(ObjectHashAggregateExec.scala:104)
> 	at org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec$$anonfun$doExecute$1$$anonfun$2.apply(ObjectHashAggregateExec.scala:103)
> 	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:827)
> 	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:827)
> 	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
> 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
> 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
> 	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
> 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
> 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
> 	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
> 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
> 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
> 	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:96)
> 	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53)
> 	at org.apache.spark.scheduler.Task.run(Task.scala:108)
> 	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:338)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> 	at java.lang.Thread.run(Thread.java:748)
> Caused by: java.nio.channels.ClosedByInterruptException
> 	at java.nio.channels.spi.AbstractInterruptibleChannel.end(AbstractInterruptibleChannel.java:202)
> 	at sun.nio.ch.FileChannelImpl.read(FileChannelImpl.java:164)
> 	at sun.nio.ch.ChannelInputStream.read(ChannelInputStream.java:65)
> 	at sun.nio.ch.ChannelInputStream.read(ChannelInputStream.java:109)
> 	at sun.nio.ch.ChannelInputStream.read(ChannelInputStream.java:103)
> 	at java.io.InputStream.skip(InputStream.java:224)
> 	at org.spark_project.guava.io.ByteStreams.skipFully(ByteStreams.java:755)
> 	at org.apache.spark.network.buffer.FileSegmentManagedBuffer.createInputStream(FileSegmentManagedBuffer.java:100)
> 	... 28 more
> {code}
>  
> And in Spark UI, I found the only failed task is 13023, which means killing speculative task make whole stage fails.
> ||[Index|http://lvshdc2en0012.lvs.paypal.com:8088/proxy/application_1521504416249_116088/stages/stage?id=4&attempt=0&task.sort=Index&task.pageSize=100]||[ID|http://lvshdc2en0012.lvs.paypal.com:8088/proxy/application_1521504416249_116088/stages/stage?id=4&attempt=0&task.sort=ID&task.pageSize=100]||[Attempt|http://lvshdc2en0012.lvs.paypal.com:8088/proxy/application_1521504416249_116088/stages/stage?id=4&attempt=0&task.sort=Attempt&task.pageSize=100]||[Status ▴|http://lvshdc2en0012.lvs.paypal.com:8088/proxy/application_1521504416249_116088/stages/stage?id=4&attempt=0&task.sort=Status&task.desc=true&task.pageSize=100]||[Locality Level|http://lvshdc2en0012.lvs.paypal.com:8088/proxy/application_1521504416249_116088/stages/stage?id=4&attempt=0&task.sort=Locality+Level&task.pageSize=100]||[Executor ID / Host|http://lvshdc2en0012.lvs.paypal.com:8088/proxy/application_1521504416249_116088/stages/stage?id=4&attempt=0&task.sort=Executor+ID+%2F+Host&task.pageSize=100]||[Launch Time|http://lvshdc2en0012.lvs.paypal.com:8088/proxy/application_1521504416249_116088/stages/stage?id=4&attempt=0&task.sort=Launch+Time&task.pageSize=100]||[Duration|http://lvshdc2en0012.lvs.paypal.com:8088/proxy/application_1521504416249_116088/stages/stage?id=4&attempt=0&task.sort=Duration&task.pageSize=100]||[GC Time|http://lvshdc2en0012.lvs.paypal.com:8088/proxy/application_1521504416249_116088/stages/stage?id=4&attempt=0&task.sort=GC+Time&task.pageSize=100]||[Shuffle Read Size / Records|http://lvshdc2en0012.lvs.paypal.com:8088/proxy/application_1521504416249_116088/stages/stage?id=4&attempt=0&task.sort=Shuffle+Read+Size+%2F+Records&task.pageSize=100]||[Write Time|http://lvshdc2en0012.lvs.paypal.com:8088/proxy/application_1521504416249_116088/stages/stage?id=4&attempt=0&task.sort=Write+Time&task.pageSize=100]||[Shuffle Write Size / Records|http://lvshdc2en0012.lvs.paypal.com:8088/proxy/application_1521504416249_116088/stages/stage?id=4&attempt=0&task.sort=Shuffle+Write+Size+%2F+Records&task.pageSize=100]||[Errors|http://lvshdc2en0012.lvs.paypal.com:8088/proxy/application_1521504416249_116088/stages/stage?id=4&attempt=0&task.sort=Errors&task.pageSize=100]||
> |2879|13023|1 (speculative)|FAILED|PROCESS_LOCAL|33 / lvshdc2dn2202.lvs.****.com
>  [stdout|http://lvshdc2dn2202.lvs.paypalinc.com:8042/node/containerlogs/container_e39_1521504416249_116088_01_000131/pp_risk_grs_datamart_batch/stdout?start=-4096]
>  [stderr|http://lvshdc2dn2202.lvs.paypalinc.com:8042/node/containerlogs/container_e39_1521504416249_116088_01_000131/pp_risk_grs_datamart_batch/stderr?start=-4096]|2018/03/26 23:12:09| | |/| |/|FetchFailed(BlockManagerId(33, lvshdc2dn2202.lvs.paypalinc.com, 33481, None), shuffleId=24, mapId=10, reduceId=2879, message=|



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org