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