jiangxin369 commented on code in PR #248:
URL: https://github.com/apache/flink-ml/pull/248#discussion_r1297235534


##########
flink-ml-iteration/flink-ml-iteration-common/src/main/java/org/apache/flink/iteration/typeinfo/IterationRecordSerializer.java:
##########
@@ -106,7 +106,7 @@ public int getLength() {
     @Override
     public void serialize(IterationRecord<T> record, DataOutputView target) 
throws IOException {
         target.writeByte((byte) record.getType().ordinal());
-        serializerNumber(record.getEpoch(), target);
+        target.writeInt(record.getEpoch());

Review Comment:
   The original `serializerNumber` cannot serialize `Integer.MAX_VALUE + 1`, 
which is the epoch of the feedback record in the last iteration. In the 
previous implementation, the feedback records are just passed in memory and 
don't need to serialize/deserialize, so the bug doesn't appear.
   
   I also did a benchmark to serialize 1 billion integers with 
`serializerNumber` and `DataOutputSerializer#writeInt`, their performances are 
quite close, so I think we can remove the current `serializerNumber` 
implementation.



##########
flink-ml-iteration/flink-ml-iteration-common/src/main/java/org/apache/flink/iteration/operator/feedback/SpillableFeedbackQueue.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.flink.iteration.operator.feedback;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputSerializer;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.runtime.io.disk.InputViewIterator;
+import org.apache.flink.runtime.io.disk.SpillingBuffer;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.table.runtime.operators.sort.ListMemorySegmentPool;
+import org.apache.flink.util.MutableObjectIterator;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * * A queue that can spill the items to disks automatically when the memory 
buffer is full.
+ *
+ * @param <T> The element type.
+ */
+@Internal
+final class SpillableFeedbackQueue<T> {
+    private final DataOutputSerializer output = new DataOutputSerializer(256);
+    private final TypeSerializer<T> serializer;
+    private final IOManager ioManager;
+    private final MemoryManager memoryManager;
+    private final int numPages;
+
+    private List<MemorySegment> segments;
+    private ListMemorySegmentPool segmentPool;
+
+    private SpillingBuffer target;
+    private long size = 0L;
+
+    SpillableFeedbackQueue(
+            IOManager ioManager,
+            MemoryManager memoryManager,
+            TypeSerializer<T> serializer,
+            long inMemoryBufferSize,
+            long pageSize)
+            throws MemoryAllocationException {
+        this.serializer = Objects.requireNonNull(serializer);
+        this.ioManager = Objects.requireNonNull(ioManager);
+        this.memoryManager = Objects.requireNonNull(memoryManager);
+
+        this.numPages = (int) (inMemoryBufferSize / pageSize);
+        resetSpillingBuffer();
+    }
+
+    void add(T item) {
+        try {
+            output.clear();
+            serializer.serialize(item, output);
+            target.write(output.getSharedBuffer(), 0, output.length());
+            size++;
+        } catch (IOException e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
+    MutableObjectIterator<T> iterate() {
+        try {
+            DataInputView input = target.flip();
+            return new InputViewIterator<>(input, this.serializer);
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    long size() {
+        return size;
+    }
+
+    void reset() throws Exception {
+        size = 0;
+        close();
+        resetSpillingBuffer();
+    }
+
+    void close() throws IOException {
+        output.clear();
+        List<MemorySegment> toRelease = target.close();
+        toRelease.addAll(segments);
+        memoryManager.release(toRelease);

Review Comment:
   No, because every segment would be removed one by one from the `segments` 
when it is used, it can be seen in `ListMemorySegmentSource#nextSegment`. So 
the return value of `SpillingBuffer.close()` and the remaining `segments` 
together are the memories that should be freed.



-- 
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]

Reply via email to