Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[Bug] Occasionally encountering IllegalReferenceCountException when releasing ShuffleIndexResult #1628

Closed
2 of 3 tasks
rickyma opened this issue Apr 8, 2024 · 16 comments · Fixed by #2179 · May be fixed by #1822
Closed
2 of 3 tasks

[Bug] Occasionally encountering IllegalReferenceCountException when releasing ShuffleIndexResult #1628

rickyma opened this issue Apr 8, 2024 · 16 comments · Fixed by #2179 · May be fixed by #1822
Assignees
Labels
help wanted Extra attention is needed

Comments

@rickyma
Copy link
Contributor

rickyma commented Apr 8, 2024

Code of Conduct

Search before asking

  • I have searched in the issues and found no similar issues.

Describe the bug

https://github.com/apache/incubator-uniffle/actions/runs/9282054580/job/25539265104?pr=1752

org.apache.uniffle.test.RepartitionWithHadoopHybridStorageRssTest

Error stacktrace:

Caused by: org.apache.uniffle.common.exception.RssFetchFailedException: Failed to read shuffle data from HOT handler
	at org.apache.uniffle.storage.handler.impl.ComposedClientReadHandler.readShuffleData(ComposedClientReadHandler.java:124)
	at org.apache.uniffle.client.impl.ShuffleReadClientImpl.read(ShuffleReadClientImpl.java:305)
	at org.apache.uniffle.client.impl.ShuffleReadClientImpl.readShuffleBlockData(ShuffleReadClientImpl.java:213)
	at org.apache.spark.shuffle.reader.RssShuffleDataIterator.hasNext(RssShuffleDataIterator.java:115)
	at org.apache.spark.util.CompletionIterator.hasNext(CompletionIterator.scala:31)
	at org.apache.spark.shuffle.reader.RssShuffleReader$MultiPartitionIterator.hasNext(RssShuffleReader.java:307)
	at org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:155)
	at org.apache.spark.Aggregator.combineCombinersByKey(Aggregator.scala:50)
	at org.apache.spark.shuffle.reader.RssShuffleReader.read(RssShuffleReader.java:136)
	at org.apache.spark.rdd.ShuffledRDD.compute(ShuffledRDD.scala:106)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:365)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:329)
	at org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52)
	at org.apache.spark.scheduler.Task.run(Task.scala:136)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:548)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1504)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:551)
	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:750)
Caused by: io.netty.util.IllegalReferenceCountException: refCnt: 0, decrement: 1
	at io.netty.util.internal.ReferenceCountUpdater.toLiveRealRefCnt(ReferenceCountUpdater.java:83)
	at io.netty.util.internal.ReferenceCountUpdater.release(ReferenceCountUpdater.java:148)
	at io.netty.buffer.AbstractReferenceCountedByteBuf.release(AbstractReferenceCountedByteBuf.java:101)
	at org.apache.uniffle.common.netty.buffer.NettyManagedBuffer.release(NettyManagedBuffer.java:59)
	at org.apache.uniffle.common.ShuffleIndexResult.release(ShuffleIndexResult.java:75)
	at org.apache.uniffle.storage.handler.impl.DataSkippableReadHandler.readShuffleData(DataSkippableReadHandler.java:82)
	at org.apache.uniffle.storage.handler.impl.ComposedClientReadHandler.readShuffleData(ComposedClientReadHandler.java:113)
	... 21 more

Reason: Unknown
Priority: Minor

Affects Version(s)

master

Uniffle Server Log Output

No response

Uniffle Engine Log Output

No response

Uniffle Server Configurations

No response

Uniffle Engine Configurations

No response

Additional context

No response

Are you willing to submit PR?

  • Yes I am willing to submit a PR!
@rickyma
Copy link
Contributor Author

rickyma commented Jun 12, 2024

@EnricoMi Could you help to find out the reason here? I still have no clue.

I've encountered this issue again this time: https://github.com/apache/incubator-uniffle/actions/runs/9486586733/job/26141279578?pr=1780

@rickyma rickyma added the help wanted Extra attention is needed label Jun 18, 2024
@wForget
Copy link
Member

wForget commented Jun 21, 2024

Maybe duplicate of #1818

@rickyma
Copy link
Contributor Author

rickyma commented Jun 21, 2024

I think they are different. This is caused by calling ShuffleIndexResult.release. #1818 is caused by calling ShuffleDataResult.release.

@wForget
Copy link
Member

wForget commented Jun 21, 2024

Maybe duplicate of #1818

Sorry, it looks like the error stack is different

@rickyma
Copy link
Contributor Author

rickyma commented Jun 21, 2024

Could you help to look at this issue too?

@rickyma rickyma added the flaky test a flaky test label Jun 21, 2024
@wForget
Copy link
Member

wForget commented Jun 21, 2024

Could you help to look at this issue too?

Sure, I will try to fix it later.

@wForget
Copy link
Member

wForget commented Jun 21, 2024

ShuffleIndexResult and ShuffleDataResult seem to be no longer used after release. Shall we set their internal buffer to null after release? cc @zuston

@rickyma
Copy link
Contributor Author

rickyma commented Jun 21, 2024

I think it is feasible, you can do it. Although setting the internal buffer to null might have a positive impact on GC, this impact is usually minimal.

@wForget
Copy link
Member

wForget commented Jun 26, 2024

https://github.com/wForget/incubator-uniffle/blob/0c80d4d62315aeb5366f9d2154f861fd59a769aa/common/src/main/java/org/apache/uniffle/common/netty/buffer/NettyManagedBuffer.java#L62-L70

After adding the following log in NettyManagedBuffer, I got some related logs from https://github.com/wForget/incubator-uniffle/actions/runs/9676598458 and did not find any repeated calls.

[2024-06-26 09:03:18.227] [Executor task launch worker for task 1.0 in stage 2.0 (TID 10)] [WARN] NettyManagedBuffer.release - Check NettyManagedBuffer release
[2024-06-26 09:03:18.227] [Executor task launch worker for task 1.0 in stage 2.0 (TID 10)] [WARN] NettyManagedBuffer.release - Executor task launch worker for task 1.0 in stage 2.0 (TID 10)
[2024-06-26 09:03:18.227] [Executor task launch worker for task 1.0 in stage 2.0 (TID 10)] [WARN] NettyManagedBuffer.release - org.apache.uniffle.common.netty.buffer.NettyManagedBuffer@1d924010
[2024-06-26 09:03:18.227] [Executor task launch worker for task 1.0 in stage 2.0 (TID 10)] [WARN] NettyManagedBuffer.release - PooledUnsafeDirectByteBuf(freed)
[2024-06-26 09:03:18.227] [Executor task launch worker for task 1.0 in stage 2.0 (TID 10)] [WARN] NettyManagedBuffer.release - size: 70
[2024-06-26 09:03:18.227] [Executor task launch worker for task 1.0 in stage 2.0 (TID 10)] [WARN] NettyManagedBuffer.release - Check NettyManagedBuffer release stack tree
java.lang.Throwable
	at org.apache.uniffle.common.netty.buffer.NettyManagedBuffer.release(NettyManagedBuffer.java:68)
	at org.apache.uniffle.common.ShuffleIndexResult.release(ShuffleIndexResult.java:76)
	at org.apache.uniffle.storage.handler.impl.DataSkippableReadHandler.readShuffleData(DataSkippableReadHandler.java:82)
	at org.apache.uniffle.storage.handler.impl.ComposedClientReadHandler.readShuffleData(ComposedClientReadHandler.java:113)
	at org.apache.uniffle.client.impl.ShuffleReadClientImpl.read(ShuffleReadClientImpl.java:308)
	at org.apache.uniffle.client.impl.ShuffleReadClientImpl.readShuffleBlockData(ShuffleReadClientImpl.java:216)
	at org.apache.spark.shuffle.reader.RssShuffleDataIterator.hasNext(RssShuffleDataIterator.java:115)
	at org.apache.spark.util.CompletionIterator.hasNext(CompletionIterator.scala:31)
	at org.apache.spark.shuffle.reader.RssShuffleReader$MultiPartitionIterator.hasNext(RssShuffleReader.java:315)
	at org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:155)
	at org.apache.spark.Aggregator.combineCombinersByKey(Aggregator.scala:50)
	at org.apache.spark.shuffle.reader.RssShuffleReader.read(RssShuffleReader.java:182)
	at org.apache.spark.rdd.ShuffledRDD.compute(ShuffledRDD.scala:106)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
	at org.apache.spark.scheduler.Task.run(Task.scala:131)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:506)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1462)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:509)
	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:750)
[2024-06-26 09:03:18.228] [Executor task launch worker for task 1.0 in stage 2.0 (TID 10)] [ERROR] Executor.logError - Exception in task 1.0 in stage 2.0 (TID 10)
org.apache.uniffle.common.exception.RssFetchFailedException: Failed to read shuffle data from HOT handler
	at org.apache.uniffle.storage.handler.impl.ComposedClientReadHandler.readShuffleData(ComposedClientReadHandler.java:124)
	at org.apache.uniffle.client.impl.ShuffleReadClientImpl.read(ShuffleReadClientImpl.java:308)
	at org.apache.uniffle.client.impl.ShuffleReadClientImpl.readShuffleBlockData(ShuffleReadClientImpl.java:216)
	at org.apache.spark.shuffle.reader.RssShuffleDataIterator.hasNext(RssShuffleDataIterator.java:115)
	at org.apache.spark.util.CompletionIterator.hasNext(CompletionIterator.scala:31)
	at org.apache.spark.shuffle.reader.RssShuffleReader$MultiPartitionIterator.hasNext(RssShuffleReader.java:315)
	at org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:155)
	at org.apache.spark.Aggregator.combineCombinersByKey(Aggregator.scala:50)
	at org.apache.spark.shuffle.reader.RssShuffleReader.read(RssShuffleReader.java:182)
	at org.apache.spark.rdd.ShuffledRDD.compute(ShuffledRDD.scala:106)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
	at org.apache.spark.scheduler.Task.run(Task.scala:131)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:506)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1462)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:509)
	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:750)
Caused by: io.netty.util.IllegalReferenceCountException: refCnt: 0, decrement: 1
	at io.netty.util.internal.ReferenceCountUpdater.toLiveRealRefCnt(ReferenceCountUpdater.java:83)
	at io.netty.util.internal.ReferenceCountUpdater.release(ReferenceCountUpdater.java:148)
	at io.netty.buffer.AbstractReferenceCountedByteBuf.release(AbstractReferenceCountedByteBuf.java:101)
	at org.apache.uniffle.common.netty.buffer.NettyManagedBuffer.release(NettyManagedBuffer.java:69)
	at org.apache.uniffle.common.ShuffleIndexResult.release(ShuffleIndexResult.java:76)
	at org.apache.uniffle.storage.handler.impl.DataSkippableReadHandler.readShuffleData(DataSkippableReadHandler.java:82)
	at org.apache.uniffle.storage.handler.impl.ComposedClientReadHandler.readShuffleData(ComposedClientReadHandler.java:113)
	... 25 more

PooledUnsafeDirectByteBuf is marked as freed, but size is not 0.

@rickyma
Copy link
Contributor Author

rickyma commented Jun 27, 2024

Could you add some logs in HadoopShuffleReadHandler.readShuffleIndex to get more infomation why it returned a freed ByteBuf whose size is not 0? Or maybe this time it's not in HadoopShuffleReadHandler but some elsewhere?

@wForget
Copy link
Member

wForget commented Jul 1, 2024

Could you add some logs in HadoopShuffleReadHandler.readShuffleIndex to get more infomation why it returned a freed ByteBuf whose size is not 0? Or maybe this time it's not in HadoopShuffleReadHandler but some elsewhere?

The PooledUnsafeDirectByteBuf object does not seem to be generated by HadoopShuffleReadHandler.readShuffleIndex, it should be generated by netty framework.

@rickyma
Copy link
Contributor Author

rickyma commented Jul 1, 2024

If you are sure that it is PooledUnsafeDirectByteBuf, then it must come from LocalFileClientReadHandler.readShuffleIndex.

Because we use PooledByteBufAllocator to receive responses like GetLocalShuffleIndexResponse from Uniffle server.

Could you please add some logs in TransportFrameDecoder, Message and GetLocalShuffleIndexRequest.decode to help dig into this issue?

@wForget
Copy link
Member

wForget commented Jul 2, 2024

I seem to have found two threads that repeatedly release it:

[2024-07-01 22:12:37.173] [netty-rpc-client-24-1] [WARN] PooledUnsafeDirectByteBuf.release - Check PooledUnsafeDirectByteBuf release empty
[2024-07-01 22:12:37.173] [netty-rpc-client-24-1] [WARN] PooledUnsafeDirectByteBuf.release - netty-rpc-client-24-1
[2024-07-01 22:12:37.173] [netty-rpc-client-24-1] [WARN] PooledUnsafeDirectByteBuf.release - PooledUnsafeDirectByteBuf(ridx: 0, widx: 1063, cap: 1063)
[2024-07-01 22:12:37.173] [netty-rpc-client-24-1] [WARN] PooledUnsafeDirectByteBuf.release - io.netty.buffer.PooledUnsafeDirectByteBuf@6d1c06f3
[2024-07-01 22:12:37.173] [netty-rpc-client-24-1] [WARN] PooledUnsafeDirectByteBuf.release - size: 1063
[2024-07-01 22:12:37.174] [netty-rpc-client-24-1] [WARN] PooledUnsafeDirectByteBuf.release - refCnt: 1
[2024-07-01 22:12:37.174] [netty-rpc-client-24-1] [WARN] PooledUnsafeDirectByteBuf.release - Check PooledUnsafeDirectByteBuf release stack tree
java.lang.Throwable
	at io.netty.buffer.PooledUnsafeDirectByteBuf.release(PooledUnsafeDirectByteBuf.java:308)
	at io.netty.channel.ChannelOutboundBuffer.remove(ChannelOutboundBuffer.java:296)
	at io.netty.channel.ChannelOutboundBuffer.removeBytes(ChannelOutboundBuffer.java:383)
	at io.netty.channel.socket.nio.NioSocketChannel.doWrite(NioSocketChannel.java:421)
	at io.netty.channel.AbstractChannel$AbstractUnsafe.flush0(AbstractChannel.java:931)
	at io.netty.channel.nio.AbstractNioChannel$AbstractNioUnsafe.flush0(AbstractNioChannel.java:359)
	at io.netty.channel.AbstractChannel$AbstractUnsafe.flush(AbstractChannel.java:895)
	at io.netty.channel.DefaultChannelPipeline$HeadContext.flush(DefaultChannelPipeline.java:1372)
	at io.netty.channel.AbstractChannelHandlerContext.invokeFlush0(AbstractChannelHandlerContext.java:935)
	at io.netty.channel.AbstractChannelHandlerContext.invokeFlush(AbstractChannelHandlerContext.java:921)
	at io.netty.channel.AbstractChannelHandlerContext.flush(AbstractChannelHandlerContext.java:907)
	at io.netty.channel.ChannelDuplexHandler.flush(ChannelDuplexHandler.java:127)
	at io.netty.channel.AbstractChannelHandlerContext.invokeFlush0(AbstractChannelHandlerContext.java:937)
	at io.netty.channel.AbstractChannelHandlerContext.invokeWriteAndFlush(AbstractChannelHandlerContext.java:957)
	at io.netty.channel.AbstractChannelHandlerContext$WriteTask.run(AbstractChannelHandlerContext.java:1263)
	at io.netty.util.concurrent.AbstractEventExecutor.runTask(AbstractEventExecutor.java:173)
	at io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:166)
	at io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:470)
	at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:566)
	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997)
	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
	at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
	at java.lang.Thread.run(Thread.java:748)


[2024-07-01 22:12:36.273] [Executor task launch worker for task 0.0 in stage 5.0 (TID 14)] [WARN] PooledUnsafeDirectByteBuf.release - Check PooledUnsafeDirectByteBuf release empty
[2024-07-01 22:12:36.273] [Executor task launch worker for task 0.0 in stage 5.0 (TID 14)] [WARN] PooledUnsafeDirectByteBuf.release - Executor task launch worker for task 0.0 in stage 5.0 (TID 14)
[2024-07-01 22:12:36.273] [Executor task launch worker for task 0.0 in stage 5.0 (TID 14)] [WARN] PooledUnsafeDirectByteBuf.release - PooledUnsafeDirectByteBuf(ridx: 0, widx: 70, cap: 70)
[2024-07-01 22:12:36.273] [Executor task launch worker for task 0.0 in stage 5.0 (TID 14)] [WARN] PooledUnsafeDirectByteBuf.release - io.netty.buffer.PooledUnsafeDirectByteBuf@6d1c06f3
[2024-07-01 22:12:36.273] [Executor task launch worker for task 0.0 in stage 5.0 (TID 14)] [WARN] PooledUnsafeDirectByteBuf.release - size: 70
[2024-07-01 22:12:36.273] [Executor task launch worker for task 0.0 in stage 5.0 (TID 14)] [WARN] PooledUnsafeDirectByteBuf.release - refCnt: 1
[2024-07-01 22:12:36.273] [Executor task launch worker for task 0.0 in stage 5.0 (TID 14)] [WARN] PooledUnsafeDirectByteBuf.release - Check PooledUnsafeDirectByteBuf release stack tree
java.lang.Throwable
	at io.netty.buffer.PooledUnsafeDirectByteBuf.release(PooledUnsafeDirectByteBuf.java:308)
	at org.apache.uniffle.common.netty.buffer.NettyManagedBuffer.release(NettyManagedBuffer.java:88)
	at org.apache.uniffle.common.ShuffleIndexResult.release(ShuffleIndexResult.java:76)
	at org.apache.uniffle.storage.handler.impl.DataSkippableReadHandler.readShuffleData(DataSkippableReadHandler.java:87)
	at org.apache.uniffle.storage.handler.impl.ComposedClientReadHandler.readShuffleData(ComposedClientReadHandler.java:113)
	at org.apache.uniffle.client.impl.ShuffleReadClientImpl.read(ShuffleReadClientImpl.java:308)
	at org.apache.uniffle.client.impl.ShuffleReadClientImpl.readShuffleBlockData(ShuffleReadClientImpl.java:216)
	at org.apache.spark.shuffle.reader.RssShuffleDataIterator.hasNext(RssShuffleDataIterator.java:115)
	at org.apache.spark.util.CompletionIterator.hasNext(CompletionIterator.scala:31)
	at org.apache.spark.shuffle.reader.RssShuffleReader$MultiPartitionIterator.hasNext(RssShuffleReader.java:315)
	at org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:155)
	at org.apache.spark.Aggregator.combineCombinersByKey(Aggregator.scala:50)
	at org.apache.spark.shuffle.reader.RssShuffleReader.read(RssShuffleReader.java:182)
	at org.apache.spark.rdd.ShuffledRDD.compute(ShuffledRDD.scala:106)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:367)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:331)
	at org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:104)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:54)
	at org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:166)
	at org.apache.spark.scheduler.Task.run(Task.scala:141)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Executor.scala:620)
	at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(SparkErrorUtils.scala:64)
	at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(SparkErrorUtils.scala:61)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:94)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:623)
	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)
[2024-07-01 22:12:37.178] [Executor task launch worker for task 0.0 in stage 5.0 (TID 14)] [ERROR] Executor.logError - Exception in task 0.0 in stage 5.0 (TID 14)
org.apache.uniffle.common.exception.RssFetchFailedException: Failed to read shuffle data from HOT handler
	at org.apache.uniffle.storage.handler.impl.ComposedClientReadHandler.readShuffleData(ComposedClientReadHandler.java:124)
	at org.apache.uniffle.client.impl.ShuffleReadClientImpl.read(ShuffleReadClientImpl.java:308)
	at org.apache.uniffle.client.impl.ShuffleReadClientImpl.readShuffleBlockData(ShuffleReadClientImpl.java:216)
	at org.apache.spark.shuffle.reader.RssShuffleDataIterator.hasNext(RssShuffleDataIterator.java:115)
	at org.apache.spark.util.CompletionIterator.hasNext(CompletionIterator.scala:31)
	at org.apache.spark.shuffle.reader.RssShuffleReader$MultiPartitionIterator.hasNext(RssShuffleReader.java:315)
	at org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:155)
	at org.apache.spark.Aggregator.combineCombinersByKey(Aggregator.scala:50)
	at org.apache.spark.shuffle.reader.RssShuffleReader.read(RssShuffleReader.java:182)
	at org.apache.spark.rdd.ShuffledRDD.compute(ShuffledRDD.scala:106)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:367)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:331)
	at org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:104)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:54)
	at org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:166)
	at org.apache.spark.scheduler.Task.run(Task.scala:141)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Executor.scala:620)
	at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(SparkErrorUtils.scala:64)
	at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(SparkErrorUtils.scala:61)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:94)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:623)
	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: io.netty.util.IllegalReferenceCountException: refCnt: 0, decrement: 1
	at io.netty.util.internal.ReferenceCountUpdater.toLiveRealRefCnt(ReferenceCountUpdater.java:83)
	at io.netty.util.internal.ReferenceCountUpdater.retryRelease0(ReferenceCountUpdater.java:173)
	at io.netty.util.internal.ReferenceCountUpdater.release(ReferenceCountUpdater.java:147)
	at io.netty.buffer.AbstractReferenceCountedByteBuf.release(AbstractReferenceCountedByteBuf.java:101)
	at io.netty.buffer.PooledUnsafeDirectByteBuf.release(PooledUnsafeDirectByteBuf.java:316)
	at org.apache.uniffle.common.netty.buffer.NettyManagedBuffer.release(NettyManagedBuffer.java:88)
	at org.apache.uniffle.common.ShuffleIndexResult.release(ShuffleIndexResult.java:76)
	at org.apache.uniffle.storage.handler.impl.DataSkippableReadHandler.readShuffleData(DataSkippableReadHandler.java:87)
	at org.apache.uniffle.storage.handler.impl.ComposedClientReadHandler.readShuffleData(ComposedClientReadHandler.java:113)
	... 24 more

@rickyma
Copy link
Contributor Author

rickyma commented Jul 2, 2024

I didn't see the connection between these two release stacktrace logs, as they seem to be unrelated. The first stack contains ChannelOutboundBuffer, so it looks more like it happened during ChannelOutbound rather than ChannelInbound. The TransportFrameDecoder, on the other hand, occurs during ChannelInbound. So, it seems like the first release doesn't have much to do with TransportFrameDecoder?

@wForget
Copy link
Member

wForget commented Jul 3, 2024

io.netty.buffer.PooledUnsafeDirectByteBuf@6d1c06f3

They seem to be the same object. PooledUnsafeDirectByteBuf seems to be reusable before release.

@xianjingfeng xianjingfeng removed the flaky test a flaky test label Aug 8, 2024
@xianjingfeng
Copy link
Member

Could we temporarily handle it in the following way?

public static void releaseByteBuffer(ByteBuffer byteBuffer) {
    if (byteBuffer == null || !byteBuffer.isDirect()) {
      return;
    }
    try {
      PlatformDependent.freeDirectBuffer(byteBuffer);
    } catch (Throwable t) {
      LOGGER.warn("Failed to release direct buffer:" + t.getMessage());
    }
  }

@jerqi jerqi closed this as completed in 4c14b97 Oct 18, 2024
maobaolong added a commit to maobaolong/incubator-uniffle that referenced this issue Nov 4, 2024
…se exception to pass flaky test (apache#2179)

### What changes were proposed in this pull request?

(Please outline the changes and how this PR fixes the issue.)

### Why are the changes needed?

- Pass the flaky test
- Avoid memory leak after double release the `ShuffleIndexResult`

Fix: apache#1628

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Flaky test can be fixed.

(cherry picked from commit 4c14b97)
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment