You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by lz...@apache.org on 2023/01/10 15:16:10 UTC

[flink-table-store] branch master updated: [FLINK-30610] Flink-table-runtime free for disk io in flink-core

This is an automated email from the ASF dual-hosted git repository.

lzljs3620320 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink-table-store.git


The following commit(s) were added to refs/heads/master by this push:
     new 6c9f4331 [FLINK-30610] Flink-table-runtime free for disk io in flink-core
6c9f4331 is described below

commit 6c9f4331e1da6d4eba5b0e423f20cd9e5fcbc10a
Author: Jingsong Lee <ji...@gmail.com>
AuthorDate: Tue Jan 10 23:16:04 2023 +0800

    [FLINK-30610] Flink-table-runtime free for disk io in flink-core
    
    This closes #472
---
 flink-table-store-common/pom.xml                   |   7 -
 .../table/store/data/AbstractPagedInputView.java   |  31 +-
 .../table/store/data/AbstractPagedOutputView.java  |  32 +-
 .../table/store/data/BinaryRowDataSerializer.java  |   5 -
 .../table/store/data/RandomAccessInputView.java    |  89 ++++++
 .../store/data/SimpleCollectingOutputView.java     |  82 +++++
 .../store/data/columnar/ColumnarRowIterator.java   |  30 +-
 .../store/file/utils/IteratorResultIterator.java   |  30 +-
 .../flink/table/store/file/utils/RecordReader.java |   1 +
 .../store/file/utils/RecordReaderIterator.java     |   0
 .../table/store/file/utils/RecordReaderUtils.java  |   0
 .../table/store/file/utils/RecyclableIterator.java |  32 +-
 .../flink/table/store/format/FileFormat.java       |  11 +-
 .../table/store/format/FormatReaderFactory.java    |  20 +-
 .../flink/table/store/utils/DecimalUtils.java      |  99 +++++++
 .../org/apache/flink/table/store/utils/Pool.java   | 106 +++++++
 .../flink/table/store/utils/SegmentsUtil.java      |  82 +++++
 .../flink/table/store/utils/StringUtils.java       |  63 ++++
 .../flink/table/store/connector/FlinkCatalog.java  |  42 ++-
 .../store/connector/sink/StoreSinkWriteImpl.java   |   7 +-
 flink-table-store-core/pom.xml                     |  14 +-
 .../apache/flink/table/store/file/KeyValue.java    |   2 +-
 .../table/store/file/casting/CastExecutors.java    |  27 +-
 .../BlockCompressionFactory.java}                  |  20 +-
 .../store/file/compression/BlockCompressor.java    |  43 +++
 .../store/file/compression/BlockDecompressor.java  |  37 +++
 .../BufferCompressionException.java}               |  28 +-
 .../BufferDecompressionException.java}             |  28 +-
 .../store/file/compression/CompressorUtils.java    |  52 ++++
 .../Lz4BlockCompressionFactory.java}               |  24 +-
 .../store/file/compression/Lz4BlockCompressor.java |  59 ++++
 .../file/compression/Lz4BlockDecompressor.java     |  71 +++++
 .../store/file/disk/AbstractFileIOChannel.java     | 120 ++++++++
 .../store/file/disk/BufferFileChannelReader.java   |  67 +++++
 .../BufferFileReader.java}                         |  17 +-
 .../BufferFileReaderImpl.java}                     |  32 +-
 .../BufferFileWriter.java}                         |  24 +-
 .../store/file/disk/BufferFileWriterImpl.java      |  44 +++
 .../store/file/disk/ChannelReaderInputView.java    | 103 +++++++
 .../file/disk/ChannelReaderInputViewIterator.java  |  71 +++++
 .../MemoryOwner.java => disk/ChannelWithMeta.java} |  31 +-
 .../store/file/disk/ChannelWriterOutputView.java   | 100 +++++++
 .../FileChannelManager.java}                       |  20 +-
 .../store/file/disk/FileChannelManagerImpl.java    | 128 ++++++++
 .../table/store/file/disk/FileChannelUtil.java     |  57 ++++
 .../flink/table/store/file/disk/FileIOChannel.java | 161 ++++++++++
 .../MemoryOwner.java => disk/IOManager.java}       |  30 +-
 .../flink/table/store/file/disk/IOManagerImpl.java | 125 ++++++++
 .../file/io/KeyValueDataFileRecordReader.java      |  18 +-
 .../store/file/io/KeyValueDataFileWriter.java      |   2 +-
 .../store/file/io/RowDataFileRecordReader.java     |  20 +-
 .../table/store/file/manifest/ManifestFile.java    |   7 +-
 .../table/store/file/manifest/ManifestList.java    |  11 +-
 .../flink/table/store/file/memory/Buffer.java      |  64 ++++
 .../store/file/memory/HeapMemorySegmentPool.java   |   1 -
 .../flink/table/store/file/memory/MemoryOwner.java |   2 -
 .../table/store/file/memory/MemoryPoolFactory.java |   1 -
 .../{MemoryOwner.java => MemorySegmentPool.java}   |  31 +-
 .../store/file/mergetree/MergeTreeWriter.java      |   4 +-
 .../file/mergetree/SortBufferWriteBuffer.java      |  10 +-
 .../compact/aggregate/FieldListaggAgg.java         |   4 +-
 .../mergetree/compact/aggregate/FieldSumAgg.java   |   4 +-
 .../file/operation/AbstractFileStoreWrite.java     |   2 +-
 .../table/store/file/operation/FileStoreWrite.java |   2 +-
 .../table/store/file/predicate/BucketSelector.java |   2 +-
 .../table/store/file/schema/UpdateSchema.java      |  37 ---
 .../file/sort/AbstractBinaryExternalMerger.java    | 224 ++++++++++++++
 .../store/file/sort/BinaryExternalMerger.java      |  18 +-
 .../store/file/sort/BinaryExternalSortBuffer.java  |  33 +--
 .../store/file/sort/BinaryInMemorySortBuffer.java  |   9 +-
 .../store/file/sort/BinaryIndexedSortable.java     |   9 +-
 .../table/store/file/sort/BinaryMergeIterator.java |  96 ++++++
 .../flink/table/store/file/sort/HeapSort.java      |  66 +++++
 .../table/store/file/sort/IndexedSortable.java     |  76 +++++
 .../MemoryOwner.java => sort/IndexedSorter.java}   |  25 +-
 .../store/file/sort/PartialOrderPriorityQueue.java | 195 ++++++++++++
 .../flink/table/store/file/sort/QuickSort.java     | 330 +++++++++++++++++++++
 .../table/store/file/sort/SpillChannelManager.java |  95 ++++++
 .../file/stats/FieldStatsArraySerializer.java      |   2 +-
 .../table/store/file/utils/BulkFormatMapping.java  |  10 +-
 .../flink/table/store/file/utils/FileUtils.java    |  26 +-
 .../table/store/file/utils/ObjectSerializer.java   |   4 +-
 .../table/store/file/utils/SerializationUtils.java |   3 +-
 .../flink/table/store/table/sink/TableWrite.java   |   2 +-
 .../table/store/table/sink/TableWriteImpl.java     |   2 +-
 .../flink/table/store/file/FileFormatTest.java     |  13 +-
 .../table/store/file/KeyValueSerializerTest.java   |   2 +-
 .../flink/table/store/file/TestFileStore.java      |   2 +-
 .../table/store/file/TestKeyValueGenerator.java    |   4 +-
 .../file/disk/BufferFileWriterReaderTest.java      | 176 +++++++++++
 .../flink/table/store/file/disk/IOManagerTest.java |  74 +++++
 .../file/format/FileStatsExtractingAvroFormat.java |   5 +-
 .../store/file/format/FlushingFileFormat.java      |   5 +-
 .../store/file/io/KeyValueFileReadWriteTest.java   |   6 +-
 .../store/file/memory/MemoryPoolFactoryTest.java   |   2 -
 .../mergetree/SortBufferWriteBufferTestBase.java   |   2 +-
 .../compact/CombiningRecordReaderTestBase.java     |   2 +-
 .../mergetree/compact/IntervalPartitionTest.java   |   2 +-
 .../file/operation/KeyValueFileStoreReadTest.java  |   2 +-
 .../file/sort/BinaryExternalSortBufferTest.java    |  16 +-
 .../store/file/sort/IntNormalizedKeyComputer.java  |   2 +-
 .../store/file/stats/TestFileStatsExtractor.java   |   5 +-
 .../store/table/AppendOnlyFileStoreTableTest.java  |   2 +-
 flink-table-store-dist/pom.xml                     |   6 -
 flink-table-store-format/pom.xml                   |   6 -
 .../store/format/avro/AbstractAvroBulkFormat.java  |  78 ++---
 .../table/store/format/avro/AvroFileFormat.java    |  23 +-
 .../table/store/format/orc/OrcFileFormat.java      |   5 +-
 .../table/store/format/orc/OrcReaderFactory.java   |  83 ++----
 .../store/format/parquet/ParquetFileFormat.java    |   5 +-
 .../format/parquet/ParquetInputFormatFactory.java  | 102 -------
 .../store/format/parquet/ParquetReaderFactory.java |  55 +---
 .../src/main/resources/META-INF/NOTICE             |  11 +
 .../table/store/format/BulkFileFormatTest.java     |  14 +-
 .../store/format/avro/AvroBulkFormatTest.java      | 170 +----------
 .../store/format/avro/AvroBulkFormatTestUtils.java |  11 +-
 .../store/format/orc/OrcReaderFactoryTest.java     | 181 +++--------
 .../store/format/parquet/ParquetReadWriteTest.java |  31 +-
 flink-table-store-shade/pom.xml                    |  12 -
 119 files changed, 3729 insertions(+), 1137 deletions(-)

diff --git a/flink-table-store-common/pom.xml b/flink-table-store-common/pom.xml
index ef7ffc77..fd5fe346 100644
--- a/flink-table-store-common/pom.xml
+++ b/flink-table-store-common/pom.xml
@@ -46,13 +46,6 @@ under the License.
             <scope>provided</scope>
         </dependency>
 
-        <dependency>
-            <groupId>org.apache.flink</groupId>
-            <artifactId>flink-connector-files</artifactId>
-            <version>${flink.version}</version>
-            <scope>provided</scope>
-        </dependency>
-
         <!-- Java compiler -->
         <dependency>
             <groupId>org.codehaus.janino</groupId>
diff --git a/flink-table-store-common/src/main/java/org/apache/flink/table/store/data/AbstractPagedInputView.java b/flink-table-store-common/src/main/java/org/apache/flink/table/store/data/AbstractPagedInputView.java
index d05a8297..0539b0b0 100644
--- a/flink-table-store-common/src/main/java/org/apache/flink/table/store/data/AbstractPagedInputView.java
+++ b/flink-table-store-common/src/main/java/org/apache/flink/table/store/data/AbstractPagedInputView.java
@@ -35,9 +35,6 @@ public abstract class AbstractPagedInputView implements DataInputView {
 
     private MemorySegment currentSegment;
 
-    protected final int
-            headerLength; // the number of bytes to skip at the beginning of each segment
-
     private int positionInSegment; // the offset in the current segment
 
     private int limitInSegment; // the limit in the current segment before switching to the next
@@ -57,14 +54,10 @@ public abstract class AbstractPagedInputView implements DataInputView {
      *
      * @param initialSegment The memory segment to start reading from.
      * @param initialLimit The position one after the last valid byte in the initial segment.
-     * @param headerLength The number of bytes to skip at the beginning of each segment for the
-     *     header. This length must be the same for all memory segments.
      */
-    protected AbstractPagedInputView(
-            MemorySegment initialSegment, int initialLimit, int headerLength) {
-        this.headerLength = headerLength;
-        this.positionInSegment = headerLength;
-        seekInput(initialSegment, headerLength, initialLimit);
+    protected AbstractPagedInputView(MemorySegment initialSegment, int initialLimit) {
+        this.positionInSegment = 0;
+        seekInput(initialSegment, 0, initialLimit);
     }
 
     /**
@@ -73,13 +66,8 @@ public abstract class AbstractPagedInputView implements DataInputView {
      *
      * <p>WARNING: The view is not readable until the first call to either {@link #advance()}, or to
      * {@link #seekInput(MemorySegment, int, int)}.
-     *
-     * @param headerLength The number of bytes to skip at the beginning of each segment for the
-     *     header.
      */
-    protected AbstractPagedInputView(int headerLength) {
-        this.headerLength = headerLength;
-    }
+    protected AbstractPagedInputView() {}
 
     // --------------------------------------------------------------------------------------------
     //                                  Page Management
@@ -164,12 +152,7 @@ public abstract class AbstractPagedInputView implements DataInputView {
         // EOF is reproducible (if nextSegment throws a reproducible EOFException)
         this.currentSegment = nextSegment(this.currentSegment);
         this.limitInSegment = getLimitForSegment(this.currentSegment);
-        this.positionInSegment = this.headerLength;
-    }
-
-    /** @return header length. */
-    public int getHeaderLength() {
-        return headerLength;
+        this.positionInSegment = 0;
     }
 
     /**
@@ -195,8 +178,8 @@ public abstract class AbstractPagedInputView implements DataInputView {
      */
     protected void clear() {
         this.currentSegment = null;
-        this.positionInSegment = this.headerLength;
-        this.limitInSegment = headerLength;
+        this.positionInSegment = 0;
+        this.limitInSegment = 0;
     }
 
     // --------------------------------------------------------------------------------------------
diff --git a/flink-table-store-common/src/main/java/org/apache/flink/table/store/data/AbstractPagedOutputView.java b/flink-table-store-common/src/main/java/org/apache/flink/table/store/data/AbstractPagedOutputView.java
index ff637efc..49cad0af 100644
--- a/flink-table-store-common/src/main/java/org/apache/flink/table/store/data/AbstractPagedOutputView.java
+++ b/flink-table-store-common/src/main/java/org/apache/flink/table/store/data/AbstractPagedOutputView.java
@@ -36,13 +36,10 @@ import java.io.UTFDataFormatException;
  */
 public abstract class AbstractPagedOutputView implements DataOutputView, MemorySegmentWritable {
 
-    private MemorySegment currentSegment; // the current memory segment to write to
+    protected MemorySegment currentSegment; // the current memory segment to write to
 
     protected final int segmentSize; // the size of the memory segments
 
-    protected final int
-            headerLength; // the number of bytes to skip at the beginning of each segment
-
     private int positionInSegment; // the offset in the current segment
 
     private byte[] utfBuffer; // the reusable array for UTF encodings
@@ -58,28 +55,14 @@ public abstract class AbstractPagedOutputView implements DataOutputView, MemoryS
      *
      * @param initialSegment The segment that the view starts writing to.
      * @param segmentSize The size of the memory segments.
-     * @param headerLength The number of bytes to skip at the beginning of each segment for the
-     *     header.
      */
-    protected AbstractPagedOutputView(
-            MemorySegment initialSegment, int segmentSize, int headerLength) {
+    protected AbstractPagedOutputView(MemorySegment initialSegment, int segmentSize) {
         if (initialSegment == null) {
             throw new NullPointerException("Initial Segment may not be null");
         }
         this.segmentSize = segmentSize;
-        this.headerLength = headerLength;
         this.currentSegment = initialSegment;
-        this.positionInSegment = headerLength;
-    }
-
-    /**
-     * @param segmentSize The size of the memory segments.
-     * @param headerLength The number of bytes to skip at the beginning of each segment for the
-     *     header.
-     */
-    protected AbstractPagedOutputView(int segmentSize, int headerLength) {
-        this.segmentSize = segmentSize;
-        this.headerLength = headerLength;
+        this.positionInSegment = 0;
     }
 
     // --------------------------------------------------------------------------------------------
@@ -137,12 +120,7 @@ public abstract class AbstractPagedOutputView implements DataOutputView, MemoryS
      */
     public void advance() throws IOException {
         this.currentSegment = nextSegment(this.currentSegment, this.positionInSegment);
-        this.positionInSegment = this.headerLength;
-    }
-
-    /** @return header length. */
-    public int getHeaderLength() {
-        return headerLength;
+        this.positionInSegment = 0;
     }
 
     /**
@@ -166,7 +144,7 @@ public abstract class AbstractPagedOutputView implements DataOutputView, MemoryS
      */
     protected void clear() {
         this.currentSegment = null;
-        this.positionInSegment = this.headerLength;
+        this.positionInSegment = 0;
     }
 
     // --------------------------------------------------------------------------------------------
diff --git a/flink-table-store-common/src/main/java/org/apache/flink/table/store/data/BinaryRowDataSerializer.java b/flink-table-store-common/src/main/java/org/apache/flink/table/store/data/BinaryRowDataSerializer.java
index 65be3bd4..1f8892b1 100644
--- a/flink-table-store-common/src/main/java/org/apache/flink/table/store/data/BinaryRowDataSerializer.java
+++ b/flink-table-store-common/src/main/java/org/apache/flink/table/store/data/BinaryRowDataSerializer.java
@@ -131,7 +131,6 @@ public class BinaryRowDataSerializer extends AbstractRowDataSerializer<BinaryRow
     @Override
     public int serializeToPages(BinaryRowData record, AbstractPagedOutputView headerLessView)
             throws IOException {
-        checkArgument(headerLessView.getHeaderLength() == 0);
         int skip = checkSkipWriteForFixLengthPart(headerLessView);
         headerLessView.writeInt(record.getSizeInBytes());
         serializeWithoutLength(record, headerLessView);
@@ -176,7 +175,6 @@ public class BinaryRowDataSerializer extends AbstractRowDataSerializer<BinaryRow
     @Override
     public BinaryRowData deserializeFromPages(
             BinaryRowData reuse, AbstractPagedInputView headerLessView) throws IOException {
-        checkArgument(headerLessView.getHeaderLength() == 0);
         checkSkipReadForFixLengthPart(headerLessView);
         return deserialize(reuse, headerLessView);
     }
@@ -184,7 +182,6 @@ public class BinaryRowDataSerializer extends AbstractRowDataSerializer<BinaryRow
     @Override
     public BinaryRowData mapFromPages(BinaryRowData reuse, AbstractPagedInputView headerLessView)
             throws IOException {
-        checkArgument(headerLessView.getHeaderLength() == 0);
         checkSkipReadForFixLengthPart(headerLessView);
         pointTo(headerLessView.readInt(), reuse, headerLessView);
         return reuse;
@@ -192,7 +189,6 @@ public class BinaryRowDataSerializer extends AbstractRowDataSerializer<BinaryRow
 
     @Override
     public void skipRecordFromPages(AbstractPagedInputView headerLessView) throws IOException {
-        checkArgument(headerLessView.getHeaderLength() == 0);
         checkSkipReadForFixLengthPart(headerLessView);
         headerLessView.skipBytes(headerLessView.readInt());
     }
@@ -220,7 +216,6 @@ public class BinaryRowDataSerializer extends AbstractRowDataSerializer<BinaryRow
      */
     public void pointTo(int length, BinaryRowData reuse, AbstractPagedInputView headerLessView)
             throws IOException {
-        checkArgument(headerLessView.getHeaderLength() == 0);
         if (length < 0) {
             throw new IOException(
                     String.format(
diff --git a/flink-table-store-common/src/main/java/org/apache/flink/table/store/data/RandomAccessInputView.java b/flink-table-store-common/src/main/java/org/apache/flink/table/store/data/RandomAccessInputView.java
new file mode 100644
index 00000000..e8688e73
--- /dev/null
+++ b/flink-table-store-common/src/main/java/org/apache/flink/table/store/data/RandomAccessInputView.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.table.store.data;
+
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.SeekableDataInputView;
+import org.apache.flink.util.MathUtils;
+
+import java.io.EOFException;
+import java.util.ArrayList;
+
+/** A {@link AbstractPagedInputView} to read pages in memory. */
+public class RandomAccessInputView extends AbstractPagedInputView implements SeekableDataInputView {
+
+    private final ArrayList<MemorySegment> segments;
+
+    private int currentSegmentIndex;
+
+    private final int segmentSizeBits;
+
+    private final int segmentSizeMask;
+
+    private final int segmentSize;
+
+    private final int limitInLastSegment;
+
+    public RandomAccessInputView(ArrayList<MemorySegment> segments, int segmentSize) {
+        this(segments, segmentSize, segmentSize);
+    }
+
+    public RandomAccessInputView(
+            ArrayList<MemorySegment> segments, int segmentSize, int limitInLastSegment) {
+        super(segments.get(0), segments.size() > 1 ? segmentSize : limitInLastSegment);
+        this.segments = segments;
+        this.currentSegmentIndex = 0;
+        this.segmentSize = segmentSize;
+        this.segmentSizeBits = MathUtils.log2strict(segmentSize);
+        this.segmentSizeMask = segmentSize - 1;
+        this.limitInLastSegment = limitInLastSegment;
+    }
+
+    @Override
+    public void setReadPosition(long position) {
+        final int bufferNum = (int) (position >>> this.segmentSizeBits);
+        final int offset = (int) (position & this.segmentSizeMask);
+
+        this.currentSegmentIndex = bufferNum;
+        seekInput(
+                this.segments.get(bufferNum),
+                offset,
+                bufferNum < this.segments.size() - 1 ? this.segmentSize : this.limitInLastSegment);
+    }
+
+    public long getReadPosition() {
+        return (((long) currentSegmentIndex) << segmentSizeBits) + getCurrentPositionInSegment();
+    }
+
+    @Override
+    protected MemorySegment nextSegment(MemorySegment current) throws EOFException {
+        if (++this.currentSegmentIndex < this.segments.size()) {
+            return this.segments.get(this.currentSegmentIndex);
+        } else {
+            throw new EOFException();
+        }
+    }
+
+    @Override
+    protected int getLimitForSegment(MemorySegment segment) {
+        return this.currentSegmentIndex == this.segments.size() - 1
+                ? this.limitInLastSegment
+                : this.segmentSize;
+    }
+}
diff --git a/flink-table-store-common/src/main/java/org/apache/flink/table/store/data/SimpleCollectingOutputView.java b/flink-table-store-common/src/main/java/org/apache/flink/table/store/data/SimpleCollectingOutputView.java
new file mode 100644
index 00000000..a1be58f3
--- /dev/null
+++ b/flink-table-store-common/src/main/java/org/apache/flink/table/store/data/SimpleCollectingOutputView.java
@@ -0,0 +1,82 @@
+/*
+ * 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.table.store.data;
+
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentSource;
+import org.apache.flink.util.MathUtils;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * The list with the full segments contains at any point all completely full segments, plus the
+ * segment that is currently filled.
+ */
+public class SimpleCollectingOutputView extends AbstractPagedOutputView {
+
+    private final List<MemorySegment> fullSegments;
+
+    private final MemorySegmentSource memorySource;
+
+    private final int segmentSizeBits;
+
+    private int segmentNum;
+
+    public SimpleCollectingOutputView(
+            List<MemorySegment> fullSegmentTarget, MemorySegmentSource memSource, int segmentSize) {
+        super(memSource.nextSegment(), segmentSize);
+        this.segmentSizeBits = MathUtils.log2strict(segmentSize);
+        this.fullSegments = fullSegmentTarget;
+        this.memorySource = memSource;
+        this.fullSegments.add(getCurrentSegment());
+    }
+
+    public void reset() {
+        if (this.fullSegments.size() != 0) {
+            throw new IllegalStateException("The target list still contains memory segments.");
+        }
+
+        clear();
+        try {
+            advance();
+        } catch (IOException ioex) {
+            throw new RuntimeException("Error getting first segment for record collector.", ioex);
+        }
+        this.segmentNum = 0;
+    }
+
+    @Override
+    protected MemorySegment nextSegment(MemorySegment current, int positionInCurrent)
+            throws EOFException {
+        final MemorySegment next = this.memorySource.nextSegment();
+        if (next != null) {
+            this.fullSegments.add(next);
+            this.segmentNum++;
+            return next;
+        } else {
+            throw new EOFException("Can't collect further: memorySource depleted");
+        }
+    }
+
+    public long getCurrentOffset() {
+        return (((long) this.segmentNum) << this.segmentSizeBits) + getCurrentPositionInSegment();
+    }
+}
diff --git a/flink-table-store-common/src/main/java/org/apache/flink/table/store/data/columnar/ColumnarRowIterator.java b/flink-table-store-common/src/main/java/org/apache/flink/table/store/data/columnar/ColumnarRowIterator.java
index 6092b716..73fde99b 100644
--- a/flink-table-store-common/src/main/java/org/apache/flink/table/store/data/columnar/ColumnarRowIterator.java
+++ b/flink-table-store-common/src/main/java/org/apache/flink/table/store/data/columnar/ColumnarRowIterator.java
@@ -19,24 +19,20 @@
 package org.apache.flink.table.store.data.columnar;
 
 import org.apache.flink.annotation.Internal;
-import org.apache.flink.connector.file.src.reader.BulkFormat;
-import org.apache.flink.connector.file.src.util.CheckpointedPosition;
-import org.apache.flink.connector.file.src.util.MutableRecordAndPosition;
-import org.apache.flink.connector.file.src.util.RecordAndPosition;
-import org.apache.flink.connector.file.src.util.RecyclableIterator;
 import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.store.file.utils.RecordReader;
+import org.apache.flink.table.store.file.utils.RecyclableIterator;
 
 import javax.annotation.Nullable;
 
 /**
- * A {@link BulkFormat.RecordIterator} that returns {@link RowData}s. The next row is set by {@link
- * ColumnarRowData#setRowId}.
+ * A {@link RecordReader.RecordIterator} that returns {@link RowData}s. The next row is set by
+ * {@link ColumnarRowData#setRowId}.
  */
 @Internal
 public class ColumnarRowIterator extends RecyclableIterator<RowData> {
 
     private final ColumnarRowData rowData;
-    private final MutableRecordAndPosition<RowData> recordAndPosition;
 
     private int num;
     private int pos;
@@ -44,31 +40,19 @@ public class ColumnarRowIterator extends RecyclableIterator<RowData> {
     public ColumnarRowIterator(ColumnarRowData rowData, @Nullable Runnable recycler) {
         super(recycler);
         this.rowData = rowData;
-        this.recordAndPosition = new MutableRecordAndPosition<>();
     }
 
-    /**
-     * @param num number rows in this batch.
-     * @param recordSkipCount The number of rows that have been returned before this batch.
-     */
-    public void set(final int num, final long recordSkipCount) {
-        set(num, CheckpointedPosition.NO_OFFSET, recordSkipCount);
-    }
-
-    /** Set number rows in this batch and updates the position. */
-    public void set(final int num, final long offset, final long recordSkipCount) {
+    public void set(int num) {
         this.num = num;
         this.pos = 0;
-        this.recordAndPosition.set(null, offset, recordSkipCount);
     }
 
     @Nullable
     @Override
-    public RecordAndPosition<RowData> next() {
+    public RowData next() {
         if (pos < num) {
             rowData.setRowId(pos++);
-            recordAndPosition.setNext(rowData);
-            return recordAndPosition;
+            return rowData;
         } else {
             return null;
         }
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java b/flink-table-store-common/src/main/java/org/apache/flink/table/store/file/utils/IteratorResultIterator.java
similarity index 55%
copy from flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java
copy to flink-table-store-common/src/main/java/org/apache/flink/table/store/file/utils/IteratorResultIterator.java
index 4a352309..a8ae6d87 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java
+++ b/flink-table-store-common/src/main/java/org/apache/flink/table/store/file/utils/IteratorResultIterator.java
@@ -16,19 +16,29 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.store.file.memory;
+package org.apache.flink.table.store.file.utils;
 
-import org.apache.flink.table.runtime.util.MemorySegmentPool;
+import javax.annotation.Nullable;
 
-/** A class provides memory related methods. */
-public interface MemoryOwner {
+import java.util.Iterator;
 
-    /** Set {@link MemorySegmentPool} for the owner. */
-    void setMemoryPool(MemorySegmentPool memoryPool);
+/** A simple {@link RecordReader.RecordIterator} that returns the elements of an iterator. */
+public final class IteratorResultIterator<E> extends RecyclableIterator<E> {
 
-    /** Memory occupancy size of this owner. */
-    long memoryOccupancy();
+    private final Iterator<E> records;
 
-    /** Flush memory of owner, release memory. */
-    void flushMemory() throws Exception;
+    public IteratorResultIterator(final Iterator<E> records, final @Nullable Runnable recycler) {
+        super(recycler);
+        this.records = records;
+    }
+
+    @Nullable
+    @Override
+    public E next() {
+        if (records.hasNext()) {
+            return records.next();
+        } else {
+            return null;
+        }
+    }
 }
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/utils/RecordReader.java b/flink-table-store-common/src/main/java/org/apache/flink/table/store/file/utils/RecordReader.java
similarity index 99%
rename from flink-table-store-core/src/main/java/org/apache/flink/table/store/file/utils/RecordReader.java
rename to flink-table-store-common/src/main/java/org/apache/flink/table/store/file/utils/RecordReader.java
index 7a52eb67..cf6ac997 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/utils/RecordReader.java
+++ b/flink-table-store-common/src/main/java/org/apache/flink/table/store/file/utils/RecordReader.java
@@ -49,6 +49,7 @@ public interface RecordReader<T> extends Closeable {
          * Gets the next record from the iterator. Returns null if this iterator has no more
          * elements.
          */
+        @Nullable
         T next() throws IOException;
 
         /**
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/utils/RecordReaderIterator.java b/flink-table-store-common/src/main/java/org/apache/flink/table/store/file/utils/RecordReaderIterator.java
similarity index 100%
rename from flink-table-store-core/src/main/java/org/apache/flink/table/store/file/utils/RecordReaderIterator.java
rename to flink-table-store-common/src/main/java/org/apache/flink/table/store/file/utils/RecordReaderIterator.java
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/utils/RecordReaderUtils.java b/flink-table-store-common/src/main/java/org/apache/flink/table/store/file/utils/RecordReaderUtils.java
similarity index 100%
rename from flink-table-store-core/src/main/java/org/apache/flink/table/store/file/utils/RecordReaderUtils.java
rename to flink-table-store-common/src/main/java/org/apache/flink/table/store/file/utils/RecordReaderUtils.java
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java b/flink-table-store-common/src/main/java/org/apache/flink/table/store/file/utils/RecyclableIterator.java
similarity index 53%
copy from flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java
copy to flink-table-store-common/src/main/java/org/apache/flink/table/store/file/utils/RecyclableIterator.java
index 4a352309..34768390 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java
+++ b/flink-table-store-common/src/main/java/org/apache/flink/table/store/file/utils/RecyclableIterator.java
@@ -16,19 +16,31 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.store.file.memory;
+package org.apache.flink.table.store.file.utils;
 
-import org.apache.flink.table.runtime.util.MemorySegmentPool;
+import org.apache.flink.annotation.Internal;
 
-/** A class provides memory related methods. */
-public interface MemoryOwner {
+import javax.annotation.Nullable;
 
-    /** Set {@link MemorySegmentPool} for the owner. */
-    void setMemoryPool(MemorySegmentPool memoryPool);
+/**
+ * Utility base class for iterators that accept a recycler.
+ *
+ * @param <E> The type of the records returned by the iterator.
+ */
+@Internal
+public abstract class RecyclableIterator<E> implements RecordReader.RecordIterator<E> {
+
+    @Nullable private final Runnable recycler;
 
-    /** Memory occupancy size of this owner. */
-    long memoryOccupancy();
+    /** Creates a {@code RecyclableIterator} with the given optional recycler. */
+    protected RecyclableIterator(@Nullable Runnable recycler) {
+        this.recycler = recycler;
+    }
 
-    /** Flush memory of owner, release memory. */
-    void flushMemory() throws Exception;
+    @Override
+    public void releaseBatch() {
+        if (recycler != null) {
+            recycler.run();
+        }
+    }
 }
diff --git a/flink-table-store-common/src/main/java/org/apache/flink/table/store/format/FileFormat.java b/flink-table-store-common/src/main/java/org/apache/flink/table/store/format/FileFormat.java
index f34d4e1b..30135bc0 100644
--- a/flink-table-store-common/src/main/java/org/apache/flink/table/store/format/FileFormat.java
+++ b/flink-table-store-common/src/main/java/org/apache/flink/table/store/format/FileFormat.java
@@ -22,8 +22,6 @@ import org.apache.flink.api.common.serialization.BulkWriter;
 import org.apache.flink.configuration.ConfigOption;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.DelegatingConfiguration;
-import org.apache.flink.connector.file.src.FileSourceSplit;
-import org.apache.flink.connector.file.src.reader.BulkFormat;
 import org.apache.flink.table.api.ValidationException;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.store.file.predicate.Predicate;
@@ -54,19 +52,19 @@ public abstract class FileFormat {
     }
 
     /**
-     * Create a {@link BulkFormat} from the type, with projection pushed down.
+     * Create a {@link FormatReaderFactory} from the type, with projection pushed down.
      *
      * @param type Type without projection.
      * @param projection See {@link org.apache.flink.table.connector.Projection#toNestedIndexes()}.
      * @param filters A list of filters in conjunctive form for filtering on a best-effort basis.
      */
-    public abstract BulkFormat<RowData, FileSourceSplit> createReaderFactory(
+    public abstract FormatReaderFactory createReaderFactory(
             RowType type, int[][] projection, @Nullable List<Predicate> filters);
 
     /** Create a {@link BulkWriter.Factory} from the type. */
     public abstract BulkWriter.Factory<RowData> createWriterFactory(RowType type);
 
-    public BulkFormat<RowData, FileSourceSplit> createReaderFactory(RowType rowType) {
+    public FormatReaderFactory createReaderFactory(RowType rowType) {
         int[][] projection = new int[rowType.getFieldCount()][];
         for (int i = 0; i < projection.length; i++) {
             projection[i] = new int[] {i};
@@ -74,8 +72,7 @@ public abstract class FileFormat {
         return createReaderFactory(rowType, projection);
     }
 
-    public BulkFormat<RowData, FileSourceSplit> createReaderFactory(
-            RowType rowType, int[][] projection) {
+    public FormatReaderFactory createReaderFactory(RowType rowType, int[][] projection) {
         return createReaderFactory(rowType, projection, new ArrayList<>());
     }
 
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java b/flink-table-store-common/src/main/java/org/apache/flink/table/store/format/FormatReaderFactory.java
similarity index 63%
copy from flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java
copy to flink-table-store-common/src/main/java/org/apache/flink/table/store/format/FormatReaderFactory.java
index 4a352309..e71b085b 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java
+++ b/flink-table-store-common/src/main/java/org/apache/flink/table/store/format/FormatReaderFactory.java
@@ -16,19 +16,17 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.store.file.memory;
+package org.apache.flink.table.store.format;
 
-import org.apache.flink.table.runtime.util.MemorySegmentPool;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.store.file.utils.RecordReader;
 
-/** A class provides memory related methods. */
-public interface MemoryOwner {
+import java.io.IOException;
+import java.io.Serializable;
 
-    /** Set {@link MemorySegmentPool} for the owner. */
-    void setMemoryPool(MemorySegmentPool memoryPool);
+/** A factory to create {@link RecordReader} for file. */
+public interface FormatReaderFactory extends Serializable {
 
-    /** Memory occupancy size of this owner. */
-    long memoryOccupancy();
-
-    /** Flush memory of owner, release memory. */
-    void flushMemory() throws Exception;
+    RecordReader<RowData> createReader(Path file) throws IOException;
 }
diff --git a/flink-table-store-common/src/main/java/org/apache/flink/table/store/utils/DecimalUtils.java b/flink-table-store-common/src/main/java/org/apache/flink/table/store/utils/DecimalUtils.java
new file mode 100644
index 00000000..59c2bf1e
--- /dev/null
+++ b/flink-table-store-common/src/main/java/org/apache/flink/table/store/utils/DecimalUtils.java
@@ -0,0 +1,99 @@
+/*
+ * 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.table.store.utils;
+
+import org.apache.flink.table.data.DecimalData;
+
+import java.math.BigDecimal;
+import java.math.RoundingMode;
+
+import static org.apache.flink.table.data.DecimalData.fromBigDecimal;
+
+/** Utilities for {@link DecimalData}. */
+public class DecimalUtils {
+
+    static final int MAX_COMPACT_PRECISION = 18;
+
+    static final long[] POW10 = new long[MAX_COMPACT_PRECISION + 1];
+
+    static {
+        POW10[0] = 1;
+        for (int i = 1; i < POW10.length; i++) {
+            POW10[i] = 10 * POW10[i - 1];
+        }
+    }
+
+    public static double doubleValue(DecimalData decimal) {
+        if (decimal.isCompact()) {
+            return ((double) decimal.toUnscaledLong()) / POW10[decimal.scale()];
+        } else {
+            return decimal.toBigDecimal().doubleValue();
+        }
+    }
+
+    public static DecimalData add(DecimalData v1, DecimalData v2, int precision, int scale) {
+        if (v1.isCompact()
+                && v2.isCompact()
+                && v1.scale() == v2.scale()
+                && DecimalData.isCompact(precision)) {
+            assert scale == v1.scale(); // no need to rescale
+            try {
+                long ls =
+                        Math.addExact(v1.toUnscaledLong(), v2.toUnscaledLong()); // checks overflow
+                return DecimalData.fromUnscaledLong(ls, precision, scale);
+            } catch (ArithmeticException e) {
+                // overflow, fall through
+            }
+        }
+        BigDecimal bd = v1.toBigDecimal().add(v2.toBigDecimal());
+        return fromBigDecimal(bd, precision, scale);
+    }
+
+    public static long castToIntegral(DecimalData dec) {
+        BigDecimal bd = dec.toBigDecimal();
+        // rounding down. This is consistent with float=>int,
+        // and consistent with SQLServer, Spark.
+        bd = bd.setScale(0, RoundingMode.DOWN);
+        return bd.longValue();
+    }
+
+    public static DecimalData castToDecimal(DecimalData dec, int precision, int scale) {
+        return fromBigDecimal(dec.toBigDecimal(), precision, scale);
+    }
+
+    public static DecimalData castFrom(DecimalData dec, int precision, int scale) {
+        return fromBigDecimal(dec.toBigDecimal(), precision, scale);
+    }
+
+    public static DecimalData castFrom(String string, int precision, int scale) {
+        return fromBigDecimal(new BigDecimal(string), precision, scale);
+    }
+
+    public static DecimalData castFrom(double val, int p, int s) {
+        return fromBigDecimal(BigDecimal.valueOf(val), p, s);
+    }
+
+    public static DecimalData castFrom(long val, int p, int s) {
+        return fromBigDecimal(BigDecimal.valueOf(val), p, s);
+    }
+
+    public static boolean castToBoolean(DecimalData dec) {
+        return dec.toBigDecimal().compareTo(BigDecimal.ZERO) != 0;
+    }
+}
diff --git a/flink-table-store-common/src/main/java/org/apache/flink/table/store/utils/Pool.java b/flink-table-store-common/src/main/java/org/apache/flink/table/store/utils/Pool.java
new file mode 100644
index 00000000..60429f4a
--- /dev/null
+++ b/flink-table-store-common/src/main/java/org/apache/flink/table/store/utils/Pool.java
@@ -0,0 +1,106 @@
+/*
+ * 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.table.store.utils;
+
+import org.apache.flink.table.store.file.utils.RecordReader;
+
+import javax.annotation.Nullable;
+
+import java.util.concurrent.ArrayBlockingQueue;
+
+/**
+ * A pool to cache and recycle heavyweight objects, to reduce object allocation.
+ *
+ * <p>This pool can be used in the {@link RecordReader}, when the returned objects are heavyweight
+ * and need to be reused for efficiency. Because the reading happens in I/O threads while the record
+ * processing happens in Flink's main processing threads, these objects cannot be reused immediately
+ * after being returned. They can be reused, once they are recycled back to the pool.
+ *
+ * @param <T> The type of object cached in the pool.
+ */
+public class Pool<T> {
+
+    private final ArrayBlockingQueue<T> pool;
+
+    private final Recycler<T> recycler;
+
+    private final int poolCapacity;
+    private int poolSize;
+
+    /**
+     * Creates a pool with the given capacity. No more than that many elements may be added to the
+     * pool.
+     */
+    public Pool(int poolCapacity) {
+        this.pool = new ArrayBlockingQueue<>(poolCapacity);
+        this.recycler = this::addBack;
+        this.poolCapacity = poolCapacity;
+        this.poolSize = 0;
+    }
+
+    /**
+     * Gets the recycler for this pool. The recycler returns its given objects back to this pool.
+     */
+    public Recycler<T> recycler() {
+        return recycler;
+    }
+
+    /**
+     * Adds an entry to the pool with an optional payload. This method fails if called more often
+     * than the pool capacity specified during construction.
+     */
+    public synchronized void add(T object) {
+        if (poolSize >= poolCapacity) {
+            throw new IllegalStateException("No space left in pool");
+        }
+        poolSize++;
+
+        addBack(object);
+    }
+
+    /** Gets the next cached entry. This blocks until the next entry is available. */
+    public T pollEntry() throws InterruptedException {
+        return pool.take();
+    }
+
+    /** Tries to get the next cached entry. If the pool is empty, this method returns null. */
+    @Nullable
+    public T tryPollEntry() {
+        return pool.poll();
+    }
+
+    /** Internal callback to put an entry back to the pool. */
+    void addBack(T object) {
+        pool.add(object);
+    }
+
+    // --------------------------------------------------------------------------------------------
+
+    /**
+     * A Recycler puts objects into the pool that the recycler is associated with.
+     *
+     * @param <T> The pooled and recycled type.
+     */
+    @FunctionalInterface
+    public interface Recycler<T> {
+
+        /** Recycles the given object to the pool that this recycler works with. */
+        void recycle(T object);
+    }
+}
diff --git a/flink-table-store-common/src/main/java/org/apache/flink/table/store/utils/SegmentsUtil.java b/flink-table-store-common/src/main/java/org/apache/flink/table/store/utils/SegmentsUtil.java
new file mode 100644
index 00000000..644a4546
--- /dev/null
+++ b/flink-table-store-common/src/main/java/org/apache/flink/table/store/utils/SegmentsUtil.java
@@ -0,0 +1,82 @@
+/*
+ * 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.table.store.utils;
+
+import org.apache.flink.core.memory.MemorySegment;
+
+/** Util for data format segments calc. */
+public class SegmentsUtil {
+
+    /** Is it just in first MemorySegment, we use quick way to do something. */
+    private static boolean inFirstSegment(MemorySegment[] segments, int offset, int numBytes) {
+        return numBytes + offset <= segments[0].size();
+    }
+
+    /**
+     * Copy segments to a new byte[].
+     *
+     * @param segments Source segments.
+     * @param offset Source segments offset.
+     * @param numBytes the number bytes to copy.
+     */
+    public static byte[] copyToBytes(MemorySegment[] segments, int offset, int numBytes) {
+        return copyToBytes(segments, offset, new byte[numBytes], 0, numBytes);
+    }
+
+    /**
+     * Copy segments to target byte[].
+     *
+     * @param segments Source segments.
+     * @param offset Source segments offset.
+     * @param bytes target byte[].
+     * @param bytesOffset target byte[] offset.
+     * @param numBytes the number bytes to copy.
+     */
+    public static byte[] copyToBytes(
+            MemorySegment[] segments, int offset, byte[] bytes, int bytesOffset, int numBytes) {
+        if (inFirstSegment(segments, offset, numBytes)) {
+            segments[0].get(offset, bytes, bytesOffset, numBytes);
+        } else {
+            copyMultiSegmentsToBytes(segments, offset, bytes, bytesOffset, numBytes);
+        }
+        return bytes;
+    }
+
+    public static void copyMultiSegmentsToBytes(
+            MemorySegment[] segments, int offset, byte[] bytes, int bytesOffset, int numBytes) {
+        int remainSize = numBytes;
+        for (MemorySegment segment : segments) {
+            int remain = segment.size() - offset;
+            if (remain > 0) {
+                int nCopy = Math.min(remain, remainSize);
+                segment.get(offset, bytes, numBytes - remainSize + bytesOffset, nCopy);
+                remainSize -= nCopy;
+                // next new segment.
+                offset = 0;
+                if (remainSize == 0) {
+                    return;
+                }
+            } else {
+                // remain is negative, let's advance to next segment
+                // now the offset = offset - segmentSize (-remain)
+                offset = -remain;
+            }
+        }
+    }
+}
diff --git a/flink-table-store-common/src/main/java/org/apache/flink/table/store/utils/StringUtils.java b/flink-table-store-common/src/main/java/org/apache/flink/table/store/utils/StringUtils.java
new file mode 100644
index 00000000..e0bcf3a9
--- /dev/null
+++ b/flink-table-store-common/src/main/java/org/apache/flink/table/store/utils/StringUtils.java
@@ -0,0 +1,63 @@
+/*
+ * 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.table.store.utils;
+
+import org.apache.flink.table.data.binary.BinaryStringData;
+
+import java.util.Arrays;
+
+import static org.apache.flink.table.data.binary.BinaryStringData.fromBytes;
+
+/** Utils for {@link BinaryStringData}. */
+public class StringUtils {
+
+    /**
+     * Concatenates input strings together into a single string. Returns NULL if any argument is
+     * NULL.
+     */
+    public static BinaryStringData concat(BinaryStringData... inputs) {
+        return concat(Arrays.asList(inputs));
+    }
+
+    public static BinaryStringData concat(Iterable<BinaryStringData> inputs) {
+        // Compute the total length of the result.
+        int totalLength = 0;
+        for (BinaryStringData input : inputs) {
+            if (input == null) {
+                return null;
+            }
+
+            input.ensureMaterialized();
+            totalLength += input.getSizeInBytes();
+        }
+
+        // Allocate a new byte array, and copy the inputs one by one into it.
+        final byte[] result = new byte[totalLength];
+        int offset = 0;
+        for (BinaryStringData input : inputs) {
+            if (input != null) {
+                int len = input.getSizeInBytes();
+                SegmentsUtil.copyToBytes(
+                        input.getSegments(), input.getOffset(), result, offset, len);
+                offset += len;
+            }
+        }
+        return fromBytes(result);
+    }
+}
diff --git a/flink-table-store-connector/src/main/java/org/apache/flink/table/store/connector/FlinkCatalog.java b/flink-table-store-connector/src/main/java/org/apache/flink/table/store/connector/FlinkCatalog.java
index 03091912..e84a20c8 100644
--- a/flink-table-store-connector/src/main/java/org/apache/flink/table/store/connector/FlinkCatalog.java
+++ b/flink-table-store-connector/src/main/java/org/apache/flink/table/store/connector/FlinkCatalog.java
@@ -19,6 +19,7 @@
 package org.apache.flink.table.store.connector;
 
 import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.catalog.AbstractCatalog;
 import org.apache.flink.table.catalog.CatalogBaseTable;
 import org.apache.flink.table.catalog.CatalogDatabase;
@@ -27,6 +28,7 @@ import org.apache.flink.table.catalog.CatalogFunction;
 import org.apache.flink.table.catalog.CatalogPartition;
 import org.apache.flink.table.catalog.CatalogPartitionSpec;
 import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.CatalogTableImpl;
 import org.apache.flink.table.catalog.ObjectPath;
 import org.apache.flink.table.catalog.exceptions.CatalogException;
 import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException;
@@ -38,6 +40,8 @@ import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
 import org.apache.flink.table.catalog.exceptions.TableNotExistException;
 import org.apache.flink.table.catalog.stats.CatalogColumnStatistics;
 import org.apache.flink.table.catalog.stats.CatalogTableStatistics;
+import org.apache.flink.table.descriptors.DescriptorProperties;
+import org.apache.flink.table.descriptors.Schema;
 import org.apache.flink.table.expressions.Expression;
 import org.apache.flink.table.factories.Factory;
 import org.apache.flink.table.store.file.catalog.Catalog;
@@ -45,16 +49,20 @@ import org.apache.flink.table.store.file.schema.SchemaChange;
 import org.apache.flink.table.store.file.schema.UpdateSchema;
 import org.apache.flink.table.store.table.FileStoreTable;
 import org.apache.flink.table.store.table.Table;
+import org.apache.flink.table.types.logical.RowType;
 
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Optional;
 
+import static org.apache.flink.table.descriptors.Schema.SCHEMA;
 import static org.apache.flink.table.factories.FactoryUtil.CONNECTOR;
 import static org.apache.flink.table.store.CoreOptions.PATH;
+import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType;
 
 /** Catalog for table store. */
 public class FlinkCatalog extends AbstractCatalog {
@@ -156,7 +164,7 @@ public class FlinkCatalog extends AbstractCatalog {
 
         if (table instanceof FileStoreTable) {
             CatalogTable catalogTable =
-                    ((FileStoreTable) table).schema().toUpdateSchema().toCatalogTable();
+                    toCatalogTable(((FileStoreTable) table).schema().toUpdateSchema());
             // add path to source and sink
             catalogTable
                     .getOptions()
@@ -302,6 +310,38 @@ public class FlinkCatalog extends AbstractCatalog {
         }
     }
 
+    private CatalogTableImpl toCatalogTable(UpdateSchema updateSchema) {
+        TableSchema schema;
+        Map<String, String> newOptions = new HashMap<>(updateSchema.options());
+
+        // try to read schema from options
+        // in the case of virtual columns and watermark
+        DescriptorProperties tableSchemaProps = new DescriptorProperties(true);
+        tableSchemaProps.putProperties(newOptions);
+        Optional<TableSchema> optional = tableSchemaProps.getOptionalTableSchema(Schema.SCHEMA);
+        if (optional.isPresent()) {
+            schema = optional.get();
+
+            // remove schema from options
+            DescriptorProperties removeProperties = new DescriptorProperties(false);
+            removeProperties.putTableSchema(SCHEMA, schema);
+            removeProperties.asMap().keySet().forEach(newOptions::remove);
+        } else {
+            TableSchema.Builder builder = TableSchema.builder();
+            for (RowType.RowField field : updateSchema.rowType().getFields()) {
+                builder.field(field.getName(), fromLogicalToDataType(field.getType()));
+            }
+            if (updateSchema.primaryKeys().size() > 0) {
+                builder.primaryKey(updateSchema.primaryKeys().toArray(new String[0]));
+            }
+
+            schema = builder.build();
+        }
+
+        return new CatalogTableImpl(
+                schema, updateSchema.partitionKeys(), newOptions, updateSchema.comment());
+    }
+
     // --------------------- unsupported methods ----------------------------
 
     @Override
diff --git a/flink-table-store-connector/src/main/java/org/apache/flink/table/store/connector/sink/StoreSinkWriteImpl.java b/flink-table-store-connector/src/main/java/org/apache/flink/table/store/connector/sink/StoreSinkWriteImpl.java
index 75186dac..ab457866 100644
--- a/flink-table-store-connector/src/main/java/org/apache/flink/table/store/connector/sink/StoreSinkWriteImpl.java
+++ b/flink-table-store-connector/src/main/java/org/apache/flink/table/store/connector/sink/StoreSinkWriteImpl.java
@@ -23,6 +23,7 @@ import org.apache.flink.runtime.state.StateInitializationContext;
 import org.apache.flink.runtime.state.StateSnapshotContext;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.data.binary.BinaryRowData;
+import org.apache.flink.table.store.file.disk.IOManagerImpl;
 import org.apache.flink.table.store.file.io.DataFileMeta;
 import org.apache.flink.table.store.table.FileStoreTable;
 import org.apache.flink.table.store.table.sink.FileCommittable;
@@ -70,7 +71,11 @@ public class StoreSinkWriteImpl implements StoreSinkWrite {
         if (commitUser == null) {
             write = null;
         } else {
-            write = table.newWrite(commitUser).withIOManager(ioManager).withOverwrite(isOverwrite);
+            write =
+                    table.newWrite(commitUser)
+                            .withIOManager(
+                                    new IOManagerImpl(ioManager.getSpillingDirectoriesPaths()))
+                            .withOverwrite(isOverwrite);
         }
     }
 
diff --git a/flink-table-store-core/pom.xml b/flink-table-store-core/pom.xml
index 657d84ce..3e62ca6c 100644
--- a/flink-table-store-core/pom.xml
+++ b/flink-table-store-core/pom.xml
@@ -55,23 +55,21 @@ under the License.
 
         <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-connector-files</artifactId>
+            <artifactId>flink-table-common</artifactId>
             <version>${flink.version}</version>
             <scope>provided</scope>
         </dependency>
 
         <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-table-common</artifactId>
-            <version>${flink.version}</version>
-            <scope>provided</scope>
+            <artifactId>flink-shaded-jackson</artifactId>
+            <version>${flink.shaded.jackson.version}-${flink.shaded.version}</version>
         </dependency>
 
         <dependency>
-            <groupId>org.apache.flink</groupId>
-            <artifactId>${flink.table.runtime}</artifactId>
-            <version>${flink.version}</version>
-            <scope>provided</scope>
+            <groupId>org.lz4</groupId>
+            <artifactId>lz4-java</artifactId>
+            <version>1.8.0</version>
         </dependency>
 
         <!-- test dependencies -->
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/KeyValue.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/KeyValue.java
index a1dc6c02..7344ec60 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/KeyValue.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/KeyValue.java
@@ -20,7 +20,7 @@ package org.apache.flink.table.store.file;
 
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.runtime.typeutils.RowDataSerializer;
+import org.apache.flink.table.store.data.RowDataSerializer;
 import org.apache.flink.table.store.file.schema.AtomicDataType;
 import org.apache.flink.table.store.file.schema.DataField;
 import org.apache.flink.table.store.utils.RowDataUtils;
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/casting/CastExecutors.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/casting/CastExecutors.java
index 28e41b1b..01e52398 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/casting/CastExecutors.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/casting/CastExecutors.java
@@ -19,12 +19,12 @@
 package org.apache.flink.table.store.file.casting;
 
 import org.apache.flink.table.data.DecimalData;
-import org.apache.flink.table.data.DecimalDataUtils;
 import org.apache.flink.table.data.StringData;
 import org.apache.flink.table.data.TimestampData;
 import org.apache.flink.table.data.binary.BinaryStringData;
-import org.apache.flink.table.data.binary.BinaryStringDataUtil;
 import org.apache.flink.table.store.utils.DateTimeUtils;
+import org.apache.flink.table.store.utils.DecimalUtils;
+import org.apache.flink.table.store.utils.StringUtils;
 import org.apache.flink.table.types.logical.BinaryType;
 import org.apache.flink.table.types.logical.CharType;
 import org.apache.flink.table.types.logical.DecimalType;
@@ -88,14 +88,14 @@ public class CastExecutors {
                                     case INTEGER:
                                     case BIGINT:
                                         {
-                                            return DecimalDataUtils.castFrom(
+                                            return DecimalUtils.castFrom(
                                                     number.longValue(),
                                                     decimalType.getPrecision(),
                                                     decimalType.getScale());
                                         }
                                     default:
                                         {
-                                            return DecimalDataUtils.castFrom(
+                                            return DecimalUtils.castFrom(
                                                     number.doubleValue(),
                                                     decimalType.getPrecision(),
                                                     decimalType.getScale());
@@ -110,25 +110,22 @@ public class CastExecutors {
                 {
                     switch (outputType.getTypeRoot()) {
                         case TINYINT:
-                            return value ->
-                                    (byte) DecimalDataUtils.castToIntegral((DecimalData) value);
+                            return value -> (byte) DecimalUtils.castToIntegral((DecimalData) value);
                         case SMALLINT:
                             return value ->
-                                    (short) DecimalDataUtils.castToIntegral((DecimalData) value);
+                                    (short) DecimalUtils.castToIntegral((DecimalData) value);
                         case INTEGER:
-                            return value ->
-                                    (int) DecimalDataUtils.castToIntegral((DecimalData) value);
+                            return value -> (int) DecimalUtils.castToIntegral((DecimalData) value);
                         case BIGINT:
-                            return value -> DecimalDataUtils.castToIntegral((DecimalData) value);
+                            return value -> DecimalUtils.castToIntegral((DecimalData) value);
                         case FLOAT:
-                            return value ->
-                                    (float) DecimalDataUtils.doubleValue((DecimalData) value);
+                            return value -> (float) DecimalUtils.doubleValue((DecimalData) value);
                         case DOUBLE:
-                            return value -> DecimalDataUtils.doubleValue((DecimalData) value);
+                            return value -> DecimalUtils.doubleValue((DecimalData) value);
                         case DECIMAL:
                             DecimalType decimalType = (DecimalType) outputType;
                             return value ->
-                                    DecimalDataUtils.castToDecimal(
+                                    DecimalUtils.castToDecimal(
                                             (DecimalData) value,
                                             decimalType.getPrecision(),
                                             decimalType.getScale());
@@ -153,7 +150,7 @@ public class CastExecutors {
                                     int padLength = targetLength - strData.numChars();
                                     BinaryStringData padString =
                                             BinaryStringData.blankString(padLength);
-                                    result = BinaryStringDataUtil.concat(strData, padString);
+                                    result = StringUtils.concat(strData, padString);
                                 } else {
                                     result = strData;
                                 }
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/compression/BlockCompressionFactory.java
similarity index 63%
copy from flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java
copy to flink-table-store-core/src/main/java/org/apache/flink/table/store/file/compression/BlockCompressionFactory.java
index 4a352309..03363711 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/compression/BlockCompressionFactory.java
@@ -16,19 +16,15 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.store.file.memory;
+package org.apache.flink.table.store.file.compression;
 
-import org.apache.flink.table.runtime.util.MemorySegmentPool;
-
-/** A class provides memory related methods. */
-public interface MemoryOwner {
-
-    /** Set {@link MemorySegmentPool} for the owner. */
-    void setMemoryPool(MemorySegmentPool memoryPool);
+/**
+ * Each compression codec has an implementation of {@link BlockCompressionFactory} to create
+ * compressors and decompressors.
+ */
+public interface BlockCompressionFactory {
 
-    /** Memory occupancy size of this owner. */
-    long memoryOccupancy();
+    BlockCompressor getCompressor();
 
-    /** Flush memory of owner, release memory. */
-    void flushMemory() throws Exception;
+    BlockDecompressor getDecompressor();
 }
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/compression/BlockCompressor.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/compression/BlockCompressor.java
new file mode 100644
index 00000000..2cdba652
--- /dev/null
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/compression/BlockCompressor.java
@@ -0,0 +1,43 @@
+/*
+ * 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.table.store.file.compression;
+
+/**
+ * A compressor which compresses a whole byte array each time. It will read from and write to byte
+ * arrays given from the outside, reducing copy time.
+ */
+public interface BlockCompressor {
+
+    /** Get the max compressed size for a given original size. */
+    int getMaxCompressedSize(int srcSize);
+
+    /**
+     * Compress data read from src, and write the compressed data to dst.
+     *
+     * @param src Uncompressed data to read from
+     * @param srcOff The start offset of uncompressed data
+     * @param srcLen The length of data which want to be compressed
+     * @param dst The target to write compressed data
+     * @param dstOff The start offset to write the compressed data
+     * @return Length of compressed data
+     * @throws BufferCompressionException if exception thrown when compressing
+     */
+    int compress(byte[] src, int srcOff, int srcLen, byte[] dst, int dstOff)
+            throws BufferCompressionException;
+}
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/compression/BlockDecompressor.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/compression/BlockDecompressor.java
new file mode 100644
index 00000000..8c7c5daf
--- /dev/null
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/compression/BlockDecompressor.java
@@ -0,0 +1,37 @@
+/*
+ * 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.table.store.file.compression;
+
+/** A decompressor which decompresses a block each time. */
+public interface BlockDecompressor {
+
+    /**
+     * Decompress source data read from src and write the decompressed data to dst.
+     *
+     * @param src Compressed data to read from
+     * @param srcOff The start offset of compressed data
+     * @param srcLen The length of data which want to be decompressed
+     * @param dst The target to write decompressed data
+     * @param dstOff The start offset to write the decompressed data
+     * @return Length of decompressed data
+     * @throws BufferDecompressionException if exception thrown when decompressing
+     */
+    int decompress(byte[] src, int srcOff, int srcLen, byte[] dst, int dstOff)
+            throws BufferDecompressionException;
+}
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/compression/BufferCompressionException.java
similarity index 56%
copy from flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java
copy to flink-table-store-core/src/main/java/org/apache/flink/table/store/file/compression/BufferCompressionException.java
index 4a352309..708c32f9 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/compression/BufferCompressionException.java
@@ -16,19 +16,27 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.store.file.memory;
+package org.apache.flink.table.store.file.compression;
 
-import org.apache.flink.table.runtime.util.MemorySegmentPool;
+/**
+ * A {@code BufferCompressionException} is thrown when the target data cannot be compressed, such as
+ * insufficient target buffer space for compression, etc.
+ */
+public class BufferCompressionException extends RuntimeException {
 
-/** A class provides memory related methods. */
-public interface MemoryOwner {
+    public BufferCompressionException() {
+        super();
+    }
 
-    /** Set {@link MemorySegmentPool} for the owner. */
-    void setMemoryPool(MemorySegmentPool memoryPool);
+    public BufferCompressionException(String message) {
+        super(message);
+    }
 
-    /** Memory occupancy size of this owner. */
-    long memoryOccupancy();
+    public BufferCompressionException(String message, Throwable e) {
+        super(message, e);
+    }
 
-    /** Flush memory of owner, release memory. */
-    void flushMemory() throws Exception;
+    public BufferCompressionException(Throwable e) {
+        super(e);
+    }
 }
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/compression/BufferDecompressionException.java
similarity index 55%
copy from flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java
copy to flink-table-store-core/src/main/java/org/apache/flink/table/store/file/compression/BufferDecompressionException.java
index 4a352309..5fbaabf8 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/compression/BufferDecompressionException.java
@@ -16,19 +16,27 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.store.file.memory;
+package org.apache.flink.table.store.file.compression;
 
-import org.apache.flink.table.runtime.util.MemorySegmentPool;
+/**
+ * A {@code BufferDecompressionException} is thrown when the target data cannot be decompressed,
+ * such as data corruption, insufficient target buffer space for decompression, etc.
+ */
+public class BufferDecompressionException extends RuntimeException {
 
-/** A class provides memory related methods. */
-public interface MemoryOwner {
+    public BufferDecompressionException() {
+        super();
+    }
 
-    /** Set {@link MemorySegmentPool} for the owner. */
-    void setMemoryPool(MemorySegmentPool memoryPool);
+    public BufferDecompressionException(String message) {
+        super(message);
+    }
 
-    /** Memory occupancy size of this owner. */
-    long memoryOccupancy();
+    public BufferDecompressionException(String message, Throwable e) {
+        super(message, e);
+    }
 
-    /** Flush memory of owner, release memory. */
-    void flushMemory() throws Exception;
+    public BufferDecompressionException(Throwable e) {
+        super(e);
+    }
 }
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/compression/CompressorUtils.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/compression/CompressorUtils.java
new file mode 100644
index 00000000..104635c5
--- /dev/null
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/compression/CompressorUtils.java
@@ -0,0 +1,52 @@
+/*
+ * 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.table.store.file.compression;
+
+/** Utils for {@link BlockCompressor}. */
+public class CompressorUtils {
+    /**
+     * We put two integers before each compressed block, the first integer represents the compressed
+     * length of the block, and the second one represents the original length of the block.
+     */
+    public static final int HEADER_LENGTH = 8;
+
+    public static void writeIntLE(int i, byte[] buf, int offset) {
+        buf[offset++] = (byte) i;
+        buf[offset++] = (byte) (i >>> 8);
+        buf[offset++] = (byte) (i >>> 16);
+        buf[offset] = (byte) (i >>> 24);
+    }
+
+    public static int readIntLE(byte[] buf, int i) {
+        return (buf[i] & 0xFF)
+                | ((buf[i + 1] & 0xFF) << 8)
+                | ((buf[i + 2] & 0xFF) << 16)
+                | ((buf[i + 3] & 0xFF) << 24);
+    }
+
+    public static void validateLength(int compressedLen, int originalLen)
+            throws BufferDecompressionException {
+        if (originalLen < 0
+                || compressedLen < 0
+                || (originalLen == 0 && compressedLen != 0)
+                || (originalLen != 0 && compressedLen == 0)) {
+            throw new BufferDecompressionException("Input is corrupted, invalid length.");
+        }
+    }
+}
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/compression/Lz4BlockCompressionFactory.java
similarity index 63%
copy from flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java
copy to flink-table-store-core/src/main/java/org/apache/flink/table/store/file/compression/Lz4BlockCompressionFactory.java
index 4a352309..63e90219 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/compression/Lz4BlockCompressionFactory.java
@@ -16,19 +16,17 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.store.file.memory;
+package org.apache.flink.table.store.file.compression;
 
-import org.apache.flink.table.runtime.util.MemorySegmentPool;
+/** Implementation of {@link BlockCompressionFactory} for Lz4 codec. */
+public class Lz4BlockCompressionFactory implements BlockCompressionFactory {
+    @Override
+    public BlockCompressor getCompressor() {
+        return new Lz4BlockCompressor();
+    }
 
-/** A class provides memory related methods. */
-public interface MemoryOwner {
-
-    /** Set {@link MemorySegmentPool} for the owner. */
-    void setMemoryPool(MemorySegmentPool memoryPool);
-
-    /** Memory occupancy size of this owner. */
-    long memoryOccupancy();
-
-    /** Flush memory of owner, release memory. */
-    void flushMemory() throws Exception;
+    @Override
+    public BlockDecompressor getDecompressor() {
+        return new Lz4BlockDecompressor();
+    }
 }
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/compression/Lz4BlockCompressor.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/compression/Lz4BlockCompressor.java
new file mode 100644
index 00000000..2a282fc8
--- /dev/null
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/compression/Lz4BlockCompressor.java
@@ -0,0 +1,59 @@
+/*
+ * 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.table.store.file.compression;
+
+import net.jpountz.lz4.LZ4Compressor;
+import net.jpountz.lz4.LZ4Factory;
+
+import static org.apache.flink.table.store.file.compression.CompressorUtils.HEADER_LENGTH;
+import static org.apache.flink.table.store.file.compression.CompressorUtils.writeIntLE;
+
+/**
+ * Encode data into LZ4 format (not compatible with the LZ4 Frame format). It reads from and writes
+ * to byte arrays provided from the outside, thus reducing copy time.
+ *
+ * <p>This class is copied and modified from {@link net.jpountz.lz4.LZ4BlockOutputStream}.
+ */
+public class Lz4BlockCompressor implements BlockCompressor {
+
+    private final LZ4Compressor compressor;
+
+    public Lz4BlockCompressor() {
+        this.compressor = LZ4Factory.fastestInstance().fastCompressor();
+    }
+
+    @Override
+    public int getMaxCompressedSize(int srcSize) {
+        return HEADER_LENGTH + compressor.maxCompressedLength(srcSize);
+    }
+
+    @Override
+    public int compress(byte[] src, int srcOff, int srcLen, byte[] dst, int dstOff)
+            throws BufferCompressionException {
+        try {
+            int compressedLength =
+                    compressor.compress(src, srcOff, srcLen, dst, dstOff + HEADER_LENGTH);
+            writeIntLE(compressedLength, dst, dstOff);
+            writeIntLE(srcLen, dst, dstOff + 4);
+            return HEADER_LENGTH + compressedLength;
+        } catch (Exception e) {
+            throw new BufferCompressionException(e);
+        }
+    }
+}
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/compression/Lz4BlockDecompressor.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/compression/Lz4BlockDecompressor.java
new file mode 100644
index 00000000..acf55b2c
--- /dev/null
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/compression/Lz4BlockDecompressor.java
@@ -0,0 +1,71 @@
+/*
+ * 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.table.store.file.compression;
+
+import net.jpountz.lz4.LZ4Exception;
+import net.jpountz.lz4.LZ4Factory;
+import net.jpountz.lz4.LZ4FastDecompressor;
+
+import static org.apache.flink.table.store.file.compression.CompressorUtils.HEADER_LENGTH;
+import static org.apache.flink.table.store.file.compression.CompressorUtils.readIntLE;
+import static org.apache.flink.table.store.file.compression.CompressorUtils.validateLength;
+
+/**
+ * Decode data written with {@link Lz4BlockCompressor}. It reads from and writes to byte arrays
+ * provided from the outside, thus reducing copy time.
+ *
+ * <p>This class is copied and modified from {@link net.jpountz.lz4.LZ4BlockInputStream}.
+ */
+public class Lz4BlockDecompressor implements BlockDecompressor {
+
+    private final LZ4FastDecompressor decompressor;
+
+    public Lz4BlockDecompressor() {
+        this.decompressor = LZ4Factory.fastestInstance().fastDecompressor();
+    }
+
+    @Override
+    public int decompress(byte[] src, int srcOff, int srcLen, byte[] dst, int dstOff)
+            throws BufferDecompressionException {
+        final int compressedLen = readIntLE(src, srcOff);
+        final int originalLen = readIntLE(src, srcOff + 4);
+        validateLength(compressedLen, originalLen);
+
+        if (dst.length - dstOff < originalLen) {
+            throw new BufferDecompressionException("Buffer length too small");
+        }
+
+        if (src.length - srcOff - HEADER_LENGTH < compressedLen) {
+            throw new BufferDecompressionException(
+                    "Source data is not integral for decompression.");
+        }
+
+        try {
+            final int compressedLen2 =
+                    decompressor.decompress(src, srcOff + HEADER_LENGTH, dst, dstOff, originalLen);
+            if (compressedLen != compressedLen2) {
+                throw new BufferDecompressionException("Input is corrupted");
+            }
+        } catch (LZ4Exception e) {
+            throw new BufferDecompressionException("Input is corrupted", e);
+        }
+
+        return originalLen;
+    }
+}
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/AbstractFileIOChannel.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/AbstractFileIOChannel.java
new file mode 100644
index 00000000..dd2f8bae
--- /dev/null
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/AbstractFileIOChannel.java
@@ -0,0 +1,120 @@
+/*
+ * 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.table.store.file.disk;
+
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.channels.FileChannel;
+
+/** Abstract {@link FileIOChannel} to share some implementation. */
+public abstract class AbstractFileIOChannel implements FileIOChannel {
+
+    /** Logger object for channel and its subclasses. */
+    protected static final Logger LOG = LoggerFactory.getLogger(FileIOChannel.class);
+
+    /** The ID of the underlying channel. */
+    protected final FileIOChannel.ID id;
+
+    /** A file channel for NIO access to the file. */
+    protected final FileChannel fileChannel;
+
+    /**
+     * Creates a new channel to the path indicated by the given ID. The channel hands IO requests to
+     * the given request queue to be processed.
+     *
+     * @param channelID The id describing the path of the file that the channel accessed.
+     * @param writeEnabled Flag describing whether the channel should be opened in read/write mode,
+     *     rather than in read-only mode.
+     * @throws IOException Thrown, if the channel could no be opened.
+     */
+    protected AbstractFileIOChannel(FileIOChannel.ID channelID, boolean writeEnabled)
+            throws IOException {
+        this.id = Preconditions.checkNotNull(channelID);
+
+        try {
+            @SuppressWarnings("resource")
+            RandomAccessFile file = new RandomAccessFile(id.getPath(), writeEnabled ? "rw" : "r");
+            this.fileChannel = file.getChannel();
+        } catch (IOException e) {
+            throw new IOException(
+                    "Channel to path '" + channelID.getPath() + "' could not be opened.", e);
+        }
+    }
+
+    // --------------------------------------------------------------------------------------------
+
+    @Override
+    public final FileIOChannel.ID getChannelID() {
+        return this.id;
+    }
+
+    @Override
+    public long getSize() throws IOException {
+        FileChannel channel = fileChannel;
+        return channel == null ? 0 : channel.size();
+    }
+
+    @Override
+    public boolean isClosed() {
+        return !this.fileChannel.isOpen();
+    }
+
+    @Override
+    public void close() throws IOException {
+        if (this.fileChannel.isOpen()) {
+            this.fileChannel.close();
+        }
+    }
+
+    @Override
+    public void deleteChannel() {
+        if (!isClosed() || this.fileChannel.isOpen()) {
+            throw new IllegalStateException("Cannot delete a channel that is open.");
+        }
+
+        // make a best effort to delete the file. Don't report exceptions.
+        try {
+            File f = new File(this.id.getPath());
+            if (f.exists()) {
+                f.delete();
+            }
+        } catch (Throwable ignored) {
+        }
+    }
+
+    @Override
+    public void closeAndDelete() throws IOException {
+        try {
+            close();
+        } finally {
+            deleteChannel();
+        }
+    }
+
+    @Override
+    public FileChannel getNioFileChannel() {
+        return fileChannel;
+    }
+}
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/BufferFileChannelReader.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/BufferFileChannelReader.java
new file mode 100644
index 00000000..164be04a
--- /dev/null
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/BufferFileChannelReader.java
@@ -0,0 +1,67 @@
+/*
+ * 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.table.store.file.disk;
+
+import org.apache.flink.table.store.file.memory.Buffer;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** Helper class to read {@link Buffer}s from files into objects. */
+public class BufferFileChannelReader {
+    private final ByteBuffer header = ByteBuffer.allocateDirect(4);
+    private final FileChannel fileChannel;
+
+    BufferFileChannelReader(FileChannel fileChannel) {
+        this.fileChannel = fileChannel;
+    }
+
+    /**
+     * Reads data from the object's file channel into the given buffer.
+     *
+     * @param buffer the buffer to read into
+     * @return whether the end of the file has been reached (<tt>true</tt>) or not (<tt>false</tt>)
+     */
+    public boolean readBufferFromFileChannel(Buffer buffer) throws IOException {
+        checkArgument(fileChannel.size() - fileChannel.position() > 0);
+
+        // Read header
+        header.clear();
+        fileChannel.read(header);
+        header.flip();
+
+        int size = header.getInt();
+        if (size > buffer.getMaxCapacity()) {
+            throw new IllegalStateException(
+                    "Buffer is too small for data: "
+                            + buffer.getMaxCapacity()
+                            + " bytes available, but "
+                            + size
+                            + " needed. This is most likely due to an serialized event, which is larger than the buffer size.");
+        }
+        checkArgument(buffer.getSize() == 0, "Buffer not empty");
+
+        fileChannel.read(buffer.getNioBuffer(0, size));
+        buffer.setSize(size);
+        return fileChannel.size() - fileChannel.position() == 0;
+    }
+}
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/BufferFileReader.java
similarity index 63%
copy from flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java
copy to flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/BufferFileReader.java
index 4a352309..7c5756ad 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/BufferFileReader.java
@@ -16,19 +16,16 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.store.file.memory;
+package org.apache.flink.table.store.file.disk;
 
-import org.apache.flink.table.runtime.util.MemorySegmentPool;
+import org.apache.flink.table.store.file.memory.Buffer;
 
-/** A class provides memory related methods. */
-public interface MemoryOwner {
+import java.io.IOException;
 
-    /** Set {@link MemorySegmentPool} for the owner. */
-    void setMemoryPool(MemorySegmentPool memoryPool);
+/** Read {@link Buffer} from file. */
+public interface BufferFileReader extends FileIOChannel {
 
-    /** Memory occupancy size of this owner. */
-    long memoryOccupancy();
+    void readInto(Buffer buffer) throws IOException;
 
-    /** Flush memory of owner, release memory. */
-    void flushMemory() throws Exception;
+    boolean hasReachedEndOfFile();
 }
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/BufferFileReaderImpl.java
similarity index 50%
copy from flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java
copy to flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/BufferFileReaderImpl.java
index 4a352309..4b450fe4 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/BufferFileReaderImpl.java
@@ -16,19 +16,31 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.store.file.memory;
+package org.apache.flink.table.store.file.disk;
 
-import org.apache.flink.table.runtime.util.MemorySegmentPool;
+import org.apache.flink.table.store.file.memory.Buffer;
 
-/** A class provides memory related methods. */
-public interface MemoryOwner {
+import java.io.IOException;
 
-    /** Set {@link MemorySegmentPool} for the owner. */
-    void setMemoryPool(MemorySegmentPool memoryPool);
+/** A synchronous {@link BufferFileReader} implementation. */
+public class BufferFileReaderImpl extends AbstractFileIOChannel implements BufferFileReader {
 
-    /** Memory occupancy size of this owner. */
-    long memoryOccupancy();
+    private final BufferFileChannelReader reader;
 
-    /** Flush memory of owner, release memory. */
-    void flushMemory() throws Exception;
+    private boolean hasReachedEndOfFile;
+
+    public BufferFileReaderImpl(ID channelID) throws IOException {
+        super(channelID, false);
+        this.reader = new BufferFileChannelReader(fileChannel);
+    }
+
+    @Override
+    public void readInto(Buffer buffer) throws IOException {
+        hasReachedEndOfFile = reader.readBufferFromFileChannel(buffer);
+    }
+
+    @Override
+    public boolean hasReachedEndOfFile() {
+        return hasReachedEndOfFile;
+    }
 }
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/BufferFileWriter.java
similarity index 58%
copy from flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java
copy to flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/BufferFileWriter.java
index 4a352309..55cda1a1 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/BufferFileWriter.java
@@ -16,19 +16,21 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.store.file.memory;
+package org.apache.flink.table.store.file.disk;
 
-import org.apache.flink.table.runtime.util.MemorySegmentPool;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.table.store.file.memory.Buffer;
 
-/** A class provides memory related methods. */
-public interface MemoryOwner {
+import java.io.IOException;
 
-    /** Set {@link MemorySegmentPool} for the owner. */
-    void setMemoryPool(MemorySegmentPool memoryPool);
+/** Write {@link MemorySegment} to file. */
+public interface BufferFileWriter extends FileIOChannel {
 
-    /** Memory occupancy size of this owner. */
-    long memoryOccupancy();
-
-    /** Flush memory of owner, release memory. */
-    void flushMemory() throws Exception;
+    /**
+     * Writes the given block. The request may be executed synchronously, or asynchronously,
+     * depending on the implementation.
+     *
+     * @throws IOException Thrown, when the writer encounters an I/O error.
+     */
+    void writeBlock(Buffer buffer) throws IOException;
 }
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/BufferFileWriterImpl.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/BufferFileWriterImpl.java
new file mode 100644
index 00000000..be863db0
--- /dev/null
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/BufferFileWriterImpl.java
@@ -0,0 +1,44 @@
+/*
+ * 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.table.store.file.disk;
+
+import org.apache.flink.table.store.file.memory.Buffer;
+import org.apache.flink.util.FileUtils;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/** A synchronous {@link BufferFileWriter} implementation. */
+public class BufferFileWriterImpl extends AbstractFileIOChannel implements BufferFileWriter {
+
+    protected BufferFileWriterImpl(ID channelID) throws IOException {
+        super(channelID, true);
+    }
+
+    @Override
+    public void writeBlock(Buffer buffer) throws IOException {
+        ByteBuffer nioBufferReadable = buffer.getMemorySegment().wrap(0, buffer.getSize()).slice();
+        ByteBuffer header = ByteBuffer.allocateDirect(4);
+        header.putInt(nioBufferReadable.remaining());
+        header.flip();
+
+        FileUtils.writeCompletely(fileChannel, header);
+        FileUtils.writeCompletely(fileChannel, nioBufferReadable);
+    }
+}
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/ChannelReaderInputView.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/ChannelReaderInputView.java
new file mode 100644
index 00000000..3097b44a
--- /dev/null
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/ChannelReaderInputView.java
@@ -0,0 +1,103 @@
+/*
+ * 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.table.store.file.disk;
+
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
+import org.apache.flink.table.store.data.AbstractPagedInputView;
+import org.apache.flink.table.store.file.compression.BlockCompressionFactory;
+import org.apache.flink.table.store.file.compression.BlockDecompressor;
+import org.apache.flink.table.store.file.memory.Buffer;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * A {@link org.apache.flink.core.memory.DataInputView} that is backed by a {@link
+ * BufferFileReader}, making it effectively a data input stream. The view reads it data in blocks
+ * from the underlying channel and decompress it before returning to caller. The view can only read
+ * data that has been written by {@link ChannelWriterOutputView}, due to block formatting.
+ */
+public class ChannelReaderInputView extends AbstractPagedInputView {
+
+    private final BlockDecompressor decompressor;
+    private final BufferFileReader reader;
+    private final MemorySegment uncompressedBuffer;
+
+    private final MemorySegment compressedBuffer;
+
+    private int numBlocksRemaining;
+    private int currentSegmentLimit;
+
+    public ChannelReaderInputView(
+            FileIOChannel.ID id,
+            IOManager ioManager,
+            BlockCompressionFactory compressionCodecFactory,
+            int compressionBlockSize,
+            int numBlocks)
+            throws IOException {
+        this.numBlocksRemaining = numBlocks;
+        this.reader = ioManager.createBufferFileReader(id);
+        uncompressedBuffer = MemorySegmentFactory.wrap(new byte[compressionBlockSize]);
+        decompressor = compressionCodecFactory.getDecompressor();
+        compressedBuffer =
+                MemorySegmentFactory.wrap(
+                        new byte
+                                [compressionCodecFactory
+                                        .getCompressor()
+                                        .getMaxCompressedSize(compressionBlockSize)]);
+    }
+
+    @Override
+    protected MemorySegment nextSegment(MemorySegment current) throws IOException {
+        // check for end-of-stream
+        if (this.numBlocksRemaining <= 0) {
+            this.reader.close();
+            throw new EOFException();
+        }
+
+        Buffer buffer = Buffer.create(compressedBuffer);
+        reader.readInto(buffer);
+        this.currentSegmentLimit =
+                decompressor.decompress(
+                        buffer.getMemorySegment().getArray(),
+                        0,
+                        buffer.getSize(),
+                        uncompressedBuffer.getArray(),
+                        0);
+        this.numBlocksRemaining--;
+        return uncompressedBuffer;
+    }
+
+    @Override
+    protected int getLimitForSegment(MemorySegment segment) {
+        return currentSegmentLimit;
+    }
+
+    public List<MemorySegment> close() throws IOException {
+        reader.close();
+        return Collections.emptyList();
+    }
+
+    public FileIOChannel getChannel() {
+        return reader;
+    }
+}
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/ChannelReaderInputViewIterator.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/ChannelReaderInputViewIterator.java
new file mode 100644
index 00000000..c98fa3f1
--- /dev/null
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/ChannelReaderInputViewIterator.java
@@ -0,0 +1,71 @@
+/*
+ * 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.table.store.file.disk;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.util.MutableObjectIterator;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.util.List;
+
+/** A simple iterator over the input read though an I/O channel. */
+public class ChannelReaderInputViewIterator<E> implements MutableObjectIterator<E> {
+    private final ChannelReaderInputView inView;
+
+    private final TypeSerializer<E> accessors;
+
+    private final List<MemorySegment> freeMemTarget;
+
+    public ChannelReaderInputViewIterator(
+            ChannelReaderInputView inView,
+            List<MemorySegment> freeMemTarget,
+            TypeSerializer<E> accessors) {
+        this.inView = inView;
+        this.freeMemTarget = freeMemTarget;
+        this.accessors = accessors;
+    }
+
+    @Override
+    public E next(E reuse) throws IOException {
+        try {
+            return this.accessors.deserialize(reuse, this.inView);
+        } catch (EOFException eofex) {
+            final List<MemorySegment> freeMem = this.inView.close();
+            if (this.freeMemTarget != null) {
+                this.freeMemTarget.addAll(freeMem);
+            }
+            return null;
+        }
+    }
+
+    @Override
+    public E next() throws IOException {
+        try {
+            return this.accessors.deserialize(this.inView);
+        } catch (EOFException eofex) {
+            final List<MemorySegment> freeMem = this.inView.close();
+            if (this.freeMemTarget != null) {
+                this.freeMemTarget.addAll(freeMem);
+            }
+            return null;
+        }
+    }
+}
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/ChannelWithMeta.java
similarity index 53%
copy from flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java
copy to flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/ChannelWithMeta.java
index 4a352309..23dfb72c 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/ChannelWithMeta.java
@@ -16,19 +16,30 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.store.file.memory;
+package org.apache.flink.table.store.file.disk;
 
-import org.apache.flink.table.runtime.util.MemorySegmentPool;
+/** Channel with block count and numBytesInLastBlock of file. */
+public class ChannelWithMeta {
 
-/** A class provides memory related methods. */
-public interface MemoryOwner {
+    private final FileIOChannel.ID channel;
+    private final int blockCount;
+    private final int numBytesInLastBlock;
 
-    /** Set {@link MemorySegmentPool} for the owner. */
-    void setMemoryPool(MemorySegmentPool memoryPool);
+    public ChannelWithMeta(FileIOChannel.ID channel, int blockCount, int numBytesInLastBlock) {
+        this.channel = channel;
+        this.blockCount = blockCount;
+        this.numBytesInLastBlock = numBytesInLastBlock;
+    }
 
-    /** Memory occupancy size of this owner. */
-    long memoryOccupancy();
+    public FileIOChannel.ID getChannel() {
+        return channel;
+    }
 
-    /** Flush memory of owner, release memory. */
-    void flushMemory() throws Exception;
+    public int getBlockCount() {
+        return blockCount;
+    }
+
+    public int getNumBytesInLastBlock() {
+        return numBytesInLastBlock;
+    }
 }
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/ChannelWriterOutputView.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/ChannelWriterOutputView.java
new file mode 100644
index 00000000..9680bce5
--- /dev/null
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/ChannelWriterOutputView.java
@@ -0,0 +1,100 @@
+/*
+ * 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.table.store.file.disk;
+
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
+import org.apache.flink.table.store.data.AbstractPagedOutputView;
+import org.apache.flink.table.store.file.compression.BlockCompressionFactory;
+import org.apache.flink.table.store.file.compression.BlockCompressor;
+import org.apache.flink.table.store.file.memory.Buffer;
+
+import java.io.IOException;
+
+/**
+ * A {@link org.apache.flink.core.memory.DataOutputView} that is backed by a {@link FileIOChannel},
+ * making it effectively a data output stream. The view will compress its data before writing it in
+ * blocks to the underlying channel.
+ */
+public final class ChannelWriterOutputView extends AbstractPagedOutputView {
+
+    private final MemorySegment compressedBuffer;
+    private final BlockCompressor compressor;
+    private final BufferFileWriter writer;
+
+    private int blockCount;
+
+    private long numBytes;
+    private long numCompressedBytes;
+
+    public ChannelWriterOutputView(
+            BufferFileWriter writer,
+            BlockCompressionFactory compressionCodecFactory,
+            int compressionBlockSize) {
+        super(MemorySegmentFactory.wrap(new byte[compressionBlockSize]), compressionBlockSize);
+
+        compressor = compressionCodecFactory.getCompressor();
+        compressedBuffer =
+                MemorySegmentFactory.wrap(
+                        new byte[compressor.getMaxCompressedSize(compressionBlockSize)]);
+        this.writer = writer;
+    }
+
+    public FileIOChannel getChannel() {
+        return writer;
+    }
+
+    public int close() throws IOException {
+        if (!writer.isClosed()) {
+            int currentPositionInSegment = getCurrentPositionInSegment();
+            writeCompressed(currentSegment, currentPositionInSegment);
+            clear();
+            this.writer.close();
+        }
+        return -1;
+    }
+
+    @Override
+    protected MemorySegment nextSegment(MemorySegment current, int positionInCurrent)
+            throws IOException {
+        writeCompressed(current, positionInCurrent);
+        return current;
+    }
+
+    private void writeCompressed(MemorySegment current, int size) throws IOException {
+        int compressedLen =
+                compressor.compress(current.getArray(), 0, size, compressedBuffer.getArray(), 0);
+        writer.writeBlock(Buffer.create(compressedBuffer, compressedLen));
+        blockCount++;
+        numBytes += size;
+        numCompressedBytes += compressedLen;
+    }
+
+    public long getNumBytes() {
+        return numBytes;
+    }
+
+    public long getNumCompressedBytes() {
+        return numCompressedBytes;
+    }
+
+    public int getBlockCount() {
+        return blockCount;
+    }
+}
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/FileChannelManager.java
similarity index 58%
copy from flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java
copy to flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/FileChannelManager.java
index 4a352309..d146837c 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/FileChannelManager.java
@@ -16,19 +16,19 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.store.file.memory;
+package org.apache.flink.table.store.file.disk;
 
-import org.apache.flink.table.runtime.util.MemorySegmentPool;
+import java.io.File;
 
-/** A class provides memory related methods. */
-public interface MemoryOwner {
+/** The manager used for creating/getting file IO channels based on config temp dirs. */
+public interface FileChannelManager extends AutoCloseable {
 
-    /** Set {@link MemorySegmentPool} for the owner. */
-    void setMemoryPool(MemorySegmentPool memoryPool);
+    /** Creates an ID identifying an underlying file channel and returns it. */
+    FileIOChannel.ID createChannel();
 
-    /** Memory occupancy size of this owner. */
-    long memoryOccupancy();
+    /** Creates an enumerator for channels that logically belong together and returns it. */
+    FileIOChannel.Enumerator createChannelEnumerator();
 
-    /** Flush memory of owner, release memory. */
-    void flushMemory() throws Exception;
+    /** Gets all the files corresponding to the config temp dirs. */
+    File[] getPaths();
 }
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/FileChannelManagerImpl.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/FileChannelManagerImpl.java
new file mode 100644
index 00000000..d3a83865
--- /dev/null
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/FileChannelManagerImpl.java
@@ -0,0 +1,128 @@
+/*
+ * 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.table.store.file.disk;
+
+import org.apache.flink.table.store.file.disk.FileIOChannel.Enumerator;
+import org.apache.flink.table.store.file.disk.FileIOChannel.ID;
+import org.apache.flink.util.FileUtils;
+import org.apache.flink.util.IOUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Arrays;
+import java.util.Random;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** The manager used for creating/deleting file channels based on config temp dirs. */
+public class FileChannelManagerImpl implements FileChannelManager {
+    private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImpl.class);
+
+    /** The temporary directories for files. */
+    private final File[] paths;
+
+    /** A random number generator for the anonymous Channel IDs. */
+    private final Random random;
+
+    /** The number of the next path to use. */
+    private final AtomicLong nextPath = new AtomicLong(0);
+
+    public FileChannelManagerImpl(String[] tempDirs, String prefix) {
+        checkNotNull(tempDirs, "The temporary directories must not be null.");
+        checkArgument(tempDirs.length > 0, "The temporary directories must not be empty.");
+
+        this.random = new Random();
+
+        // Creates directories after registering shutdown hook to ensure the directories can be
+        // removed if required.
+        this.paths = createFiles(tempDirs, prefix);
+    }
+
+    private static File[] createFiles(String[] tempDirs, String prefix) {
+        File[] files = new File[tempDirs.length];
+        for (int i = 0; i < tempDirs.length; i++) {
+            File baseDir = new File(tempDirs[i]);
+            String subfolder = String.format("flink-%s-%s", prefix, UUID.randomUUID());
+            File storageDir = new File(baseDir, subfolder);
+
+            if (!storageDir.exists() && !storageDir.mkdirs()) {
+                throw new RuntimeException(
+                        "Could not create storage directory for FileChannelManager: "
+                                + storageDir.getAbsolutePath());
+            }
+            files[i] = storageDir;
+
+            LOG.debug(
+                    "FileChannelManager uses directory {} for spill files.",
+                    storageDir.getAbsolutePath());
+        }
+        return files;
+    }
+
+    @Override
+    public ID createChannel() {
+        int num = (int) (nextPath.getAndIncrement() % paths.length);
+        return new ID(paths[num], num, random);
+    }
+
+    @Override
+    public Enumerator createChannelEnumerator() {
+        return new Enumerator(paths, random);
+    }
+
+    @Override
+    public File[] getPaths() {
+        return Arrays.copyOf(paths, paths.length);
+    }
+
+    /** Remove all the temp directories. */
+    @Override
+    public void close() throws Exception {
+        IOUtils.closeAll(
+                Arrays.stream(paths)
+                        .filter(File::exists)
+                        .map(this::getFileCloser)
+                        .collect(Collectors.toList()));
+    }
+
+    private AutoCloseable getFileCloser(File path) {
+        return () -> {
+            try {
+                FileUtils.deleteDirectory(path);
+                LOG.info(
+                        "FileChannelManager removed spill file directory {}",
+                        path.getAbsolutePath());
+            } catch (IOException e) {
+                String errorMessage =
+                        String.format(
+                                "FileChannelManager failed to properly clean up temp file directory: %s",
+                                path);
+                throw new UncheckedIOException(errorMessage, e);
+            }
+        };
+    }
+}
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/FileChannelUtil.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/FileChannelUtil.java
new file mode 100644
index 00000000..898b5351
--- /dev/null
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/FileChannelUtil.java
@@ -0,0 +1,57 @@
+/*
+ * 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.table.store.file.disk;
+
+import org.apache.flink.table.store.file.compression.BlockCompressionFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+/** File channel util for runtime. */
+public class FileChannelUtil {
+
+    public static ChannelReaderInputView createInputView(
+            IOManager ioManager,
+            ChannelWithMeta channel,
+            List<FileIOChannel> channels,
+            BlockCompressionFactory compressionCodecFactory,
+            int compressionBlockSize)
+            throws IOException {
+        ChannelReaderInputView in =
+                new ChannelReaderInputView(
+                        channel.getChannel(),
+                        ioManager,
+                        compressionCodecFactory,
+                        compressionBlockSize,
+                        channel.getBlockCount());
+        channels.add(in.getChannel());
+        return in;
+    }
+
+    public static ChannelWriterOutputView createOutputView(
+            IOManager ioManager,
+            FileIOChannel.ID channel,
+            BlockCompressionFactory compressionCodecFactory,
+            int compressionBlockSize)
+            throws IOException {
+        BufferFileWriter bufferWriter = ioManager.createBufferFileWriter(channel);
+        return new ChannelWriterOutputView(
+                bufferWriter, compressionCodecFactory, compressionBlockSize);
+    }
+}
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/FileIOChannel.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/FileIOChannel.java
new file mode 100644
index 00000000..8760144c
--- /dev/null
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/FileIOChannel.java
@@ -0,0 +1,161 @@
+/*
+ * 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.table.store.file.disk;
+
+import org.apache.flink.util.StringUtils;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * A Channel represents a collection of files that belong logically to the same resource. An example
+ * is a collection of files that contain sorted runs of data from the same stream, that will later
+ * on be merged together.
+ */
+public interface FileIOChannel {
+
+    /**
+     * Gets the channel ID of this I/O channel.
+     *
+     * @return The channel ID.
+     */
+    ID getChannelID();
+
+    /** Gets the size (in bytes) of the file underlying the channel. */
+    long getSize() throws IOException;
+
+    /**
+     * Checks whether the channel has been closed.
+     *
+     * @return True if the channel has been closed, false otherwise.
+     */
+    boolean isClosed();
+
+    /**
+     * Closes the channel. For asynchronous implementations, this method waits until all pending
+     * requests are handled. Even if an exception interrupts the closing, the underlying
+     * <tt>FileChannel</tt> is closed.
+     *
+     * @throws IOException Thrown, if an error occurred while waiting for pending requests.
+     */
+    void close() throws IOException;
+
+    /**
+     * Deletes the file underlying this I/O channel.
+     *
+     * @throws IllegalStateException Thrown, when the channel is still open.
+     */
+    void deleteChannel();
+
+    FileChannel getNioFileChannel();
+
+    /**
+     * Closes the channel and deletes the underlying file. For asynchronous implementations, this
+     * method waits until all pending requests are handled.
+     *
+     * @throws IOException Thrown, if an error occurred while waiting for pending requests.
+     */
+    void closeAndDelete() throws IOException;
+
+    // --------------------------------------------------------------------------------------------
+    // --------------------------------------------------------------------------------------------
+
+    /** An ID identifying an underlying file channel. */
+    class ID {
+
+        private static final int RANDOM_BYTES_LENGTH = 16;
+
+        private final File path;
+
+        private final int bucketNum;
+
+        private ID(File path, int bucketNum) {
+            this.path = path;
+            this.bucketNum = bucketNum;
+        }
+
+        public ID(File basePath, int bucketNum, Random random) {
+            this.path = new File(basePath, randomString(random) + ".channel");
+            this.bucketNum = bucketNum;
+        }
+
+        /** Returns the path to the underlying temporary file. */
+        public String getPath() {
+            return path.getAbsolutePath();
+        }
+
+        /** Returns the path to the underlying temporary file as a File. */
+        public File getPathFile() {
+            return path;
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (obj instanceof ID) {
+                ID other = (ID) obj;
+                return this.path.equals(other.path) && this.bucketNum == other.bucketNum;
+            } else {
+                return false;
+            }
+        }
+
+        @Override
+        public int hashCode() {
+            return path.hashCode();
+        }
+
+        @Override
+        public String toString() {
+            return path.getAbsolutePath();
+        }
+
+        private static String randomString(Random random) {
+            byte[] bytes = new byte[RANDOM_BYTES_LENGTH];
+            random.nextBytes(bytes);
+            return StringUtils.byteToHexString(bytes);
+        }
+    }
+
+    /** An enumerator for channels that logically belong together. */
+    final class Enumerator {
+
+        private static final AtomicInteger GLOBAL_NUMBER = new AtomicInteger();
+
+        private final File[] paths;
+
+        private final String namePrefix;
+
+        private int localCounter;
+
+        public Enumerator(File[] basePaths, Random random) {
+            this.paths = basePaths;
+            this.namePrefix = FileIOChannel.ID.randomString(random);
+            this.localCounter = 0;
+        }
+
+        public FileIOChannel.ID next() {
+            int bucketNum = GLOBAL_NUMBER.getAndIncrement() % paths.length;
+            String filename = String.format("%s.%06d.channel", namePrefix, (localCounter++));
+            return new FileIOChannel.ID(new File(paths[bucketNum], filename), bucketNum);
+        }
+    }
+}
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/IOManager.java
similarity index 52%
copy from flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java
copy to flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/IOManager.java
index 4a352309..af64663d 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/IOManager.java
@@ -16,19 +16,29 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.store.file.memory;
+package org.apache.flink.table.store.file.disk;
 
-import org.apache.flink.table.runtime.util.MemorySegmentPool;
+import org.apache.flink.table.store.file.disk.FileIOChannel.Enumerator;
+import org.apache.flink.table.store.file.disk.FileIOChannel.ID;
 
-/** A class provides memory related methods. */
-public interface MemoryOwner {
+import java.io.IOException;
 
-    /** Set {@link MemorySegmentPool} for the owner. */
-    void setMemoryPool(MemorySegmentPool memoryPool);
+/** The facade for the provided disk I/O services. */
+public interface IOManager extends AutoCloseable {
 
-    /** Memory occupancy size of this owner. */
-    long memoryOccupancy();
+    ID createChannel();
 
-    /** Flush memory of owner, release memory. */
-    void flushMemory() throws Exception;
+    Enumerator createChannelEnumerator();
+
+    BufferFileWriter createBufferFileWriter(ID channelID) throws IOException;
+
+    BufferFileReader createBufferFileReader(ID channelID) throws IOException;
+
+    static IOManager create(String tempDir) {
+        return create(new String[] {tempDir});
+    }
+
+    static IOManager create(String[] tempDirs) {
+        return new IOManagerImpl(tempDirs);
+    }
 }
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/IOManagerImpl.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/IOManagerImpl.java
new file mode 100644
index 00000000..80a92c1c
--- /dev/null
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/disk/IOManagerImpl.java
@@ -0,0 +1,125 @@
+/*
+ * 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.table.store.file.disk;
+
+import org.apache.flink.table.store.file.disk.FileIOChannel.Enumerator;
+import org.apache.flink.table.store.file.disk.FileIOChannel.ID;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.stream.Collectors;
+
+/** The facade for the provided I/O manager services. */
+public class IOManagerImpl implements IOManager {
+    protected static final Logger LOG = LoggerFactory.getLogger(IOManager.class);
+
+    private static final String DIR_NAME_PREFIX = "io";
+
+    private final FileChannelManager fileChannelManager;
+
+    // -------------------------------------------------------------------------
+    //               Constructors / Destructors
+    // -------------------------------------------------------------------------
+
+    /**
+     * Constructs a new IOManager.
+     *
+     * @param tempDirs The basic directories for files underlying anonymous channels.
+     */
+    public IOManagerImpl(String[] tempDirs) {
+        this.fileChannelManager =
+                new FileChannelManagerImpl(Preconditions.checkNotNull(tempDirs), DIR_NAME_PREFIX);
+        if (LOG.isInfoEnabled()) {
+            LOG.info(
+                    "Created a new {} for spilling of task related data to disk (joins, sorting, ...). Used directories:\n\t{}",
+                    FileChannelManager.class.getSimpleName(),
+                    Arrays.stream(fileChannelManager.getPaths())
+                            .map(File::getAbsolutePath)
+                            .collect(Collectors.joining("\n\t")));
+        }
+    }
+
+    /** Removes all temporary files. */
+    @Override
+    public void close() throws Exception {
+        fileChannelManager.close();
+    }
+
+    @Override
+    public ID createChannel() {
+        return fileChannelManager.createChannel();
+    }
+
+    @Override
+    public Enumerator createChannelEnumerator() {
+        return fileChannelManager.createChannelEnumerator();
+    }
+
+    /**
+     * Deletes the file underlying the given channel. If the channel is still open, this call may
+     * fail.
+     *
+     * @param channel The channel to be deleted.
+     */
+    public static void deleteChannel(ID channel) {
+        if (channel != null) {
+            if (channel.getPathFile().exists() && !channel.getPathFile().delete()) {
+                LOG.warn("IOManager failed to delete temporary file {}", channel.getPath());
+            }
+        }
+    }
+
+    /**
+     * Gets the directories that the I/O manager spills to.
+     *
+     * @return The directories that the I/O manager spills to.
+     */
+    public File[] getSpillingDirectories() {
+        return fileChannelManager.getPaths();
+    }
+
+    /**
+     * Gets the directories that the I/O manager spills to, as path strings.
+     *
+     * @return The directories that the I/O manager spills to, as path strings.
+     */
+    public String[] getSpillingDirectoriesPaths() {
+        File[] paths = fileChannelManager.getPaths();
+        String[] strings = new String[paths.length];
+        for (int i = 0; i < strings.length; i++) {
+            strings[i] = paths[i].getAbsolutePath();
+        }
+        return strings;
+    }
+
+    @Override
+    public BufferFileWriter createBufferFileWriter(FileIOChannel.ID channelID) throws IOException {
+        return new BufferFileWriterImpl(channelID);
+    }
+
+    @Override
+    public BufferFileReader createBufferFileReader(FileIOChannel.ID channelID) throws IOException {
+        return new BufferFileReaderImpl(channelID);
+    }
+}
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/io/KeyValueDataFileRecordReader.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/io/KeyValueDataFileRecordReader.java
index 0402b955..970d3bb5 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/io/KeyValueDataFileRecordReader.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/io/KeyValueDataFileRecordReader.java
@@ -18,15 +18,13 @@
 
 package org.apache.flink.table.store.file.io;
 
-import org.apache.flink.connector.file.src.FileSourceSplit;
-import org.apache.flink.connector.file.src.reader.BulkFormat;
-import org.apache.flink.connector.file.src.util.RecordAndPosition;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.store.file.KeyValue;
 import org.apache.flink.table.store.file.KeyValueSerializer;
 import org.apache.flink.table.store.file.utils.FileUtils;
 import org.apache.flink.table.store.file.utils.RecordReader;
+import org.apache.flink.table.store.format.FormatReaderFactory;
 import org.apache.flink.table.types.logical.RowType;
 
 import javax.annotation.Nullable;
@@ -36,13 +34,13 @@ import java.io.IOException;
 /** {@link RecordReader} for reading {@link KeyValue} data files. */
 public class KeyValueDataFileRecordReader implements RecordReader<KeyValue> {
 
-    private final BulkFormat.Reader<RowData> reader;
+    private final RecordReader<RowData> reader;
     private final KeyValueSerializer serializer;
     private final int level;
     @Nullable private final int[] indexMapping;
 
     public KeyValueDataFileRecordReader(
-            BulkFormat<RowData, FileSourceSplit> readerFactory,
+            FormatReaderFactory readerFactory,
             Path path,
             RowType keyType,
             RowType valueType,
@@ -58,7 +56,7 @@ public class KeyValueDataFileRecordReader implements RecordReader<KeyValue> {
     @Nullable
     @Override
     public RecordIterator<KeyValue> readBatch() throws IOException {
-        BulkFormat.RecordIterator<RowData> iterator = reader.readBatch();
+        RecordReader.RecordIterator<RowData> iterator = reader.readBatch();
         return iterator == null ? null : new KeyValueDataFileRecordIterator(iterator, indexMapping);
     }
 
@@ -69,22 +67,22 @@ public class KeyValueDataFileRecordReader implements RecordReader<KeyValue> {
 
     private class KeyValueDataFileRecordIterator extends AbstractFileRecordIterator<KeyValue> {
 
-        private final BulkFormat.RecordIterator<RowData> iterator;
+        private final RecordReader.RecordIterator<RowData> iterator;
 
         private KeyValueDataFileRecordIterator(
-                BulkFormat.RecordIterator<RowData> iterator, @Nullable int[] indexMapping) {
+                RecordReader.RecordIterator<RowData> iterator, @Nullable int[] indexMapping) {
             super(indexMapping);
             this.iterator = iterator;
         }
 
         @Override
         public KeyValue next() throws IOException {
-            RecordAndPosition<RowData> result = iterator.next();
+            RowData result = iterator.next();
 
             if (result == null) {
                 return null;
             } else {
-                return serializer.fromRow(mappingRowData(result.getRecord())).setLevel(level);
+                return serializer.fromRow(mappingRowData(result)).setLevel(level);
             }
         }
 
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/io/KeyValueDataFileWriter.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/io/KeyValueDataFileWriter.java
index 8bada63d..4a8741eb 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/io/KeyValueDataFileWriter.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/io/KeyValueDataFileWriter.java
@@ -22,7 +22,7 @@ import org.apache.flink.api.common.serialization.BulkWriter;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.data.binary.BinaryRowData;
-import org.apache.flink.table.runtime.typeutils.RowDataSerializer;
+import org.apache.flink.table.store.data.RowDataSerializer;
 import org.apache.flink.table.store.file.KeyValue;
 import org.apache.flink.table.store.file.stats.BinaryTableStats;
 import org.apache.flink.table.store.file.stats.FieldStatsArraySerializer;
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/io/RowDataFileRecordReader.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/io/RowDataFileRecordReader.java
index 8eff6918..6ba97e27 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/io/RowDataFileRecordReader.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/io/RowDataFileRecordReader.java
@@ -18,13 +18,11 @@
 
 package org.apache.flink.table.store.file.io;
 
-import org.apache.flink.connector.file.src.FileSourceSplit;
-import org.apache.flink.connector.file.src.reader.BulkFormat;
-import org.apache.flink.connector.file.src.util.RecordAndPosition;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.store.file.utils.FileUtils;
 import org.apache.flink.table.store.file.utils.RecordReader;
+import org.apache.flink.table.store.format.FormatReaderFactory;
 
 import javax.annotation.Nullable;
 
@@ -33,13 +31,11 @@ import java.io.IOException;
 /** Reads {@link RowData} from data files. */
 public class RowDataFileRecordReader implements RecordReader<RowData> {
 
-    private final BulkFormat.Reader<RowData> reader;
+    private final RecordReader<RowData> reader;
     @Nullable private final int[] indexMapping;
 
     public RowDataFileRecordReader(
-            Path path,
-            BulkFormat<RowData, FileSourceSplit> readerFactory,
-            @Nullable int[] indexMapping)
+            Path path, FormatReaderFactory readerFactory, @Nullable int[] indexMapping)
             throws IOException {
         this.reader = FileUtils.createFormatReader(readerFactory, path);
         this.indexMapping = indexMapping;
@@ -48,7 +44,7 @@ public class RowDataFileRecordReader implements RecordReader<RowData> {
     @Nullable
     @Override
     public RecordReader.RecordIterator<RowData> readBatch() throws IOException {
-        BulkFormat.RecordIterator<RowData> iterator = reader.readBatch();
+        RecordIterator<RowData> iterator = reader.readBatch();
         return iterator == null ? null : new RowDataFileRecordIterator(iterator, indexMapping);
     }
 
@@ -59,19 +55,19 @@ public class RowDataFileRecordReader implements RecordReader<RowData> {
 
     private static class RowDataFileRecordIterator extends AbstractFileRecordIterator<RowData> {
 
-        private final BulkFormat.RecordIterator<RowData> iterator;
+        private final RecordIterator<RowData> iterator;
 
         private RowDataFileRecordIterator(
-                BulkFormat.RecordIterator<RowData> iterator, @Nullable int[] indexMapping) {
+                RecordIterator<RowData> iterator, @Nullable int[] indexMapping) {
             super(indexMapping);
             this.iterator = iterator;
         }
 
         @Override
         public RowData next() throws IOException {
-            RecordAndPosition<RowData> result = iterator.next();
+            RowData result = iterator.next();
 
-            return result == null ? null : mappingRowData(result.getRecord());
+            return result == null ? null : mappingRowData(result);
         }
 
         @Override
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/manifest/ManifestFile.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/manifest/ManifestFile.java
index 77b26de2..fc6f0515 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/manifest/ManifestFile.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/manifest/ManifestFile.java
@@ -20,8 +20,6 @@ package org.apache.flink.table.store.file.manifest;
 
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.serialization.BulkWriter;
-import org.apache.flink.connector.file.src.FileSourceSplit;
-import org.apache.flink.connector.file.src.reader.BulkFormat;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.store.file.io.RollingFileWriter;
@@ -33,6 +31,7 @@ import org.apache.flink.table.store.file.utils.FileUtils;
 import org.apache.flink.table.store.file.utils.VersionedObjectSerializer;
 import org.apache.flink.table.store.format.FieldStatsCollector;
 import org.apache.flink.table.store.format.FileFormat;
+import org.apache.flink.table.store.format.FormatReaderFactory;
 import org.apache.flink.table.types.logical.RowType;
 
 import java.io.IOException;
@@ -48,7 +47,7 @@ public class ManifestFile {
     private final long schemaId;
     private final RowType partitionType;
     private final ManifestEntrySerializer serializer;
-    private final BulkFormat<RowData, FileSourceSplit> readerFactory;
+    private final FormatReaderFactory readerFactory;
     private final BulkWriter.Factory<RowData> writerFactory;
     private final FileStorePathFactory pathFactory;
     private final long suggestedFileSize;
@@ -58,7 +57,7 @@ public class ManifestFile {
             long schemaId,
             RowType partitionType,
             ManifestEntrySerializer serializer,
-            BulkFormat<RowData, FileSourceSplit> readerFactory,
+            FormatReaderFactory readerFactory,
             BulkWriter.Factory<RowData> writerFactory,
             FileStorePathFactory pathFactory,
             long suggestedFileSize) {
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/manifest/ManifestList.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/manifest/ManifestList.java
index 41a25b98..e9db1576 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/manifest/ManifestList.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/manifest/ManifestList.java
@@ -19,8 +19,6 @@
 package org.apache.flink.table.store.file.manifest;
 
 import org.apache.flink.api.common.serialization.BulkWriter;
-import org.apache.flink.connector.file.src.FileSourceSplit;
-import org.apache.flink.connector.file.src.reader.BulkFormat;
 import org.apache.flink.core.fs.FSDataOutputStream;
 import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.core.fs.Path;
@@ -29,6 +27,7 @@ import org.apache.flink.table.store.file.utils.FileStorePathFactory;
 import org.apache.flink.table.store.file.utils.FileUtils;
 import org.apache.flink.table.store.file.utils.VersionedObjectSerializer;
 import org.apache.flink.table.store.format.FileFormat;
+import org.apache.flink.table.store.format.FormatReaderFactory;
 import org.apache.flink.table.types.logical.RowType;
 
 import java.io.IOException;
@@ -41,13 +40,13 @@ import java.util.List;
 public class ManifestList {
 
     private final ManifestFileMetaSerializer serializer;
-    private final BulkFormat<RowData, FileSourceSplit> readerFactory;
+    private final FormatReaderFactory readerFactory;
     private final BulkWriter.Factory<RowData> writerFactory;
     private final FileStorePathFactory pathFactory;
 
     private ManifestList(
             ManifestFileMetaSerializer serializer,
-            BulkFormat<RowData, FileSourceSplit> readerFactory,
+            FormatReaderFactory readerFactory,
             BulkWriter.Factory<RowData> writerFactory,
             FileStorePathFactory pathFactory) {
         this.serializer = serializer;
@@ -103,8 +102,8 @@ public class ManifestList {
     }
 
     /**
-     * Creator of {@link ManifestList}. It reueses {@link BulkFormat} and {@link BulkWriter.Factory}
-     * from {@link FileFormat}.
+     * Creator of {@link ManifestList}. It reueses {@link FormatReaderFactory} and {@link
+     * BulkWriter.Factory} from {@link FileFormat}.
      */
     public static class Factory {
 
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/Buffer.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/Buffer.java
new file mode 100644
index 00000000..53a0ac3e
--- /dev/null
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/Buffer.java
@@ -0,0 +1,64 @@
+/*
+ * 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.table.store.file.memory;
+
+import org.apache.flink.core.memory.MemorySegment;
+
+import java.nio.ByteBuffer;
+
+/** A buffer with size. */
+public class Buffer {
+
+    private final MemorySegment segment;
+
+    private int size;
+
+    public Buffer(MemorySegment segment, int size) {
+        this.segment = segment;
+        this.size = size;
+    }
+
+    public static Buffer create(MemorySegment segment) {
+        return create(segment, 0);
+    }
+
+    public static Buffer create(MemorySegment segment, int size) {
+        return new Buffer(segment, size);
+    }
+
+    public MemorySegment getMemorySegment() {
+        return segment;
+    }
+
+    public int getSize() {
+        return size;
+    }
+
+    public int getMaxCapacity() {
+        return segment.size();
+    }
+
+    public ByteBuffer getNioBuffer(int index, int length) {
+        return segment.wrap(index, length).slice();
+    }
+
+    public void setSize(int size) {
+        this.size = size;
+    }
+}
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/HeapMemorySegmentPool.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/HeapMemorySegmentPool.java
index 0dc48e7e..e5a07beb 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/HeapMemorySegmentPool.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/HeapMemorySegmentPool.java
@@ -20,7 +20,6 @@ package org.apache.flink.table.store.file.memory;
 
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.core.memory.MemorySegmentFactory;
-import org.apache.flink.table.runtime.util.MemorySegmentPool;
 
 import java.util.LinkedList;
 import java.util.List;
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java
index 4a352309..24d991a6 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java
@@ -18,8 +18,6 @@
 
 package org.apache.flink.table.store.file.memory;
 
-import org.apache.flink.table.runtime.util.MemorySegmentPool;
-
 /** A class provides memory related methods. */
 public interface MemoryOwner {
 
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryPoolFactory.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryPoolFactory.java
index bad0aa54..1a379e67 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryPoolFactory.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryPoolFactory.java
@@ -19,7 +19,6 @@
 package org.apache.flink.table.store.file.memory;
 
 import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.table.runtime.util.MemorySegmentPool;
 
 import java.util.List;
 
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemorySegmentPool.java
similarity index 57%
copy from flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java
copy to flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemorySegmentPool.java
index 4a352309..47f128d4 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemorySegmentPool.java
@@ -18,17 +18,30 @@
 
 package org.apache.flink.table.store.file.memory;
 
-import org.apache.flink.table.runtime.util.MemorySegmentPool;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentSource;
 
-/** A class provides memory related methods. */
-public interface MemoryOwner {
+import java.util.List;
 
-    /** Set {@link MemorySegmentPool} for the owner. */
-    void setMemoryPool(MemorySegmentPool memoryPool);
+/** MemorySegment pool to hold pages in memory. */
+public interface MemorySegmentPool extends MemorySegmentSource {
 
-    /** Memory occupancy size of this owner. */
-    long memoryOccupancy();
+    int DEFAULT_PAGE_SIZE = 32 * 1024;
 
-    /** Flush memory of owner, release memory. */
-    void flushMemory() throws Exception;
+    /**
+     * Get the page size of each page this pool holds.
+     *
+     * @return the page size
+     */
+    int pageSize();
+
+    /**
+     * Return all pages back into this pool.
+     *
+     * @param memory the pages which want to be returned.
+     */
+    void returnAll(List<MemorySegment> memory);
+
+    /** @return Free page number. */
+    int freePages();
 }
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/mergetree/MergeTreeWriter.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/mergetree/MergeTreeWriter.java
index 8432d109..2acf5d18 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/mergetree/MergeTreeWriter.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/mergetree/MergeTreeWriter.java
@@ -19,19 +19,19 @@
 package org.apache.flink.table.store.file.mergetree;
 
 import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.runtime.util.MemorySegmentPool;
 import org.apache.flink.table.store.CoreOptions.ChangelogProducer;
 import org.apache.flink.table.store.file.KeyValue;
 import org.apache.flink.table.store.file.compact.CompactManager;
 import org.apache.flink.table.store.file.compact.CompactResult;
+import org.apache.flink.table.store.file.disk.IOManager;
 import org.apache.flink.table.store.file.io.CompactIncrement;
 import org.apache.flink.table.store.file.io.DataFileMeta;
 import org.apache.flink.table.store.file.io.KeyValueFileWriterFactory;
 import org.apache.flink.table.store.file.io.NewFilesIncrement;
 import org.apache.flink.table.store.file.io.RollingFileWriter;
 import org.apache.flink.table.store.file.memory.MemoryOwner;
+import org.apache.flink.table.store.file.memory.MemorySegmentPool;
 import org.apache.flink.table.store.file.mergetree.compact.MergeFunction;
 import org.apache.flink.table.store.file.utils.RecordWriter;
 import org.apache.flink.table.types.logical.RowType;
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/mergetree/SortBufferWriteBuffer.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/mergetree/SortBufferWriteBuffer.java
index 8156f06d..c25d2bc1 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/mergetree/SortBufferWriteBuffer.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/mergetree/SortBufferWriteBuffer.java
@@ -19,18 +19,18 @@
 package org.apache.flink.table.store.file.mergetree;
 
 import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.data.binary.BinaryRowData;
-import org.apache.flink.table.runtime.typeutils.BinaryRowDataSerializer;
-import org.apache.flink.table.runtime.typeutils.InternalSerializers;
-import org.apache.flink.table.runtime.typeutils.RowDataSerializer;
-import org.apache.flink.table.runtime.util.MemorySegmentPool;
 import org.apache.flink.table.store.codegen.CodeGenUtils;
 import org.apache.flink.table.store.codegen.NormalizedKeyComputer;
 import org.apache.flink.table.store.codegen.RecordComparator;
+import org.apache.flink.table.store.data.BinaryRowDataSerializer;
+import org.apache.flink.table.store.data.InternalSerializers;
+import org.apache.flink.table.store.data.RowDataSerializer;
 import org.apache.flink.table.store.file.KeyValue;
 import org.apache.flink.table.store.file.KeyValueSerializer;
+import org.apache.flink.table.store.file.disk.IOManager;
+import org.apache.flink.table.store.file.memory.MemorySegmentPool;
 import org.apache.flink.table.store.file.mergetree.compact.MergeFunction;
 import org.apache.flink.table.store.file.mergetree.compact.ReducerMergeFunctionWrapper;
 import org.apache.flink.table.store.file.sort.BinaryExternalSortBuffer;
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/mergetree/compact/aggregate/FieldListaggAgg.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/mergetree/compact/aggregate/FieldListaggAgg.java
index 69b10b5f..b15f12ca 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/mergetree/compact/aggregate/FieldListaggAgg.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/mergetree/compact/aggregate/FieldListaggAgg.java
@@ -20,7 +20,7 @@ package org.apache.flink.table.store.file.mergetree.compact.aggregate;
 
 import org.apache.flink.table.data.StringData;
 import org.apache.flink.table.data.binary.BinaryStringData;
-import org.apache.flink.table.data.binary.BinaryStringDataUtil;
+import org.apache.flink.table.store.utils.StringUtils;
 import org.apache.flink.table.types.logical.LogicalType;
 
 /** listagg aggregate a field of a row. */
@@ -46,7 +46,7 @@ public class FieldListaggAgg extends FieldAggregator {
                     StringData mergeFieldSD = (StringData) accumulator;
                     StringData inFieldSD = (StringData) inputField;
                     concatenate =
-                            BinaryStringDataUtil.concat(
+                            StringUtils.concat(
                                     (BinaryStringData) mergeFieldSD,
                                     new BinaryStringData(DELIMITER),
                                     (BinaryStringData) inFieldSD);
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/mergetree/compact/aggregate/FieldSumAgg.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/mergetree/compact/aggregate/FieldSumAgg.java
index 234eb5fe..3560a5f0 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/mergetree/compact/aggregate/FieldSumAgg.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/mergetree/compact/aggregate/FieldSumAgg.java
@@ -19,7 +19,7 @@
 package org.apache.flink.table.store.file.mergetree.compact.aggregate;
 
 import org.apache.flink.table.data.DecimalData;
-import org.apache.flink.table.data.DecimalDataUtils;
+import org.apache.flink.table.store.utils.DecimalUtils;
 import org.apache.flink.table.types.logical.LogicalType;
 
 /** sum aggregate a field of a row. */
@@ -45,7 +45,7 @@ public class FieldSumAgg extends FieldAggregator {
                     assert mergeFieldDD.precision() == inFieldDD.precision()
                             : "Inconsistent precision of aggregate DecimalData!";
                     sum =
-                            DecimalDataUtils.add(
+                            DecimalUtils.add(
                                     mergeFieldDD,
                                     inFieldDD,
                                     mergeFieldDD.precision(),
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/operation/AbstractFileStoreWrite.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/operation/AbstractFileStoreWrite.java
index c498d56d..b1d5991d 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/operation/AbstractFileStoreWrite.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/operation/AbstractFileStoreWrite.java
@@ -19,9 +19,9 @@
 package org.apache.flink.table.store.file.operation;
 
 import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.table.data.binary.BinaryRowData;
 import org.apache.flink.table.store.file.Snapshot;
+import org.apache.flink.table.store.file.disk.IOManager;
 import org.apache.flink.table.store.file.io.DataFileMeta;
 import org.apache.flink.table.store.file.manifest.ManifestEntry;
 import org.apache.flink.table.store.file.utils.RecordWriter;
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/operation/FileStoreWrite.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/operation/FileStoreWrite.java
index 87105b2c..fc8cb843 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/operation/FileStoreWrite.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/operation/FileStoreWrite.java
@@ -18,9 +18,9 @@
 
 package org.apache.flink.table.store.file.operation;
 
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.table.data.binary.BinaryRowData;
 import org.apache.flink.table.store.file.FileStore;
+import org.apache.flink.table.store.file.disk.IOManager;
 import org.apache.flink.table.store.file.io.DataFileMeta;
 import org.apache.flink.table.store.file.utils.RecordWriter;
 import org.apache.flink.table.store.table.sink.FileCommittable;
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/predicate/BucketSelector.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/predicate/BucketSelector.java
index f242092d..6339f97d 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/predicate/BucketSelector.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/predicate/BucketSelector.java
@@ -21,7 +21,7 @@ package org.apache.flink.table.store.file.predicate;
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.table.data.GenericRowData;
 import org.apache.flink.table.data.binary.BinaryRowData;
-import org.apache.flink.table.runtime.typeutils.RowDataSerializer;
+import org.apache.flink.table.store.data.RowDataSerializer;
 import org.apache.flink.table.store.table.sink.BucketComputer;
 import org.apache.flink.table.types.logical.RowType;
 
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/schema/UpdateSchema.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/schema/UpdateSchema.java
index 87477589..3ae99ac7 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/schema/UpdateSchema.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/schema/UpdateSchema.java
@@ -20,9 +20,7 @@ package org.apache.flink.table.store.file.schema;
 
 import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.catalog.CatalogTable;
-import org.apache.flink.table.catalog.CatalogTableImpl;
 import org.apache.flink.table.descriptors.DescriptorProperties;
-import org.apache.flink.table.descriptors.Schema;
 import org.apache.flink.table.types.logical.RowType;
 import org.apache.flink.util.Preconditions;
 
@@ -31,12 +29,8 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Optional;
 import java.util.Set;
 
-import static org.apache.flink.table.descriptors.Schema.SCHEMA;
-import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType;
-
 /** A update schema. */
 public class UpdateSchema {
 
@@ -140,37 +134,6 @@ public class UpdateSchema {
                 + '}';
     }
 
-    public CatalogTableImpl toCatalogTable() {
-        TableSchema schema;
-        Map<String, String> newOptions = new HashMap<>(options);
-
-        // try to read schema from options
-        // in the case of virtual columns and watermark
-        DescriptorProperties tableSchemaProps = new DescriptorProperties(true);
-        tableSchemaProps.putProperties(newOptions);
-        Optional<TableSchema> optional = tableSchemaProps.getOptionalTableSchema(Schema.SCHEMA);
-        if (optional.isPresent()) {
-            schema = optional.get();
-
-            // remove schema from options
-            DescriptorProperties removeProperties = new DescriptorProperties(false);
-            removeProperties.putTableSchema(SCHEMA, schema);
-            removeProperties.asMap().keySet().forEach(newOptions::remove);
-        } else {
-            TableSchema.Builder builder = TableSchema.builder();
-            for (RowType.RowField field : rowType.getFields()) {
-                builder.field(field.getName(), fromLogicalToDataType(field.getType()));
-            }
-            if (primaryKeys.size() > 0) {
-                builder.primaryKey(primaryKeys.toArray(new String[0]));
-            }
-
-            schema = builder.build();
-        }
-
-        return new CatalogTableImpl(schema, partitionKeys, newOptions, comment);
-    }
-
     public static UpdateSchema fromCatalogTable(CatalogTable catalogTable) {
         TableSchema schema = catalogTable.getSchema();
         RowType rowType = (RowType) schema.toPhysicalRowDataType().getLogicalType();
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/AbstractBinaryExternalMerger.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/AbstractBinaryExternalMerger.java
new file mode 100644
index 00000000..e5e0424d
--- /dev/null
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/AbstractBinaryExternalMerger.java
@@ -0,0 +1,224 @@
+/*
+ * 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.table.store.file.sort;
+
+import org.apache.flink.table.store.data.AbstractPagedOutputView;
+import org.apache.flink.table.store.file.compression.BlockCompressionFactory;
+import org.apache.flink.table.store.file.disk.ChannelReaderInputView;
+import org.apache.flink.table.store.file.disk.ChannelWithMeta;
+import org.apache.flink.table.store.file.disk.ChannelWriterOutputView;
+import org.apache.flink.table.store.file.disk.FileChannelUtil;
+import org.apache.flink.table.store.file.disk.FileIOChannel;
+import org.apache.flink.table.store.file.disk.IOManager;
+import org.apache.flink.util.MutableObjectIterator;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+
+/**
+ * Spilled files Merger of {@link BinaryExternalSortBuffer}. It merges {@link #maxFanIn} spilled
+ * files at most once.
+ *
+ * @param <Entry> Type of Entry to Merge sort.
+ */
+public abstract class AbstractBinaryExternalMerger<Entry> implements Closeable {
+
+    private static final Logger LOG = LoggerFactory.getLogger(AbstractBinaryExternalMerger.class);
+
+    private volatile boolean closed;
+
+    private final int maxFanIn;
+    private final SpillChannelManager channelManager;
+    private final BlockCompressionFactory compressionCodecFactory;
+    private final int compressionBlockSize;
+
+    protected final int pageSize;
+    protected final IOManager ioManager;
+
+    public AbstractBinaryExternalMerger(
+            IOManager ioManager,
+            int pageSize,
+            int maxFanIn,
+            SpillChannelManager channelManager,
+            BlockCompressionFactory compressionCodecFactory,
+            int compressionBlockSize) {
+        this.ioManager = ioManager;
+        this.pageSize = pageSize;
+        this.maxFanIn = maxFanIn;
+        this.channelManager = channelManager;
+        this.compressionCodecFactory = compressionCodecFactory;
+        this.compressionBlockSize = compressionBlockSize;
+    }
+
+    @Override
+    public void close() {
+        this.closed = true;
+    }
+
+    /**
+     * Returns an iterator that iterates over the merged result from all given channels.
+     *
+     * @param channelIDs The channels that are to be merged and returned.
+     * @return An iterator over the merged records of the input channels.
+     * @throws IOException Thrown, if the readers encounter an I/O problem.
+     */
+    public BinaryMergeIterator<Entry> getMergingIterator(
+            List<ChannelWithMeta> channelIDs, List<FileIOChannel> openChannels) throws IOException {
+        // create one iterator per channel id
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Performing merge of " + channelIDs.size() + " sorted streams.");
+        }
+
+        final List<MutableObjectIterator<Entry>> iterators = new ArrayList<>(channelIDs.size() + 1);
+
+        for (ChannelWithMeta channel : channelIDs) {
+            ChannelReaderInputView view =
+                    FileChannelUtil.createInputView(
+                            ioManager,
+                            channel,
+                            openChannels,
+                            compressionCodecFactory,
+                            compressionBlockSize);
+            iterators.add(channelReaderInputViewIterator(view));
+        }
+
+        return new BinaryMergeIterator<>(
+                iterators, mergeReusedEntries(channelIDs.size()), mergeComparator());
+    }
+
+    /**
+     * Merges the given sorted runs to a smaller number of sorted runs.
+     *
+     * @param channelIDs The IDs of the sorted runs that need to be merged.
+     * @return A list of the IDs of the merged channels.
+     * @throws IOException Thrown, if the readers or writers encountered an I/O problem.
+     */
+    public List<ChannelWithMeta> mergeChannelList(List<ChannelWithMeta> channelIDs)
+            throws IOException {
+        // A channel list with length maxFanIn<sup>i</sup> can be merged to maxFanIn files in i-1
+        // rounds where every merge
+        // is a full merge with maxFanIn input channels. A partial round includes merges with fewer
+        // than maxFanIn
+        // inputs. It is most efficient to perform the partial round first.
+        final double scale = Math.ceil(Math.log(channelIDs.size()) / Math.log(maxFanIn)) - 1;
+
+        final int numStart = channelIDs.size();
+        final int numEnd = (int) Math.pow(maxFanIn, scale);
+
+        final int numMerges = (int) Math.ceil((numStart - numEnd) / (double) (maxFanIn - 1));
+
+        final int numNotMerged = numEnd - numMerges;
+        final int numToMerge = numStart - numNotMerged;
+
+        // unmerged channel IDs are copied directly to the result list
+        final List<ChannelWithMeta> mergedChannelIDs = new ArrayList<>(numEnd);
+        mergedChannelIDs.addAll(channelIDs.subList(0, numNotMerged));
+
+        final int channelsToMergePerStep = (int) Math.ceil(numToMerge / (double) numMerges);
+
+        final List<ChannelWithMeta> channelsToMergeThisStep =
+                new ArrayList<>(channelsToMergePerStep);
+        int channelNum = numNotMerged;
+        while (!closed && channelNum < channelIDs.size()) {
+            channelsToMergeThisStep.clear();
+
+            for (int i = 0;
+                    i < channelsToMergePerStep && channelNum < channelIDs.size();
+                    i++, channelNum++) {
+                channelsToMergeThisStep.add(channelIDs.get(channelNum));
+            }
+
+            mergedChannelIDs.add(mergeChannels(channelsToMergeThisStep));
+        }
+
+        return mergedChannelIDs;
+    }
+
+    /**
+     * Merges the sorted runs described by the given Channel IDs into a single sorted run.
+     *
+     * @param channelIDs The IDs of the runs' channels.
+     * @return The ID and number of blocks of the channel that describes the merged run.
+     */
+    private ChannelWithMeta mergeChannels(List<ChannelWithMeta> channelIDs) throws IOException {
+        // the list with the target iterators
+        List<FileIOChannel> openChannels = new ArrayList<>(channelIDs.size());
+        final BinaryMergeIterator<Entry> mergeIterator =
+                getMergingIterator(channelIDs, openChannels);
+
+        // create a new channel writer
+        final FileIOChannel.ID mergedChannelID = ioManager.createChannel();
+        channelManager.addChannel(mergedChannelID);
+        ChannelWriterOutputView output = null;
+
+        int numBytesInLastBlock;
+        int numBlocksWritten;
+        try {
+            output =
+                    FileChannelUtil.createOutputView(
+                            ioManager,
+                            mergedChannelID,
+                            compressionCodecFactory,
+                            compressionBlockSize);
+            writeMergingOutput(mergeIterator, output);
+            numBytesInLastBlock = output.close();
+            numBlocksWritten = output.getBlockCount();
+        } catch (IOException e) {
+            if (output != null) {
+                output.close();
+                output.getChannel().deleteChannel();
+            }
+            throw e;
+        }
+
+        // remove, close and delete channels
+        for (FileIOChannel channel : openChannels) {
+            channelManager.removeChannel(channel.getChannelID());
+            try {
+                channel.closeAndDelete();
+            } catch (Throwable ignored) {
+            }
+        }
+
+        return new ChannelWithMeta(mergedChannelID, numBlocksWritten, numBytesInLastBlock);
+    }
+
+    // -------------------------------------------------------------------------------------------
+
+    /** @return entry iterator reading from inView. */
+    protected abstract MutableObjectIterator<Entry> channelReaderInputViewIterator(
+            ChannelReaderInputView inView);
+
+    /** @return merging comparator used in merging. */
+    protected abstract Comparator<Entry> mergeComparator();
+
+    /** @return reused entry object used in merging. */
+    protected abstract List<Entry> mergeReusedEntries(int size);
+
+    /** read the merged stream and write the data back. */
+    protected abstract void writeMergingOutput(
+            MutableObjectIterator<Entry> mergeIterator, AbstractPagedOutputView output)
+            throws IOException;
+}
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/BinaryExternalMerger.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/BinaryExternalMerger.java
index 489f909d..ab0941a9 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/BinaryExternalMerger.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/BinaryExternalMerger.java
@@ -18,16 +18,14 @@
 
 package org.apache.flink.table.store.file.sort;
 
-import org.apache.flink.runtime.io.compression.BlockCompressionFactory;
-import org.apache.flink.runtime.io.disk.ChannelReaderInputViewIterator;
-import org.apache.flink.runtime.io.disk.iomanager.AbstractChannelReaderInputView;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.memory.AbstractPagedOutputView;
 import org.apache.flink.table.data.binary.BinaryRowData;
-import org.apache.flink.table.runtime.operators.sort.AbstractBinaryExternalMerger;
-import org.apache.flink.table.runtime.operators.sort.SpillChannelManager;
-import org.apache.flink.table.runtime.typeutils.BinaryRowDataSerializer;
 import org.apache.flink.table.store.codegen.RecordComparator;
+import org.apache.flink.table.store.data.AbstractPagedOutputView;
+import org.apache.flink.table.store.data.BinaryRowDataSerializer;
+import org.apache.flink.table.store.file.compression.BlockCompressionFactory;
+import org.apache.flink.table.store.file.disk.ChannelReaderInputView;
+import org.apache.flink.table.store.file.disk.ChannelReaderInputViewIterator;
+import org.apache.flink.table.store.file.disk.IOManager;
 import org.apache.flink.util.MutableObjectIterator;
 
 import java.io.IOException;
@@ -48,7 +46,6 @@ public class BinaryExternalMerger extends AbstractBinaryExternalMerger<BinaryRow
             SpillChannelManager channelManager,
             BinaryRowDataSerializer serializer,
             RecordComparator comparator,
-            boolean compressionEnable,
             BlockCompressionFactory compressionCodecFactory,
             int compressionBlockSize) {
         super(
@@ -56,7 +53,6 @@ public class BinaryExternalMerger extends AbstractBinaryExternalMerger<BinaryRow
                 pageSize,
                 maxFanIn,
                 channelManager,
-                compressionEnable,
                 compressionCodecFactory,
                 compressionBlockSize);
         this.serializer = serializer;
@@ -65,7 +61,7 @@ public class BinaryExternalMerger extends AbstractBinaryExternalMerger<BinaryRow
 
     @Override
     protected MutableObjectIterator<BinaryRowData> channelReaderInputViewIterator(
-            AbstractChannelReaderInputView inView) {
+            ChannelReaderInputView inView) {
         return new ChannelReaderInputViewIterator<>(inView, null, serializer.duplicate());
     }
 
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/BinaryExternalSortBuffer.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/BinaryExternalSortBuffer.java
index 9e49c447..5bbe3e20 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/BinaryExternalSortBuffer.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/BinaryExternalSortBuffer.java
@@ -20,20 +20,17 @@ package org.apache.flink.table.store.file.sort;
 
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.configuration.MemorySize;
-import org.apache.flink.runtime.io.compression.BlockCompressionFactory;
-import org.apache.flink.runtime.io.compression.Lz4BlockCompressionFactory;
-import org.apache.flink.runtime.io.disk.iomanager.AbstractChannelWriterOutputView;
-import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.operators.sort.QuickSort;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.data.binary.BinaryRowData;
-import org.apache.flink.table.runtime.io.ChannelWithMeta;
-import org.apache.flink.table.runtime.operators.sort.BinaryMergeIterator;
-import org.apache.flink.table.runtime.operators.sort.SpillChannelManager;
-import org.apache.flink.table.runtime.typeutils.BinaryRowDataSerializer;
-import org.apache.flink.table.runtime.util.FileChannelUtil;
 import org.apache.flink.table.store.codegen.RecordComparator;
+import org.apache.flink.table.store.data.BinaryRowDataSerializer;
+import org.apache.flink.table.store.file.compression.BlockCompressionFactory;
+import org.apache.flink.table.store.file.compression.Lz4BlockCompressionFactory;
+import org.apache.flink.table.store.file.disk.ChannelWithMeta;
+import org.apache.flink.table.store.file.disk.ChannelWriterOutputView;
+import org.apache.flink.table.store.file.disk.FileChannelUtil;
+import org.apache.flink.table.store.file.disk.FileIOChannel;
+import org.apache.flink.table.store.file.disk.IOManager;
 import org.apache.flink.util.MutableObjectIterator;
 
 import java.io.IOException;
@@ -44,12 +41,10 @@ import java.util.List;
 public class BinaryExternalSortBuffer implements SortBuffer {
 
     private final BinaryRowDataSerializer serializer;
-    private final int pageSize;
     private final BinaryInMemorySortBuffer inMemorySortBuffer;
     private final IOManager ioManager;
     private SpillChannelManager channelManager;
     private final int maxNumFileHandles;
-    private final boolean compressionEnable;
     private final BlockCompressionFactory compressionCodecFactory;
     private final int compressionBlockSize;
     private final BinaryExternalMerger merger;
@@ -67,12 +62,10 @@ public class BinaryExternalSortBuffer implements SortBuffer {
             IOManager ioManager,
             int maxNumFileHandles) {
         this.serializer = serializer;
-        this.pageSize = pageSize;
         this.inMemorySortBuffer = inMemorySortBuffer;
         this.ioManager = ioManager;
         this.channelManager = new SpillChannelManager();
         this.maxNumFileHandles = maxNumFileHandles;
-        this.compressionEnable = true;
         this.compressionCodecFactory = new Lz4BlockCompressionFactory();
         this.compressionBlockSize = (int) MemorySize.parse("64 kb").getBytes();
         this.merger =
@@ -83,7 +76,6 @@ public class BinaryExternalSortBuffer implements SortBuffer {
                         channelManager,
                         (BinaryRowDataSerializer) serializer.duplicate(),
                         comparator,
-                        compressionEnable,
                         compressionCodecFactory,
                         compressionBlockSize);
         this.enumerator = ioManager.createChannelEnumerator();
@@ -189,19 +181,14 @@ public class BinaryExternalSortBuffer implements SortBuffer {
         FileIOChannel.ID channel = enumerator.next();
         channelManager.addChannel(channel);
 
-        AbstractChannelWriterOutputView output = null;
+        ChannelWriterOutputView output = null;
         int bytesInLastBuffer;
         int blockCount;
 
         try {
             output =
                     FileChannelUtil.createOutputView(
-                            ioManager,
-                            channel,
-                            compressionEnable,
-                            compressionCodecFactory,
-                            compressionBlockSize,
-                            pageSize);
+                            ioManager, channel, compressionCodecFactory, compressionBlockSize);
             new QuickSort().sort(inMemorySortBuffer);
             inMemorySortBuffer.writeToOutput(output);
             bytesInLastBuffer = output.close();
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/BinaryInMemorySortBuffer.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/BinaryInMemorySortBuffer.java
index b1bb1aff..6b219bb0 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/BinaryInMemorySortBuffer.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/BinaryInMemorySortBuffer.java
@@ -19,15 +19,14 @@
 package org.apache.flink.table.store.file.sort;
 
 import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.runtime.io.disk.SimpleCollectingOutputView;
-import org.apache.flink.runtime.operators.sort.QuickSort;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.data.binary.BinaryRowData;
-import org.apache.flink.table.runtime.typeutils.AbstractRowDataSerializer;
-import org.apache.flink.table.runtime.typeutils.BinaryRowDataSerializer;
-import org.apache.flink.table.runtime.util.MemorySegmentPool;
 import org.apache.flink.table.store.codegen.NormalizedKeyComputer;
 import org.apache.flink.table.store.codegen.RecordComparator;
+import org.apache.flink.table.store.data.AbstractRowDataSerializer;
+import org.apache.flink.table.store.data.BinaryRowDataSerializer;
+import org.apache.flink.table.store.data.SimpleCollectingOutputView;
+import org.apache.flink.table.store.file.memory.MemorySegmentPool;
 import org.apache.flink.util.MutableObjectIterator;
 
 import java.io.EOFException;
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/BinaryIndexedSortable.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/BinaryIndexedSortable.java
index 32ecf5b7..96070c9c 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/BinaryIndexedSortable.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/BinaryIndexedSortable.java
@@ -19,15 +19,14 @@
 package org.apache.flink.table.store.file.sort;
 
 import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.runtime.io.disk.RandomAccessInputView;
-import org.apache.flink.runtime.memory.AbstractPagedOutputView;
-import org.apache.flink.runtime.operators.sort.IndexedSortable;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.data.binary.BinaryRowData;
-import org.apache.flink.table.runtime.typeutils.BinaryRowDataSerializer;
-import org.apache.flink.table.runtime.util.MemorySegmentPool;
 import org.apache.flink.table.store.codegen.NormalizedKeyComputer;
 import org.apache.flink.table.store.codegen.RecordComparator;
+import org.apache.flink.table.store.data.AbstractPagedOutputView;
+import org.apache.flink.table.store.data.BinaryRowDataSerializer;
+import org.apache.flink.table.store.data.RandomAccessInputView;
+import org.apache.flink.table.store.file.memory.MemorySegmentPool;
 
 import java.io.IOException;
 import java.util.ArrayList;
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/BinaryMergeIterator.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/BinaryMergeIterator.java
new file mode 100644
index 00000000..83d3039c
--- /dev/null
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/BinaryMergeIterator.java
@@ -0,0 +1,96 @@
+/*
+ * 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.table.store.file.sort;
+
+import org.apache.flink.util.MutableObjectIterator;
+
+import java.io.IOException;
+import java.util.Comparator;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** Binary version of {@code MergeIterator}. Use {@code RecordComparator} to compare record. */
+public class BinaryMergeIterator<Entry> implements MutableObjectIterator<Entry> {
+
+    // heap over the head elements of the stream
+    private final PartialOrderPriorityQueue<HeadStream<Entry>> heap;
+    private HeadStream<Entry> currHead;
+
+    public BinaryMergeIterator(
+            List<MutableObjectIterator<Entry>> iterators,
+            List<Entry> reusableEntries,
+            Comparator<Entry> comparator)
+            throws IOException {
+        checkArgument(iterators.size() == reusableEntries.size());
+        this.heap =
+                new PartialOrderPriorityQueue<>(
+                        (o1, o2) -> comparator.compare(o1.getHead(), o2.getHead()),
+                        iterators.size());
+        for (int i = 0; i < iterators.size(); i++) {
+            this.heap.add(new HeadStream<>(iterators.get(i), reusableEntries.get(i)));
+        }
+    }
+
+    @Override
+    public Entry next(Entry reuse) throws IOException {
+        // Ignore reuse, because each HeadStream has its own reuse BinaryRowData.
+        return next();
+    }
+
+    @Override
+    public Entry next() throws IOException {
+        if (currHead != null) {
+            if (!currHead.nextHead()) {
+                this.heap.poll();
+            } else {
+                this.heap.adjustTop();
+            }
+        }
+
+        if (this.heap.size() > 0) {
+            currHead = this.heap.peek();
+            return currHead.getHead();
+        } else {
+            return null;
+        }
+    }
+
+    private static final class HeadStream<Entry> {
+
+        private final MutableObjectIterator<Entry> iterator;
+        private Entry head;
+
+        private HeadStream(MutableObjectIterator<Entry> iterator, Entry head) throws IOException {
+            this.iterator = iterator;
+            this.head = head;
+            if (!nextHead()) {
+                throw new IllegalStateException();
+            }
+        }
+
+        private Entry getHead() {
+            return this.head;
+        }
+
+        private boolean nextHead() throws IOException {
+            return (this.head = this.iterator.next(head)) != null;
+        }
+    }
+}
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/HeapSort.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/HeapSort.java
new file mode 100644
index 00000000..f1741fc8
--- /dev/null
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/HeapSort.java
@@ -0,0 +1,66 @@
+/*
+ * 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.table.store.file.sort;
+
+/**
+ * This file is based on source code from the Hadoop Project (http://hadoop.apache.org/), licensed
+ * by the Apache Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership.
+ */
+public final class HeapSort implements IndexedSorter {
+    public HeapSort() {}
+
+    private static void downHeap(final IndexedSortable s, final int b, int i, final int n) {
+        for (int idx = i << 1; idx < n; idx = i << 1) {
+            if (idx + 1 < n && s.compare(b + idx, b + idx + 1) < 0) {
+                if (s.compare(b + i, b + idx + 1) < 0) {
+                    s.swap(b + i, b + idx + 1);
+                } else {
+                    return;
+                }
+                i = idx + 1;
+            } else if (s.compare(b + i, b + idx) < 0) {
+                s.swap(b + i, b + idx);
+                i = idx;
+            } else {
+                return;
+            }
+        }
+    }
+
+    public void sort(final IndexedSortable s, final int p, final int r) {
+        final int n = r - p;
+        // build heap w/ reverse comparator, then write in-place from end
+        final int t = Integer.highestOneBit(n);
+        for (int i = t; i > 1; i >>>= 1) {
+            for (int j = i >>> 1; j < i; ++j) {
+                downHeap(s, p - 1, j, n + 1);
+            }
+        }
+        for (int i = r - 1; i > p; --i) {
+            s.swap(p, i);
+            downHeap(s, p - 1, 1, i - p + 1);
+        }
+    }
+
+    @Override
+    public void sort(IndexedSortable s) {
+        sort(s, 0, s.size());
+    }
+}
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/IndexedSortable.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/IndexedSortable.java
new file mode 100644
index 00000000..0adaa73f
--- /dev/null
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/IndexedSortable.java
@@ -0,0 +1,76 @@
+/*
+ * 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.table.store.file.sort;
+
+/** Indexed sortable to provide compare and swap. */
+public interface IndexedSortable {
+
+    /**
+     * Compare items at the given addresses consistent with the semantics of {@link
+     * java.util.Comparator#compare(Object, Object)}.
+     */
+    int compare(int i, int j);
+
+    /**
+     * Compare records at the given addresses consistent with the semantics of {@link
+     * java.util.Comparator#compare(Object, Object)}.
+     *
+     * @param segmentNumberI index of memory segment containing first record
+     * @param segmentOffsetI offset into memory segment containing first record
+     * @param segmentNumberJ index of memory segment containing second record
+     * @param segmentOffsetJ offset into memory segment containing second record
+     * @return a negative integer, zero, or a positive integer as the first argument is less than,
+     *     equal to, or greater than the second.
+     */
+    int compare(int segmentNumberI, int segmentOffsetI, int segmentNumberJ, int segmentOffsetJ);
+
+    /** Swap items at the given addresses. */
+    void swap(int i, int j);
+
+    /**
+     * Swap records at the given addresses.
+     *
+     * @param segmentNumberI index of memory segment containing first record
+     * @param segmentOffsetI offset into memory segment containing first record
+     * @param segmentNumberJ index of memory segment containing second record
+     * @param segmentOffsetJ offset into memory segment containing second record
+     */
+    void swap(int segmentNumberI, int segmentOffsetI, int segmentNumberJ, int segmentOffsetJ);
+
+    /**
+     * Gets the number of elements in the sortable.
+     *
+     * @return The number of elements.
+     */
+    int size();
+
+    /**
+     * Gets the size of each record, the number of bytes separating the head of successive records.
+     *
+     * @return The record size
+     */
+    int recordSize();
+
+    /**
+     * Gets the number of elements in each memory segment.
+     *
+     * @return The number of records per segment
+     */
+    int recordsPerSegment();
+}
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/IndexedSorter.java
similarity index 60%
copy from flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java
copy to flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/IndexedSorter.java
index 4a352309..eee7e4e1 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/memory/MemoryOwner.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/IndexedSorter.java
@@ -16,19 +16,20 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.store.file.memory;
+package org.apache.flink.table.store.file.sort;
 
-import org.apache.flink.table.runtime.util.MemorySegmentPool;
+/** Indexed sorter to provide sort. */
+public interface IndexedSorter {
 
-/** A class provides memory related methods. */
-public interface MemoryOwner {
+    /**
+     * Sort the items accessed through the given IndexedSortable over the given range of logical
+     * indices. From the perspective of the sort algorithm, each index between l (inclusive) and r
+     * (exclusive) is an addressable entry.
+     *
+     * @see IndexedSortable#compare
+     * @see IndexedSortable#swap
+     */
+    void sort(IndexedSortable s, int l, int r);
 
-    /** Set {@link MemorySegmentPool} for the owner. */
-    void setMemoryPool(MemorySegmentPool memoryPool);
-
-    /** Memory occupancy size of this owner. */
-    long memoryOccupancy();
-
-    /** Flush memory of owner, release memory. */
-    void flushMemory() throws Exception;
+    void sort(IndexedSortable s);
 }
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/PartialOrderPriorityQueue.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/PartialOrderPriorityQueue.java
new file mode 100644
index 00000000..ce8394a3
--- /dev/null
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/PartialOrderPriorityQueue.java
@@ -0,0 +1,195 @@
+/*
+ * 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.table.store.file.sort;
+
+import java.util.AbstractQueue;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.Queue;
+
+/**
+ * This class implements a priority-queue, which maintains a partial ordering of its elements such
+ * that the least element can always be found in constant time. Put()'s and pop()'s require
+ * log(size) time.
+ */
+public class PartialOrderPriorityQueue<T> extends AbstractQueue<T> implements Queue<T> {
+    /** The heap, organized as an array. */
+    private final T[] heap;
+
+    /** The comparator used to establish the order between the streams. */
+    private final Comparator<T> comparator;
+
+    /** The maximum size of the heap. */
+    private final int capacity;
+
+    /** The current number of elements in the queue. */
+    private int size;
+
+    @SuppressWarnings("unchecked")
+    public PartialOrderPriorityQueue(Comparator<T> comparator, int capacity) {
+        this.comparator = comparator;
+        this.capacity = capacity + 1;
+        this.size = 0;
+        this.heap = (T[]) new Object[this.capacity];
+    }
+
+    /**
+     * Determines the ordering of objects in this priority queue.
+     *
+     * @param a The first element.
+     * @param b The second element.
+     * @return True, if a &lt; b, false otherwise.
+     */
+    private boolean lessThan(T a, T b) {
+        return comparator.compare(a, b) < 0;
+    }
+
+    /**
+     * Returns the remaining capacity of the backing array.
+     *
+     * @return The remaining capacity of the backing array.
+     */
+    public int remainingCapacity() {
+        return capacity - size;
+    }
+
+    /**
+     * Adds a buffer to a PriorityQueue in log(size) time. If one tries to add more objects than
+     * maxSize from initialize a RuntimeException (ArrayIndexOutOfBound) is thrown.
+     */
+    public final void put(T element) {
+        size++;
+        heap[size] = element;
+        upHeap();
+    }
+
+    /**
+     * Adds element to the PriorityQueue in log(size) time if either the PriorityQueue is not full,
+     * or not lessThan(element, top()).
+     *
+     * @param element The element to insert,
+     * @return True, if element is added, false otherwise.
+     */
+    public boolean offer(T element) {
+        if (size < capacity) {
+            put(element);
+            return true;
+        } else if (size > 0 && !lessThan(element, peek())) {
+            heap[1] = element;
+            adjustTop();
+            return true;
+        } else {
+            return false;
+        }
+    }
+
+    /**
+     * Returns the least element of the PriorityQueue in constant time, but does not remove it from
+     * the priority queue.
+     *
+     * @return The least element.
+     */
+    public final T peek() {
+        if (size > 0) {
+            return heap[1];
+        } else {
+            return null;
+        }
+    }
+
+    /**
+     * Removes and returns the least element of the PriorityQueue in log(size) time.
+     *
+     * @return The least element.
+     */
+    public final T poll() {
+        if (size > 0) {
+            T result = heap[1]; // save first value
+            heap[1] = heap[size]; // move last to first
+            heap[size] = null; // permit GC of objects
+            size--;
+            downHeap(); // adjust heap
+            return result;
+        } else {
+            return null;
+        }
+    }
+
+    /** Should be called when the Object at top changes values. */
+    public final void adjustTop() {
+        downHeap();
+    }
+
+    /**
+     * Returns the number of elements currently stored in the PriorityQueue.
+     *
+     * @return The number of elements in the queue.
+     */
+    public final int size() {
+        return size;
+    }
+
+    /** Removes all entries from the PriorityQueue. */
+    public final void clear() {
+        for (int i = 0; i <= size; i++) {
+            heap[i] = null;
+        }
+        size = 0;
+    }
+
+    private void upHeap() {
+        int i = size;
+        T node = heap[i]; // save bottom node
+        int j = i >>> 1;
+        while (j > 0 && lessThan(node, heap[j])) {
+            heap[i] = heap[j]; // shift parents down
+            i = j;
+            j = j >>> 1;
+        }
+        heap[i] = node; // install saved node
+    }
+
+    private void downHeap() {
+        int i = 1;
+        T node = heap[i]; // save top node
+        int j = i << 1; // find smaller child
+        int k = j + 1;
+        if (k <= size && lessThan(heap[k], heap[j])) {
+            j = k;
+        }
+
+        while (j <= size && lessThan(heap[j], node)) {
+            heap[i] = heap[j]; // shift up child
+            i = j;
+            j = i << 1;
+            k = j + 1;
+            if (k <= size && lessThan(heap[k], heap[j])) {
+                j = k;
+            }
+        }
+
+        heap[i] = node; // install saved node
+    }
+
+    @Override
+    public Iterator<T> iterator() {
+        return Arrays.asList(heap).iterator();
+    }
+}
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/QuickSort.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/QuickSort.java
new file mode 100644
index 00000000..27840bbd
--- /dev/null
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/QuickSort.java
@@ -0,0 +1,330 @@
+/*
+ * 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.table.store.file.sort;
+
+/** Quick sort {@link IndexedSorter}. */
+public final class QuickSort implements IndexedSorter {
+
+    private static final IndexedSorter alt = new HeapSort();
+
+    public QuickSort() {}
+
+    /**
+     * Fix the records into sorted order, swapping when the first record is greater than the second
+     * record.
+     *
+     * @param s paged sortable
+     * @param pN page number of first record
+     * @param pO page offset of first record
+     * @param rN page number of second record
+     * @param rO page offset of second record
+     */
+    private static void fix(IndexedSortable s, int pN, int pO, int rN, int rO) {
+        if (s.compare(pN, pO, rN, rO) > 0) {
+            s.swap(pN, pO, rN, rO);
+        }
+    }
+
+    /** Deepest recursion before giving up and doing a heapsort. Returns 2 * ceil(log(n)). */
+    private static int getMaxDepth(int x) {
+        if (x <= 0) {
+            throw new IllegalArgumentException("Undefined for " + x);
+        }
+        return (32 - Integer.numberOfLeadingZeros(x - 1)) << 2;
+    }
+
+    /**
+     * Sort the given range of items using quick sort. {@inheritDoc} If the recursion depth falls
+     * below {@link #getMaxDepth}, then switch to {@link HeapSort}.
+     */
+    public void sort(final IndexedSortable s, int p, int r) {
+        int recordsPerSegment = s.recordsPerSegment();
+        int recordSize = s.recordSize();
+        int maxOffset = recordSize * (recordsPerSegment - 1);
+
+        int pN = p / recordsPerSegment;
+        int pO = (p % recordsPerSegment) * recordSize;
+
+        int rN = r / recordsPerSegment;
+        int rO = (r % recordsPerSegment) * recordSize;
+
+        sortInternal(
+                s,
+                recordsPerSegment,
+                recordSize,
+                maxOffset,
+                p,
+                pN,
+                pO,
+                r,
+                rN,
+                rO,
+                getMaxDepth(r - p));
+    }
+
+    public void sort(IndexedSortable s) {
+        sort(s, 0, s.size());
+    }
+
+    /**
+     * Sort the given range of items using quick sort. If the recursion depth falls below {@link
+     * #getMaxDepth}, then switch to {@link HeapSort}.
+     *
+     * @param s paged sortable
+     * @param recordsPerSegment number of records per memory segment
+     * @param recordSize number of bytes per record
+     * @param maxOffset offset of a last record in a memory segment
+     * @param p index of first record in range
+     * @param pN page number of first record in range
+     * @param pO page offset of first record in range
+     * @param r index of last-plus-one'th record in range
+     * @param rN page number of last-plus-one'th record in range
+     * @param rO page offset of last-plus-one'th record in range
+     * @param depth recursion depth
+     * @see #sort(IndexedSortable, int, int)
+     */
+    private static void sortInternal(
+            final IndexedSortable s,
+            int recordsPerSegment,
+            int recordSize,
+            int maxOffset,
+            int p,
+            int pN,
+            int pO,
+            int r,
+            int rN,
+            int rO,
+            int depth) {
+        while (true) {
+            if (r - p < 13) {
+                // switch to insertion sort
+                int i = p + 1, iN, iO;
+                if (pO == maxOffset) {
+                    iN = pN + 1;
+                    iO = 0;
+                } else {
+                    iN = pN;
+                    iO = pO + recordSize;
+                }
+
+                while (i < r) {
+                    int j = i, jN = iN, jO = iO;
+                    int jd = j - 1, jdN, jdO;
+                    if (jO == 0) {
+                        jdN = jN - 1;
+                        jdO = maxOffset;
+                    } else {
+                        jdN = jN;
+                        jdO = jO - recordSize;
+                    }
+
+                    while (j > p && s.compare(jdN, jdO, jN, jO) > 0) {
+                        s.swap(jN, jO, jdN, jdO);
+
+                        j = jd;
+                        jN = jdN;
+                        jO = jdO;
+                        jd--;
+                        if (jdO == 0) {
+                            jdN--;
+                            jdO = maxOffset;
+                        } else {
+                            jdO -= recordSize;
+                        }
+                    }
+
+                    i++;
+                    if (iO == maxOffset) {
+                        iN++;
+                        iO = 0;
+                    } else {
+                        iO += recordSize;
+                    }
+                }
+                return;
+            }
+
+            if (--depth < 0) {
+                // switch to heap sort
+                alt.sort(s, p, r);
+                return;
+            }
+
+            int rdN, rdO;
+            if (rO == 0) {
+                rdN = rN - 1;
+                rdO = maxOffset;
+            } else {
+                rdN = rN;
+                rdO = rO - recordSize;
+            }
+            int m = (p + r) >>> 1,
+                    mN = m / recordsPerSegment,
+                    mO = (m % recordsPerSegment) * recordSize;
+
+            // select, move pivot into first position
+            fix(s, mN, mO, pN, pO);
+            fix(s, mN, mO, rdN, rdO);
+            fix(s, pN, pO, rdN, rdO);
+
+            // Divide
+            int i = p, iN = pN, iO = pO;
+            int j = r, jN = rN, jO = rO;
+            int ll = p, llN = pN, llO = pO;
+            int rr = r, rrN = rN, rrO = rO;
+            int cr;
+            while (true) {
+                i++;
+                if (iO == maxOffset) {
+                    iN++;
+                    iO = 0;
+                } else {
+                    iO += recordSize;
+                }
+
+                while (i < j) {
+                    if ((cr = s.compare(iN, iO, pN, pO)) > 0) {
+                        break;
+                    }
+
+                    if (0 == cr) {
+                        ll++;
+                        if (llO == maxOffset) {
+                            llN++;
+                            llO = 0;
+                        } else {
+                            llO += recordSize;
+                        }
+
+                        if (ll != i) {
+                            s.swap(llN, llO, iN, iO);
+                        }
+                    }
+
+                    i++;
+                    if (iO == maxOffset) {
+                        iN++;
+                        iO = 0;
+                    } else {
+                        iO += recordSize;
+                    }
+                }
+
+                j--;
+                if (jO == 0) {
+                    jN--;
+                    jO = maxOffset;
+                } else {
+                    jO -= recordSize;
+                }
+
+                while (j > i) {
+                    if ((cr = s.compare(pN, pO, jN, jO)) > 0) {
+                        break;
+                    }
+
+                    if (0 == cr) {
+                        rr--;
+                        if (rrO == 0) {
+                            rrN--;
+                            rrO = maxOffset;
+                        } else {
+                            rrO -= recordSize;
+                        }
+
+                        if (rr != j) {
+                            s.swap(rrN, rrO, jN, jO);
+                        }
+                    }
+
+                    j--;
+                    if (jO == 0) {
+                        jN--;
+                        jO = maxOffset;
+                    } else {
+                        jO -= recordSize;
+                    }
+                }
+                if (i < j) {
+                    s.swap(iN, iO, jN, jO);
+                } else {
+                    break;
+                }
+            }
+            j = i;
+            jN = iN;
+            jO = iO;
+            // swap pivot- and all eq values- into position
+            while (ll >= p) {
+                i--;
+                if (iO == 0) {
+                    iN--;
+                    iO = maxOffset;
+                } else {
+                    iO -= recordSize;
+                }
+
+                s.swap(llN, llO, iN, iO);
+
+                ll--;
+                if (llO == 0) {
+                    llN--;
+                    llO = maxOffset;
+                } else {
+                    llO -= recordSize;
+                }
+            }
+            while (rr < r) {
+                s.swap(rrN, rrO, jN, jO);
+
+                rr++;
+                if (rrO == maxOffset) {
+                    rrN++;
+                    rrO = 0;
+                } else {
+                    rrO += recordSize;
+                }
+                j++;
+                if (jO == maxOffset) {
+                    jN++;
+                    jO = 0;
+                } else {
+                    jO += recordSize;
+                }
+            }
+
+            // Conquer
+            // Recurse on smaller interval first to keep stack shallow
+            assert i != j;
+            if (i - p < r - j) {
+                sortInternal(
+                        s, recordsPerSegment, recordSize, maxOffset, p, pN, pO, i, iN, iO, depth);
+                p = j;
+                pN = jN;
+                pO = jO;
+            } else {
+                sortInternal(
+                        s, recordsPerSegment, recordSize, maxOffset, j, jN, jO, r, rN, rO, depth);
+                r = i;
+                rN = iN;
+                rO = iO;
+            }
+        }
+    }
+}
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/SpillChannelManager.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/SpillChannelManager.java
new file mode 100644
index 00000000..ede9290e
--- /dev/null
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/sort/SpillChannelManager.java
@@ -0,0 +1,95 @@
+/*
+ * 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.table.store.file.sort;
+
+import org.apache.flink.table.store.file.disk.FileIOChannel;
+
+import java.io.Closeable;
+import java.io.File;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** Channel manager to manage the life cycle of spill channels. */
+public class SpillChannelManager implements Closeable {
+
+    private final HashSet<FileIOChannel.ID> channels;
+    private final HashSet<FileIOChannel> openChannels;
+
+    private volatile boolean closed;
+
+    public SpillChannelManager() {
+        this.channels = new HashSet<>(64);
+        this.openChannels = new HashSet<>(64);
+    }
+
+    /** Add a new File channel. */
+    public synchronized void addChannel(FileIOChannel.ID id) {
+        checkArgument(!closed);
+        channels.add(id);
+    }
+
+    /** Open File channels. */
+    public synchronized void addOpenChannels(List<FileIOChannel> toOpen) {
+        checkArgument(!closed);
+        for (FileIOChannel channel : toOpen) {
+            openChannels.add(channel);
+            channels.remove(channel.getChannelID());
+        }
+    }
+
+    public synchronized void removeChannel(FileIOChannel.ID id) {
+        checkArgument(!closed);
+        channels.remove(id);
+    }
+
+    @Override
+    public synchronized void close() {
+
+        if (this.closed) {
+            return;
+        }
+
+        this.closed = true;
+
+        for (Iterator<FileIOChannel> channels = this.openChannels.iterator();
+                channels.hasNext(); ) {
+            final FileIOChannel channel = channels.next();
+            channels.remove();
+            try {
+                channel.closeAndDelete();
+            } catch (Throwable ignored) {
+            }
+        }
+
+        for (Iterator<FileIOChannel.ID> channels = this.channels.iterator(); channels.hasNext(); ) {
+            final FileIOChannel.ID channel = channels.next();
+            channels.remove();
+            try {
+                final File f = new File(channel.getPath());
+                if (f.exists()) {
+                    f.delete();
+                }
+            } catch (Throwable ignored) {
+            }
+        }
+    }
+}
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/stats/FieldStatsArraySerializer.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/stats/FieldStatsArraySerializer.java
index 7e02e5bf..28794d80 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/stats/FieldStatsArraySerializer.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/stats/FieldStatsArraySerializer.java
@@ -20,7 +20,7 @@ package org.apache.flink.table.store.file.stats;
 
 import org.apache.flink.table.data.GenericRowData;
 import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.runtime.typeutils.RowDataSerializer;
+import org.apache.flink.table.store.data.RowDataSerializer;
 import org.apache.flink.table.store.file.casting.CastExecutor;
 import org.apache.flink.table.store.format.FieldStats;
 import org.apache.flink.table.store.utils.RowDataUtils;
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/utils/BulkFormatMapping.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/utils/BulkFormatMapping.java
index 3e831c98..70f4fb5d 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/utils/BulkFormatMapping.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/utils/BulkFormatMapping.java
@@ -18,9 +18,6 @@
 
 package org.apache.flink.table.store.file.utils;
 
-import org.apache.flink.connector.file.src.FileSourceSplit;
-import org.apache.flink.connector.file.src.reader.BulkFormat;
-import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.store.file.KeyValue;
 import org.apache.flink.table.store.file.predicate.Predicate;
 import org.apache.flink.table.store.file.schema.DataField;
@@ -29,6 +26,7 @@ import org.apache.flink.table.store.file.schema.RowDataType;
 import org.apache.flink.table.store.file.schema.SchemaEvolutionUtil;
 import org.apache.flink.table.store.file.schema.TableSchema;
 import org.apache.flink.table.store.format.FileFormatDiscover;
+import org.apache.flink.table.store.format.FormatReaderFactory;
 import org.apache.flink.table.store.utils.Projection;
 import org.apache.flink.table.types.logical.RowType;
 
@@ -39,9 +37,9 @@ import java.util.List;
 /** Class with index mapping and bulk format. */
 public class BulkFormatMapping {
     @Nullable private final int[] indexMapping;
-    private final BulkFormat<RowData, FileSourceSplit> bulkFormat;
+    private final FormatReaderFactory bulkFormat;
 
-    public BulkFormatMapping(int[] indexMapping, BulkFormat<RowData, FileSourceSplit> bulkFormat) {
+    public BulkFormatMapping(int[] indexMapping, FormatReaderFactory bulkFormat) {
         this.indexMapping = indexMapping;
         this.bulkFormat = bulkFormat;
     }
@@ -51,7 +49,7 @@ public class BulkFormatMapping {
         return indexMapping;
     }
 
-    public BulkFormat<RowData, FileSourceSplit> getReaderFactory() {
+    public FormatReaderFactory getReaderFactory() {
         return bulkFormat;
     }
 
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/utils/FileUtils.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/utils/FileUtils.java
index f9af89d2..cfd1cd1d 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/utils/FileUtils.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/utils/FileUtils.java
@@ -18,17 +18,13 @@
 
 package org.apache.flink.table.store.file.utils;
 
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.connector.base.source.reader.SourceReaderOptions;
-import org.apache.flink.connector.file.src.FileSourceSplit;
-import org.apache.flink.connector.file.src.reader.BulkFormat;
-import org.apache.flink.connector.file.src.util.Utils;
 import org.apache.flink.core.fs.FSDataInputStream;
 import org.apache.flink.core.fs.FSDataOutputStream;
 import org.apache.flink.core.fs.FileStatus;
 import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.store.format.FormatReaderFactory;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -54,12 +50,6 @@ public class FileUtils {
     private static final Logger LOG = LoggerFactory.getLogger(FileUtils.class);
     private static final int LIST_MAX_RETRY = 30;
 
-    private static final Configuration DEFAULT_READER_CONFIG = new Configuration();
-
-    static {
-        DEFAULT_READER_CONFIG.setInteger(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY, 1);
-    }
-
     public static final ForkJoinPool COMMON_IO_FORK_JOIN_POOL;
 
     // if we want to name threads in the fork join pool we need all these
@@ -77,12 +67,10 @@ public class FileUtils {
     }
 
     public static <T> List<T> readListFromFile(
-            Path path,
-            ObjectSerializer<T> serializer,
-            BulkFormat<RowData, FileSourceSplit> readerFactory)
+            Path path, ObjectSerializer<T> serializer, FormatReaderFactory readerFactory)
             throws IOException {
         List<T> result = new ArrayList<>();
-        Utils.forEachRemaining(
+        RecordReaderUtils.forEachRemaining(
                 createFormatReader(readerFactory, path),
                 row -> result.add(serializer.fromRow(row)));
         return result;
@@ -181,8 +169,8 @@ public class FileUtils {
                 .map(name -> Long.parseLong(name.substring(prefix.length())));
     }
 
-    public static BulkFormat.Reader<RowData> createFormatReader(
-            BulkFormat<RowData, FileSourceSplit> format, Path file) throws IOException {
+    public static RecordReader<RowData> createFormatReader(FormatReaderFactory format, Path file)
+            throws IOException {
         if (!file.getFileSystem().exists(file)) {
             throw new FileNotFoundException(
                     String.format(
@@ -194,8 +182,6 @@ public class FileUtils {
                             file));
         }
 
-        long fileSize = FileUtils.getFileSize(file);
-        FileSourceSplit split = new FileSourceSplit("ignore", file, 0, fileSize);
-        return format.createReader(FileUtils.DEFAULT_READER_CONFIG, split);
+        return format.createReader(file);
     }
 }
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/utils/ObjectSerializer.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/utils/ObjectSerializer.java
index de9b0255..f6baf6e6 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/utils/ObjectSerializer.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/utils/ObjectSerializer.java
@@ -23,8 +23,8 @@ import org.apache.flink.core.memory.DataInputViewStreamWrapper;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
 import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.runtime.typeutils.InternalSerializers;
-import org.apache.flink.table.runtime.typeutils.RowDataSerializer;
+import org.apache.flink.table.store.data.InternalSerializers;
+import org.apache.flink.table.store.data.RowDataSerializer;
 import org.apache.flink.table.types.logical.RowType;
 
 import java.io.ByteArrayInputStream;
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/utils/SerializationUtils.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/utils/SerializationUtils.java
index e18cedd3..c75596b2 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/utils/SerializationUtils.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/utils/SerializationUtils.java
@@ -22,7 +22,6 @@ import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.table.data.binary.BinaryRowData;
-import org.apache.flink.table.runtime.typeutils.BinaryRowDataSerializer;
 import org.apache.flink.table.types.logical.VarBinaryType;
 import org.apache.flink.table.types.logical.VarCharType;
 
@@ -70,7 +69,7 @@ public class SerializationUtils {
     }
 
     /**
-     * Serialize {@link BinaryRowData}, the difference between this and {@link
+     * Serialize {@link BinaryRowData}, the difference between this and {@code
      * BinaryRowDataSerializer} is that arity is also serialized here, so the deserialization is
      * schemaless.
      */
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/table/sink/TableWrite.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/table/sink/TableWrite.java
index 13bfdd0c..6ed8a8fd 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/table/sink/TableWrite.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/table/sink/TableWrite.java
@@ -19,9 +19,9 @@
 package org.apache.flink.table.store.table.sink;
 
 import org.apache.flink.annotation.Internal;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.data.binary.BinaryRowData;
+import org.apache.flink.table.store.file.disk.IOManager;
 import org.apache.flink.table.store.file.io.DataFileMeta;
 
 import java.util.List;
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/table/sink/TableWriteImpl.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/table/sink/TableWriteImpl.java
index e5557650..6c769763 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/table/sink/TableWriteImpl.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/table/sink/TableWriteImpl.java
@@ -18,9 +18,9 @@
 
 package org.apache.flink.table.store.table.sink;
 
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.data.binary.BinaryRowData;
+import org.apache.flink.table.store.file.disk.IOManager;
 import org.apache.flink.table.store.file.io.DataFileMeta;
 import org.apache.flink.table.store.file.operation.FileStoreWrite;
 
diff --git a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/FileFormatTest.java b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/FileFormatTest.java
index 32f2a12e..4f37fd4a 100644
--- a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/FileFormatTest.java
+++ b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/FileFormatTest.java
@@ -20,15 +20,14 @@ package org.apache.flink.table.store.file;
 
 import org.apache.flink.api.common.serialization.BulkWriter;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.connector.file.src.FileSourceSplit;
-import org.apache.flink.connector.file.src.reader.BulkFormat;
-import org.apache.flink.connector.file.src.util.Utils;
 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.table.data.GenericRowData;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.store.CoreOptions;
+import org.apache.flink.table.store.file.utils.RecordReader;
+import org.apache.flink.table.store.file.utils.RecordReaderUtils;
 import org.apache.flink.table.store.format.FileFormat;
 import org.apache.flink.table.types.logical.IntType;
 import org.apache.flink.table.types.logical.RowType;
@@ -69,13 +68,9 @@ public class FileFormatTest {
         out.close();
 
         // read
-        BulkFormat.Reader<RowData> reader =
-                avro.createReaderFactory(rowType)
-                        .createReader(
-                                new Configuration(),
-                                new FileSourceSplit("", path, 0, fs.getFileStatus(path).getLen()));
+        RecordReader<RowData> reader = avro.createReaderFactory(rowType).createReader(path);
         List<RowData> result = new ArrayList<>();
-        Utils.forEachRemaining(
+        RecordReaderUtils.forEachRemaining(
                 reader,
                 rowData -> result.add(GenericRowData.of(rowData.getInt(0), rowData.getInt(1))));
 
diff --git a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/KeyValueSerializerTest.java b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/KeyValueSerializerTest.java
index 75a44155..7349a8f4 100644
--- a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/KeyValueSerializerTest.java
+++ b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/KeyValueSerializerTest.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.table.store.file;
 
-import org.apache.flink.table.runtime.typeutils.RowDataSerializer;
+import org.apache.flink.table.store.data.RowDataSerializer;
 import org.apache.flink.table.store.file.utils.ObjectSerializer;
 import org.apache.flink.table.store.file.utils.ObjectSerializerTestBase;
 
diff --git a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/TestFileStore.java b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/TestFileStore.java
index 0c61503f..f20ef1ad 100644
--- a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/TestFileStore.java
+++ b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/TestFileStore.java
@@ -22,8 +22,8 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.MemorySize;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.table.data.binary.BinaryRowData;
-import org.apache.flink.table.runtime.typeutils.RowDataSerializer;
 import org.apache.flink.table.store.CoreOptions;
+import org.apache.flink.table.store.data.RowDataSerializer;
 import org.apache.flink.table.store.file.io.DataFileMeta;
 import org.apache.flink.table.store.file.manifest.ManifestCommittable;
 import org.apache.flink.table.store.file.manifest.ManifestEntry;
diff --git a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/TestKeyValueGenerator.java b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/TestKeyValueGenerator.java
index 3d6996c3..93fa9347 100644
--- a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/TestKeyValueGenerator.java
+++ b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/TestKeyValueGenerator.java
@@ -24,8 +24,8 @@ import org.apache.flink.table.data.GenericRowData;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.data.StringData;
 import org.apache.flink.table.data.binary.BinaryRowData;
-import org.apache.flink.table.runtime.generated.RecordComparator;
-import org.apache.flink.table.runtime.typeutils.RowDataSerializer;
+import org.apache.flink.table.store.codegen.RecordComparator;
+import org.apache.flink.table.store.data.RowDataSerializer;
 import org.apache.flink.table.store.file.schema.DataField;
 import org.apache.flink.table.store.file.schema.KeyValueFieldsExtractor;
 import org.apache.flink.table.store.file.schema.SchemaManager;
diff --git a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/disk/BufferFileWriterReaderTest.java b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/disk/BufferFileWriterReaderTest.java
new file mode 100644
index 00000000..7cb3350a
--- /dev/null
+++ b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/disk/BufferFileWriterReaderTest.java
@@ -0,0 +1,176 @@
+/*
+ * 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.table.store.file.disk;
+
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
+import org.apache.flink.table.store.file.memory.Buffer;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.LinkedList;
+import java.util.Random;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.fail;
+
+/** Test for {@link BufferFileReader} and {@link BufferFileWriter}. */
+public class BufferFileWriterReaderTest {
+
+    private static final int BUFFER_SIZE = 32 * 1024;
+
+    private static final Random random = new Random();
+
+    @TempDir Path tempDir;
+
+    private IOManager ioManager;
+
+    private BufferFileWriter writer;
+
+    private BufferFileReader reader;
+
+    @AfterEach
+    public void shutdown() throws Exception {
+        if (writer != null) {
+            writer.deleteChannel();
+        }
+
+        if (reader != null) {
+            reader.deleteChannel();
+        }
+
+        ioManager.close();
+    }
+
+    @BeforeEach
+    public void setUpWriterAndReader() {
+        this.ioManager = IOManager.create(tempDir.toFile().getAbsolutePath());
+        FileIOChannel.ID channel = ioManager.createChannel();
+
+        try {
+            writer = ioManager.createBufferFileWriter(channel);
+            reader = ioManager.createBufferFileReader(channel);
+        } catch (IOException e) {
+            if (writer != null) {
+                writer.deleteChannel();
+            }
+
+            if (reader != null) {
+                reader.deleteChannel();
+            }
+
+            fail("Failed to setup writer and reader.");
+        }
+    }
+
+    @Test
+    public void testWriteRead() throws IOException {
+        int numBuffers = 1024;
+        int currentNumber = 0;
+
+        final int minBufferSize = BUFFER_SIZE / 4;
+
+        // Write buffers filled with ascending numbers...
+        LinkedList<Buffer> buffers = new LinkedList<>();
+        for (int i = 0; i < numBuffers; i++) {
+            final Buffer buffer = createBuffer();
+
+            int size = getNextMultipleOf(getRandomNumberInRange(minBufferSize, BUFFER_SIZE), 4);
+
+            currentNumber = fillBufferWithAscendingNumbers(buffer, currentNumber, size);
+
+            writer.writeBlock(buffer);
+            buffers.add(buffer);
+        }
+
+        // Make sure that the writes are finished
+        writer.close();
+
+        // Read buffers back in...
+        for (int i = 0; i < numBuffers; i++) {
+            assertThat(reader.hasReachedEndOfFile()).isFalse();
+            reader.readInto(createBuffer());
+        }
+
+        reader.close();
+
+        assertThat(reader.hasReachedEndOfFile()).isTrue();
+
+        // Verify that the content is the same
+        assertThat(numBuffers).isEqualTo(buffers.size());
+
+        currentNumber = 0;
+        Buffer buffer;
+
+        while ((buffer = buffers.poll()) != null) {
+            currentNumber = verifyBufferFilledWithAscendingNumbers(buffer, currentNumber);
+        }
+    }
+
+    private int getRandomNumberInRange(int min, int max) {
+        return random.nextInt((max - min) + 1) + min;
+    }
+
+    private int getNextMultipleOf(int number, int multiple) {
+        final int mod = number % multiple;
+
+        if (mod == 0) {
+            return number;
+        }
+
+        return number + multiple - mod;
+    }
+
+    private Buffer createBuffer() {
+        return Buffer.create(MemorySegmentFactory.allocateUnpooledSegment(BUFFER_SIZE));
+    }
+
+    static int fillBufferWithAscendingNumbers(Buffer buffer, int currentNumber, int size) {
+        checkArgument(size % 4 == 0);
+
+        MemorySegment segment = buffer.getMemorySegment();
+
+        for (int i = 0; i < size; i += 4) {
+            segment.putInt(i, currentNumber++);
+        }
+        buffer.setSize(size);
+
+        return currentNumber;
+    }
+
+    static int verifyBufferFilledWithAscendingNumbers(Buffer buffer, int currentNumber) {
+        MemorySegment segment = buffer.getMemorySegment();
+
+        int size = buffer.getSize();
+
+        for (int i = 0; i < size; i += 4) {
+            if (segment.getInt(i) != currentNumber++) {
+                throw new IllegalStateException("Read unexpected number from buffer.");
+            }
+        }
+
+        return currentNumber;
+    }
+}
diff --git a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/disk/IOManagerTest.java b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/disk/IOManagerTest.java
new file mode 100644
index 00000000..06ebe75d
--- /dev/null
+++ b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/disk/IOManagerTest.java
@@ -0,0 +1,74 @@
+/*
+ * 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.table.store.file.disk;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.io.File;
+import java.nio.file.Path;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Test for {@link IOManager}. */
+public class IOManagerTest {
+
+    @TempDir Path tempDir;
+
+    @Test
+    public void channelEnumerator() throws Exception {
+        File tempPath = tempDir.toFile();
+
+        String[] tempDirs =
+                new String[] {
+                    new File(tempPath, "a").getAbsolutePath(),
+                    new File(tempPath, "b").getAbsolutePath(),
+                    new File(tempPath, "c").getAbsolutePath(),
+                    new File(tempPath, "d").getAbsolutePath(),
+                    new File(tempPath, "e").getAbsolutePath(),
+                };
+
+        int[] counters = new int[tempDirs.length];
+        try (IOManager ioMan = IOManager.create(tempDirs)) {
+            FileIOChannel.Enumerator enumerator = ioMan.createChannelEnumerator();
+
+            for (int i = 0; i < 3 * tempDirs.length; i++) {
+                FileIOChannel.ID id = enumerator.next();
+
+                File path = id.getPathFile();
+
+                assertThat(path.isAbsolute()).isTrue();
+                assertThat(path.isDirectory()).isFalse();
+
+                assertThat(tempPath.equals(path.getParentFile().getParentFile().getParentFile()))
+                        .isTrue();
+
+                for (int k = 0; k < tempDirs.length; k++) {
+                    if (path.getParentFile().getParent().equals(tempDirs[k])) {
+                        counters[k]++;
+                    }
+                }
+            }
+
+            for (int k = 0; k < tempDirs.length; k++) {
+                assertThat(counters[k]).isEqualTo(3);
+            }
+        }
+    }
+}
diff --git a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/format/FileStatsExtractingAvroFormat.java b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/format/FileStatsExtractingAvroFormat.java
index d63305e2..9ce7a28a 100644
--- a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/format/FileStatsExtractingAvroFormat.java
+++ b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/format/FileStatsExtractingAvroFormat.java
@@ -20,13 +20,12 @@ package org.apache.flink.table.store.file.format;
 
 import org.apache.flink.api.common.serialization.BulkWriter;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.connector.file.src.FileSourceSplit;
-import org.apache.flink.connector.file.src.reader.BulkFormat;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.store.file.predicate.Predicate;
 import org.apache.flink.table.store.file.stats.TestFileStatsExtractor;
 import org.apache.flink.table.store.format.FileFormat;
 import org.apache.flink.table.store.format.FileStatsExtractor;
+import org.apache.flink.table.store.format.FormatReaderFactory;
 import org.apache.flink.table.types.logical.RowType;
 
 import javax.annotation.Nullable;
@@ -45,7 +44,7 @@ public class FileStatsExtractingAvroFormat extends FileFormat {
     }
 
     @Override
-    public BulkFormat<RowData, FileSourceSplit> createReaderFactory(
+    public FormatReaderFactory createReaderFactory(
             RowType type, int[][] projection, @Nullable List<Predicate> filters) {
         return avro.createReaderFactory(type, projection, filters);
     }
diff --git a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/format/FlushingFileFormat.java b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/format/FlushingFileFormat.java
index 0ffa1e12..f2cf1d62 100644
--- a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/format/FlushingFileFormat.java
+++ b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/format/FlushingFileFormat.java
@@ -20,11 +20,10 @@ package org.apache.flink.table.store.file.format;
 
 import org.apache.flink.api.common.serialization.BulkWriter;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.connector.file.src.FileSourceSplit;
-import org.apache.flink.connector.file.src.reader.BulkFormat;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.store.file.predicate.Predicate;
 import org.apache.flink.table.store.format.FileFormat;
+import org.apache.flink.table.store.format.FormatReaderFactory;
 import org.apache.flink.table.types.logical.RowType;
 
 import javax.annotation.Nullable;
@@ -43,7 +42,7 @@ public class FlushingFileFormat extends FileFormat {
     }
 
     @Override
-    public BulkFormat<RowData, FileSourceSplit> createReaderFactory(
+    public FormatReaderFactory createReaderFactory(
             RowType type, int[][] projection, @Nullable List<Predicate> filters) {
         return format.createReaderFactory(type, projection, filters);
     }
diff --git a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/io/KeyValueFileReadWriteTest.java b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/io/KeyValueFileReadWriteTest.java
index f39a5a9b..84e59f64 100644
--- a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/io/KeyValueFileReadWriteTest.java
+++ b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/io/KeyValueFileReadWriteTest.java
@@ -18,12 +18,12 @@
 
 package org.apache.flink.table.store.file.io;
 
-import org.apache.flink.connector.file.table.FileSystemConnectorOptions;
 import org.apache.flink.core.fs.FileStatus;
 import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.table.data.GenericRowData;
-import org.apache.flink.table.runtime.typeutils.RowDataSerializer;
+import org.apache.flink.table.store.CoreOptions;
+import org.apache.flink.table.store.data.RowDataSerializer;
 import org.apache.flink.table.store.file.KeyValue;
 import org.apache.flink.table.store.file.KeyValueSerializerTest;
 import org.apache.flink.table.store.file.TestKeyValueGenerator;
@@ -238,7 +238,7 @@ public class KeyValueFileReadWriteTest {
                 new FileStorePathFactory(
                         new Path(path),
                         RowType.of(),
-                        FileSystemConnectorOptions.PARTITION_DEFAULT_NAME.defaultValue(),
+                        CoreOptions.PARTITION_DEFAULT_NAME.defaultValue(),
                         format);
         int suggestedFileSize = ThreadLocalRandom.current().nextInt(8192) + 1024;
         return KeyValueFileWriterFactory.builder(
diff --git a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/memory/MemoryPoolFactoryTest.java b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/memory/MemoryPoolFactoryTest.java
index 891fdad6..6b3e185e 100644
--- a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/memory/MemoryPoolFactoryTest.java
+++ b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/memory/MemoryPoolFactoryTest.java
@@ -18,8 +18,6 @@
 
 package org.apache.flink.table.store.file.memory;
 
-import org.apache.flink.table.runtime.util.MemorySegmentPool;
-
 import org.junit.jupiter.api.Test;
 
 import java.util.ArrayList;
diff --git a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/mergetree/SortBufferWriteBufferTestBase.java b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/mergetree/SortBufferWriteBufferTestBase.java
index 85010f83..d80751d5 100644
--- a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/mergetree/SortBufferWriteBufferTestBase.java
+++ b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/mergetree/SortBufferWriteBufferTestBase.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.table.store.file.mergetree;
 
-import org.apache.flink.table.runtime.generated.RecordComparator;
+import org.apache.flink.table.store.codegen.RecordComparator;
 import org.apache.flink.table.store.file.KeyValue;
 import org.apache.flink.table.store.file.memory.HeapMemorySegmentPool;
 import org.apache.flink.table.store.file.mergetree.compact.DeduplicateMergeFunction;
diff --git a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/mergetree/compact/CombiningRecordReaderTestBase.java b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/mergetree/compact/CombiningRecordReaderTestBase.java
index e7a2e2f3..2865fc3e 100644
--- a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/mergetree/compact/CombiningRecordReaderTestBase.java
+++ b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/mergetree/compact/CombiningRecordReaderTestBase.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.table.store.file.mergetree.compact;
 
-import org.apache.flink.table.runtime.generated.RecordComparator;
+import org.apache.flink.table.store.codegen.RecordComparator;
 import org.apache.flink.table.store.file.KeyValue;
 import org.apache.flink.table.store.file.utils.RecordReader;
 import org.apache.flink.table.store.file.utils.ReusingTestData;
diff --git a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/mergetree/compact/IntervalPartitionTest.java b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/mergetree/compact/IntervalPartitionTest.java
index 689ac09e..e1397a05 100644
--- a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/mergetree/compact/IntervalPartitionTest.java
+++ b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/mergetree/compact/IntervalPartitionTest.java
@@ -20,7 +20,7 @@ package org.apache.flink.table.store.file.mergetree.compact;
 
 import org.apache.flink.table.data.binary.BinaryRowData;
 import org.apache.flink.table.data.writer.BinaryRowWriter;
-import org.apache.flink.table.runtime.generated.RecordComparator;
+import org.apache.flink.table.store.codegen.RecordComparator;
 import org.apache.flink.table.store.file.io.DataFileMeta;
 import org.apache.flink.table.store.file.mergetree.SortedRun;
 import org.apache.flink.table.store.file.stats.StatsTestUtils;
diff --git a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/operation/KeyValueFileStoreReadTest.java b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/operation/KeyValueFileStoreReadTest.java
index 7f4ecf93..25ab9359 100644
--- a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/operation/KeyValueFileStoreReadTest.java
+++ b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/operation/KeyValueFileStoreReadTest.java
@@ -22,7 +22,7 @@ import org.apache.flink.core.fs.Path;
 import org.apache.flink.table.api.DataTypes;
 import org.apache.flink.table.data.GenericRowData;
 import org.apache.flink.table.data.binary.BinaryRowData;
-import org.apache.flink.table.runtime.typeutils.RowDataSerializer;
+import org.apache.flink.table.store.data.RowDataSerializer;
 import org.apache.flink.table.store.file.KeyValue;
 import org.apache.flink.table.store.file.TestFileStore;
 import org.apache.flink.table.store.file.TestKeyValueGenerator;
diff --git a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/sort/BinaryExternalSortBufferTest.java b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/sort/BinaryExternalSortBufferTest.java
index 481c2f3c..f4644327 100644
--- a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/sort/BinaryExternalSortBufferTest.java
+++ b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/sort/BinaryExternalSortBufferTest.java
@@ -18,16 +18,14 @@
 
 package org.apache.flink.table.store.file.sort;
 
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
-import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.table.data.StringData;
 import org.apache.flink.table.data.binary.BinaryRowData;
 import org.apache.flink.table.data.writer.BinaryRowWriter;
-import org.apache.flink.table.runtime.typeutils.AbstractRowDataSerializer;
-import org.apache.flink.table.runtime.typeutils.BinaryRowDataSerializer;
-import org.apache.flink.table.runtime.util.MemorySegmentPool;
+import org.apache.flink.table.store.data.AbstractRowDataSerializer;
+import org.apache.flink.table.store.data.BinaryRowDataSerializer;
+import org.apache.flink.table.store.file.disk.IOManager;
 import org.apache.flink.table.store.file.memory.HeapMemorySegmentPool;
+import org.apache.flink.table.store.file.memory.MemorySegmentPool;
 import org.apache.flink.util.MutableObjectIterator;
 
 import org.junit.jupiter.api.AfterEach;
@@ -69,7 +67,7 @@ public class BinaryExternalSortBufferTest {
 
     @BeforeEach
     public void beforeTest() {
-        ioManager = new IOManagerAsync(tempDir.toString());
+        ioManager = IOManager.create(tempDir.toString());
         initMemorySegmentPool(MEMORY_SIZE);
         this.serializer = new BinaryRowDataSerializer(2);
     }
@@ -82,7 +80,7 @@ public class BinaryExternalSortBufferTest {
 
     private void initMemorySegmentPool(long maxMemory) {
         this.memorySegmentPool =
-                new HeapMemorySegmentPool(maxMemory, MemoryManager.DEFAULT_PAGE_SIZE);
+                new HeapMemorySegmentPool(maxMemory, MemorySegmentPool.DEFAULT_PAGE_SIZE);
         this.totalPages = memorySegmentPool.freePages();
     }
 
@@ -279,7 +277,7 @@ public class BinaryExternalSortBufferTest {
         return new BinaryExternalSortBuffer(
                 serializer,
                 IntRecordComparator.INSTANCE,
-                MemoryManager.DEFAULT_PAGE_SIZE,
+                MemorySegmentPool.DEFAULT_PAGE_SIZE,
                 inMemorySortBuffer,
                 ioManager,
                 maxNumFileHandles);
diff --git a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/sort/IntNormalizedKeyComputer.java b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/sort/IntNormalizedKeyComputer.java
index af920330..15406fcd 100644
--- a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/sort/IntNormalizedKeyComputer.java
+++ b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/sort/IntNormalizedKeyComputer.java
@@ -20,8 +20,8 @@ package org.apache.flink.table.store.file.sort;
 
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.runtime.operators.sort.SortUtil;
 import org.apache.flink.table.store.codegen.NormalizedKeyComputer;
+import org.apache.flink.table.store.utils.SortUtil;
 
 /** Example for int {@link NormalizedKeyComputer}. */
 public class IntNormalizedKeyComputer implements NormalizedKeyComputer {
diff --git a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/stats/TestFileStatsExtractor.java b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/stats/TestFileStatsExtractor.java
index 4d854aef..abcd5963 100644
--- a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/stats/TestFileStatsExtractor.java
+++ b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/stats/TestFileStatsExtractor.java
@@ -18,8 +18,6 @@
 
 package org.apache.flink.table.store.file.stats;
 
-import org.apache.flink.connector.file.src.FileSourceSplit;
-import org.apache.flink.connector.file.src.reader.BulkFormat;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.store.file.utils.FileUtils;
@@ -28,6 +26,7 @@ import org.apache.flink.table.store.format.FieldStats;
 import org.apache.flink.table.store.format.FieldStatsCollector;
 import org.apache.flink.table.store.format.FileFormat;
 import org.apache.flink.table.store.format.FileStatsExtractor;
+import org.apache.flink.table.store.format.FormatReaderFactory;
 import org.apache.flink.table.types.logical.RowType;
 
 import java.io.IOException;
@@ -50,7 +49,7 @@ public class TestFileStatsExtractor implements FileStatsExtractor {
     @Override
     public FieldStats[] extract(Path path) throws IOException {
         IdentityObjectSerializer serializer = new IdentityObjectSerializer(rowType);
-        BulkFormat<RowData, FileSourceSplit> readerFactory = format.createReaderFactory(rowType);
+        FormatReaderFactory readerFactory = format.createReaderFactory(rowType);
         List<RowData> records = FileUtils.readListFromFile(path, serializer, readerFactory);
         FieldStatsCollector statsCollector = new FieldStatsCollector(rowType);
         for (RowData record : records) {
diff --git a/flink-table-store-core/src/test/java/org/apache/flink/table/store/table/AppendOnlyFileStoreTableTest.java b/flink-table-store-core/src/test/java/org/apache/flink/table/store/table/AppendOnlyFileStoreTableTest.java
index e1cca63f..5458e956 100644
--- a/flink-table-store-core/src/test/java/org/apache/flink/table/store/table/AppendOnlyFileStoreTableTest.java
+++ b/flink-table-store-core/src/test/java/org/apache/flink/table/store/table/AppendOnlyFileStoreTableTest.java
@@ -21,8 +21,8 @@ package org.apache.flink.table.store.table;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.data.binary.BinaryRowData;
-import org.apache.flink.table.runtime.typeutils.RowDataSerializer;
 import org.apache.flink.table.store.CoreOptions;
+import org.apache.flink.table.store.data.RowDataSerializer;
 import org.apache.flink.table.store.file.WriteMode;
 import org.apache.flink.table.store.file.operation.ScanKind;
 import org.apache.flink.table.store.file.predicate.Predicate;
diff --git a/flink-table-store-dist/pom.xml b/flink-table-store-dist/pom.xml
index f0db602a..b9f57301 100644
--- a/flink-table-store-dist/pom.xml
+++ b/flink-table-store-dist/pom.xml
@@ -101,12 +101,6 @@ under the License.
                                 otherwise if user has the same format/connector jar in the classpath,
                                 FactoryUtil will complain about multiple matching factories.
                                 -->
-                                <filter>
-                                    <artifact>org.apache.flink:flink-connector-files</artifact>
-                                    <excludes>
-                                        <exclude>META-INF/services/**</exclude>
-                                    </excludes>
-                                </filter>
                                 <filter>
                                     <artifact>org.apache.flink:${flink.connector.kafka}</artifact>
                                     <excludes>
diff --git a/flink-table-store-format/pom.xml b/flink-table-store-format/pom.xml
index 6f256d0a..3ace2d3e 100644
--- a/flink-table-store-format/pom.xml
+++ b/flink-table-store-format/pom.xml
@@ -60,12 +60,6 @@ under the License.
             <scope>provided</scope>
         </dependency>
 
-        <dependency>
-            <groupId>org.apache.flink</groupId>
-            <artifactId>flink-connector-files</artifactId>
-            <version>${flink.version}</version>
-        </dependency>
-
         <!-- Hadoop -->
 
         <dependency>
diff --git a/flink-table-store-format/src/main/java/org/apache/flink/table/store/format/avro/AbstractAvroBulkFormat.java b/flink-table-store-format/src/main/java/org/apache/flink/table/store/format/avro/AbstractAvroBulkFormat.java
index ef232179..f915183b 100644
--- a/flink-table-store-format/src/main/java/org/apache/flink/table/store/format/avro/AbstractAvroBulkFormat.java
+++ b/flink-table-store-format/src/main/java/org/apache/flink/table/store/format/avro/AbstractAvroBulkFormat.java
@@ -19,14 +19,13 @@
 package org.apache.flink.table.store.format.avro;
 
 import org.apache.flink.annotation.Internal;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.connector.file.src.FileSourceSplit;
-import org.apache.flink.connector.file.src.reader.BulkFormat;
-import org.apache.flink.connector.file.src.util.CheckpointedPosition;
-import org.apache.flink.connector.file.src.util.IteratorResultIterator;
-import org.apache.flink.connector.file.src.util.Pool;
 import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.core.fs.Path;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.store.file.utils.IteratorResultIterator;
+import org.apache.flink.table.store.file.utils.RecordReader;
+import org.apache.flink.table.store.format.FormatReaderFactory;
+import org.apache.flink.table.store.utils.Pool;
 
 import org.apache.avro.Schema;
 import org.apache.avro.file.DataFileReader;
@@ -41,13 +40,12 @@ import java.util.Iterator;
 import java.util.function.Function;
 
 /**
- * Provides a {@link BulkFormat} for Avro records.
+ * Provides a {@link FormatReaderFactory} for Avro records.
  *
  * <p>NOTE: Copied from Flink.
  */
 @Internal
-public abstract class AbstractAvroBulkFormat<A, T, SplitT extends FileSourceSplit>
-        implements BulkFormat<T, SplitT> {
+public abstract class AbstractAvroBulkFormat<A> implements FormatReaderFactory {
 
     private static final long serialVersionUID = 1L;
 
@@ -58,51 +56,28 @@ public abstract class AbstractAvroBulkFormat<A, T, SplitT extends FileSourceSpli
     }
 
     @Override
-    public AvroReader createReader(Configuration config, SplitT split) throws IOException {
-        return createReader(split, createReusedAvroRecord(), createConverter());
+    public AvroReader createReader(Path file) throws IOException {
+        return createReader(file, createReusedAvroRecord(), createConverter());
     }
 
-    @Override
-    public AvroReader restoreReader(Configuration config, SplitT split) throws IOException {
-        return createReader(split, createReusedAvroRecord(), createConverter());
-    }
-
-    @Override
-    public boolean isSplittable() {
-        return true;
-    }
-
-    private AvroReader createReader(SplitT split, A reuse, Function<A, T> converter)
+    private AvroReader createReader(Path file, A reuse, Function<A, RowData> converter)
             throws IOException {
-        long end = split.offset() + split.length();
-        if (split.getReaderPosition().isPresent()) {
-            CheckpointedPosition position = split.getReaderPosition().get();
-            return new AvroReader(
-                    split.path(),
-                    split.offset(),
-                    end,
-                    position.getOffset(),
-                    position.getRecordsAfterOffset(),
-                    reuse,
-                    converter);
-        } else {
-            return new AvroReader(split.path(), split.offset(), end, -1, 0, reuse, converter);
-        }
+        long end = file.getFileSystem().getFileStatus(file).getLen();
+        return new AvroReader(file, 0, end, -1, 0, reuse, converter);
     }
 
     protected abstract A createReusedAvroRecord();
 
-    protected abstract Function<A, T> createConverter();
+    protected abstract Function<A, RowData> createConverter();
 
-    private class AvroReader implements BulkFormat.Reader<T> {
+    private class AvroReader implements RecordReader<RowData> {
 
         private final DataFileReader<A> reader;
-        private final Function<A, T> converter;
+        private final Function<A, RowData> converter;
 
         private final long end;
         private final Pool<A> pool;
 
-        private long currentBlockStart;
         private long currentRecordsToSkip;
 
         private AvroReader(
@@ -112,7 +87,7 @@ public abstract class AbstractAvroBulkFormat<A, T, SplitT extends FileSourceSpli
                 long blockStart,
                 long recordsToSkip,
                 A reuse,
-                Function<A, T> converter)
+                Function<A, RowData> converter)
                 throws IOException {
             this.reader = createReaderFromPath(path);
             if (blockStart >= 0) {
@@ -129,7 +104,6 @@ public abstract class AbstractAvroBulkFormat<A, T, SplitT extends FileSourceSpli
             this.pool = new Pool<>(1);
             this.pool.add(reuse);
 
-            this.currentBlockStart = reader.previousSync();
             this.currentRecordsToSkip = recordsToSkip;
         }
 
@@ -144,7 +118,7 @@ public abstract class AbstractAvroBulkFormat<A, T, SplitT extends FileSourceSpli
 
         @Nullable
         @Override
-        public RecordIterator<T> readBatch() throws IOException {
+        public RecordIterator<RowData> readBatch() throws IOException {
             A reuse;
             try {
                 reuse = pool.pollEntry();
@@ -159,20 +133,14 @@ public abstract class AbstractAvroBulkFormat<A, T, SplitT extends FileSourceSpli
                 return null;
             }
 
-            currentBlockStart = reader.previousSync();
-            Iterator<T> iterator =
+            Iterator<RowData> iterator =
                     new AvroBlockIterator(
                             reader.getBlockCount() - currentRecordsToSkip,
                             reader,
                             reuse,
                             converter);
-            long recordsToSkip = currentRecordsToSkip;
             currentRecordsToSkip = 0;
-            return new IteratorResultIterator<>(
-                    iterator,
-                    currentBlockStart,
-                    recordsToSkip,
-                    () -> pool.recycler().recycle(reuse));
+            return new IteratorResultIterator<>(iterator, () -> pool.recycler().recycle(reuse));
         }
 
         private boolean readNextBlock() throws IOException {
@@ -187,18 +155,18 @@ public abstract class AbstractAvroBulkFormat<A, T, SplitT extends FileSourceSpli
         }
     }
 
-    private class AvroBlockIterator implements Iterator<T> {
+    private class AvroBlockIterator implements Iterator<RowData> {
 
         private long numRecordsRemaining;
         private final DataFileReader<A> reader;
         private final A reuse;
-        private final Function<A, T> converter;
+        private final Function<A, RowData> converter;
 
         private AvroBlockIterator(
                 long numRecordsRemaining,
                 DataFileReader<A> reader,
                 A reuse,
-                Function<A, T> converter) {
+                Function<A, RowData> converter) {
             this.numRecordsRemaining = numRecordsRemaining;
             this.reader = reader;
             this.reuse = reuse;
@@ -211,7 +179,7 @@ public abstract class AbstractAvroBulkFormat<A, T, SplitT extends FileSourceSpli
         }
 
         @Override
-        public T next() {
+        public RowData next() {
             try {
                 numRecordsRemaining--;
                 // reader.next merely deserialize bytes in memory to java objects
diff --git a/flink-table-store-format/src/main/java/org/apache/flink/table/store/format/avro/AvroFileFormat.java b/flink-table-store-format/src/main/java/org/apache/flink/table/store/format/avro/AvroFileFormat.java
index 2b3e413c..8a908ae5 100644
--- a/flink-table-store-format/src/main/java/org/apache/flink/table/store/format/avro/AvroFileFormat.java
+++ b/flink-table-store-format/src/main/java/org/apache/flink/table/store/format/avro/AvroFileFormat.java
@@ -19,18 +19,15 @@
 package org.apache.flink.table.store.format.avro;
 
 import org.apache.flink.api.common.serialization.BulkWriter;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.configuration.ConfigOption;
 import org.apache.flink.configuration.ConfigOptions;
 import org.apache.flink.configuration.ReadableConfig;
-import org.apache.flink.connector.file.src.FileSourceSplit;
-import org.apache.flink.connector.file.src.reader.BulkFormat;
 import org.apache.flink.core.fs.FSDataOutputStream;
 import org.apache.flink.table.data.GenericRowData;
 import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
 import org.apache.flink.table.store.file.predicate.Predicate;
 import org.apache.flink.table.store.format.FileFormat;
+import org.apache.flink.table.store.format.FormatReaderFactory;
 import org.apache.flink.table.store.utils.Projection;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.RowType;
@@ -71,7 +68,7 @@ public class AvroFileFormat extends FileFormat {
     }
 
     @Override
-    public BulkFormat<RowData, FileSourceSplit> createReaderFactory(
+    public FormatReaderFactory createReaderFactory(
             RowType type, int[][] projection, @Nullable List<Predicate> filters) {
         // avro is a file format that keeps schemas in file headers,
         // if the schema given to the reader is not equal to the schema in header,
@@ -80,8 +77,7 @@ public class AvroFileFormat extends FileFormat {
         //
         // for detailed discussion see comments in https://github.com/apache/flink/pull/18657
         LogicalType producedType = Projection.of(projection).project(type);
-        return new AvroGenericRecordBulkFormat(
-                (RowType) producedType.copy(false), InternalTypeInfo.of(producedType));
+        return new AvroGenericRecordBulkFormat((RowType) producedType.copy(false));
     }
 
     @Override
@@ -89,19 +85,15 @@ public class AvroFileFormat extends FileFormat {
         return new RowDataAvroWriterFactory(type, formatOptions.get(AVRO_OUTPUT_CODEC));
     }
 
-    private static class AvroGenericRecordBulkFormat
-            extends AbstractAvroBulkFormat<GenericRecord, RowData, FileSourceSplit> {
+    private static class AvroGenericRecordBulkFormat extends AbstractAvroBulkFormat<GenericRecord> {
 
         private static final long serialVersionUID = 1L;
 
         private final RowType producedRowType;
-        private final TypeInformation<RowData> producedTypeInfo;
 
-        public AvroGenericRecordBulkFormat(
-                RowType producedRowType, TypeInformation<RowData> producedTypeInfo) {
+        public AvroGenericRecordBulkFormat(RowType producedRowType) {
             super(AvroSchemaConverter.convertToSchema(producedRowType));
             this.producedRowType = producedRowType;
-            this.producedTypeInfo = producedTypeInfo;
         }
 
         @Override
@@ -115,11 +107,6 @@ public class AvroFileFormat extends FileFormat {
                     AvroToRowDataConverters.createRowConverter(producedRowType);
             return record -> record == null ? null : (GenericRowData) converter.convert(record);
         }
-
-        @Override
-        public TypeInformation<RowData> getProducedType() {
-            return producedTypeInfo;
-        }
     }
 
     /**
diff --git a/flink-table-store-format/src/main/java/org/apache/flink/table/store/format/orc/OrcFileFormat.java b/flink-table-store-format/src/main/java/org/apache/flink/table/store/format/orc/OrcFileFormat.java
index bf593a56..a53fbbc2 100644
--- a/flink-table-store-format/src/main/java/org/apache/flink/table/store/format/orc/OrcFileFormat.java
+++ b/flink-table-store-format/src/main/java/org/apache/flink/table/store/format/orc/OrcFileFormat.java
@@ -22,13 +22,12 @@ import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.serialization.BulkWriter;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.ReadableConfig;
-import org.apache.flink.connector.file.src.FileSourceSplit;
-import org.apache.flink.connector.file.src.reader.BulkFormat;
 import org.apache.flink.table.api.DataTypes;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.store.file.predicate.Predicate;
 import org.apache.flink.table.store.format.FileFormat;
 import org.apache.flink.table.store.format.FileStatsExtractor;
+import org.apache.flink.table.store.format.FormatReaderFactory;
 import org.apache.flink.table.store.format.orc.filter.OrcFileStatsExtractor;
 import org.apache.flink.table.store.format.orc.filter.OrcFilters;
 import org.apache.flink.table.store.format.orc.filter.OrcPredicateFunctionVisitor;
@@ -81,7 +80,7 @@ public class OrcFileFormat extends FileFormat {
     }
 
     @Override
-    public BulkFormat<RowData, FileSourceSplit> createReaderFactory(
+    public FormatReaderFactory createReaderFactory(
             RowType type, int[][] projection, @Nullable List<Predicate> filters) {
         List<OrcFilters.Predicate> orcPredicates = new ArrayList<>();
 
diff --git a/flink-table-store-format/src/main/java/org/apache/flink/table/store/format/orc/OrcReaderFactory.java b/flink-table-store-format/src/main/java/org/apache/flink/table/store/format/orc/OrcReaderFactory.java
index 5ce6a835..44e80245 100644
--- a/flink-table-store-format/src/main/java/org/apache/flink/table/store/format/orc/OrcReaderFactory.java
+++ b/flink-table-store-format/src/main/java/org/apache/flink/table/store/format/orc/OrcReaderFactory.java
@@ -18,20 +18,18 @@
 
 package org.apache.flink.table.store.format.orc;
 
-import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.connector.base.source.reader.SourceReaderOptions;
-import org.apache.flink.connector.file.src.FileSourceSplit;
-import org.apache.flink.connector.file.src.reader.BulkFormat;
-import org.apache.flink.connector.file.src.util.Pool;
-import org.apache.flink.connector.file.src.util.RecordAndPosition;
+import org.apache.flink.core.fs.Path;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.store.data.columnar.ColumnVector;
 import org.apache.flink.table.store.data.columnar.ColumnarRowData;
 import org.apache.flink.table.store.data.columnar.ColumnarRowIterator;
 import org.apache.flink.table.store.data.columnar.VectorizedColumnBatch;
+import org.apache.flink.table.store.file.utils.RecordReader.RecordIterator;
+import org.apache.flink.table.store.format.FormatReaderFactory;
 import org.apache.flink.table.store.format.fs.HadoopReadOnlyFileSystem;
 import org.apache.flink.table.store.format.orc.filter.OrcFilters;
+import org.apache.flink.table.store.utils.Pool;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.RowType;
 
@@ -54,7 +52,7 @@ import static org.apache.flink.table.store.format.orc.reader.OrcSplitReaderUtil.
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /** An ORC reader that produces a stream of {@link ColumnarRowData} records. */
-public class OrcReaderFactory implements BulkFormat<RowData, FileSourceSplit> {
+public class OrcReaderFactory implements FormatReaderFactory {
 
     private static final long serialVersionUID = 1L;
 
@@ -93,39 +91,21 @@ public class OrcReaderFactory implements BulkFormat<RowData, FileSourceSplit> {
     // ------------------------------------------------------------------------
 
     @Override
-    public OrcVectorizedReader createReader(
-            final org.apache.flink.configuration.Configuration config, final FileSourceSplit split)
-            throws IOException {
-
-        final int numBatchesToCirculate =
-                config.getInteger(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY);
-        final Pool<OrcReaderBatch> poolOfBatches = createPoolOfBatches(numBatchesToCirculate);
-
-        final RecordReader orcReader =
+    public OrcVectorizedReader createReader(Path file) throws IOException {
+        Pool<OrcReaderBatch> poolOfBatches = createPoolOfBatches(1);
+        RecordReader orcReader =
                 createRecordReader(
                         hadoopConfigWrapper.getHadoopConfig(),
                         schema,
                         selectedFields,
                         conjunctPredicates,
-                        split.path(),
-                        split.offset(),
-                        split.length());
+                        file,
+                        0,
+                        file.getFileSystem().getFileStatus(file).getLen());
 
         return new OrcVectorizedReader(orcReader, poolOfBatches);
     }
 
-    @Override
-    public OrcVectorizedReader restoreReader(
-            final org.apache.flink.configuration.Configuration config,
-            final FileSourceSplit split) {
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public boolean isSplittable() {
-        return true;
-    }
-
     /**
      * Creates the {@link OrcReaderBatch} structure, which is responsible for holding the data
      * structures that hold the batch data (column vectors, row arrays, ...) and the batch
@@ -147,12 +127,6 @@ public class OrcReaderFactory implements BulkFormat<RowData, FileSourceSplit> {
         return new OrcReaderBatch(orcBatch, flinkColumnBatch, recycler);
     }
 
-    /** Gets the type produced by this format. */
-    @Override
-    public TypeInformation<RowData> getProducedType() {
-        throw new UnsupportedOperationException();
-    }
-
     // ------------------------------------------------------------------------
 
     private Pool<OrcReaderBatch> createPoolOfBatches(int numBatches) {
@@ -169,16 +143,7 @@ public class OrcReaderFactory implements BulkFormat<RowData, FileSourceSplit> {
 
     // ------------------------------------------------------------------------
 
-    /**
-     * The {@code OrcReaderBatch} class holds the data structures containing the batch data (column
-     * vectors, row arrays, ...) and performs the batch conversion from the ORC representation to
-     * the result format.
-     *
-     * <p>This base class only holds the ORC Column Vectors, subclasses hold additionally the result
-     * structures and implement the conversion in {@link
-     * OrcReaderBatch#convertAndGetIterator(VectorizedRowBatch, long)}.
-     */
-    protected static class OrcReaderBatch {
+    private static class OrcReaderBatch {
 
         private final VectorizedRowBatch orcVectorizedRowBatch;
         private final Pool.Recycler<OrcReaderBatch> recycler;
@@ -210,26 +175,12 @@ public class OrcReaderFactory implements BulkFormat<RowData, FileSourceSplit> {
             return orcVectorizedRowBatch;
         }
 
-        /**
-         * Converts the ORC VectorizedRowBatch into the result structure and returns an iterator
-         * over the entries.
-         *
-         * <p>This method may, for example, return a single element iterator that returns the entire
-         * batch as one, or (as another example) return an iterator over the rows projected from
-         * this column batch.
-         *
-         * <p>The position information in the result needs to be constructed as follows: The value
-         * of {@code startingOffset} is the offset value ({@link RecordAndPosition#getOffset()}) for
-         * all rows in the batch. Each row then increments the records-to-skip value ({@link
-         * RecordAndPosition#getRecordSkipCount()}).
-         */
-        private RecordIterator<RowData> convertAndGetIterator(
-                final VectorizedRowBatch orcBatch, final long startingOffset) {
+        private RecordIterator<RowData> convertAndGetIterator(VectorizedRowBatch orcBatch) {
             // no copying from the ORC column vectors to the Flink columns vectors necessary,
             // because they point to the same data arrays internally design
             int batchSize = orcBatch.size;
             flinkColumnBatch.setNumRows(batchSize);
-            result.set(batchSize, startingOffset, 0);
+            result.set(batchSize);
             return result;
         }
     }
@@ -249,7 +200,8 @@ public class OrcReaderFactory implements BulkFormat<RowData, FileSourceSplit> {
      * batch is addressed by the starting row number of the batch, plus the number of records to be
      * skipped before.
      */
-    private static final class OrcVectorizedReader implements BulkFormat.Reader<RowData> {
+    private static final class OrcVectorizedReader
+            implements org.apache.flink.table.store.file.utils.RecordReader<RowData> {
 
         private final RecordReader orcReader;
         private final Pool<OrcReaderBatch> pool;
@@ -265,13 +217,12 @@ public class OrcReaderFactory implements BulkFormat<RowData, FileSourceSplit> {
             final OrcReaderBatch batch = getCachedEntry();
             final VectorizedRowBatch orcVectorBatch = batch.orcVectorizedRowBatch();
 
-            final long orcRowNumber = orcReader.getRowNumber();
             if (!nextBatch(orcReader, orcVectorBatch)) {
                 batch.recycle();
                 return null;
             }
 
-            return batch.convertAndGetIterator(orcVectorBatch, orcRowNumber);
+            return batch.convertAndGetIterator(orcVectorBatch);
         }
 
         @Override
diff --git a/flink-table-store-format/src/main/java/org/apache/flink/table/store/format/parquet/ParquetFileFormat.java b/flink-table-store-format/src/main/java/org/apache/flink/table/store/format/parquet/ParquetFileFormat.java
index 266039e7..c5d8d9ca 100644
--- a/flink-table-store-format/src/main/java/org/apache/flink/table/store/format/parquet/ParquetFileFormat.java
+++ b/flink-table-store-format/src/main/java/org/apache/flink/table/store/format/parquet/ParquetFileFormat.java
@@ -21,12 +21,11 @@ package org.apache.flink.table.store.format.parquet;
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.serialization.BulkWriter;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.connector.file.src.FileSourceSplit;
-import org.apache.flink.connector.file.src.reader.BulkFormat;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.store.file.predicate.Predicate;
 import org.apache.flink.table.store.format.FileFormat;
 import org.apache.flink.table.store.format.FileStatsExtractor;
+import org.apache.flink.table.store.format.FormatReaderFactory;
 import org.apache.flink.table.store.format.parquet.writer.RowDataParquetBuilder;
 import org.apache.flink.table.store.utils.Projection;
 import org.apache.flink.table.types.logical.RowType;
@@ -52,7 +51,7 @@ public class ParquetFileFormat extends FileFormat {
     }
 
     @Override
-    public BulkFormat<RowData, FileSourceSplit> createReaderFactory(
+    public FormatReaderFactory createReaderFactory(
             RowType type, int[][] projection, List<Predicate> filters) {
         return new ParquetReaderFactory(
                 getParquetConfiguration(formatOptions),
diff --git a/flink-table-store-format/src/main/java/org/apache/flink/table/store/format/parquet/ParquetInputFormatFactory.java b/flink-table-store-format/src/main/java/org/apache/flink/table/store/format/parquet/ParquetInputFormatFactory.java
deleted file mode 100644
index c4260226..00000000
--- a/flink-table-store-format/src/main/java/org/apache/flink/table/store/format/parquet/ParquetInputFormatFactory.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * 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.table.store.format.parquet;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.connector.file.src.FileSourceSplit;
-import org.apache.flink.connector.file.src.reader.BulkFormat;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.store.utils.ReflectionUtils;
-import org.apache.flink.table.types.logical.RowType;
-
-import org.apache.hadoop.conf.Configuration;
-
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.util.Arrays;
-import java.util.Collections;
-
-/** Factory to create parquet input format for different Flink versions. */
-public class ParquetInputFormatFactory {
-
-    public static BulkFormat<RowData, FileSourceSplit> create(
-            Configuration conf,
-            RowType producedRowType,
-            TypeInformation<RowData> producedTypeInfo,
-            boolean isUtcTimestamp) {
-        Class<?> formatClass;
-        try {
-            formatClass =
-                    Class.forName("org.apache.flink.formats.parquet.ParquetColumnarRowInputFormat");
-
-            Method method =
-                    Arrays.stream(formatClass.getDeclaredMethods())
-                            .filter(m -> "createPartitionedFormat".equals(m.getName()))
-                            .findAny()
-                            .orElseThrow(NoSuchMethodException::new);
-            int paramCnt = method.getParameterCount();
-            return paramCnt == 8
-                    ? createFrom115(method, conf, producedRowType, producedTypeInfo, isUtcTimestamp)
-                    : createFrom114(method, conf, producedRowType, isUtcTimestamp);
-        } catch (ClassNotFoundException | NoSuchMethodException e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    private static BulkFormat<RowData, FileSourceSplit> createFrom115(
-            Method method,
-            Configuration conf,
-            RowType producedRowType,
-            TypeInformation<RowData> producedTypeInfo,
-            boolean isUtcTimestamp)
-            throws NoSuchMethodException {
-        try {
-            return ReflectionUtils.invokeStaticMethod(
-                    method,
-                    conf,
-                    producedRowType,
-                    producedTypeInfo,
-                    Collections.emptyList(),
-                    null,
-                    2048,
-                    isUtcTimestamp,
-                    true);
-        } catch (InvocationTargetException | IllegalAccessException e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    private static BulkFormat<RowData, FileSourceSplit> createFrom114(
-            Method method, Configuration conf, RowType producedRowType, boolean isUtcTimestamp)
-            throws NoSuchMethodException {
-        try {
-            return ReflectionUtils.invokeStaticMethod(
-                    method,
-                    conf,
-                    producedRowType,
-                    Collections.emptyList(),
-                    null,
-                    2048,
-                    isUtcTimestamp,
-                    true);
-        } catch (InvocationTargetException | IllegalAccessException e) {
-            throw new RuntimeException(e);
-        }
-    }
-}
diff --git a/flink-table-store-format/src/main/java/org/apache/flink/table/store/format/parquet/ParquetReaderFactory.java b/flink-table-store-format/src/main/java/org/apache/flink/table/store/format/parquet/ParquetReaderFactory.java
index c52824fa..b9f0ca29 100644
--- a/flink-table-store-format/src/main/java/org/apache/flink/table/store/format/parquet/ParquetReaderFactory.java
+++ b/flink-table-store-format/src/main/java/org/apache/flink/table/store/format/parquet/ParquetReaderFactory.java
@@ -18,11 +18,7 @@
 
 package org.apache.flink.table.store.format.parquet;
 
-import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.connector.file.src.FileSourceSplit;
-import org.apache.flink.connector.file.src.reader.BulkFormat;
-import org.apache.flink.connector.file.src.util.Pool;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.store.data.columnar.ColumnVector;
@@ -30,8 +26,12 @@ import org.apache.flink.table.store.data.columnar.ColumnarRowData;
 import org.apache.flink.table.store.data.columnar.ColumnarRowIterator;
 import org.apache.flink.table.store.data.columnar.VectorizedColumnBatch;
 import org.apache.flink.table.store.data.columnar.writable.WritableColumnVector;
+import org.apache.flink.table.store.file.utils.RecordReader;
+import org.apache.flink.table.store.file.utils.RecordReader.RecordIterator;
+import org.apache.flink.table.store.format.FormatReaderFactory;
 import org.apache.flink.table.store.format.parquet.reader.ColumnReader;
 import org.apache.flink.table.store.format.parquet.reader.ParquetDecimalVector;
+import org.apache.flink.table.store.utils.Pool;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.LogicalTypeRoot;
 import org.apache.flink.table.types.logical.RowType;
@@ -51,7 +51,6 @@ import org.slf4j.LoggerFactory;
 import javax.annotation.Nullable;
 
 import java.io.IOException;
-import java.io.UnsupportedEncodingException;
 import java.util.Arrays;
 import java.util.HashSet;
 import java.util.List;
@@ -62,10 +61,10 @@ import static org.apache.flink.table.store.format.parquet.reader.ParquetSplitRea
 import static org.apache.parquet.hadoop.UnmaterializableRecordCounter.BAD_RECORD_THRESHOLD_CONF_KEY;
 
 /**
- * Parquet {@link BulkFormat} that reads data from the file to {@link VectorizedColumnBatch} in
- * vectorized mode.
+ * Parquet {@link FormatReaderFactory} that reads data from the file to {@link
+ * VectorizedColumnBatch} in vectorized mode.
  */
-public class ParquetReaderFactory implements BulkFormat<RowData, FileSourceSplit> {
+public class ParquetReaderFactory implements FormatReaderFactory {
 
     private static final Logger LOG = LoggerFactory.getLogger(ParquetReaderFactory.class);
 
@@ -93,13 +92,9 @@ public class ParquetReaderFactory implements BulkFormat<RowData, FileSourceSplit
     }
 
     @Override
-    public ParquetReader createReader(
-            final org.apache.flink.configuration.Configuration config, final FileSourceSplit split)
-            throws IOException {
-
-        final Path filePath = split.path();
-        final long splitOffset = split.offset();
-        final long splitLength = split.length();
+    public ParquetReader createReader(Path filePath) throws IOException {
+        final long splitOffset = 0;
+        final long splitLength = filePath.getFileSystem().getFileStatus(filePath).getLen();
 
         ParquetReadOptions.Builder builder =
                 ParquetReadOptions.builder().withRange(splitOffset, splitOffset + splitLength);
@@ -137,18 +132,6 @@ public class ParquetReaderFactory implements BulkFormat<RowData, FileSourceSplit
         }
     }
 
-    @Override
-    public ParquetReader restoreReader(
-            final org.apache.flink.configuration.Configuration config, final FileSourceSplit split)
-            throws IOException {
-        throw new UnsupportedEncodingException();
-    }
-
-    @Override
-    public boolean isSplittable() {
-        return true;
-    }
-
     /** Clips `parquetSchema` according to `fieldNames`. */
     private MessageType clipParquetSchema(GroupType parquetSchema) {
         Type[] types = new Type[projectedFields.length];
@@ -247,7 +230,7 @@ public class ParquetReaderFactory implements BulkFormat<RowData, FileSourceSplit
         return new VectorizedColumnBatch(vectors);
     }
 
-    private class ParquetReader implements BulkFormat.Reader<RowData> {
+    private class ParquetReader implements RecordReader<RowData> {
 
         private ParquetFileReader reader;
 
@@ -292,13 +275,12 @@ public class ParquetReaderFactory implements BulkFormat<RowData, FileSourceSplit
         public RecordIterator<RowData> readBatch() throws IOException {
             final ParquetReaderBatch batch = getCachedEntry();
 
-            final long rowsReturnedBefore = rowsReturned;
             if (!nextBatch(batch)) {
                 batch.recycle();
                 return null;
             }
 
-            return batch.convertAndGetIterator(rowsReturnedBefore);
+            return batch.convertAndGetIterator();
         }
 
         /** Advances to the next batch of rows. Returns false if there are no more. */
@@ -379,10 +361,6 @@ public class ParquetReaderFactory implements BulkFormat<RowData, FileSourceSplit
         return new ParquetReaderBatch(writableVectors, columnarBatch, recycler);
     }
 
-    /**
-     * Reader batch that provides writing and reading capabilities. Provides {@link RecordIterator}
-     * reading interface from {@link #convertAndGetIterator(long)}.
-     */
     private static class ParquetReaderBatch {
 
         private final WritableColumnVector[] writableVectors;
@@ -406,14 +384,9 @@ public class ParquetReaderFactory implements BulkFormat<RowData, FileSourceSplit
             recycler.recycle(this);
         }
 
-        public RecordIterator<RowData> convertAndGetIterator(long rowsReturned) {
-            result.set(columnarBatch.getNumRows(), rowsReturned);
+        public RecordIterator<RowData> convertAndGetIterator() {
+            result.set(columnarBatch.getNumRows());
             return result;
         }
     }
-
-    @Override
-    public TypeInformation<RowData> getProducedType() {
-        throw new UnsupportedOperationException();
-    }
 }
diff --git a/flink-table-store-format/src/main/resources/META-INF/NOTICE b/flink-table-store-format/src/main/resources/META-INF/NOTICE
index 89edd908..d98fe55d 100644
--- a/flink-table-store-format/src/main/resources/META-INF/NOTICE
+++ b/flink-table-store-format/src/main/resources/META-INF/NOTICE
@@ -6,6 +6,12 @@ The Apache Software Foundation (http://www.apache.org/).
 
 This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt)
 
+- org.apache.orc:orc-core:1.5.6
+- org.apache.orc:orc-shims:1.5.6
+- org.apache.hive:hive-storage-api:2.6.0
+- io.airlift:aircompressor:0.10
+- commons-lang:commons-lang:2.6
+
 - org.apache.avro:avro:1.11.1
 - com.fasterxml.jackson.core:jackson-core:2.13.4
 - com.fasterxml.jackson.core:jackson-databind:2.13.4.2
@@ -20,3 +26,8 @@ This project bundles the following dependencies under the Apache Software Licens
 - org.apache.parquet:parquet-format-structures:1.12.3
 - org.apache.parquet:parquet-jackson:1.12.3
 - commons-pool:commons-pool:1.6
+
+This project bundles the following dependencies under the BSD license.
+See bundled license files for details.
+
+- com.google.protobuf:protobuf-java:2.5.0
diff --git a/flink-table-store-format/src/test/java/org/apache/flink/table/store/format/BulkFileFormatTest.java b/flink-table-store-format/src/test/java/org/apache/flink/table/store/format/BulkFileFormatTest.java
index eefcec76..0dfa45ea 100644
--- a/flink-table-store-format/src/test/java/org/apache/flink/table/store/format/BulkFileFormatTest.java
+++ b/flink-table-store-format/src/test/java/org/apache/flink/table/store/format/BulkFileFormatTest.java
@@ -20,14 +20,13 @@ package org.apache.flink.table.store.format;
 
 import org.apache.flink.api.common.serialization.BulkWriter;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.connector.file.src.FileSourceSplit;
-import org.apache.flink.connector.file.src.reader.BulkFormat;
-import org.apache.flink.connector.file.src.util.Utils;
 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.table.data.GenericRowData;
 import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.store.file.utils.RecordReader;
+import org.apache.flink.table.store.file.utils.RecordReaderUtils;
 import org.apache.flink.table.types.logical.IntType;
 import org.apache.flink.table.types.logical.RowType;
 
@@ -81,14 +80,9 @@ public class BulkFileFormatTest {
         out.close();
 
         // read
-        BulkFormat.Reader<RowData> reader =
-                fileFormat
-                        .createReaderFactory(rowType)
-                        .createReader(
-                                new Configuration(),
-                                new FileSourceSplit("", path, 0, fs.getFileStatus(path).getLen()));
+        RecordReader<RowData> reader = fileFormat.createReaderFactory(rowType).createReader(path);
         List<RowData> result = new ArrayList<>();
-        Utils.forEachRemaining(
+        RecordReaderUtils.forEachRemaining(
                 reader,
                 rowData -> result.add(GenericRowData.of(rowData.getInt(0), rowData.getInt(0))));
 
diff --git a/flink-table-store-format/src/test/java/org/apache/flink/table/store/format/avro/AvroBulkFormatTest.java b/flink-table-store-format/src/test/java/org/apache/flink/table/store/format/avro/AvroBulkFormatTest.java
index c6ef91de..3006eea6 100644
--- a/flink-table-store-format/src/test/java/org/apache/flink/table/store/format/avro/AvroBulkFormatTest.java
+++ b/flink-table-store-format/src/test/java/org/apache/flink/table/store/format/avro/AvroBulkFormatTest.java
@@ -18,20 +18,15 @@
 
 package org.apache.flink.table.store.format.avro;
 
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.connector.file.src.FileSourceSplit;
-import org.apache.flink.connector.file.src.reader.BulkFormat;
-import org.apache.flink.connector.file.src.util.CheckpointedPosition;
-import org.apache.flink.connector.file.src.util.RecordAndPosition;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.table.data.GenericRowData;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.store.file.utils.RecordReader;
+import org.apache.flink.table.store.file.utils.RecordReaderUtils;
 import org.apache.flink.util.FileUtils;
-import org.apache.flink.util.StringUtils;
 
 import org.apache.avro.Schema;
-import org.apache.avro.file.DataFileConstants;
 import org.apache.avro.file.DataFileWriter;
 import org.apache.avro.generic.GenericDatumWriter;
 import org.apache.avro.generic.GenericRecord;
@@ -44,11 +39,9 @@ import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.nio.file.Files;
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.List;
-import java.util.UUID;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import static org.apache.flink.table.store.format.avro.AvroBulkFormatTestUtils.ROW_TYPE;
 import static org.assertj.core.api.Assertions.assertThat;
@@ -128,157 +121,10 @@ class AvroBulkFormatTest {
     void testReadWholeFileWithOneSplit() throws IOException {
         AvroBulkFormatTestUtils.TestingAvroBulkFormat bulkFormat =
                 new AvroBulkFormatTestUtils.TestingAvroBulkFormat();
-        assertSplit(
-                bulkFormat,
-                Collections.singletonList(
-                        new SplitInfo(
-                                0,
-                                tmpFile.length(),
-                                Arrays.asList(
-                                        new BatchInfo(0, 3),
-                                        new BatchInfo(3, 5),
-                                        new BatchInfo(5, 6)))));
-    }
-
-    @Test
-    void testReadWholeFileWithMultipleSplits() throws IOException {
-        AvroBulkFormatTestUtils.TestingAvroBulkFormat bulkFormat =
-                new AvroBulkFormatTestUtils.TestingAvroBulkFormat();
-        long splitLength = tmpFile.length() / 3;
-        assertSplit(
-                bulkFormat,
-                Arrays.asList(
-                        new SplitInfo(
-                                0, splitLength, Collections.singletonList(new BatchInfo(0, 3))),
-                        new SplitInfo(splitLength, splitLength * 2, Collections.emptyList()),
-                        new SplitInfo(
-                                splitLength * 2,
-                                tmpFile.length(),
-                                Arrays.asList(new BatchInfo(3, 5), new BatchInfo(5, 6)))));
-    }
-
-    @Test
-    void testSplitsAtCriticalLocations() throws IOException {
-        AvroBulkFormatTestUtils.TestingAvroBulkFormat bulkFormat =
-                new AvroBulkFormatTestUtils.TestingAvroBulkFormat();
-        assertSplit(
-                bulkFormat,
-                Arrays.asList(
-                        // ends just before the new block
-                        new SplitInfo(
-                                BLOCK_STARTS.get(0) - DataFileConstants.SYNC_SIZE,
-                                BLOCK_STARTS.get(1) - DataFileConstants.SYNC_SIZE,
-                                Collections.singletonList(new BatchInfo(0, 3))),
-                        // ends just at the beginning of new block
-                        new SplitInfo(
-                                BLOCK_STARTS.get(1) - DataFileConstants.SYNC_SIZE,
-                                BLOCK_STARTS.get(2) - DataFileConstants.SYNC_SIZE + 1,
-                                Arrays.asList(new BatchInfo(3, 5), new BatchInfo(5, 6)))));
-    }
-
-    @Test
-    void testRestoreReader() throws IOException {
-        AvroBulkFormatTestUtils.TestingAvroBulkFormat bulkFormat =
-                new AvroBulkFormatTestUtils.TestingAvroBulkFormat();
-        long splitLength = tmpFile.length() / 3;
-        String splitId = UUID.randomUUID().toString();
-
-        FileSourceSplit split =
-                new FileSourceSplit(
-                        splitId, new Path(tmpFile.toString()), splitLength * 2, tmpFile.length());
-        BulkFormat.Reader<RowData> reader = bulkFormat.createReader(new Configuration(), split);
-        long offset1 = assertBatch(reader, new BatchInfo(3, 5));
-        assertBatch(reader, new BatchInfo(5, 6));
-        assertThat(reader.readBatch()).isNull();
-        reader.close();
-
-        split =
-                new FileSourceSplit(
-                        splitId,
-                        new Path(tmpFile.toString()),
-                        splitLength * 2,
-                        tmpFile.length(),
-                        StringUtils.EMPTY_STRING_ARRAY,
-                        new CheckpointedPosition(offset1, 1));
-        reader = bulkFormat.restoreReader(new Configuration(), split);
-        long offset2 = assertBatch(reader, new BatchInfo(3, 5), 1);
-        assertBatch(reader, new BatchInfo(5, 6));
-        assertThat(reader.readBatch()).isNull();
-        reader.close();
-
-        assertThat(offset2).isEqualTo(offset1);
-    }
-
-    private void assertSplit(
-            AvroBulkFormatTestUtils.TestingAvroBulkFormat bulkFormat, List<SplitInfo> splitInfos)
-            throws IOException {
-        for (SplitInfo splitInfo : splitInfos) {
-            FileSourceSplit split =
-                    new FileSourceSplit(
-                            UUID.randomUUID().toString(),
-                            new Path(tmpFile.toString()),
-                            splitInfo.start,
-                            splitInfo.end - splitInfo.start);
-            BulkFormat.Reader<RowData> reader = bulkFormat.createReader(new Configuration(), split);
-            List<Long> offsets = new ArrayList<>();
-            for (BatchInfo batch : splitInfo.batches) {
-                offsets.add(assertBatch(reader, batch));
-            }
-            assertThat(reader.readBatch()).isNull();
-            for (int j = 1; j < offsets.size(); j++) {
-                assertThat(offsets.get(j - 1) < offsets.get(j)).isTrue();
-            }
-            reader.close();
-        }
-    }
-
-    private long assertBatch(BulkFormat.Reader<RowData> reader, BatchInfo batchInfo)
-            throws IOException {
-        return assertBatch(reader, batchInfo, 0);
-    }
-
-    private long assertBatch(
-            BulkFormat.Reader<RowData> reader, BatchInfo batchInfo, int initialSkipCount)
-            throws IOException {
-        long ret = -1;
-        int skipCount = initialSkipCount;
-        BulkFormat.RecordIterator<RowData> iterator = reader.readBatch();
-        for (RecordAndPosition<RowData> recordAndPos = iterator.next();
-                recordAndPos != null;
-                recordAndPos = iterator.next()) {
-            if (ret == -1) {
-                ret = recordAndPos.getOffset();
-            }
-            assertThat(recordAndPos.getRecord())
-                    .isEqualTo(TEST_DATA.get(batchInfo.start + skipCount));
-            assertThat(recordAndPos.getOffset()).isEqualTo(ret);
-            skipCount++;
-            assertThat(recordAndPos.getRecordSkipCount()).isEqualTo(skipCount);
-        }
-        assertThat(skipCount).isEqualTo(batchInfo.end - batchInfo.start);
-        iterator.releaseBatch();
-        return ret;
-    }
-
-    private static class SplitInfo {
-        private final long start;
-        private final long end;
-        private final List<BatchInfo> batches;
-
-        private SplitInfo(long start, long end, List<BatchInfo> batches) {
-            this.start = start;
-            this.end = end;
-            this.batches = batches;
-        }
-    }
-
-    private static class BatchInfo {
-        private final int start;
-        private final int end;
-
-        private BatchInfo(int start, int end) {
-            this.start = start;
-            this.end = end;
-        }
+        RecordReader<RowData> reader = bulkFormat.createReader(new Path(tmpFile.toString()));
+        AtomicInteger i = new AtomicInteger(0);
+        RecordReaderUtils.forEachRemaining(
+                reader,
+                rowData -> assertThat(rowData).isEqualTo(TEST_DATA.get(i.getAndIncrement())));
     }
 }
diff --git a/flink-table-store-format/src/test/java/org/apache/flink/table/store/format/avro/AvroBulkFormatTestUtils.java b/flink-table-store-format/src/test/java/org/apache/flink/table/store/format/avro/AvroBulkFormatTestUtils.java
index 57279a11..08cedf35 100644
--- a/flink-table-store-format/src/test/java/org/apache/flink/table/store/format/avro/AvroBulkFormatTestUtils.java
+++ b/flink-table-store-format/src/test/java/org/apache/flink/table/store/format/avro/AvroBulkFormatTestUtils.java
@@ -18,11 +18,8 @@
 
 package org.apache.flink.table.store.format.avro;
 
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.connector.file.src.FileSourceSplit;
 import org.apache.flink.table.api.DataTypes;
 import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.RowType;
 
@@ -43,8 +40,7 @@ public class AvroBulkFormatTestUtils {
                     new String[] {"a", "b"});
 
     /** {@link AbstractAvroBulkFormat} for tests. */
-    public static class TestingAvroBulkFormat
-            extends AbstractAvroBulkFormat<GenericRecord, RowData, FileSourceSplit> {
+    public static class TestingAvroBulkFormat extends AbstractAvroBulkFormat<GenericRecord> {
 
         protected TestingAvroBulkFormat() {
             super(AvroSchemaConverter.convertToSchema(ROW_TYPE));
@@ -61,10 +57,5 @@ public class AvroBulkFormatTestUtils {
                     AvroToRowDataConverters.createRowConverter(ROW_TYPE);
             return record -> record == null ? null : (RowData) converter.convert(record);
         }
-
-        @Override
-        public TypeInformation<RowData> getProducedType() {
-            return InternalTypeInfo.of(ROW_TYPE);
-        }
     }
 }
diff --git a/flink-table-store-format/src/test/java/org/apache/flink/table/store/format/orc/OrcReaderFactoryTest.java b/flink-table-store-format/src/test/java/org/apache/flink/table/store/format/orc/OrcReaderFactoryTest.java
index 41a61684..1d425504 100644
--- a/flink-table-store-format/src/test/java/org/apache/flink/table/store/format/orc/OrcReaderFactoryTest.java
+++ b/flink-table-store-format/src/test/java/org/apache/flink/table/store/format/orc/OrcReaderFactoryTest.java
@@ -18,21 +18,18 @@
 
 package org.apache.flink.table.store.format.orc;
 
-import org.apache.flink.connector.file.src.FileSourceSplit;
-import org.apache.flink.connector.file.src.reader.BulkFormat;
-import org.apache.flink.connector.file.src.util.Utils;
-import org.apache.flink.core.fs.FileStatus;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.table.api.DataTypes;
 import org.apache.flink.table.data.DecimalDataUtils;
 import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.store.file.utils.RecordReader;
+import org.apache.flink.table.store.file.utils.RecordReaderUtils;
 import org.apache.flink.table.store.format.orc.filter.OrcFilters;
 import org.apache.flink.table.types.logical.DecimalType;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.RowType;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
 import org.junit.jupiter.api.BeforeAll;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.io.TempDir;
@@ -41,7 +38,6 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.nio.file.Files;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
@@ -94,19 +90,16 @@ class OrcReaderFactoryTest {
         AtomicInteger cnt = new AtomicInteger(0);
         AtomicLong totalF0 = new AtomicLong(0);
 
-        // read all splits
-        for (FileSourceSplit split : createSplits(flatFile, 4)) {
-            forEach(
-                    format,
-                    split,
-                    row -> {
-                        assertThat(row.isNullAt(0)).isFalse();
-                        assertThat(row.isNullAt(1)).isFalse();
-                        totalF0.addAndGet(row.getInt(0));
-                        assertThat(row.getString(1).toString()).isNotNull();
-                        cnt.incrementAndGet();
-                    });
-        }
+        forEach(
+                format,
+                flatFile,
+                row -> {
+                    assertThat(row.isNullAt(0)).isFalse();
+                    assertThat(row.isNullAt(1)).isFalse();
+                    totalF0.addAndGet(row.getInt(0));
+                    assertThat(row.getString(1).toString()).isNotNull();
+                    cnt.incrementAndGet();
+                });
 
         // check that all rows have been read
         assertThat(cnt.get()).isEqualTo(1920800);
@@ -120,21 +113,18 @@ class OrcReaderFactoryTest {
         AtomicInteger cnt = new AtomicInteger(0);
         AtomicLong totalF0 = new AtomicLong(0);
 
-        // read all splits
-        for (FileSourceSplit split : createSplits(flatFile, 4)) {
-            forEach(
-                    format,
-                    split,
-                    row -> {
-                        assertThat(row.isNullAt(0)).isFalse();
-                        assertThat(row.isNullAt(1)).isFalse();
-                        assertThat(row.isNullAt(2)).isFalse();
-                        assertThat(row.getString(0).toString()).isNotNull();
-                        totalF0.addAndGet(row.getInt(1));
-                        assertThat(row.getString(2).toString()).isNotNull();
-                        cnt.incrementAndGet();
-                    });
-        }
+        forEach(
+                format,
+                flatFile,
+                row -> {
+                    assertThat(row.isNullAt(0)).isFalse();
+                    assertThat(row.isNullAt(1)).isFalse();
+                    assertThat(row.isNullAt(2)).isFalse();
+                    assertThat(row.getString(0).toString()).isNotNull();
+                    totalF0.addAndGet(row.getInt(1));
+                    assertThat(row.getString(2).toString()).isNotNull();
+                    cnt.incrementAndGet();
+                });
 
         // check that all rows have been read
         assertThat(cnt.get()).isEqualTo(1920800);
@@ -148,88 +138,30 @@ class OrcReaderFactoryTest {
         AtomicInteger cnt = new AtomicInteger(0);
         AtomicInteger nullCount = new AtomicInteger(0);
 
-        // read all splits
-        for (FileSourceSplit split : createSplits(decimalFile, 4)) {
-            forEach(
-                    format,
-                    split,
-                    row -> {
-                        if (cnt.get() == 0) {
-                            // validate first row
-                            assertThat(row).isNotNull();
-                            assertThat(row.getArity()).isEqualTo(1);
-                            assertThat(row.getDecimal(0, 10, 5))
-                                    .isEqualTo(DecimalDataUtils.castFrom(-1000.5d, 10, 5));
+        forEach(
+                format,
+                decimalFile,
+                row -> {
+                    if (cnt.get() == 0) {
+                        // validate first row
+                        assertThat(row).isNotNull();
+                        assertThat(row.getArity()).isEqualTo(1);
+                        assertThat(row.getDecimal(0, 10, 5))
+                                .isEqualTo(DecimalDataUtils.castFrom(-1000.5d, 10, 5));
+                    } else {
+                        if (!row.isNullAt(0)) {
+                            assertThat(row.getDecimal(0, 10, 5)).isNotNull();
                         } else {
-                            if (!row.isNullAt(0)) {
-                                assertThat(row.getDecimal(0, 10, 5)).isNotNull();
-                            } else {
-                                nullCount.incrementAndGet();
-                            }
+                            nullCount.incrementAndGet();
                         }
-                        cnt.incrementAndGet();
-                    });
-        }
+                    }
+                    cnt.incrementAndGet();
+                });
 
         assertThat(cnt.get()).isEqualTo(6000);
         assertThat(nullCount.get()).isEqualTo(2000);
     }
 
-    @Test
-    void testReadFileAndRestore() throws IOException {
-        OrcReaderFactory format = createFormat(FLAT_FILE_TYPE, new int[] {0, 1});
-
-        // pick a middle split
-        FileSourceSplit split = createSplits(flatFile, 3).get(1);
-
-        int expectedCnt = 660000;
-
-        innerTestRestore(format, split, expectedCnt, 656700330000L);
-    }
-
-    @Test
-    void testReadFileAndRestoreWithFilter() throws IOException {
-        List<OrcFilters.Predicate> filter =
-                Collections.singletonList(
-                        new OrcFilters.Or(
-                                new OrcFilters.Between(
-                                        "_col0", PredicateLeaf.Type.LONG, 0L, 975000L),
-                                new OrcFilters.Equals("_col0", PredicateLeaf.Type.LONG, 980001L),
-                                new OrcFilters.Between(
-                                        "_col0", PredicateLeaf.Type.LONG, 990000L, 1800000L)));
-        OrcReaderFactory format = createFormat(FLAT_FILE_TYPE, new int[] {0, 1}, filter);
-
-        // pick a middle split
-        FileSourceSplit split = createSplits(flatFile, 1).get(0);
-
-        int expectedCnt = 1795000;
-        long expectedTotalF0 = 1615113397500L;
-
-        innerTestRestore(format, split, expectedCnt, expectedTotalF0);
-    }
-
-    private void innerTestRestore(
-            OrcReaderFactory format, FileSourceSplit split, int expectedCnt, long expectedTotalF0)
-            throws IOException {
-        AtomicInteger cnt = new AtomicInteger(0);
-        AtomicLong totalF0 = new AtomicLong(0);
-
-        Consumer<RowData> consumer =
-                row -> {
-                    assertThat(row.isNullAt(0)).isFalse();
-                    assertThat(row.isNullAt(1)).isFalse();
-                    totalF0.addAndGet(row.getInt(0));
-                    assertThat(row.getString(1).toString()).isNotNull();
-                    cnt.incrementAndGet();
-                };
-
-        Utils.forEachRemaining(createReader(format, split), consumer);
-
-        // check that all rows have been read
-        assertThat(cnt.get()).isEqualTo(expectedCnt);
-        assertThat(totalF0.get()).isEqualTo(expectedTotalF0);
-    }
-
     protected OrcReaderFactory createFormat(RowType formatType, int[] selectedFields) {
         return createFormat(formatType, selectedFields, new ArrayList<>());
     }
@@ -242,14 +174,14 @@ class OrcReaderFactoryTest {
                 new Configuration(), formatType, selectedFields, conjunctPredicates, BATCH_SIZE);
     }
 
-    private BulkFormat.Reader<RowData> createReader(OrcReaderFactory format, FileSourceSplit split)
+    private RecordReader<RowData> createReader(OrcReaderFactory format, Path split)
             throws IOException {
-        return format.createReader(new org.apache.flink.configuration.Configuration(), split);
+        return format.createReader(split);
     }
 
-    private void forEach(OrcReaderFactory format, FileSourceSplit split, Consumer<RowData> action)
+    private void forEach(OrcReaderFactory format, Path file, Consumer<RowData> action)
             throws IOException {
-        Utils.forEachRemaining(createReader(format, split), action);
+        RecordReaderUtils.forEachRemaining(format.createReader(file), action);
     }
 
     static Path copyFileFromResource(String resourceName, java.nio.file.Path file) {
@@ -265,27 +197,4 @@ class OrcReaderFactoryTest {
             throw new RuntimeException(e);
         }
     }
-
-    private static List<FileSourceSplit> createSplits(Path path, int minNumSplits)
-            throws IOException {
-        final List<FileSourceSplit> splits = new ArrayList<>(minNumSplits);
-        final FileStatus fileStatus = path.getFileSystem().getFileStatus(path);
-        final long len = fileStatus.getLen();
-        final long preferSplitSize = len / minNumSplits + (len % minNumSplits == 0 ? 0 : 1);
-        int splitNum = 0;
-        long position = 0;
-        while (position < len) {
-            long splitLen = Math.min(preferSplitSize, len - position);
-            splits.add(
-                    new FileSourceSplit(
-                            String.valueOf(splitNum++),
-                            path,
-                            position,
-                            splitLen,
-                            fileStatus.getModificationTime(),
-                            len));
-            position += splitLen;
-        }
-        return splits;
-    }
 }
diff --git a/flink-table-store-format/src/test/java/org/apache/flink/table/store/format/parquet/ParquetReadWriteTest.java b/flink-table-store-format/src/test/java/org/apache/flink/table/store/format/parquet/ParquetReadWriteTest.java
index 5341874b..eae9db2d 100644
--- a/flink-table-store-format/src/test/java/org/apache/flink/table/store/format/parquet/ParquetReadWriteTest.java
+++ b/flink-table-store-format/src/test/java/org/apache/flink/table/store/format/parquet/ParquetReadWriteTest.java
@@ -20,8 +20,6 @@ package org.apache.flink.table.store.format.parquet;
 
 import org.apache.flink.api.common.serialization.BulkWriter;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.connector.file.src.FileSourceSplit;
-import org.apache.flink.connector.file.src.reader.BulkFormat;
 import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.table.data.DecimalData;
@@ -31,6 +29,7 @@ import org.apache.flink.table.data.GenericRowData;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.data.StringData;
 import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.store.file.utils.RecordReader;
 import org.apache.flink.table.store.format.parquet.writer.RowDataParquetBuilder;
 import org.apache.flink.table.types.logical.ArrayType;
 import org.apache.flink.table.types.logical.BigIntType;
@@ -68,7 +67,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.stream.Collectors;
 
-import static org.apache.flink.connector.file.src.util.Utils.forEachRemaining;
+import static org.apache.flink.table.store.file.utils.RecordReaderUtils.forEachRemaining;
 import static org.assertj.core.api.Assertions.assertThat;
 
 /** Test for {@link ParquetReaderFactory}. */
@@ -227,9 +226,7 @@ public class ParquetReadWriteTest {
 
         AtomicInteger cnt = new AtomicInteger(0);
         forEachRemaining(
-                format.createReader(
-                        EMPTY_CONF,
-                        new FileSourceSplit("id", testPath, 0, Long.MAX_VALUE, 0, Long.MAX_VALUE)),
+                format.createReader(testPath),
                 row -> {
                     int i = cnt.get();
                     assertThat(row.getDouble(0)).isEqualTo(i);
@@ -265,9 +262,7 @@ public class ParquetReadWriteTest {
 
         AtomicInteger cnt = new AtomicInteger(0);
         forEachRemaining(
-                format.createReader(
-                        EMPTY_CONF,
-                        new FileSourceSplit("id", testPath, 0, Long.MAX_VALUE, 0, Long.MAX_VALUE)),
+                format.createReader(testPath),
                 row -> {
                     int i = cnt.get();
                     assertThat(row.getDouble(0)).isEqualTo(i);
@@ -282,14 +277,8 @@ public class ParquetReadWriteTest {
             throws IOException {
         List<RowData> rows = records.stream().map(this::newRow).collect(Collectors.toList());
         Path testPath = createTempParquetFile(folder, rows, rowGroupSize);
-
-        long fileLen = testPath.getFileSystem().getFileStatus(testPath).getLen();
-        int len1 = testReadingSplit(subList(records, 0), testPath, 0, fileLen / 3);
-        int len2 = testReadingSplit(subList(records, len1), testPath, fileLen / 3, fileLen * 2 / 3);
-        int len3 =
-                testReadingSplit(
-                        subList(records, len1 + len2), testPath, fileLen * 2 / 3, Long.MAX_VALUE);
-        assertThat(len1 + len2 + len3).isEqualTo(records.size());
+        int len = testReadingFile(subList(records, 0), testPath);
+        assertThat(len).isEqualTo(records.size());
     }
 
     private Path createTempParquetFile(File folder, List<RowData> rows, int rowGroupSize)
@@ -311,9 +300,7 @@ public class ParquetReadWriteTest {
         return path;
     }
 
-    private int testReadingSplit(
-            List<Integer> expected, Path path, long splitStart, long splitLength)
-            throws IOException {
+    private int testReadingFile(List<Integer> expected, Path path) throws IOException {
         ParquetReaderFactory format = new ParquetReaderFactory(new Configuration(), ROW_TYPE, 500);
 
         // validate java serialization
@@ -323,9 +310,7 @@ public class ParquetReadWriteTest {
             throw new IOException(e);
         }
 
-        BulkFormat.Reader<RowData> reader =
-                format.createReader(
-                        EMPTY_CONF, new FileSourceSplit("ignore", path, splitStart, splitLength));
+        RecordReader<RowData> reader = format.createReader(path);
 
         AtomicInteger cnt = new AtomicInteger(0);
         final AtomicReference<RowData> previousRow = new AtomicReference<>();
diff --git a/flink-table-store-shade/pom.xml b/flink-table-store-shade/pom.xml
index 207999cd..4e31e41b 100644
--- a/flink-table-store-shade/pom.xml
+++ b/flink-table-store-shade/pom.xml
@@ -237,7 +237,6 @@ under the License.
                                     <include>org.apache.flink:flink-core</include>
                                     <include>org.apache.flink:flink-runtime</include>
                                     <include>org.apache.flink:${flink.streaming.java}</include>
-                                    <include>org.apache.flink:flink-connector-files</include>
                                     <include>org.apache.flink:flink-hadoop-fs</include>
                                     <include>org.apache.flink:flink-shaded-jackson</include>
                                     <include>org.apache.flink:flink-shaded-guava</include>
@@ -263,17 +262,6 @@ under the License.
                                 </includes>
                             </artifactSet>
                             <filters>
-                                <!--
-                                Throw away all META-INF/services,
-                                otherwise if user has the same format/connector jar in the classpath,
-                                FactoryUtil will complain about multiple matching factories.
-                                -->
-                                <filter>
-                                    <artifact>org.apache.flink:flink-connector-files</artifact>
-                                    <excludes>
-                                        <exclude>META-INF/services/**</exclude>
-                                    </excludes>
-                                </filter>
                                 <!-- Another copy of the Apache license, which we don't need. -->
                                 <filter>
                                     <artifact>*</artifact>