yunfengzhou-hub commented on code in PR #97:
URL: https://github.com/apache/flink-ml/pull/97#discussion_r888745233


##########
flink-ml-iteration/src/main/java/org/apache/flink/iteration/datacache/nonkeyed/MemorySegmentWriter.java:
##########
@@ -0,0 +1,170 @@
+/*
+ * 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.datacache.nonkeyed;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+
+/** A class that writes cache data to memory segments. */
+@Internal
+public class MemorySegmentWriter<T> implements SegmentWriter<T> {
+    private final LimitedSizeMemoryManager memoryManager;
+
+    private final Path path;
+
+    private final TypeSerializer<T> serializer;
+
+    private final ManagedMemoryOutputStream outputStream;
+
+    private final DataOutputView outputView;
+
+    private int count;
+
+    public MemorySegmentWriter(
+            Path path,
+            LimitedSizeMemoryManager memoryManager,
+            TypeSerializer<T> serializer,
+            long expectedSize)
+            throws MemoryAllocationException {
+        this.serializer = serializer;
+        this.memoryManager = Preconditions.checkNotNull(memoryManager);
+        this.path = path;
+        this.outputStream = new ManagedMemoryOutputStream(memoryManager, 
expectedSize);
+        this.outputView = new DataOutputViewStreamWrapper(outputStream);
+        this.count = 0;
+    }
+
+    @Override
+    public boolean addRecord(T record) {
+        try {
+            serializer.serialize(record, outputView);
+            count++;
+            return true;
+        } catch (IOException e) {
+            return false;
+        }
+    }
+
+    @Override
+    public int getCount() {
+        return this.count;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public Optional<Segment> finish() throws IOException {
+        if (count > 0) {
+            return Optional.of(
+                    new Segment(
+                            path,
+                            count,
+                            outputStream.getKey(),
+                            outputStream.getSegments(),
+                            (TypeSerializer<Object>) serializer));
+        } else {
+            memoryManager.releaseAll(outputStream.getKey());
+            return Optional.empty();
+        }
+    }
+
+    private static class ManagedMemoryOutputStream extends OutputStream {
+        private final LimitedSizeMemoryManager memoryManager;
+
+        private final int pageSize;
+
+        private final Object key = new Object();
+
+        private final List<MemorySegment> segments = new ArrayList<>();
+
+        private int segmentIndex;
+
+        private int segmentOffset;
+
+        private int globalOffset;
+
+        public ManagedMemoryOutputStream(LimitedSizeMemoryManager 
memoryManager, long expectedSize)
+                throws MemoryAllocationException {
+            this.memoryManager = memoryManager;
+            this.pageSize = memoryManager.getPageSize();
+            this.segmentIndex = 0;
+            this.segmentOffset = 0;
+
+            Preconditions.checkArgument(expectedSize >= 0);
+            if (expectedSize > 0) {

Review Comment:
   Do you mean we should allocate all segments from the beginning, or we should 
not do it but the current implementation has forced such practice?
   
   In my opinion, we should not do it so as to improve memory utilization. And 
the current implementation has achieved it, since it will require memory as 
more records are added instead of occupying all memory space during 
initialization. `expectedSize` is only used when recreating cache during 
reading process, in which case we would have known the size to be occupied from 
the start.
   
   According to other offline discussions I'll modify the code to use flink's 
`LazyMemorySegmentPool` instead of `LimitedSizeMemoryManager`, and does the 
re-caching process when recovering from snapshot instead of during reading. But 
the statements above remain valid.



-- 
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-unsubscr...@flink.apache.org

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

Reply via email to