[ https://issues.apache.org/jira/browse/SPARK-11617?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15001721#comment-15001721 ]
LingZhou commented on SPARK-11617: ---------------------------------- Ok, it's in two programs. In one programs, it's like 15/11/12 13:22:51 INFO scheduler.TaskSetManager: Starting task 270.0 in stage 1.0 (TID 2781, gsr490, partition 270,PROCESS_LOCAL, 1994 bytes) 15/11/12 13:22:51 INFO scheduler.TaskSetManager: Starting task 271.0 in stage 1.0 (TID 2782, gsr492, partition 271,PROCESS_LOCAL, 1994 bytes) 15/11/12 13:22:51 WARN scheduler.TaskSetManager: Lost task 132.0 in stage 1.0 (TID 2643, gsr490): 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.PooledUnsafeDirectByteBuf.getBytes(PooledUnsafeDirectByteBuf.java:138) 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 org.xerial.snappy.SnappyInputStream.readNext(SnappyInputStream.java:361) at org.xerial.snappy.SnappyInputStream.hasNextChunk(SnappyInputStream.java:381) at org.xerial.snappy.SnappyInputStream.rawRead(SnappyInputStream.java:182) at org.xerial.snappy.SnappyInputStream.read(SnappyInputStream.java:163) at com.esotericsoftware.kryo.io.Input.fill(Input.java:140) at com.esotericsoftware.kryo.io.Input.require(Input.java:169) at com.esotericsoftware.kryo.io.Input.readUtf8_slow(Input.java:524) at com.esotericsoftware.kryo.io.Input.readUtf8(Input.java:517) at com.esotericsoftware.kryo.io.Input.readString(Input.java:447) at com.esotericsoftware.kryo.serializers.DefaultSerializers$StringSerializer.read(DefaultSerializers.java:157) at com.esotericsoftware.kryo.serializers.DefaultSerializers$StringSerializer.read(DefaultSerializers.java:146) at com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:729) at org.apache.spark.serializer.KryoDeserializationStream.readObject(KryoSerializer.scala:224) at org.apache.spark.serializer.DeserializationStream.readKey(Serializer.scala:169) at org.apache.spark.serializer.DeserializationStream$$anon$2.getNext(Serializer.scala:201) at org.apache.spark.serializer.DeserializationStream$$anon$2.getNext(Serializer.scala:198) at org.apache.spark.util.NextIterator.hasNext(NextIterator.scala:73) 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.ExternalSorter.insertAll(ExternalSorter.scala:197) at org.apache.spark.shuffle.BlockStoreShuffleReader.read(BlockStoreShuffleReader.scala:103) 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.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:300) at org.apache.spark.rdd.RDD.iterator(RDD.scala:264) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) 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:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) in another program it's like 15/11/12 13:35:06 INFO scheduler.TaskSetManager: Starting task 268.2 in stage 3.0 (TID 20032, gsr489, partition 268,PROCESS_LOCAL, 1961 bytes) 15/11/12 13:35:06 INFO scheduler.TaskSetManager: Lost task 281.1 in stage 3.0 (TID 19895) on executor gsr489: io.netty.util.IllegalReferenceCountException (refCnt: 0) [duplicate 274] 15/11/12 13:35:06 INFO scheduler.TaskSetManager: Starting task 281.2 in stage 3.0 (TID 20033, gsr491, partition 281,PROCESS_LOCAL, 1961 bytes) 15/11/12 13:35:06 INFO scheduler.TaskSetManager: Lost task 113.2 in stage 3.0 (TID 19982) on executor gsr491: io.netty.util.IllegalReferenceCountException (refCnt: 0) [duplicate 275] 15/11/12 13:35:06 INFO scheduler.TaskSetManager: Starting task 113.3 in stage 3.0 (TID 20034, gsr490, partition 113,PROCESS_LOCAL, 1961 bytes) 15/11/12 13:35:06 INFO scheduler.TaskSetManager: Lost task 412.1 in stage 3.0 (TID 19830) on executor gsr490: io.netty.util.IllegalReferenceCountException (refCnt: 0) [duplicate 276] 15/11/12 13:35:06 INFO scheduler.TaskSetManager: Starting task 412.2 in stage 3.0 (TID 20035, gsr491, partition 412,PROCESS_LOCAL, 1961 bytes) 15/11/12 13:35:06 INFO scheduler.TaskSetManager: Lost task 211.0 in stage 3.0 (TID 19411) on executor gsr491: io.netty.util.IllegalReferenceCountException (refCnt: 0) [duplicate 277] 15/11/12 13:35:06 INFO scheduler.TaskSetManager: Starting task 211.1 in stage 3.0 (TID 20036, gsr487, partition 211,PROCESS_LOCAL, 1961 bytes) 15/11/12 13:35:06 INFO scheduler.TaskSetManager: Lost task 22.1 in stage 3.0 (TID 19745) on executor gsr487: io.netty.util.IllegalReferenceCountException (refCnt: 0) [duplicate 278] 15/11/12 13:35:07 INFO scheduler.TaskSetManager: Starting task 22.2 in stage 3.0 (TID 20037, gsr492, partition 22,PROCESS_LOCAL, 1961 bytes) 15/11/12 13:35:07 INFO scheduler.TaskSetManager: Lost task 343.1 in stage 3.0 (TID 19759) on executor gsr492: io.netty.util.IllegalReferenceCountException (refCnt: 0) [duplicate 279] 15/11/12 13:35:07 INFO storage.BlockManagerInfo: Removed broadcast_0_piece0 on gsr493:57074 in memory (size: 1456.0 B, free: 125.8 MB) 15/11/12 13:35:07 INFO scheduler.TaskSetManager: Starting task 343.2 in stage 3.0 (TID 20038, gsr489, partition 343,PROCESS_LOCAL, 1961 bytes) 15/11/12 13:35:07 INFO scheduler.TaskSetManager: Lost task 377.2 in stage 3.0 (TID 19927) on executor gsr489: io.netty.util.IllegalReferenceCountException (refCnt: 0) [duplicate 280] 15/11/12 13:35:07 INFO storage.BlockManagerInfo: Removed broadcast_0_piece0 on gsr491:41858 in memory (size: 1456.0 B, free: 51.5 MB) 15/11/12 13:35:07 INFO scheduler.TaskSetManager: Starting task 377.3 in stage 3.0 (TID 20039, gsr488, partition 377,PROCESS_LOCAL, 1961 bytes) 15/11/12 13:35:07 WARN scheduler.TaskSetManager: Lost task 237.1 in stage 3.0 (TID 19851, gsr488): java.lang.IndexOutOfBoundsException: index: 39501, length: 4 (expected: range(0, 0)) at io.netty.buffer.AbstractByteBuf.checkIndex(AbstractByteBuf.java:1134) at io.netty.buffer.AbstractByteBuf.checkDstIndex(AbstractByteBuf.java:1148) at io.netty.buffer.CompositeByteBuf.getBytes(CompositeByteBuf.java:676) 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 org.xerial.snappy.SnappyInputStream.readNext(SnappyInputStream.java:361) at org.xerial.snappy.SnappyInputStream.hasNextChunk(SnappyInputStream.java:381) at org.xerial.snappy.SnappyInputStream.rawRead(SnappyInputStream.java:182) at org.xerial.snappy.SnappyInputStream.read(SnappyInputStream.java:163) at java.io.ObjectInputStream$PeekInputStream.read(ObjectInputStream.java:2313) at java.io.ObjectInputStream$PeekInputStream.readFully(ObjectInputStream.java:2326) at java.io.ObjectInputStream$BlockDataInputStream.readShort(ObjectInputStream.java:2797) at java.io.ObjectInputStream.readStreamHeader(ObjectInputStream.java:802) 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.ExternalSorter.insertAll(ExternalSorter.scala:197) at org.apache.spark.shuffle.BlockStoreShuffleReader.read(BlockStoreShuffleReader.scala:103) 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:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) > MEMORY LEAK: ByteBuf.release() was not called before it's garbage-collected > --------------------------------------------------------------------------- > > Key: SPARK-11617 > URL: https://issues.apache.org/jira/browse/SPARK-11617 > Project: Spark > Issue Type: Bug > Components: Spark Core, YARN > Affects Versions: 1.6.0 > Reporter: LingZhou > > The problem may be related to > [SPARK-11235][NETWORK] Add ability to stream data using network lib. > while running on yarn-client mode, there are error messages: > 15/11/09 10:23:55 ERROR util.ResourceLeakDetector: LEAK: ByteBuf.release() > was not called before it's garbage-collected. Enable advanced leak reporting > to find out where the leak occurred. To enable advanced leak reporting, > specify the JVM option '-Dio.netty.leakDetectionLevel=advanced' or call > ResourceLeakDetector.setLevel() See > http://netty.io/wiki/reference-counted-objects.html for more information. > and then it will cause > cluster.YarnSchedulerBackend$YarnSchedulerEndpoint: Container killed by YARN > for exceeding memory limits. 9.0 GB of 9 GB physical memory used. Consider > boosting spark.yarn.executor.memoryOverhead. > and WARN scheduler.TaskSetManager: Lost task 105.0 in stage 1.0 (TID 2616, > gsr489): java.lang.IndexOutOfBoundsException: index: 130828, length: 16833 > (expected: range(0, 524288)). -- 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