[ https://issues.apache.org/jira/browse/SPARK-27530?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16825770#comment-16825770 ]
Josh Rosen edited comment on SPARK-27530 at 4/25/19 6:32 AM: ------------------------------------------------------------- This specific error message was added in SPARK-24160. As discussed in that ticket's PR, zero-sized blocks should never be requested, so the receipt of a zero-sized block indicates either a bug in the shuffle-request-issuing logic or a data corruption / truncation bug somewhere in the shuffle stack. *Update*: I see someone linked SPARK-27216 to this ticket; that sounds like a plausible cause / fix. was (Author: joshrosen): This specific error message was added in SPARK-24160. As discussed in that ticket's PR, zero-sized blocks should never be requested, so the receipt of a zero-sized block indicates either a bug in the shuffle-request-issuing logic or a data corruption / truncation bug somewhere in the shuffle stack. *Update*: I see someone linked SPARK-27216 to this ticket; that sounds like a plausible cause. > FetchFailedException: Received a zero-size buffer for block shuffle > ------------------------------------------------------------------- > > Key: SPARK-27530 > URL: https://issues.apache.org/jira/browse/SPARK-27530 > Project: Spark > Issue Type: Improvement > Components: Shuffle, Spark Core > Affects Versions: 2.4.0 > Reporter: Adrian Muraru > Priority: Major > > I'm getting this in a large shuffle: > {code:java} > org.apache.spark.shuffle.FetchFailedException: Received a zero-size buffer > for block shuffle_2_9167_1861 from BlockManagerId(2665, ip-172-25-44-74, > 39439, None) (expectedApproxSize = 33708, isNetworkReqDone=false) > at > org.apache.spark.storage.ShuffleBlockFetcherIterator.throwFetchFailedException(ShuffleBlockFetcherIterator.scala:554) > at > org.apache.spark.storage.ShuffleBlockFetcherIterator.next(ShuffleBlockFetcherIterator.scala:438) > at > org.apache.spark.storage.ShuffleBlockFetcherIterator.next(ShuffleBlockFetcherIterator.scala:64) > at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:435) > at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:441) > at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:409) > at > org.apache.spark.util.CompletionIterator.hasNext(CompletionIterator.scala:30) > at > org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:37) > at > org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:156) > at org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:41) > at > org.apache.spark.shuffle.BlockStoreShuffleReader.read(BlockStoreShuffleReader.scala:90) > at org.apache.spark.rdd.ShuffledRDD.compute(ShuffledRDD.scala:105) > at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324) > at org.apache.spark.rdd.RDD.iterator(RDD.scala:288) > at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) > at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324) > at org.apache.spark.rdd.RDD.iterator(RDD.scala:288) > at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) > at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324) > at org.apache.spark.rdd.RDD.iterator(RDD.scala:288) > at org.apache.spark.rdd.UnionRDD.compute(UnionRDD.scala:105) > at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324) > at org.apache.spark.rdd.RDD.iterator(RDD.scala:288) > at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) > at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324) > at org.apache.spark.rdd.RDD.iterator(RDD.scala:288) > at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99) > at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:55) > at org.apache.spark.scheduler.Task.run(Task.scala:121) > at > org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:402) > at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360) > at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:408) > 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.io.IOException: Received a zero-size buffer for block > shuffle_2_9167_1861 from BlockManagerId(2665, ip-172-25-44-74, 39439, None) > (expectedApproxSize = 33708, isNetworkReqDone=false){code} > -- This message was sent by Atlassian JIRA (v7.6.3#76005) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org