[#1472][part-3] fix(client): Fix occasional IllegalReferenceCountException issues in extremely rare scenarios (#1522)
### What changes were proposed in this pull request?
Improve the robustness of methods `ShuffleDataResult.release()` and `ShuffleIndexResult.release()` to fix occasional IllegalReferenceCountException issues in extremely rare scenarios.
### Why are the changes needed?
A sub PR for: https://github.com/apache/incubator-uniffle/pull/1519
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Existing UTs.
diff --git a/common/src/main/java/org/apache/uniffle/common/netty/TransportFrameDecoder.java b/common/src/main/java/org/apache/uniffle/common/netty/TransportFrameDecoder.java
index cfb0c40..deab9fe 100644
--- a/common/src/main/java/org/apache/uniffle/common/netty/TransportFrameDecoder.java
+++ b/common/src/main/java/org/apache/uniffle/common/netty/TransportFrameDecoder.java
@@ -22,6 +22,7 @@
import com.google.common.base.Preconditions;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.CompositeByteBuf;
+import io.netty.buffer.EmptyByteBuf;
import io.netty.buffer.Unpooled;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInboundHandlerAdapter;
@@ -84,7 +85,7 @@
if (msg == null || msg.body() == null || msg.body().byteBuf() == null) {
return true;
}
- return msg.body().byteBuf().readableBytes() == 0;
+ return msg.body().byteBuf() instanceof EmptyByteBuf;
}
private void clear() {
diff --git a/common/src/main/java/org/apache/uniffle/common/netty/buffer/NettyManagedBuffer.java b/common/src/main/java/org/apache/uniffle/common/netty/buffer/NettyManagedBuffer.java
index 5328642..547a4e8 100644
--- a/common/src/main/java/org/apache/uniffle/common/netty/buffer/NettyManagedBuffer.java
+++ b/common/src/main/java/org/apache/uniffle/common/netty/buffer/NettyManagedBuffer.java
@@ -25,7 +25,7 @@
public class NettyManagedBuffer extends ManagedBuffer {
public static final NettyManagedBuffer EMPTY_BUFFER =
- new NettyManagedBuffer(Unpooled.buffer(0, 0));
+ new NettyManagedBuffer(Unpooled.EMPTY_BUFFER);
private ByteBuf buf;