Skip to content

Commit 6555280

Browse files
authored
[#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.
1 parent 32187f8 commit 6555280

1 file changed

Lines changed: 0 additions & 6 deletions

File tree

client/src/main/java/org/apache/uniffle/client/impl/ShuffleReadClientImpl.java

Lines changed: 0 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -310,9 +310,6 @@ private int read() {
310310
if (sdr == null) {
311311
return 0;
312312
}
313-
if (readBuffer != null) {
314-
RssUtils.releaseByteBuffer(readBuffer);
315-
}
316313
readBuffer = sdr.getDataBuffer();
317314
if (readBuffer == null || readBuffer.capacity() == 0) {
318315
return 0;
@@ -331,9 +328,6 @@ public void close() {
331328
if (sdr != null) {
332329
sdr.release();
333330
}
334-
if (readBuffer != null) {
335-
RssUtils.releaseByteBuffer(readBuffer);
336-
}
337331
if (clientReadHandler != null) {
338332
clientReadHandler.close();
339333
}

0 commit comments

Comments
 (0)