[ https://issues.apache.org/jira/browse/SPARK-11648?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15000103#comment-15000103 ]
Nishkam Ravi commented on SPARK-11648: -------------------------------------- Traced this back to 'stream data using network lib' commit from Nov 4th. Looks like a ByteBuf lifecycle management issue. cc'ing [~vanzin] > IllegalReferenceCountException in Spark workloads > ------------------------------------------------- > > Key: SPARK-11648 > URL: https://issues.apache.org/jira/browse/SPARK-11648 > Project: Spark > Issue Type: Bug > Components: Spark Core > Affects Versions: 1.6.0 > Reporter: Nishkam Ravi > > This exception is thrown for multiple workloads. Can be reproduced with > WordCount/PageRank/TeraSort. > ----------------------------- > Stack trace: > 15/11/10 01:11:31 WARN TaskSetManager: Lost task 6.0 in stage 1.0 (TID 459, > 10.20.78.15): io.netty.util.IllegalReferenceCountException: refCnt: 0 > at > io.netty.buffer.AbstractByteBuf.ensureAccessible(AbstractByteBuf.java:1178) > at io.netty.buffer.AbstractByteBuf.checkIndex(AbstractByteBuf.java:1129) > at io.netty.buffer.SlicedByteBuf.getBytes(SlicedByteBuf.java:180) > at io.netty.buffer.CompositeByteBuf.getBytes(CompositeByteBuf.java:687) > at io.netty.buffer.CompositeByteBuf.getBytes(CompositeByteBuf.java:42) > at io.netty.buffer.SlicedByteBuf.getBytes(SlicedByteBuf.java:181) > at io.netty.buffer.AbstractByteBuf.readBytes(AbstractByteBuf.java:677) > at io.netty.buffer.ByteBufInputStream.read(ByteBufInputStream.java:120) > at > org.apache.spark.storage.BufferReleasingInputStream.read(ShuffleBlockFetcherIterator.scala:360) > at com.ning.compress.lzf.ChunkDecoder.readHeader(ChunkDecoder.java:213) > at > com.ning.compress.lzf.impl.UnsafeChunkDecoder.decodeChunk(UnsafeChunkDecoder.java:49) > at > com.ning.compress.lzf.LZFInputStream.readyBuffer(LZFInputStream.java:363) > at com.ning.compress.lzf.LZFInputStream.read(LZFInputStream.java:193) > at > java.io.ObjectInputStream$PeekInputStream.read(ObjectInputStream.java:2310) > at > java.io.ObjectInputStream$PeekInputStream.readFully(ObjectInputStream.java:2323) > at > java.io.ObjectInputStream$BlockDataInputStream.readShort(ObjectInputStream.java:2794) > at > java.io.ObjectInputStream.readStreamHeader(ObjectInputStream.java:801) > at java.io.ObjectInputStream.<init>(ObjectInputStream.java:299) > at > org.apache.spark.serializer.JavaDeserializationStream$$anon$1.<init>(JavaSerializer.scala:64) > at > org.apache.spark.serializer.JavaDeserializationStream.<init>(JavaSerializer.scala:64) > at > org.apache.spark.serializer.JavaSerializerInstance.deserializeStream(JavaSerializer.scala:123) > at > org.apache.spark.shuffle.BlockStoreShuffleReader$$anonfun$3.apply(BlockStoreShuffleReader.scala:64) > at > org.apache.spark.shuffle.BlockStoreShuffleReader$$anonfun$3.apply(BlockStoreShuffleReader.scala:60) > at scala.collection.Iterator$$anon$13.hasNext(Iterator.scala:371) > at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:327) > at > org.apache.spark.util.CompletionIterator.hasNext(CompletionIterator.scala:32) > at > org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:39) > at > org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:152) > at > org.apache.spark.Aggregator.combineCombinersByKey(Aggregator.scala:58) > at > org.apache.spark.shuffle.BlockStoreShuffleReader.read(BlockStoreShuffleReader.scala:83) > at org.apache.spark.rdd.ShuffledRDD.compute(ShuffledRDD.scala:98) > at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:300) > at org.apache.spark.rdd.RDD.iterator(RDD.scala:264) > at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66) > at org.apache.spark.scheduler.Task.run(Task.scala:88) > at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214) > 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) -- 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