You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@celeborn.apache.org by "RexXiong (via GitHub)" <gi...@apache.org> on 2023/04/04 01:24:53 UTC

[GitHub] [incubator-celeborn] RexXiong commented on a diff in pull request #1407: [CELEBORN-431][FLINK] Support dynamic buffer allocation in reading map partition.

RexXiong commented on code in PR #1407:
URL: https://github.com/apache/incubator-celeborn/pull/1407#discussion_r1155935381


##########
common/src/main/java/org/apache/celeborn/common/network/server/BufferStreamManager.java:
##########
@@ -319,28 +329,71 @@ public synchronized void setupDataPartitionReader(
       streamReaders.put(streamId, dataPartitionReader);
 
       // create initial buffers for read
-      if (allocateResources && buffers == null) {
+      if (allocateResources && bufferQueue.size() == 0) {
         memoryManager.requestReadBuffers(
-            minReadBuffers,
-            maxReadBuffers,
-            fileInfo.getBufferSize(),
-            (allocatedBuffers, throwable) ->
-                MapDataPartition.this.onBuffer(new LinkedBlockingDeque<>(allocatedBuffers)));
+            new ReadBufferRequest(
+                minReadBuffers,
+                maxReadBuffers,
+                fileInfo.getBufferSize(),
+                (allocatedBuffers, throwable) -> onBuffer(allocatedBuffers)));
       } else {
         triggerRead();
       }
     }
 
     // Read logic is executed on another thread.
-    public void onBuffer(Queue<ByteBuf> buffers) {
-      this.buffers = buffers;
+    public void onBuffer(List<ByteBuf> buffers) {
+      if (isReleased || bufferQueue.isReleased()) {

Review Comment:
   the new apply buffers would be leaked here?



##########
common/src/main/java/org/apache/celeborn/common/network/server/memory/BufferQueue.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.celeborn.common.network.server.memory;
+
+import static org.apache.commons.crypto.utils.Utils.checkArgument;
+import static org.apache.hadoop.shaded.com.google.common.base.Preconditions.checkState;
+
+import java.util.Collection;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import io.netty.buffer.ByteBuf;
+
+@NotThreadSafe

Review Comment:
   ThreadSafe?



##########
common/src/test/java/org/apache/celeborn/common/network/server/BufferStreamManagerSuiteJ.java:
##########
@@ -106,6 +106,8 @@ public void testStreamRegisterAndCleanup() throws Exception {
     numInFlightRequests.decrementAndGet();
     bufferStreamManager.connectionTerminated(channel);
     timeOutOrMeetCondition(() -> bufferStreamManager.numRecycleStreams() == 0);
+    // Let delayed queue element timeout
+    Thread.sleep(1000);

Review Comment:
   Can use timeOutOrMeetCondition to replace



##########
common/src/main/java/org/apache/celeborn/common/network/server/BufferStreamManager.java:
##########
@@ -319,28 +329,71 @@ public synchronized void setupDataPartitionReader(
       streamReaders.put(streamId, dataPartitionReader);

Review Comment:
   streamReaders can replace readers or readers need skip or remove when the reader is already released.



##########
common/src/main/java/org/apache/celeborn/common/network/server/BufferStreamManager.java:
##########
@@ -415,7 +467,7 @@ public boolean releaseStream(Long streamId) {
       DataPartitionReader dataPartitionReader = streamReaders.get(streamId);
       dataPartitionReader.release();
       if (dataPartitionReader.isFinished()) {
-        logger.debug("release all for stream: {}", streamId);
+        logger.info("release all for stream: {}", streamId);

Review Comment:
   keep debug level



-- 
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: issues-unsubscribe@celeborn.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org