colinmjj commented on code in PR #67:
URL: https://github.com/apache/incubator-uniffle/pull/67#discussion_r928360881


##########
client-spark/common/src/main/java/org/apache/spark/shuffle/writer/WriteBufferManager.java:
##########
@@ -117,29 +117,16 @@ public List<ShuffleBlockInfo> addRecord(int partitionId, 
Object key, Object valu
       return null;
     }
     List<ShuffleBlockInfo> result = Lists.newArrayList();
-    if (buffers.containsKey(partitionId)) {
-      WriterBuffer wb = buffers.get(partitionId);
-      if (wb.askForMemory(serializedDataLength)) {
-        if (serializedDataLength > bufferSegmentSize) {
-          requestMemory(serializedDataLength);
-        } else {
-          requestMemory(bufferSegmentSize);
-        }
-      }
-      wb.addRecord(serializedData, serializedDataLength);
-      if (wb.getMemoryUsed() > bufferSize) {
-        result.add(createShuffleBlock(partitionId, wb));
-        copyTime += wb.getCopyTime();
-        buffers.remove(partitionId);
-        LOG.debug("Single buffer is full for shuffleId[" + shuffleId
-            + "] partition[" + partitionId + "] with memoryUsed[" + 
wb.getMemoryUsed()
-            + "], dataLength[" + wb.getDataLength() + "]");
-      }
-    } else {
-      requestMemory(bufferSegmentSize);
-      WriterBuffer wb = new WriterBuffer(bufferSegmentSize);
-      wb.addRecord(serializedData, serializedDataLength);

Review Comment:
   For memory leak, how about:
   ```
   int bufferSize = bufferSegmentSize
   if (serializedDataLength > bufferSize) {
     bufferSize = serializedDataLength
   }
   requestMemory(bufferSize);
   WriterBuffer wb = new WriterBuffer(bufferSize);
   ..........
   ..........
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to