This is an automated email from the ASF dual-hosted git repository.
roryqi pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/uniffle.git
The following commit(s) were added to refs/heads/master by this push:
new 6555280b4 [#2082] fix: spark executor task error when reading shuffle
data when using java open jdk11 (#2436)
6555280b4 is described below
commit 6555280b428a55a889f20dc76bdc555a85f1a9f1
Author: roryqi <[email protected]>
AuthorDate: Thu Apr 10 00:09:27 2025 +0800
[#2082] fix: spark executor task error when reading shuffle data when using
java open jdk11 (#2436)
### What changes were proposed in this pull request?
Remove unnecessary release for ByteBuf
### Why are the changes needed?
Fix: #2082
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
CI passed. User verified this fix in the production environment.
---
.../java/org/apache/uniffle/client/impl/ShuffleReadClientImpl.java | 6 ------
1 file changed, 6 deletions(-)
diff --git
a/client/src/main/java/org/apache/uniffle/client/impl/ShuffleReadClientImpl.java
b/client/src/main/java/org/apache/uniffle/client/impl/ShuffleReadClientImpl.java
index e1aa0f958..6a1701c95 100644
---
a/client/src/main/java/org/apache/uniffle/client/impl/ShuffleReadClientImpl.java
+++
b/client/src/main/java/org/apache/uniffle/client/impl/ShuffleReadClientImpl.java
@@ -310,9 +310,6 @@ public class ShuffleReadClientImpl implements
ShuffleReadClient {
if (sdr == null) {
return 0;
}
- if (readBuffer != null) {
- RssUtils.releaseByteBuffer(readBuffer);
- }
readBuffer = sdr.getDataBuffer();
if (readBuffer == null || readBuffer.capacity() == 0) {
return 0;
@@ -331,9 +328,6 @@ public class ShuffleReadClientImpl implements
ShuffleReadClient {
if (sdr != null) {
sdr.release();
}
- if (readBuffer != null) {
- RssUtils.releaseByteBuffer(readBuffer);
- }
if (clientReadHandler != null) {
clientReadHandler.close();
}