lindong28 commented on code in PR #97:
URL: https://github.com/apache/flink-ml/pull/97#discussion_r889952819


##########
flink-ml-iteration/src/main/java/org/apache/flink/iteration/datacache/nonkeyed/Segment.java:
##########
@@ -18,38 +18,80 @@
 
 package org.apache.flink.iteration.datacache.nonkeyed;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.memory.MemorySegment;
 
-import java.io.Serializable;
+import java.util.List;
 import java.util.Objects;
 
-/** A segment represents a single file for the cache. */
-public class Segment implements Serializable {
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
 
+/** A segment contains the information about a cache unit. */
+@Internal
+public class Segment {
+
+    /** The path to the file containing cached records. */
     private final Path path;
 
-    /** The count of the records in the file. */
+    /** The count of the records in the segment. */
     private final int count;
 
-    /** The total length of file. */
-    private final long size;
+    /** The total length of file containing cached records. */
+    private long fsSize = -1L;
+
+    /** The memory segments containing cached records. */
+    private List<MemorySegment> cache;
+
+    Segment(Path path, int count, long fsSize) {
+        this.path = checkNotNull(path);
+        checkArgument(count > 0);
+        this.count = count;
+        checkArgument(fsSize > 0);
+        this.fsSize = fsSize;
+    }
 
-    public Segment(Path path, int count, long size) {
-        this.path = path;
+    Segment(Path path, int count, List<MemorySegment> cache) {
+        this.path = checkNotNull(path);
+        checkArgument(count > 0);
         this.count = count;
-        this.size = size;
+        this.cache = checkNotNull(cache);
+    }
+
+    void setCache(List<MemorySegment> cache) {
+        this.cache = checkNotNull(cache);
     }
 
-    public Path getPath() {
+    void setDiskInfo(long fsSize) {

Review Comment:
   Would it be more intuitive to rename this method as `setFsSize()`?



##########
flink-ml-iteration/src/main/java/org/apache/flink/iteration/datacache/nonkeyed/MemorySegmentWriter.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.table.runtime.util.MemorySegmentPool;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+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
+class MemorySegmentWriter<T> implements SegmentWriter<T> {
+
+    /** The tool to serialize received records into bytes. */
+    private final TypeSerializer<T> serializer;
+
+    /** The pre-allocated path to hold cached records into file system. */
+    private final Path path;
+
+    /** The pool to allocate memory segments from. */
+    private final MemorySegmentPool segmentPool;
+
+    /** The output stream to write serialized content to memory segments. */
+    private final ManagedMemoryOutputStream outputStream;
+
+    /** The wrapper view of output stream to be used with TypeSerializer API. 
*/
+    private final DataOutputView outputView;
+
+    /** The number of records added so far. */
+    private int count;
+
+    MemorySegmentWriter(
+            TypeSerializer<T> serializer,
+            Path path,
+            MemorySegmentPool segmentPool,
+            long expectedSize)
+            throws MemoryAllocationException {
+        this.serializer = serializer;
+        this.path = path;
+        this.segmentPool = segmentPool;
+        this.outputStream = new ManagedMemoryOutputStream(segmentPool, 
expectedSize);
+        this.outputView = new DataOutputViewStreamWrapper(outputStream);
+        this.count = 0;
+    }
+
+    @Override
+    public boolean addRecord(T record) throws IOException {
+        if (outputStream.getPos() >= DataCacheWriter.MAX_SEGMENT_SIZE) {
+            return false;
+        }
+        try {
+            serializer.serialize(record, outputView);
+            count++;
+            return true;
+        } catch (IOException e) {
+            if (e.getCause() instanceof MemoryAllocationException) {
+                return false;
+            }
+            throw e;
+        }
+    }
+
+    @Override
+    public Optional<Segment> finish() throws IOException {
+        if (count > 0) {
+            return Optional.of(new Segment(path, count, 
outputStream.getSegments()));
+        } else {
+            segmentPool.returnAll(outputStream.getSegments());
+            return Optional.empty();
+        }
+    }
+
+    /** An output stream subclass that accepts bytes and writes them to memory 
segments. */
+    private static class ManagedMemoryOutputStream extends OutputStream {
+
+        /** The pool to allocate memory segments from. */
+        private final MemorySegmentPool segmentPool;
+
+        /** The number of bytes in a memory segment. */
+        private final int pageSize;
+
+        /** The memory segments containing written bytes. */
+        private final List<MemorySegment> segments = new ArrayList<>();
+
+        /** The index of the segment that currently accepts written bytes. */
+        private int segmentIndex;
+
+        /** THe number of bytes in the current segment that have been written. 
*/
+        private int segmentOffset;
+
+        /** THe number of bytes that have been written so far. */
+        private long globalOffset;
+
+        public ManagedMemoryOutputStream(MemorySegmentPool segmentPool, long 
expectedSize)
+                throws MemoryAllocationException {
+            this.segmentPool = segmentPool;
+            this.pageSize = segmentPool.pageSize();
+            this.segmentIndex = 0;
+            this.segmentOffset = 0;
+
+            Preconditions.checkArgument(expectedSize >= 0);
+            ensureCapacity(Math.max(expectedSize, 1L));
+        }
+
+        public long getPos() {
+            return globalOffset;
+        }
+
+        public List<MemorySegment> getSegments() {
+            return segments;
+        }
+
+        @Override
+        public void write(int b) throws IOException {
+            write(new byte[] {(byte) b}, 0, 1);
+        }
+
+        @Override
+        public void write(@Nullable byte[] b, int off, int len) throws 
IOException {
+            try {
+                ensureCapacity(globalOffset + len);
+            } catch (MemoryAllocationException e) {
+                throw new IOException(e);
+            }
+            writeRecursive(b, off, len);
+        }
+
+        private void ensureCapacity(long capacity) throws 
MemoryAllocationException {
+            Preconditions.checkArgument(capacity > 0);
+            int required =
+                    (int) (capacity % pageSize == 0 ? capacity / pageSize : 
capacity / pageSize + 1)
+                            - segments.size();
+
+            List<MemorySegment> allocatedSegments = new ArrayList<>();
+            for (int i = 0; i < required; i++) {
+                MemorySegment memorySegment = segmentPool.nextSegment();
+                if (memorySegment == null) {
+                    segmentPool.returnAll(allocatedSegments);
+                    throw new MemoryAllocationException();
+                }
+                allocatedSegments.add(memorySegment);
+            }
+
+            segments.addAll(allocatedSegments);
+        }
+
+        private void writeRecursive(byte[] b, int off, int len) {

Review Comment:
   Could we change this function to be iterative to improve performance?



##########
flink-ml-iteration/src/main/java/org/apache/flink/iteration/datacache/nonkeyed/MemorySegmentWriter.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.table.runtime.util.MemorySegmentPool;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+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
+class MemorySegmentWriter<T> implements SegmentWriter<T> {
+
+    /** The tool to serialize received records into bytes. */
+    private final TypeSerializer<T> serializer;
+
+    /** The pre-allocated path to hold cached records into file system. */
+    private final Path path;
+
+    /** The pool to allocate memory segments from. */
+    private final MemorySegmentPool segmentPool;
+
+    /** The output stream to write serialized content to memory segments. */
+    private final ManagedMemoryOutputStream outputStream;
+
+    /** The wrapper view of output stream to be used with TypeSerializer API. 
*/
+    private final DataOutputView outputView;
+
+    /** The number of records added so far. */
+    private int count;
+
+    MemorySegmentWriter(
+            TypeSerializer<T> serializer,
+            Path path,
+            MemorySegmentPool segmentPool,
+            long expectedSize)
+            throws MemoryAllocationException {
+        this.serializer = serializer;
+        this.path = path;
+        this.segmentPool = segmentPool;
+        this.outputStream = new ManagedMemoryOutputStream(segmentPool, 
expectedSize);
+        this.outputView = new DataOutputViewStreamWrapper(outputStream);
+        this.count = 0;
+    }
+
+    @Override
+    public boolean addRecord(T record) throws IOException {
+        if (outputStream.getPos() >= DataCacheWriter.MAX_SEGMENT_SIZE) {
+            return false;
+        }
+        try {
+            serializer.serialize(record, outputView);
+            count++;
+            return true;
+        } catch (IOException e) {
+            if (e.getCause() instanceof MemoryAllocationException) {
+                return false;
+            }
+            throw e;
+        }
+    }
+
+    @Override
+    public Optional<Segment> finish() throws IOException {
+        if (count > 0) {
+            return Optional.of(new Segment(path, count, 
outputStream.getSegments()));
+        } else {
+            segmentPool.returnAll(outputStream.getSegments());
+            return Optional.empty();
+        }
+    }
+
+    /** An output stream subclass that accepts bytes and writes them to memory 
segments. */
+    private static class ManagedMemoryOutputStream extends OutputStream {
+
+        /** The pool to allocate memory segments from. */
+        private final MemorySegmentPool segmentPool;
+
+        /** The number of bytes in a memory segment. */
+        private final int pageSize;
+
+        /** The memory segments containing written bytes. */
+        private final List<MemorySegment> segments = new ArrayList<>();
+
+        /** The index of the segment that currently accepts written bytes. */
+        private int segmentIndex;
+
+        /** THe number of bytes in the current segment that have been written. 
*/
+        private int segmentOffset;
+
+        /** THe number of bytes that have been written so far. */
+        private long globalOffset;
+
+        public ManagedMemoryOutputStream(MemorySegmentPool segmentPool, long 
expectedSize)
+                throws MemoryAllocationException {
+            this.segmentPool = segmentPool;
+            this.pageSize = segmentPool.pageSize();
+            this.segmentIndex = 0;
+            this.segmentOffset = 0;
+
+            Preconditions.checkArgument(expectedSize >= 0);
+            ensureCapacity(Math.max(expectedSize, 1L));
+        }
+
+        public long getPos() {
+            return globalOffset;
+        }
+
+        public List<MemorySegment> getSegments() {
+            return segments;
+        }
+
+        @Override
+        public void write(int b) throws IOException {
+            write(new byte[] {(byte) b}, 0, 1);
+        }
+
+        @Override
+        public void write(@Nullable byte[] b, int off, int len) throws 
IOException {
+            try {
+                ensureCapacity(globalOffset + len);
+            } catch (MemoryAllocationException e) {
+                throw new IOException(e);
+            }
+            writeRecursive(b, off, len);
+        }
+
+        private void ensureCapacity(long capacity) throws 
MemoryAllocationException {
+            Preconditions.checkArgument(capacity > 0);
+            int required =
+                    (int) (capacity % pageSize == 0 ? capacity / pageSize : 
capacity / pageSize + 1)
+                            - segments.size();
+
+            List<MemorySegment> allocatedSegments = new ArrayList<>();
+            for (int i = 0; i < required; i++) {
+                MemorySegment memorySegment = segmentPool.nextSegment();
+                if (memorySegment == null) {
+                    segmentPool.returnAll(allocatedSegments);

Review Comment:
   Should we also return `segments` to the segmentPool? This could be useful 
when `write(...)` encounters limit.



##########
flink-ml-iteration/src/main/java/org/apache/flink/iteration/datacache/nonkeyed/MemorySegmentWriter.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.table.runtime.util.MemorySegmentPool;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+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
+class MemorySegmentWriter<T> implements SegmentWriter<T> {
+
+    /** The tool to serialize received records into bytes. */
+    private final TypeSerializer<T> serializer;
+
+    /** The pre-allocated path to hold cached records into file system. */
+    private final Path path;
+
+    /** The pool to allocate memory segments from. */
+    private final MemorySegmentPool segmentPool;
+
+    /** The output stream to write serialized content to memory segments. */
+    private final ManagedMemoryOutputStream outputStream;
+
+    /** The wrapper view of output stream to be used with TypeSerializer API. 
*/
+    private final DataOutputView outputView;
+
+    /** The number of records added so far. */
+    private int count;
+
+    MemorySegmentWriter(
+            TypeSerializer<T> serializer,
+            Path path,
+            MemorySegmentPool segmentPool,
+            long expectedSize)
+            throws MemoryAllocationException {
+        this.serializer = serializer;
+        this.path = path;
+        this.segmentPool = segmentPool;
+        this.outputStream = new ManagedMemoryOutputStream(segmentPool, 
expectedSize);
+        this.outputView = new DataOutputViewStreamWrapper(outputStream);
+        this.count = 0;
+    }
+
+    @Override
+    public boolean addRecord(T record) throws IOException {
+        if (outputStream.getPos() >= DataCacheWriter.MAX_SEGMENT_SIZE) {
+            return false;
+        }
+        try {
+            serializer.serialize(record, outputView);
+            count++;
+            return true;
+        } catch (IOException e) {
+            if (e.getCause() instanceof MemoryAllocationException) {
+                return false;
+            }
+            throw e;
+        }
+    }
+
+    @Override
+    public Optional<Segment> finish() throws IOException {
+        if (count > 0) {
+            return Optional.of(new Segment(path, count, 
outputStream.getSegments()));
+        } else {
+            segmentPool.returnAll(outputStream.getSegments());
+            return Optional.empty();
+        }
+    }
+
+    /** An output stream subclass that accepts bytes and writes them to memory 
segments. */
+    private static class ManagedMemoryOutputStream extends OutputStream {
+
+        /** The pool to allocate memory segments from. */
+        private final MemorySegmentPool segmentPool;
+
+        /** The number of bytes in a memory segment. */
+        private final int pageSize;
+
+        /** The memory segments containing written bytes. */
+        private final List<MemorySegment> segments = new ArrayList<>();
+
+        /** The index of the segment that currently accepts written bytes. */
+        private int segmentIndex;
+
+        /** THe number of bytes in the current segment that have been written. 
*/
+        private int segmentOffset;
+
+        /** THe number of bytes that have been written so far. */
+        private long globalOffset;
+
+        public ManagedMemoryOutputStream(MemorySegmentPool segmentPool, long 
expectedSize)
+                throws MemoryAllocationException {
+            this.segmentPool = segmentPool;
+            this.pageSize = segmentPool.pageSize();
+            this.segmentIndex = 0;
+            this.segmentOffset = 0;
+
+            Preconditions.checkArgument(expectedSize >= 0);
+            ensureCapacity(Math.max(expectedSize, 1L));
+        }
+
+        public long getPos() {
+            return globalOffset;
+        }
+
+        public List<MemorySegment> getSegments() {
+            return segments;
+        }
+
+        @Override
+        public void write(int b) throws IOException {
+            write(new byte[] {(byte) b}, 0, 1);
+        }
+
+        @Override
+        public void write(@Nullable byte[] b, int off, int len) throws 
IOException {
+            try {
+                ensureCapacity(globalOffset + len);

Review Comment:
   Instead of repeatedly allocate `List<MemorySegment>` instance and do divide 
operation for each write(...), would it be simpler to just maintain 
`allocatedBytes` and just check `globalOffset + len <= allocatedBytes` for must 
write() operation?



##########
flink-ml-iteration/src/main/java/org/apache/flink/iteration/datacache/nonkeyed/FileSegmentWriter.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.FSDataOutputStream;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+
+import java.io.BufferedOutputStream;
+import java.io.IOException;
+import java.util.Optional;
+
+/** A class that writes cache data to a target file in given file system. */
+@Internal
+class FileSegmentWriter<T> implements SegmentWriter<T> {
+
+    /** The tool to serialize received records into bytes. */
+    private final TypeSerializer<T> serializer;
+
+    /** The path to the target file. */
+    private final Path path;
+
+    /** The output stream that writes to the target file. */
+    private final FSDataOutputStream outputStream;
+
+    /** A buffer that wraps the output stream to optimize performance. */
+    private final BufferedOutputStream bufferedOutputStream;
+
+    /** The wrapper view of output stream to be used with TypeSerializer API. 
*/
+    private final DataOutputView outputView;
+
+    /** The number of records added so far. */
+    private int count;
+
+    FileSegmentWriter(TypeSerializer<T> serializer, Path path) throws 
IOException {
+        this.serializer = serializer;
+        this.path = path;
+        this.outputStream = path.getFileSystem().create(path, 
FileSystem.WriteMode.NO_OVERWRITE);
+        this.bufferedOutputStream = new BufferedOutputStream(outputStream);
+        this.outputView = new 
DataOutputViewStreamWrapper(bufferedOutputStream);
+    }
+
+    @Override
+    public boolean addRecord(T record) throws IOException {
+        if (outputStream.getPos() >= DataCacheWriter.MAX_SEGMENT_SIZE) {
+            return false;
+        }
+        serializer.serialize(record, outputView);
+        count++;
+        return true;
+    }
+
+    @Override
+    public Optional<Segment> finish() throws IOException {
+        this.bufferedOutputStream.flush();

Review Comment:
   nits: we use `this` when the member variable name collide with the input 
parameter name. This typically happen in the constructor.  It looks like we 
don't need to use `this` here.



##########
flink-ml-iteration/src/main/java/org/apache/flink/iteration/datacache/nonkeyed/DataCacheReader.java:
##########
@@ -20,120 +20,91 @@
 
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.core.fs.FSDataInputStream;
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataInputViewStreamWrapper;
 
 import javax.annotation.Nullable;
 
 import java.io.IOException;
 import java.util.Iterator;
 import java.util.List;
 
-/** Reads the cached data from a list of paths. */
+/** Reads the cached data from a list of segments. */
 public class DataCacheReader<T> implements Iterator<T> {
 
+    /** The tool to deserialize bytes into records. */
     private final TypeSerializer<T> serializer;
 
-    private final FileSystem fileSystem;
-
+    /** The segments where to read the records from. */
     private final List<Segment> segments;
 
-    @Nullable private SegmentReader currentSegmentReader;
+    /** The current reader for next records. */
+    @Nullable private SegmentReader<T> currentReader;
 
-    public DataCacheReader(
-            TypeSerializer<T> serializer, FileSystem fileSystem, List<Segment> 
segments)
-            throws IOException {
-        this(serializer, fileSystem, segments, new Tuple2<>(0, 0));
+    /** The index of the segment that current reader reads from. */
+    private int segmentIndex;
+
+    /** The number of records that have been read through current reader so 
far. */
+    private int segmentCount;
+
+    public DataCacheReader(TypeSerializer<T> serializer, List<Segment> 
segments) {
+        this(serializer, segments, Tuple2.of(0, 0));
     }
 
     public DataCacheReader(
             TypeSerializer<T> serializer,
-            FileSystem fileSystem,
             List<Segment> segments,
-            Tuple2<Integer, Integer> readerPosition)
-            throws IOException {
-
+            Tuple2<Integer, Integer> readerPosition) {
         this.serializer = serializer;
-        this.fileSystem = fileSystem;
         this.segments = segments;
+        this.segmentIndex = readerPosition.f0;
+        this.segmentCount = readerPosition.f1;
 
-        if (readerPosition.f0 < segments.size()) {
-            this.currentSegmentReader = new SegmentReader(readerPosition.f0, 
readerPosition.f1);
-        }
+        createSegmentReader(readerPosition.f0, readerPosition.f1);

Review Comment:
   The `readerPosition.f1` refers to the number of records that have been read 
from the `DataCache`, right?
   
   But `createSegmentReader(...)` passes this position directly to the 
`MemorySegmentReader` constructor, which interprets this value as the number of 
records that should be skipped `within this segment`.
   
   Should this inconsistency be fixed?



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