You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by tw...@apache.org on 2021/12/03 16:55:47 UTC
[flink] 08/08: [FLINK-24687][table][connectors] Move FileSystemTableSink, FileSystemTableSource to flink-connector-files and columnar support to flink-table-common
This is an automated email from the ASF dual-hosted git repository.
twalthr pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git
commit 9bbadb9b105b233b7565af120020ebd8dce69a4f
Author: slinkydeveloper <fr...@gmail.com>
AuthorDate: Wed Nov 24 15:13:07 2021 +0100
[FLINK-24687][table][connectors] Move FileSystemTableSink, FileSystemTableSource to flink-connector-files and columnar support to flink-table-common
Now table packages don't depend on flink-connector-files anymore. Fix orc and parquet format to use only common classes and not planner nor runtime classes.
- [connector-files] Add @Internal to all public classes and interfaces
- [orc][parquet][hive] Drop scala suffix from flink-orc and flink-parquet
- [architecture-tests] Updated the violations file
- [connector-elasticsearch-base] Add flink-connector-base as dependency, which was previously brought in through flink-table-api-java-bridge -> flink-table-api-java -> flink-table-common -> flink-connector-files -> flink-connector-base.
- [orc][parquet] Add issue link for partition keys handling
- [table-uber][dist] Now flink-connector-files is not shaded inside table-uber anymore but it's loaded in /lib in the distribution as flink-connector-files
- [docs] Update sql_connectors.yml
This closes #17897.
---
docs/content/docs/connectors/table/filesystem.md | 1 +
docs/data/sql_connectors.yml | 46 ++++++++++--------
.../e577412e-8d38-496c-a680-b842112e4b94 | 1 -
.../flink-connector-elasticsearch-base/pom.xml | 5 ++
flink-connectors/flink-connector-files/pom.xml | 38 ++++++++++++++-
.../file/table}/AbstractFileSystemTable.java | 9 ++--
.../flink/connector/file/table}/BinPacking.java | 5 +-
.../connector/file/table}/ColumnarRowIterator.java | 6 ++-
.../file/table}/ContinuousPartitionFetcher.java | 2 +-
.../file/table}/DefaultPartTimeExtractor.java | 4 +-
.../file/table}/DeserializationSchemaAdapter.java | 6 ++-
.../file/table}/DynamicPartitionWriter.java | 2 +-
.../file/table}/EmptyMetaStoreFactory.java | 2 +-
.../connector/file/table}/EnrichedRowData.java | 2 +-
.../file/table}/FileInfoExtractorBulkFormat.java | 6 +--
.../connector/file/table}/FileSystemCommitter.java | 6 +--
.../file/table}/FileSystemConnectorOptions.java | 2 +-
.../connector/file/table}/FileSystemFactory.java | 2 +-
.../file/table}/FileSystemOutputFormat.java | 2 +-
.../file/table}/FileSystemTableFactory.java | 10 ++--
.../connector/file/table}/FileSystemTableSink.java | 33 +++++++------
.../file/table}/FileSystemTableSource.java | 25 +++++++---
.../file/table}/GroupedPartitionWriter.java | 2 +-
.../connector/file/table}/LimitableBulkFormat.java | 4 +-
.../file/table}/MetastoreCommitPolicy.java | 6 ++-
.../connector/file/table}/OutputFormatFactory.java | 2 +-
.../file/table}/PartitionCommitPolicy.java | 2 +-
.../connector/file/table}/PartitionComputer.java | 2 +-
.../connector/file/table}/PartitionFetcher.java | 2 +-
.../file/table}/PartitionFieldExtractor.java | 4 +-
.../connector/file/table}/PartitionLoader.java | 5 +-
.../connector/file/table}/PartitionReader.java | 2 +-
.../file/table}/PartitionTempFileManager.java | 2 +-
.../file/table}/PartitionTimeExtractor.java | 2 +-
.../connector/file/table}/PartitionWriter.java | 2 +-
.../file/table}/PartitionWriterFactory.java | 4 +-
.../file/table}/ProjectingBulkFormat.java | 2 +-
.../file/table}/RowDataPartitionComputer.java | 2 +-
.../file/table}/RowPartitionComputer.java | 2 +-
.../file/table}/SerializationSchemaAdapter.java | 2 +-
.../file/table}/SingleDirectoryWriter.java | 2 +-
.../file/table}/SuccessFileCommitPolicy.java | 4 +-
.../file/table}/TableMetaStoreFactory.java | 2 +-
.../table/factories/BulkReaderFormatFactory.java | 7 ++-
.../table/factories/BulkWriterFormatFactory.java | 4 +-
.../table/factories/FileSystemFormatFactory.java | 3 +-
.../file/table}/format/BulkDecodingFormat.java | 4 +-
.../table}/stream/AbstractStreamingWriter.java | 2 +-
.../file/table}/stream/PartitionCommitInfo.java | 5 +-
.../table}/stream/PartitionCommitPredicate.java | 11 +++--
.../file/table}/stream/PartitionCommitTrigger.java | 11 +++--
.../file/table}/stream/PartitionCommitter.java | 20 ++++----
.../stream/PartitionTimeCommitPredicate.java | 16 ++++---
.../table}/stream/PartitionTimeCommitTrigger.java | 7 +--
.../table}/stream/ProcTimeCommitPredicate.java | 6 ++-
.../file/table}/stream/ProcTimeCommitTrigger.java | 6 ++-
.../file/table}/stream/StreamingFileWriter.java | 8 ++--
.../file/table}/stream/StreamingSink.java | 26 ++++++-----
.../connector/file/table}/stream/TaskTracker.java | 5 +-
.../table}/stream/compact/CompactBucketWriter.java | 4 +-
.../table}/stream/compact/CompactBulkReader.java | 4 +-
.../file/table}/stream/compact/CompactContext.java | 4 +-
.../table}/stream/compact/CompactCoordinator.java | 20 ++++----
.../table}/stream/compact/CompactFileWriter.java | 14 +++---
.../table}/stream/compact/CompactMessages.java | 4 +-
.../table}/stream/compact/CompactOperator.java | 12 +++--
.../file/table}/stream/compact/CompactReader.java | 5 +-
.../file/table}/stream/compact/CompactWriter.java | 5 +-
.../compact/FileInputFormatCompactReader.java | 4 +-
.../org.apache.flink.table.factories.Factory | 2 +-
.../connector/file/table}/BinPackingTest.java | 15 +++---
.../connector/file/table}/EnrichedRowDataTest.java | 2 +-
.../file/table}/FileSystemCommitterTest.java | 2 +-
.../file/table}/FileSystemOutputFormatTest.java | 2 +-
.../file/table}/LimitableBulkFormatTest.java | 2 +-
.../connector/file/table}/PartitionWriterTest.java | 4 +-
.../file/table}/RowPartitionComputerTest.java | 2 +-
.../table}/stream/StreamingFileWriterTest.java | 12 ++---
.../stream/compact/AbstractCompactTestBase.java | 2 +-
.../stream/compact/CompactCoordinatorTest.java | 16 +++----
.../stream/compact/CompactFileWriterTest.java | 8 ++--
.../table}/stream/compact/CompactOperatorTest.java | 10 ++--
.../file/table}/stream/compact/TestByteFormat.java | 3 +-
flink-connectors/flink-connector-hive/pom.xml | 16 +++----
.../hive/ContinuousHiveSplitEnumerator.java | 2 +-
.../connectors/hive}/FileSystemLookupFunction.java | 4 +-
.../connectors/hive/HadoopFileSystemFactory.java | 2 +-
.../connectors/hive/HiveDynamicTableFactory.java | 6 +--
.../connectors/hive/HiveLookupTableSource.java | 13 +++---
.../hive/HiveRowDataPartitionComputer.java | 2 +-
.../connectors/hive/HiveRowPartitionComputer.java | 2 +-
.../apache/flink/connectors/hive/HiveSource.java | 2 +-
.../flink/connectors/hive/HiveSourceBuilder.java | 14 +++---
.../connectors/hive/HiveTableMetaStoreFactory.java | 2 +-
.../flink/connectors/hive/HiveTableSink.java | 22 ++++-----
.../flink/connectors/hive/HiveTableSource.java | 8 ++--
.../hive/read/HiveCompactReaderFactory.java | 6 +--
.../hive/read/HiveContinuousPartitionContext.java | 2 +-
.../hive/read/HiveContinuousPartitionFetcher.java | 6 +--
.../connectors/hive/read/HiveInputFormat.java | 11 +++--
.../hive/read/HiveInputFormatPartitionReader.java | 2 +-
.../connectors/hive/read/HivePartitionContext.java | 2 +-
.../hive/read/HivePartitionFetcherContextBase.java | 14 +++---
.../hive/read/HiveVectorizedOrcSplitReader.java | 2 +-
.../read/HiveVectorizedParquetSplitReader.java | 2 +-
.../hive/write/HiveOutputFormatFactory.java | 2 +-
.../hive/HiveDynamicTableFactoryTest.java | 19 ++++----
.../connectors/hive/HiveLookupJoinITCase.java | 19 ++++----
.../flink/connectors/hive/HiveTableSinkITCase.java | 10 ++--
.../connectors/hive/HiveTemporalJoinITCase.java | 2 +-
.../connectors/hive/PartitionMonitorTest.java | 4 +-
.../connectors/hive/TestCustomCommitPolicy.java | 2 +-
.../hive/read/HivePartitionFetcherTest.java | 4 +-
flink-dist/pom.xml | 7 +++
flink-dist/src/main/assemblies/bin.xml | 9 +++-
flink-examples/flink-examples-table/pom.xml | 5 ++
.../flink-avro-confluent-registry/pom.xml | 12 +++++
flink-formats/flink-avro/pom.xml | 11 ++++-
.../flink/formats/avro/AvroFileFormatFactory.java | 2 +-
.../formats/avro/AvroFileSystemFormatFactory.java | 2 +-
flink-formats/flink-csv/pom.xml | 11 +++++
.../formats/csv/CsvFileSystemFormatFactory.java | 2 +-
flink-formats/flink-json/pom.xml | 11 +++++
flink-formats/flink-orc-nohive/pom.xml | 26 +++++++++--
.../nohive/OrcNoHiveColumnarRowInputFormat.java | 25 ++++++----
.../flink/orc/nohive/OrcNoHiveSplitReaderUtil.java | 4 +-
.../orc/nohive/vector/AbstractOrcNoHiveVector.java | 8 ++--
.../orc/nohive/vector/OrcNoHiveBytesVector.java | 2 +-
.../orc/nohive/vector/OrcNoHiveDecimalVector.java | 2 +-
.../orc/nohive/vector/OrcNoHiveDoubleVector.java | 4 +-
.../orc/nohive/vector/OrcNoHiveLongVector.java | 10 ++--
.../nohive/vector/OrcNoHiveTimestampVector.java | 2 +-
flink-formats/flink-orc/pom.xml | 15 ++++--
.../flink/orc/OrcColumnarRowInputFormat.java | 36 +++++++++------
.../flink/orc/OrcColumnarRowSplitReader.java | 4 +-
.../org/apache/flink/orc/OrcFileFormatFactory.java | 11 +++--
.../org/apache/flink/orc/OrcSplitReaderUtil.java | 4 +-
.../flink/orc/vector/AbstractOrcColumnVector.java | 8 ++--
.../flink/orc/vector/ColumnBatchFactory.java | 2 +-
.../flink/orc/vector/OrcArrayColumnVector.java | 6 +--
.../flink/orc/vector/OrcBytesColumnVector.java | 2 +-
.../flink/orc/vector/OrcDecimalColumnVector.java | 2 +-
.../flink/orc/vector/OrcDoubleColumnVector.java | 4 +-
.../orc/vector/OrcLegacyTimestampColumnVector.java | 2 +-
.../flink/orc/vector/OrcLongColumnVector.java | 10 ++--
.../flink/orc/vector/OrcMapColumnVector.java | 6 +--
.../flink/orc/vector/OrcRowColumnVector.java | 8 ++--
.../flink/orc/vector/OrcTimestampColumnVector.java | 2 +-
.../flink/orc/OrcColumnarRowInputFormatTest.java | 9 ++--
flink-formats/flink-parquet/pom.xml | 14 ++++--
.../parquet/ParquetColumnarRowInputFormat.java | 38 ++++++++-------
.../formats/parquet/ParquetFileFormatFactory.java | 9 ++--
.../parquet/ParquetVectorizedInputFormat.java | 6 +--
.../formats/parquet/vector/ColumnBatchFactory.java | 4 +-
.../vector/ParquetColumnarRowSplitReader.java | 8 ++--
.../parquet/vector/ParquetDecimalVector.java | 10 ++--
.../formats/parquet/vector/ParquetDictionary.java | 2 +-
.../parquet/vector/ParquetSplitReaderUtil.java | 24 +++++-----
.../vector/reader/AbstractColumnReader.java | 4 +-
.../parquet/vector/reader/BooleanColumnReader.java | 4 +-
.../parquet/vector/reader/ByteColumnReader.java | 4 +-
.../parquet/vector/reader/BytesColumnReader.java | 4 +-
.../parquet/vector/reader/ColumnReader.java | 2 +-
.../parquet/vector/reader/DoubleColumnReader.java | 4 +-
.../vector/reader/FixedLenBytesColumnReader.java | 8 ++--
.../parquet/vector/reader/FloatColumnReader.java | 4 +-
.../parquet/vector/reader/IntColumnReader.java | 2 +-
.../parquet/vector/reader/LongColumnReader.java | 4 +-
.../parquet/vector/reader/RunLengthDecoder.java | 4 +-
.../parquet/vector/reader/ShortColumnReader.java | 4 +-
.../vector/reader/TimestampColumnReader.java | 4 +-
.../parquet/ParquetColumnarRowInputFormatTest.java | 7 ++-
.../vector/ParquetColumnarRowSplitReaderTest.java | 4 +-
flink-formats/flink-sql-orc/pom.xml | 6 +--
flink-formats/flink-sql-parquet/pom.xml | 6 +--
.../flink/table/runtime/arrow/ArrowReader.java | 6 +--
.../flink/table/runtime/arrow/ArrowUtils.java | 2 +-
.../arrow/vectors/ArrowArrayColumnVector.java | 6 +--
.../arrow/vectors/ArrowBigIntColumnVector.java | 2 +-
.../arrow/vectors/ArrowBooleanColumnVector.java | 2 +-
.../arrow/vectors/ArrowDateColumnVector.java | 2 +-
.../arrow/vectors/ArrowDecimalColumnVector.java | 2 +-
.../arrow/vectors/ArrowDoubleColumnVector.java | 2 +-
.../arrow/vectors/ArrowFloatColumnVector.java | 2 +-
.../arrow/vectors/ArrowIntColumnVector.java | 2 +-
.../arrow/vectors/ArrowRowColumnVector.java | 8 ++--
.../arrow/vectors/ArrowSmallIntColumnVector.java | 2 +-
.../arrow/vectors/ArrowTimeColumnVector.java | 2 +-
.../arrow/vectors/ArrowTimestampColumnVector.java | 2 +-
.../arrow/vectors/ArrowTinyIntColumnVector.java | 2 +-
.../arrow/vectors/ArrowVarBinaryColumnVector.java | 2 +-
.../arrow/vectors/ArrowVarCharColumnVector.java | 2 +-
.../flink/table/runtime/arrow/ArrowUtilsTest.java | 2 +-
.../table/sinks/CsvAppendTableSinkFactory.java | 3 +-
.../table/sinks/CsvBatchTableSinkFactory.java | 3 +-
.../org/apache/flink/table/sinks/CsvTableSink.java | 3 +-
.../flink/table/sinks/CsvTableSinkFactoryBase.java | 3 +-
.../table/sources/CsvAppendTableSourceFactory.java | 3 +-
.../table/sources/CsvBatchTableSourceFactory.java | 3 +-
.../apache/flink/table/sources/CsvTableSource.java | 3 +-
.../table/sources/CsvTableSourceFactoryBase.java | 3 +-
flink-table/flink-table-common/pom.xml | 6 ---
.../table/data/columnar}/ColumnarArrayData.java | 54 +++++++++++++---------
.../table/data/columnar}/ColumnarMapData.java | 23 +++++----
.../table/data/columnar}/ColumnarRowData.java | 17 +++++--
.../data/columnar}/vector/ArrayColumnVector.java | 4 +-
.../data/columnar}/vector/BooleanColumnVector.java | 5 +-
.../data/columnar}/vector/ByteColumnVector.java | 5 +-
.../data/columnar}/vector/BytesColumnVector.java | 5 +-
.../table/data/columnar}/vector/ColumnVector.java | 5 +-
.../data/columnar}/vector/DecimalColumnVector.java | 4 +-
.../table/data/columnar}/vector/Dictionary.java | 4 +-
.../data/columnar}/vector/DoubleColumnVector.java | 5 +-
.../data/columnar}/vector/FloatColumnVector.java | 5 +-
.../data/columnar}/vector/IntColumnVector.java | 5 +-
.../data/columnar}/vector/LongColumnVector.java | 5 +-
.../data/columnar}/vector/MapColumnVector.java | 4 +-
.../data/columnar}/vector/RowColumnVector.java | 6 ++-
.../data/columnar}/vector/ShortColumnVector.java | 5 +-
.../columnar}/vector/TimestampColumnVector.java | 4 +-
.../columnar}/vector/VectorizedColumnBatch.java | 6 ++-
.../columnar}/vector/heap/AbstractHeapVector.java | 6 ++-
.../columnar}/vector/heap/HeapBooleanVector.java | 6 ++-
.../data/columnar}/vector/heap/HeapByteVector.java | 6 ++-
.../columnar}/vector/heap/HeapBytesVector.java | 6 ++-
.../columnar}/vector/heap/HeapDoubleVector.java | 6 ++-
.../columnar}/vector/heap/HeapFloatVector.java | 6 ++-
.../data/columnar}/vector/heap/HeapIntVector.java | 6 ++-
.../data/columnar}/vector/heap/HeapLongVector.java | 6 ++-
.../columnar}/vector/heap/HeapShortVector.java | 6 ++-
.../columnar}/vector/heap/HeapTimestampVector.java | 6 ++-
.../vector/writable/AbstractWritableVector.java | 8 ++--
.../vector/writable/WritableBooleanVector.java | 6 ++-
.../vector/writable/WritableByteVector.java | 6 ++-
.../vector/writable/WritableBytesVector.java | 6 ++-
.../vector/writable/WritableColumnVector.java | 8 ++--
.../vector/writable/WritableDoubleVector.java | 6 ++-
.../vector/writable/WritableFloatVector.java | 6 ++-
.../vector/writable/WritableIntVector.java | 6 ++-
.../vector/writable/WritableLongVector.java | 6 ++-
.../vector/writable/WritableShortVector.java | 6 ++-
.../vector/writable/WritableTimestampVector.java | 6 ++-
.../table/descriptors/FileSystemValidator.java | 4 +-
.../data/columnar}/vector/ColumnVectorTest.java | 36 +++++++--------
.../vector/VectorizedColumnBatchTest.java | 22 ++++-----
flink-table/flink-table-planner/pom.xml | 8 ++++
.../file/table}/FileSystemTableFactoryTest.java | 4 +-
.../file/table}/FileSystemTableSinkTest.java | 2 +-
.../file/table}/FileSystemTableSourceTest.java | 2 +-
.../testcsv/TestCsvDeserializationSchema.java | 0
.../formats/testcsv/TestCsvFormatFactory.java | 0
.../testcsv/TestCsvSerializationSchema.java | 0
.../table/planner/factories/TestFileFactory.java | 2 +-
.../runtime/stream/sql/CompactionITCaseBase.java | 2 +-
.../org.apache.flink.table.factories.Factory | 1 +
.../file/table}/FileSystemTableSourceTest.xml | 0
.../batch/sql/PartitionableSinkITCase.scala | 2 +-
.../runtime/stream/FsStreamingSinkITCaseBase.scala | 4 +-
.../runtime/typeutils/ArrayDataSerializer.java | 2 +-
.../org.apache.flink.table.factories.Factory | 1 -
.../runtime/typeutils/ArrayDataSerializerTest.java | 4 +-
flink-table/flink-table-uber/pom.xml | 3 --
262 files changed, 1041 insertions(+), 690 deletions(-)
diff --git a/docs/content/docs/connectors/table/filesystem.md b/docs/content/docs/connectors/table/filesystem.md
index 5652fb6..ae07443 100644
--- a/docs/content/docs/connectors/table/filesystem.md
+++ b/docs/content/docs/connectors/table/filesystem.md
@@ -30,6 +30,7 @@ This connector provides access to partitioned files in filesystems
supported by the [Flink FileSystem abstraction]({{< ref "docs/deployment/filesystems/overview" >}}).
The file system connector itself is included in Flink and does not require an additional dependency.
+The corresponding jar can be found in the Flink distribution inside the `/lib` directory.
A corresponding format needs to be specified for reading and writing rows from and to a file system.
The file system connector allows for reading and writing from a local or distributed filesystem. A filesystem table can be defined as:
diff --git a/docs/data/sql_connectors.yml b/docs/data/sql_connectors.yml
index 989b5f2..b3e6d53 100644
--- a/docs/data/sql_connectors.yml
+++ b/docs/data/sql_connectors.yml
@@ -50,15 +50,15 @@ avro-confluent:
orc:
name: ORC
- maven: flink-orc$scala_version
+ maven: flink-orc
category: format
- sql_url: https://repo.maven.apache.org/maven2/org/apache/flink/flink-sql-orc$scala_version/$version/flink-sql-orc$scala_version-$version.jar
+ sql_url: https://repo.maven.apache.org/maven2/org/apache/flink/flink-sql-orc/$version/flink-sql-orc-$version.jar
parquet:
name: Parquet
- maven: flink-parquet$scala_version
+ maven: flink-parquet
category: format
- sql_url: https://repo.maven.apache.org/maven2/org/apache/flink/flink-sql-parquet$scala_version/$version/flink-sql-parquet$scala_version-$version.jar
+ sql_url: https://repo.maven.apache.org/maven2/org/apache/flink/flink-sql-parquet/$version/flink-sql-parquet-$version.jar
debezium-avro-confluent:
name: Debezium
@@ -102,53 +102,59 @@ raw:
category: format
builtin: true
+files:
+ name: Files
+ category: connector
+ maven: flink-connector-files
+ sql_url: https://repo.maven.apache.org/maven2/org/apache/flink/flink-connector-files/$version/flink-connector-files-$version.jar
+
elastic:
name: Elasticsearch
category: connector
versions:
- version: 6.x
- maven: flink-connector-elasticsearch6$scala_version
- sql_url: https://repo.maven.apache.org/maven2/org/apache/flink/flink-sql-connector-elasticsearch6$scala_version/$version/flink-sql-connector-elasticsearch6$scala_version-$version.jar
+ maven: flink-connector-elasticsearch6
+ sql_url: https://repo.maven.apache.org/maven2/org/apache/flink/flink-sql-connector-elasticsearch6/$version/flink-sql-connector-elasticsearch6-$version.jar
- version: 7.x and later versions
- maven: flink-connector-elasticsearch7$scala_version
- sql_url: https://repo.maven.apache.org/maven2/org/apache/flink/flink-sql-connector-elasticsearch7$scala_version/$version/flink-sql-connector-elasticsearch7$scala_version-$version.jar
+ maven: flink-connector-elasticsearch7
+ sql_url: https://repo.maven.apache.org/maven2/org/apache/flink/flink-sql-connector-elasticsearch7/$version/flink-sql-connector-elasticsearch7-$version.jar
hbase:
name: HBase
category: connector
versions:
- version: 1.4.x
- maven: flink-connector-hbase-1.4$scala_version
- sql_url: https://repo.maven.apache.org/maven2/org/apache/flink/flink-sql-connector-hbase-1.4$scala_version/$version/flink-sql-connector-hbase-1.4$scala_version-$version.jar
+ maven: flink-connector-hbase-1.4
+ sql_url: https://repo.maven.apache.org/maven2/org/apache/flink/flink-sql-connector-hbase-1.4/$version/flink-sql-connector-hbase-1.4-$version.jar
- version: 2.2.x
- maven: flink-connector-hbase-2.2$scala_version
- sql_url: https://repo.maven.apache.org/maven2/org/apache/flink/flink-sql-connector-hbase-2.2$scala_version/$version/flink-sql-connector-hbase-2.2$scala_version-$version.jar
+ maven: flink-connector-hbase-2.2
+ sql_url: https://repo.maven.apache.org/maven2/org/apache/flink/flink-sql-connector-hbase-2.2/$version/flink-sql-connector-hbase-2.2-$version.jar
jdbc:
name: JDBC
category: connector
- maven: flink-connector-jdbc$scala_version
- sql_url: https://repo.maven.apache.org/maven2/org/apache/flink/flink-connector-jdbc$scala_version/$version/flink-connector-jdbc$scala_version-$version.jar
+ maven: flink-connector-jdbc
+ sql_url: https://repo.maven.apache.org/maven2/org/apache/flink/flink-connector-jdbc/$version/flink-connector-jdbc-$version.jar
kafka:
name: Kafka
category: connector
versions:
- version: universal
- maven: flink-connector-kafka$scala_version
- sql_url: https://repo.maven.apache.org/maven2/org/apache/flink/flink-sql-connector-kafka$scala_version/$version/flink-sql-connector-kafka$scala_version-$version.jar
+ maven: flink-connector-kafka
+ sql_url: https://repo.maven.apache.org/maven2/org/apache/flink/flink-sql-connector-kafka/$version/flink-sql-connector-kafka-$version.jar
upsert-kafka:
name: Upsert Kafka
category: connector
versions:
- version: universal
- maven: flink-connector-kafka$scala_version
- sql_url: https://repo.maven.apache.org/maven2/org/apache/flink/flink-sql-connector-kafka$scala_version/$version/flink-sql-connector-kafka$scala_version-$version.jar
+ maven: flink-connector-kafka
+ sql_url: https://repo.maven.apache.org/maven2/org/apache/flink/flink-sql-connector-kafka/$version/flink-sql-connector-kafka-$version.jar
kinesis:
name: Kinesis
category: connector
- maven: flink-connector-kinesis$scala_version
- sql_url: https://repo.maven.apache.org/maven2/org/apache/flink/flink-sql-connector-kinesis$scala_version/$version/flink-sql-connector-kinesis$scala_version-$version.jar
+ maven: flink-connector-kinesis
+ sql_url: https://repo.maven.apache.org/maven2/org/apache/flink/flink-sql-connector-kinesis/$version/flink-sql-connector-kinesis-$version.jar
diff --git a/flink-architecture-tests/violations/e577412e-8d38-496c-a680-b842112e4b94 b/flink-architecture-tests/violations/e577412e-8d38-496c-a680-b842112e4b94
index fbfd7c5..bb9f488 100644
--- a/flink-architecture-tests/violations/e577412e-8d38-496c-a680-b842112e4b94
+++ b/flink-architecture-tests/violations/e577412e-8d38-496c-a680-b842112e4b94
@@ -1,4 +1,3 @@
Class <org.apache.flink.connector.jdbc.internal.options.JdbcConnectorOptions> does not reside in a package 'org.apache.flink..table' in (JdbcConnectorOptions.java:0)
Class <org.apache.flink.formats.raw.RawFormatOptions> does not reside in a package 'org.apache.flink..table' in (RawFormatOptions.java:0)
-Class <org.apache.flink.table.filesystem.FileSystemConnectorOptions> does not reside in a package 'org.apache.flink..table' in (FileSystemConnectorOptions.java:0)
org.apache.flink.connector.jdbc.internal.options.JdbcConnectorOptions does not satisfy: annotated with @PublicEvolving or annotated with @Public
\ No newline at end of file
diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml
index bff4b09..612048b 100644
--- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml
+++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml
@@ -46,6 +46,11 @@ under the License.
<dependency>
<groupId>org.apache.flink</groupId>
+ <artifactId>flink-connector-base</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-java</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
diff --git a/flink-connectors/flink-connector-files/pom.xml b/flink-connectors/flink-connector-files/pom.xml
index 6aee565..4bbc25d 100644
--- a/flink-connectors/flink-connector-files/pom.xml
+++ b/flink-connectors/flink-connector-files/pom.xml
@@ -58,6 +58,21 @@ under the License.
<version>${project.version}</version>
</dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-streaming-java</artifactId>
+ <version>${project.version}</version>
+ <scope>provided</scope>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-table-api-java-bridge</artifactId>
+ <version>${project.version}</version>
+ <scope>provided</scope>
+ <optional>true</optional>
+ </dependency>
+
<!-- test dependencies -->
<dependency>
@@ -86,12 +101,13 @@ under the License.
<groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-java</artifactId>
<version>${project.version}</version>
+ <type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
- <artifactId>flink-streaming-java</artifactId>
+ <artifactId>flink-table-common</artifactId>
<version>${project.version}</version>
<type>test-jar</type>
<scope>test</scope>
@@ -112,6 +128,26 @@ under the License.
</execution>
</executions>
</plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-shade-plugin</artifactId>
+ <executions>
+ <execution>
+ <id>shade-flink</id>
+ <phase>package</phase>
+ <goals>
+ <goal>shade</goal>
+ </goals>
+ <configuration>
+ <artifactSet>
+ <includes>
+ <include>org.apache.flink:flink-connector-base</include>
+ </includes>
+ </artifactSet>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
</plugins>
</build>
</project>
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/AbstractFileSystemTable.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/AbstractFileSystemTable.java
similarity index 88%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/AbstractFileSystemTable.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/AbstractFileSystemTable.java
index b96a371..f78bee6 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/AbstractFileSystemTable.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/AbstractFileSystemTable.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.DelegatingConfiguration;
@@ -31,9 +31,6 @@ import org.apache.flink.table.types.DataType;
import java.util.List;
import java.util.stream.Collectors;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.PARTITION_DEFAULT_NAME;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.PATH;
-
/** Abstract File system table for providing some common methods. */
abstract class AbstractFileSystemTable {
@@ -52,8 +49,8 @@ abstract class AbstractFileSystemTable {
this.tableOptions = new Configuration();
context.getCatalogTable().getOptions().forEach(tableOptions::setString);
this.schema = context.getCatalogTable().getResolvedSchema();
- this.path = new Path(tableOptions.get(PATH));
- this.defaultPartName = tableOptions.get(PARTITION_DEFAULT_NAME);
+ this.path = new Path(tableOptions.get(FileSystemConnectorOptions.PATH));
+ this.defaultPartName = tableOptions.get(FileSystemConnectorOptions.PARTITION_DEFAULT_NAME);
this.partitionKeys = context.getCatalogTable().getPartitionKeys();
}
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/util/BinPacking.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/BinPacking.java
similarity index 96%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/util/BinPacking.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/BinPacking.java
index 73f9a1c..a8ab2ec 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/util/BinPacking.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/BinPacking.java
@@ -16,7 +16,9 @@
* limitations under the License.
*/
-package org.apache.flink.table.runtime.util;
+package org.apache.flink.connector.file.table;
+
+import org.apache.flink.annotation.Internal;
import java.util.ArrayList;
import java.util.Deque;
@@ -25,6 +27,7 @@ import java.util.List;
import java.util.function.Function;
/** A bin packing implementation. */
+@Internal
public class BinPacking {
private BinPacking() {}
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/ColumnarRowIterator.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/ColumnarRowIterator.java
similarity index 94%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/ColumnarRowIterator.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/ColumnarRowIterator.java
index 2707526..ea421d4 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/ColumnarRowIterator.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/ColumnarRowIterator.java
@@ -16,15 +16,16 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
+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.ColumnarRowData;
import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.columnar.ColumnarRowData;
import javax.annotation.Nullable;
@@ -32,6 +33,7 @@ import javax.annotation.Nullable;
* A {@link BulkFormat.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;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/ContinuousPartitionFetcher.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/ContinuousPartitionFetcher.java
similarity index 98%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/ContinuousPartitionFetcher.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/ContinuousPartitionFetcher.java
index ae2020e..e616ac5 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/ContinuousPartitionFetcher.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/ContinuousPartitionFetcher.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.typeutils.TypeSerializer;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/DefaultPartTimeExtractor.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/DefaultPartTimeExtractor.java
similarity index 97%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/DefaultPartTimeExtractor.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/DefaultPartTimeExtractor.java
index 27f496f..cb2e255 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/DefaultPartTimeExtractor.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/DefaultPartTimeExtractor.java
@@ -16,8 +16,9 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.table.data.TimestampData;
import javax.annotation.Nullable;
@@ -44,6 +45,7 @@ import static java.time.temporal.ChronoField.YEAR;
* Default {@link PartitionTimeExtractor}. See {@link
* FileSystemConnectorOptions#PARTITION_TIME_EXTRACTOR_TIMESTAMP_PATTERN}.
*/
+@Internal
public class DefaultPartTimeExtractor implements PartitionTimeExtractor {
private static final DateTimeFormatter TIMESTAMP_FORMATTER =
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/DeserializationSchemaAdapter.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/DeserializationSchemaAdapter.java
similarity index 97%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/DeserializationSchemaAdapter.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/DeserializationSchemaAdapter.java
index 73851fa..3448aa3 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/DeserializationSchemaAdapter.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/DeserializationSchemaAdapter.java
@@ -16,8 +16,9 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.io.DelimitedInputFormat;
import org.apache.flink.api.common.serialization.DeserializationSchema;
import org.apache.flink.api.common.typeinfo.TypeInformation;
@@ -41,9 +42,10 @@ import java.util.Arrays;
import java.util.Queue;
import static org.apache.flink.connector.file.src.util.CheckpointedPosition.NO_OFFSET;
-import static org.apache.flink.table.data.vector.VectorizedColumnBatch.DEFAULT_SIZE;
+import static org.apache.flink.table.data.columnar.vector.VectorizedColumnBatch.DEFAULT_SIZE;
/** Adapter to turn a {@link DeserializationSchema} into a {@link BulkFormat}. */
+@Internal
public class DeserializationSchemaAdapter implements BulkFormat<RowData, FileSourceSplit> {
private static final int BATCH_SIZE = 100;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/DynamicPartitionWriter.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/DynamicPartitionWriter.java
similarity index 98%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/DynamicPartitionWriter.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/DynamicPartitionWriter.java
index 47167cd..c33db2f 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/DynamicPartitionWriter.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/DynamicPartitionWriter.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.io.OutputFormat;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/EmptyMetaStoreFactory.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/EmptyMetaStoreFactory.java
similarity index 97%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/EmptyMetaStoreFactory.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/EmptyMetaStoreFactory.java
index 38baff5..4525e98 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/EmptyMetaStoreFactory.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/EmptyMetaStoreFactory.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.annotation.Internal;
import org.apache.flink.core.fs.Path;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/EnrichedRowData.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/EnrichedRowData.java
similarity index 99%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/EnrichedRowData.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/EnrichedRowData.java
index dda054c..e333d37 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/EnrichedRowData.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/EnrichedRowData.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.annotation.Internal;
import org.apache.flink.table.data.ArrayData;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/FileInfoExtractorBulkFormat.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileInfoExtractorBulkFormat.java
similarity index 97%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/FileInfoExtractorBulkFormat.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileInfoExtractorBulkFormat.java
index 5dcf0ff..7bda0c0 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/FileInfoExtractorBulkFormat.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileInfoExtractorBulkFormat.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.configuration.Configuration;
@@ -26,7 +26,6 @@ import org.apache.flink.connector.file.src.util.RecordMapperWrapperRecordIterato
import org.apache.flink.table.api.DataTypes;
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.types.DataType;
import org.apache.flink.table.utils.PartitionPathUtils;
@@ -57,11 +56,12 @@ class FileInfoExtractorBulkFormat implements BulkFormat<RowData, FileSourceSplit
public FileInfoExtractorBulkFormat(
BulkFormat<RowData, FileSourceSplit> wrapped,
DataType producedDataType,
+ TypeInformation<RowData> producedTypeInformation,
Map<String, FileSystemTableSource.FileInfoAccessor> metadataColumns,
List<String> partitionColumns,
String defaultPartName) {
this.wrapped = wrapped;
- this.producedType = InternalTypeInfo.of(producedDataType.getLogicalType());
+ this.producedType = producedTypeInformation;
this.defaultPartName = defaultPartName;
// Compute index mapping for the extended row and the functions to compute metadata
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/FileSystemCommitter.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemCommitter.java
similarity index 93%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/FileSystemCommitter.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemCommitter.java
index 795cd5e..2cb8a90 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/FileSystemCommitter.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemCommitter.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.annotation.Internal;
import org.apache.flink.core.fs.FileSystem;
@@ -27,8 +27,8 @@ import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
-import static org.apache.flink.table.filesystem.PartitionTempFileManager.collectPartSpecToPaths;
-import static org.apache.flink.table.filesystem.PartitionTempFileManager.listTaskTemporaryPaths;
+import static org.apache.flink.connector.file.table.PartitionTempFileManager.collectPartSpecToPaths;
+import static org.apache.flink.connector.file.table.PartitionTempFileManager.listTaskTemporaryPaths;
/**
* File system file committer implementation. It moves all files to output path from temporary path.
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/FileSystemConnectorOptions.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemConnectorOptions.java
similarity index 99%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/FileSystemConnectorOptions.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemConnectorOptions.java
index 0d8e9d0..aebb363 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/FileSystemConnectorOptions.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemConnectorOptions.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.configuration.ConfigOption;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/FileSystemFactory.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemFactory.java
similarity index 97%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/FileSystemFactory.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemFactory.java
index b35b2c5..a151657 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/FileSystemFactory.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemFactory.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.annotation.Internal;
import org.apache.flink.core.fs.FileSystem;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/FileSystemOutputFormat.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemOutputFormat.java
similarity index 99%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/FileSystemOutputFormat.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemOutputFormat.java
index 7c632c7..5eb03f1 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/FileSystemOutputFormat.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemOutputFormat.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.io.FinalizeOnMaster;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/FileSystemTableFactory.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableFactory.java
similarity index 96%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/FileSystemTableFactory.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableFactory.java
index c06d211..52088ad 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/FileSystemTableFactory.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableFactory.java
@@ -16,17 +16,19 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.table.factories.BulkReaderFormatFactory;
+import org.apache.flink.connector.file.table.factories.BulkWriterFormatFactory;
+import org.apache.flink.connector.file.table.factories.FileSystemFormatFactory;
import org.apache.flink.table.api.ValidationException;
import org.apache.flink.table.connector.format.DecodingFormat;
import org.apache.flink.table.connector.format.EncodingFormat;
import org.apache.flink.table.connector.sink.DynamicTableSink;
import org.apache.flink.table.connector.source.DynamicTableSource;
-import org.apache.flink.table.factories.BulkReaderFormatFactory;
-import org.apache.flink.table.factories.BulkWriterFormatFactory;
import org.apache.flink.table.factories.DecodingFormatFactory;
import org.apache.flink.table.factories.DeserializationFormatFactory;
import org.apache.flink.table.factories.DynamicTableSinkFactory;
@@ -34,7 +36,6 @@ import org.apache.flink.table.factories.DynamicTableSourceFactory;
import org.apache.flink.table.factories.EncodingFormatFactory;
import org.apache.flink.table.factories.Factory;
import org.apache.flink.table.factories.FactoryUtil;
-import org.apache.flink.table.factories.FileSystemFormatFactory;
import org.apache.flink.table.factories.SerializationFormatFactory;
import org.apache.flink.table.factories.TableFactory;
@@ -54,6 +55,7 @@ import static java.time.ZoneId.SHORT_IDS;
* or a catalog table. 2.Support insert into (append) and insert overwrite. 3.Support static and
* dynamic partition inserting.
*/
+@Internal
public class FileSystemTableFactory implements DynamicTableSourceFactory, DynamicTableSinkFactory {
public static final String IDENTIFIER = "filesystem";
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/FileSystemTableSink.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableSink.java
similarity index 94%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/FileSystemTableSink.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableSink.java
index bbd7425..09705cd 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/FileSystemTableSink.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableSink.java
@@ -16,8 +16,9 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.io.FileInputFormat;
import org.apache.flink.api.common.io.InputFormat;
import org.apache.flink.api.common.io.OutputFormat;
@@ -30,6 +31,12 @@ 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.connector.file.table.factories.FileSystemFormatFactory;
+import org.apache.flink.connector.file.table.stream.PartitionCommitInfo;
+import org.apache.flink.connector.file.table.stream.StreamingSink;
+import org.apache.flink.connector.file.table.stream.compact.CompactBulkReader;
+import org.apache.flink.connector.file.table.stream.compact.CompactReader;
+import org.apache.flink.connector.file.table.stream.compact.FileInputFormatCompactReader;
import org.apache.flink.core.fs.FSDataOutputStream;
import org.apache.flink.core.fs.FileSystem;
import org.apache.flink.core.fs.Path;
@@ -60,12 +67,6 @@ import org.apache.flink.table.data.RowData;
import org.apache.flink.table.expressions.ResolvedExpression;
import org.apache.flink.table.factories.DynamicTableFactory;
import org.apache.flink.table.factories.FactoryUtil;
-import org.apache.flink.table.factories.FileSystemFormatFactory;
-import org.apache.flink.table.filesystem.stream.PartitionCommitInfo;
-import org.apache.flink.table.filesystem.stream.StreamingSink;
-import org.apache.flink.table.filesystem.stream.compact.CompactBulkReader;
-import org.apache.flink.table.filesystem.stream.compact.CompactReader;
-import org.apache.flink.table.filesystem.stream.compact.FileInputFormatCompactReader;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.logical.LogicalType;
import org.apache.flink.table.utils.PartitionPathUtils;
@@ -88,12 +89,15 @@ import java.util.stream.Collectors;
import java.util.stream.IntStream;
import java.util.stream.Stream;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.SINK_ROLLING_POLICY_CHECK_INTERVAL;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.SINK_ROLLING_POLICY_FILE_SIZE;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.SINK_ROLLING_POLICY_ROLLOVER_INTERVAL;
-import static org.apache.flink.table.filesystem.stream.compact.CompactOperator.convertToUncompacted;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.AUTO_COMPACTION;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.COMPACTION_FILE_SIZE;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.SINK_ROLLING_POLICY_CHECK_INTERVAL;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.SINK_ROLLING_POLICY_FILE_SIZE;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.SINK_ROLLING_POLICY_ROLLOVER_INTERVAL;
+import static org.apache.flink.connector.file.table.stream.compact.CompactOperator.convertToUncompacted;
/** File system {@link DynamicTableSink}. */
+@Internal
public class FileSystemTableSink extends AbstractFileSystemTable
implements DynamicTableSink, SupportsPartitioning, SupportsOverwrite {
@@ -191,8 +195,7 @@ public class FileSystemTableSink extends AbstractFileSystemTable
FileSystemFactory fsFactory = FileSystem::get;
RowDataPartitionComputer computer = partitionComputer();
- boolean autoCompaction =
- tableOptions.getBoolean(FileSystemConnectorOptions.AUTO_COMPACTION);
+ boolean autoCompaction = tableOptions.getBoolean(AUTO_COMPACTION);
Object writer = createWriter(sinkContext);
boolean isEncoder = writer instanceof Encoder;
TableBucketAssigner assigner = new TableBucketAssigner(computer);
@@ -238,7 +241,7 @@ public class FileSystemTableSink extends AbstractFileSystemTable
if (autoCompaction) {
long compactionSize =
tableOptions
- .getOptional(FileSystemConnectorOptions.COMPACTION_FILE_SIZE)
+ .getOptional(COMPACTION_FILE_SIZE)
.orElse(tableOptions.get(SINK_ROLLING_POLICY_FILE_SIZE))
.getBytes();
@@ -308,6 +311,7 @@ public class FileSystemTableSink extends AbstractFileSystemTable
bulkReaderFormat.createRuntimeDecoder(
createSourceContext(context), physicalDataType),
producedDataType,
+ context.createTypeInformation(producedDataType),
Collections.emptyMap(),
partitionKeys,
defaultPartName);
@@ -320,6 +324,7 @@ public class FileSystemTableSink extends AbstractFileSystemTable
new FileInfoExtractorBulkFormat(
new DeserializationSchemaAdapter(decoder),
producedDataType,
+ context.createTypeInformation(producedDataType),
Collections.emptyMap(),
partitionKeys,
defaultPartName);
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/FileSystemTableSource.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableSource.java
similarity index 96%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/FileSystemTableSource.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableSource.java
index db922e8..e518d0e 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/FileSystemTableSource.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableSource.java
@@ -16,8 +16,9 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.io.InputFormat;
import org.apache.flink.api.common.serialization.DeserializationSchema;
import org.apache.flink.api.java.io.CollectionInputFormat;
@@ -26,6 +27,8 @@ import org.apache.flink.configuration.ReadableConfig;
import org.apache.flink.connector.file.src.FileSource;
import org.apache.flink.connector.file.src.FileSourceSplit;
import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.connector.file.table.factories.FileSystemFormatFactory;
+import org.apache.flink.connector.file.table.format.BulkDecodingFormat;
import org.apache.flink.core.fs.Path;
import org.apache.flink.streaming.api.functions.source.InputFormatSourceFunction;
import org.apache.flink.table.api.DataTypes;
@@ -34,7 +37,6 @@ import org.apache.flink.table.api.TableSchema;
import org.apache.flink.table.api.ValidationException;
import org.apache.flink.table.connector.ChangelogMode;
import org.apache.flink.table.connector.Projection;
-import org.apache.flink.table.connector.format.BulkDecodingFormat;
import org.apache.flink.table.connector.format.DecodingFormat;
import org.apache.flink.table.connector.format.ProjectableDecodingFormat;
import org.apache.flink.table.connector.source.InputFormatProvider;
@@ -52,8 +54,6 @@ import org.apache.flink.table.data.TimestampData;
import org.apache.flink.table.expressions.ResolvedExpression;
import org.apache.flink.table.factories.DynamicTableFactory;
import org.apache.flink.table.factories.FactoryUtil;
-import org.apache.flink.table.factories.FileSystemFormatFactory;
-import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.utils.PartitionPathUtils;
import org.apache.flink.table.utils.TableSchemaUtils;
@@ -75,6 +75,7 @@ import java.util.stream.IntStream;
import java.util.stream.Stream;
/** File system table source. */
+@Internal
public class FileSystemTableSource extends AbstractFileSystemTable
implements ScanTableSource,
SupportsProjectionPushDown,
@@ -169,7 +170,7 @@ public class FileSystemTableSource extends AbstractFileSystemTable
return SourceFunctionProvider.of(
new InputFormatSourceFunction<>(
getInputFormat(),
- InternalTypeInfo.of(producedDataType.getLogicalType())),
+ scanContext.createTypeInformation(producedDataType.getLogicalType())),
true);
}
@@ -201,7 +202,11 @@ public class FileSystemTableSource extends AbstractFileSystemTable
format =
wrapBulkFormat(
- format, producedDataType, metadataToExtract, partitionKeysToExtract);
+ scanContext,
+ format,
+ producedDataType,
+ metadataToExtract,
+ partitionKeysToExtract);
return createSourceProvider(format);
} else if (deserializationFormat != null) {
BulkFormat<RowData, FileSourceSplit> format;
@@ -227,7 +232,11 @@ public class FileSystemTableSource extends AbstractFileSystemTable
format =
wrapBulkFormat(
- format, producedDataType, metadataToExtract, partitionKeysToExtract);
+ scanContext,
+ format,
+ producedDataType,
+ metadataToExtract,
+ partitionKeysToExtract);
return createSourceProvider(format);
} else {
throw new TableException("Can not find format factory.");
@@ -239,6 +248,7 @@ public class FileSystemTableSource extends AbstractFileSystemTable
* if needed.
*/
private BulkFormat<RowData, FileSourceSplit> wrapBulkFormat(
+ ScanContext context,
BulkFormat<RowData, FileSourceSplit> bulkFormat,
DataType producedDataType,
List<ReadableFileInfo> metadata,
@@ -248,6 +258,7 @@ public class FileSystemTableSource extends AbstractFileSystemTable
new FileInfoExtractorBulkFormat(
bulkFormat,
producedDataType,
+ context.createTypeInformation(producedDataType),
metadata.stream()
.collect(
Collectors.toMap(
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/GroupedPartitionWriter.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/GroupedPartitionWriter.java
similarity index 98%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/GroupedPartitionWriter.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/GroupedPartitionWriter.java
index e5038c9..2ab92c9 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/GroupedPartitionWriter.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/GroupedPartitionWriter.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.io.OutputFormat;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/LimitableBulkFormat.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/LimitableBulkFormat.java
similarity index 98%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/LimitableBulkFormat.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/LimitableBulkFormat.java
index 3a5f183..feaeb1d 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/LimitableBulkFormat.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/LimitableBulkFormat.java
@@ -16,8 +16,9 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.configuration.Configuration;
@@ -31,6 +32,7 @@ import java.io.IOException;
import java.util.concurrent.atomic.AtomicLong;
/** A {@link BulkFormat} that can limit output record number. */
+@Internal
public class LimitableBulkFormat<T, SplitT extends FileSourceSplit>
implements BulkFormat<T, SplitT> {
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/MetastoreCommitPolicy.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/MetastoreCommitPolicy.java
similarity index 92%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/MetastoreCommitPolicy.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/MetastoreCommitPolicy.java
index dc01de0..85cafc8 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/MetastoreCommitPolicy.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/MetastoreCommitPolicy.java
@@ -16,9 +16,10 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
-import org.apache.flink.table.filesystem.TableMetaStoreFactory.TableMetaStore;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.file.table.TableMetaStoreFactory.TableMetaStore;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -31,6 +32,7 @@ import java.util.LinkedHashMap;
* <p>If this is for file system table, the metastore is a empty implemantation. If this is for hive
* table, the metastore is for connecting to hive metastore.
*/
+@Internal
public class MetastoreCommitPolicy implements PartitionCommitPolicy {
private static final Logger LOG = LoggerFactory.getLogger(MetastoreCommitPolicy.class);
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/OutputFormatFactory.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/OutputFormatFactory.java
similarity index 96%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/OutputFormatFactory.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/OutputFormatFactory.java
index f7ac649..6a4a3aa 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/OutputFormatFactory.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/OutputFormatFactory.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.io.OutputFormat;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/PartitionCommitPolicy.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/PartitionCommitPolicy.java
similarity index 99%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/PartitionCommitPolicy.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/PartitionCommitPolicy.java
index 67c56df..7a82400 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/PartitionCommitPolicy.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/PartitionCommitPolicy.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.annotation.Experimental;
import org.apache.flink.core.fs.FileSystem;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/PartitionComputer.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/PartitionComputer.java
similarity index 97%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/PartitionComputer.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/PartitionComputer.java
index dc01342..b3e0cfe 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/PartitionComputer.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/PartitionComputer.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.annotation.Internal;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/PartitionFetcher.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/PartitionFetcher.java
similarity index 98%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/PartitionFetcher.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/PartitionFetcher.java
index 7f00632..323f949 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/PartitionFetcher.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/PartitionFetcher.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.annotation.Internal;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/PartitionFieldExtractor.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/PartitionFieldExtractor.java
similarity index 95%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/PartitionFieldExtractor.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/PartitionFieldExtractor.java
index 4da3b97..cbbdeaf 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/PartitionFieldExtractor.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/PartitionFieldExtractor.java
@@ -16,8 +16,9 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.connector.file.src.FileSourceSplit;
import org.apache.flink.table.types.logical.LogicalType;
import org.apache.flink.table.utils.PartitionPathUtils;
@@ -27,6 +28,7 @@ import java.util.LinkedHashMap;
/** Interface to extract partition field from split. */
@FunctionalInterface
+@Internal
public interface PartitionFieldExtractor<T extends FileSourceSplit> extends Serializable {
Object extract(T split, String fieldName, LogicalType fieldType);
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/PartitionLoader.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/PartitionLoader.java
similarity index 96%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/PartitionLoader.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/PartitionLoader.java
index 9119dd4..f8be5e5 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/PartitionLoader.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/PartitionLoader.java
@@ -16,13 +16,12 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.annotation.Internal;
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.filesystem.TableMetaStoreFactory.TableMetaStore;
import org.apache.flink.util.Preconditions;
import java.io.Closeable;
@@ -49,7 +48,7 @@ public class PartitionLoader implements Closeable {
private final boolean overwrite;
private final FileSystem fs;
- private final TableMetaStore metaStore;
+ private final TableMetaStoreFactory.TableMetaStore metaStore;
public PartitionLoader(boolean overwrite, FileSystem fs, TableMetaStoreFactory factory)
throws Exception {
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/PartitionReader.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/PartitionReader.java
similarity index 97%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/PartitionReader.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/PartitionReader.java
index 1c64637..51f5a56 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/PartitionReader.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/PartitionReader.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.annotation.Internal;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/PartitionTempFileManager.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/PartitionTempFileManager.java
similarity index 99%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/PartitionTempFileManager.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/PartitionTempFileManager.java
index 141e0ea..36cdfa7 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/PartitionTempFileManager.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/PartitionTempFileManager.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.annotation.Internal;
import org.apache.flink.core.fs.FileStatus;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/PartitionTimeExtractor.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/PartitionTimeExtractor.java
similarity index 97%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/PartitionTimeExtractor.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/PartitionTimeExtractor.java
index 803581b..0a13679 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/PartitionTimeExtractor.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/PartitionTimeExtractor.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.annotation.Experimental;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/PartitionWriter.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/PartitionWriter.java
similarity index 97%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/PartitionWriter.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/PartitionWriter.java
index 8c0029c..bf057cf 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/PartitionWriter.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/PartitionWriter.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.io.OutputFormat;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/PartitionWriterFactory.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/PartitionWriterFactory.java
similarity index 94%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/PartitionWriterFactory.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/PartitionWriterFactory.java
index 1f1c1d0..597595b 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/PartitionWriterFactory.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/PartitionWriterFactory.java
@@ -16,10 +16,10 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.annotation.Internal;
-import org.apache.flink.table.filesystem.PartitionWriter.Context;
+import org.apache.flink.connector.file.table.PartitionWriter.Context;
import java.io.Serializable;
import java.util.LinkedHashMap;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/ProjectingBulkFormat.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/ProjectingBulkFormat.java
similarity index 98%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/ProjectingBulkFormat.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/ProjectingBulkFormat.java
index b60dc46..85c70db 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/ProjectingBulkFormat.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/ProjectingBulkFormat.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.configuration.Configuration;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/RowDataPartitionComputer.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/RowDataPartitionComputer.java
similarity index 99%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/RowDataPartitionComputer.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/RowDataPartitionComputer.java
index c737c83..ca7fcac 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/RowDataPartitionComputer.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/RowDataPartitionComputer.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.annotation.Internal;
import org.apache.flink.table.data.GenericRowData;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/RowPartitionComputer.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/RowPartitionComputer.java
similarity index 99%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/RowPartitionComputer.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/RowPartitionComputer.java
index b1cfefa..4f68796 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/RowPartitionComputer.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/RowPartitionComputer.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.annotation.Internal;
import org.apache.flink.table.types.DataType;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/SerializationSchemaAdapter.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/SerializationSchemaAdapter.java
similarity index 98%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/SerializationSchemaAdapter.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/SerializationSchemaAdapter.java
index 3754369..23e32e3 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/SerializationSchemaAdapter.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/SerializationSchemaAdapter.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.serialization.Encoder;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/SingleDirectoryWriter.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/SingleDirectoryWriter.java
similarity index 98%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/SingleDirectoryWriter.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/SingleDirectoryWriter.java
index c079ec6..11b1a39 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/SingleDirectoryWriter.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/SingleDirectoryWriter.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.io.OutputFormat;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/SuccessFileCommitPolicy.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/SuccessFileCommitPolicy.java
similarity index 94%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/SuccessFileCommitPolicy.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/SuccessFileCommitPolicy.java
index 2a8e876..b96080b 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/SuccessFileCommitPolicy.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/SuccessFileCommitPolicy.java
@@ -16,8 +16,9 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.core.fs.FileSystem;
import org.apache.flink.core.fs.Path;
@@ -28,6 +29,7 @@ import org.slf4j.LoggerFactory;
* Partition commit policy to add success file to directory. Success file is configurable and empty
* file.
*/
+@Internal
public class SuccessFileCommitPolicy implements PartitionCommitPolicy {
private static final Logger LOG = LoggerFactory.getLogger(SuccessFileCommitPolicy.class);
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/TableMetaStoreFactory.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/TableMetaStoreFactory.java
similarity index 98%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/TableMetaStoreFactory.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/TableMetaStoreFactory.java
index 34ae4c7..c1d3a38 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/TableMetaStoreFactory.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/TableMetaStoreFactory.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.annotation.Internal;
import org.apache.flink.core.fs.Path;
diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/BulkReaderFormatFactory.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/factories/BulkReaderFormatFactory.java
similarity index 84%
rename from flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/BulkReaderFormatFactory.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/factories/BulkReaderFormatFactory.java
index 3e09ea1..1d1c7bf 100644
--- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/BulkReaderFormatFactory.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/factories/BulkReaderFormatFactory.java
@@ -16,14 +16,17 @@
* limitations under the License.
*/
-package org.apache.flink.table.factories;
+package org.apache.flink.connector.file.table.factories;
import org.apache.flink.annotation.Internal;
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.connector.format.BulkDecodingFormat;
+import org.apache.flink.connector.file.table.format.BulkDecodingFormat;
import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.DecodingFormatFactory;
+import org.apache.flink.table.factories.DynamicTableFactory;
+import org.apache.flink.table.factories.FactoryUtil;
/**
* Base interface for configuring a {@link BulkFormat} for file system connector.
diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/BulkWriterFormatFactory.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/factories/BulkWriterFormatFactory.java
similarity index 88%
rename from flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/BulkWriterFormatFactory.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/factories/BulkWriterFormatFactory.java
index 0a4e5c3..3c181a6 100644
--- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/BulkWriterFormatFactory.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/factories/BulkWriterFormatFactory.java
@@ -16,11 +16,13 @@
* limitations under the License.
*/
-package org.apache.flink.table.factories;
+package org.apache.flink.connector.file.table.factories;
import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.serialization.BulkWriter;
import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.EncodingFormatFactory;
+import org.apache.flink.table.factories.FactoryUtil;
/**
* Base interface for configuring a {@link BulkWriter.Factory} for file system connector.
diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/FileSystemFormatFactory.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/factories/FileSystemFormatFactory.java
similarity index 97%
rename from flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/FileSystemFormatFactory.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/factories/FileSystemFormatFactory.java
index 8bf42d5..e450761 100644
--- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/FileSystemFormatFactory.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/factories/FileSystemFormatFactory.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.factories;
+package org.apache.flink.connector.file.table.factories;
import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.io.InputFormat;
@@ -25,6 +25,7 @@ import org.apache.flink.core.fs.Path;
import org.apache.flink.table.api.TableSchema;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.expressions.ResolvedExpression;
+import org.apache.flink.table.factories.Factory;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.logical.LogicalType;
import org.apache.flink.table.types.logical.RowType;
diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/format/BulkDecodingFormat.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/format/BulkDecodingFormat.java
similarity index 89%
rename from flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/format/BulkDecodingFormat.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/format/BulkDecodingFormat.java
index fd20756..0108f16 100644
--- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/format/BulkDecodingFormat.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/format/BulkDecodingFormat.java
@@ -16,11 +16,13 @@
* limitations under the License.
*/
-package org.apache.flink.table.connector.format;
+package org.apache.flink.connector.file.table.format;
import org.apache.flink.annotation.Internal;
import org.apache.flink.connector.file.src.FileSourceSplit;
import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.table.connector.format.DecodingFormat;
+import org.apache.flink.table.connector.format.Format;
import org.apache.flink.table.connector.source.DynamicTableSource;
import org.apache.flink.table.expressions.ResolvedExpression;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/AbstractStreamingWriter.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/AbstractStreamingWriter.java
similarity index 99%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/AbstractStreamingWriter.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/AbstractStreamingWriter.java
index 2a96c62..237ecf2 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/AbstractStreamingWriter.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/AbstractStreamingWriter.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem.stream;
+package org.apache.flink.connector.file.table.stream;
import org.apache.flink.core.fs.Path;
import org.apache.flink.runtime.state.StateInitializationContext;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/PartitionCommitInfo.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/PartitionCommitInfo.java
similarity index 95%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/PartitionCommitInfo.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/PartitionCommitInfo.java
index 98c524d..abc6ede 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/PartitionCommitInfo.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/PartitionCommitInfo.java
@@ -16,7 +16,9 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem.stream;
+package org.apache.flink.connector.file.table.stream;
+
+import org.apache.flink.annotation.Internal;
import java.io.Serializable;
import java.util.List;
@@ -27,6 +29,7 @@ import java.util.List;
* <p>Need to ensure that the partitions are ready to commit. That is to say, the files in the
* partition have become readable rather than temporary.
*/
+@Internal
public class PartitionCommitInfo implements Serializable {
private static final long serialVersionUID = 1L;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/PartitionCommitPredicate.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/PartitionCommitPredicate.java
similarity index 90%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/PartitionCommitPredicate.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/PartitionCommitPredicate.java
index 3e17207..40d4b7e 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/PartitionCommitPredicate.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/PartitionCommitPredicate.java
@@ -16,19 +16,19 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem.stream;
+package org.apache.flink.connector.file.table.stream;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.configuration.Configuration;
-import org.apache.flink.table.filesystem.FileSystemConnectorOptions.PartitionCommitTriggerType;
+import org.apache.flink.connector.file.table.FileSystemConnectorOptions;
import java.util.List;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_TRIGGER;
-
/**
* Partition commit predicate. See {@link PartitionTimeCommitPredicate}. See {@link
* ProcTimeCommitPredicate}
*/
+@Internal
public interface PartitionCommitPredicate {
boolean isPartitionCommittable(PredicateContext predicateContext);
@@ -86,7 +86,8 @@ public interface PartitionCommitPredicate {
static PartitionCommitPredicate create(
Configuration conf, ClassLoader cl, List<String> partitionKeys) {
- PartitionCommitTriggerType trigger = conf.get(SINK_PARTITION_COMMIT_TRIGGER);
+ FileSystemConnectorOptions.PartitionCommitTriggerType trigger =
+ conf.get(FileSystemConnectorOptions.SINK_PARTITION_COMMIT_TRIGGER);
switch (trigger) {
case PARTITION_TIME:
return createPartitionTimeCommitPredicate(conf, cl, partitionKeys);
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/PartitionCommitTrigger.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/PartitionCommitTrigger.java
similarity index 87%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/PartitionCommitTrigger.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/PartitionCommitTrigger.java
index cbc0ce7..7b62cbf 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/PartitionCommitTrigger.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/PartitionCommitTrigger.java
@@ -16,22 +16,24 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem.stream;
+package org.apache.flink.connector.file.table.stream;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.state.OperatorStateStore;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.table.FileSystemConnectorOptions;
import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
-import org.apache.flink.table.filesystem.FileSystemConnectorOptions.PartitionCommitTriggerType;
import java.io.IOException;
import java.util.List;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_TRIGGER;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_TRIGGER;
/**
* Partition commit trigger. See {@link PartitionTimeCommitTrigger}. See {@link
* ProcTimeCommitTrigger}.
*/
+@Internal
public interface PartitionCommitTrigger {
/** Add a pending partition. */
@@ -55,7 +57,8 @@ public interface PartitionCommitTrigger {
ProcessingTimeService procTimeService)
throws Exception {
PartitionCommitPredicate partitionCommitPredicate;
- PartitionCommitTriggerType trigger = conf.get(SINK_PARTITION_COMMIT_TRIGGER);
+ FileSystemConnectorOptions.PartitionCommitTriggerType trigger =
+ conf.get(SINK_PARTITION_COMMIT_TRIGGER);
switch (trigger) {
case PARTITION_TIME:
partitionCommitPredicate =
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/PartitionCommitter.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/PartitionCommitter.java
similarity index 91%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/PartitionCommitter.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/PartitionCommitter.java
index 8eb0b8e..08bc828 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/PartitionCommitter.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/PartitionCommitter.java
@@ -16,10 +16,16 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem.stream;
+package org.apache.flink.connector.file.table.stream;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.state.CheckpointListener;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.table.EmptyMetaStoreFactory;
+import org.apache.flink.connector.file.table.FileSystemFactory;
+import org.apache.flink.connector.file.table.MetastoreCommitPolicy;
+import org.apache.flink.connector.file.table.PartitionCommitPolicy;
+import org.apache.flink.connector.file.table.TableMetaStoreFactory;
import org.apache.flink.core.fs.Path;
import org.apache.flink.runtime.state.StateInitializationContext;
import org.apache.flink.runtime.state.StateSnapshotContext;
@@ -28,20 +34,15 @@ import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.table.catalog.ObjectIdentifier;
-import org.apache.flink.table.filesystem.EmptyMetaStoreFactory;
-import org.apache.flink.table.filesystem.FileSystemFactory;
-import org.apache.flink.table.filesystem.MetastoreCommitPolicy;
-import org.apache.flink.table.filesystem.PartitionCommitPolicy;
-import org.apache.flink.table.filesystem.TableMetaStoreFactory;
import java.io.IOException;
import java.util.ArrayList;
import java.util.LinkedHashMap;
import java.util.List;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_POLICY_CLASS;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_POLICY_KIND;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_SUCCESS_FILE_NAME;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_POLICY_CLASS;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_POLICY_KIND;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_SUCCESS_FILE_NAME;
import static org.apache.flink.table.utils.PartitionPathUtils.extractPartitionSpecFromPath;
import static org.apache.flink.table.utils.PartitionPathUtils.generatePartitionPath;
@@ -58,6 +59,7 @@ import static org.apache.flink.table.utils.PartitionPathUtils.generatePartitionP
* 3.Extracting committable partitions from {@link PartitionCommitTrigger}. 4.Using {@link
* PartitionCommitPolicy} chain to commit partitions.
*/
+@Internal
public class PartitionCommitter extends AbstractStreamOperator<Void>
implements OneInputStreamOperator<PartitionCommitInfo, Void> {
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/PartitionTimeCommitPredicate.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/PartitionTimeCommitPredicate.java
similarity index 81%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/PartitionTimeCommitPredicate.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/PartitionTimeCommitPredicate.java
index 3e48c2e..b539614 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/PartitionTimeCommitPredicate.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/PartitionTimeCommitPredicate.java
@@ -16,27 +16,29 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem.stream;
+package org.apache.flink.connector.file.table.stream;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.table.PartitionTimeExtractor;
import org.apache.flink.core.fs.Path;
-import org.apache.flink.table.filesystem.PartitionTimeExtractor;
import java.time.LocalDateTime;
import java.time.ZoneId;
import java.util.List;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.PARTITION_TIME_EXTRACTOR_CLASS;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.PARTITION_TIME_EXTRACTOR_KIND;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.PARTITION_TIME_EXTRACTOR_TIMESTAMP_PATTERN;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_DELAY;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_WATERMARK_TIME_ZONE;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.PARTITION_TIME_EXTRACTOR_CLASS;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.PARTITION_TIME_EXTRACTOR_KIND;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.PARTITION_TIME_EXTRACTOR_TIMESTAMP_PATTERN;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_DELAY;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_WATERMARK_TIME_ZONE;
import static org.apache.flink.table.utils.PartitionPathUtils.extractPartitionValues;
/**
* Partition commit predicate by partition time and watermark, if 'watermark' > 'partition-time' +
* 'delay', the partition is committable.
*/
+@Internal
public class PartitionTimeCommitPredicate implements PartitionCommitPredicate {
private final PartitionTimeExtractor extractor;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/PartitionTimeCommitTrigger.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/PartitionTimeCommitTrigger.java
similarity index 96%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/PartitionTimeCommitTrigger.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/PartitionTimeCommitTrigger.java
index 46ea88b..739dfbf 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/PartitionTimeCommitTrigger.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/PartitionTimeCommitTrigger.java
@@ -16,8 +16,9 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem.stream;
+package org.apache.flink.connector.file.table.stream;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.state.ListState;
import org.apache.flink.api.common.state.ListStateDescriptor;
import org.apache.flink.api.common.state.OperatorStateStore;
@@ -25,6 +26,7 @@ import org.apache.flink.api.common.typeutils.base.ListSerializer;
import org.apache.flink.api.common.typeutils.base.LongSerializer;
import org.apache.flink.api.common.typeutils.base.MapSerializer;
import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.connector.file.table.stream.PartitionCommitPredicate.PredicateContext;
import org.apache.flink.util.StringUtils;
import java.util.ArrayList;
@@ -36,8 +38,6 @@ import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
-import static org.apache.flink.table.filesystem.stream.PartitionCommitPredicate.PredicateContext;
-
/**
* Partition commit trigger by partition time and watermark. It'll commit the partition predicated
* to be committable by {@link PartitionCommitPredicate}
@@ -45,6 +45,7 @@ import static org.apache.flink.table.filesystem.stream.PartitionCommitPredicate.
* <p>Compares watermark, and watermark is related to records and checkpoint, so we need store
* watermark information for checkpoint.
*/
+@Internal
public class PartitionTimeCommitTrigger implements PartitionCommitTrigger {
private static final ListStateDescriptor<List<String>> PENDING_PARTITIONS_STATE_DESC =
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/ProcTimeCommitPredicate.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/ProcTimeCommitPredicate.java
similarity index 87%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/ProcTimeCommitPredicate.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/ProcTimeCommitPredicate.java
index be03c3f..7577f8a 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/ProcTimeCommitPredicate.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/ProcTimeCommitPredicate.java
@@ -16,16 +16,18 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem.stream;
+package org.apache.flink.connector.file.table.stream;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.configuration.Configuration;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_DELAY;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_DELAY;
/**
* Partition commit trigger by creation time and processing time service, if 'current processing
* time' > 'partition creation time' + 'delay', the partition is committable.
*/
+@Internal
public class ProcTimeCommitPredicate implements PartitionCommitPredicate {
private final long commitDelay;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/ProcTimeCommitTrigger.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/ProcTimeCommitTrigger.java
similarity index 95%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/ProcTimeCommitTrigger.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/ProcTimeCommitTrigger.java
index 814a4e9..ebd7293 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/ProcTimeCommitTrigger.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/ProcTimeCommitTrigger.java
@@ -16,8 +16,9 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem.stream;
+package org.apache.flink.connector.file.table.stream;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.state.ListState;
import org.apache.flink.api.common.state.ListStateDescriptor;
import org.apache.flink.api.common.state.OperatorStateStore;
@@ -33,12 +34,13 @@ import java.util.Iterator;
import java.util.List;
import java.util.Map;
-import static org.apache.flink.table.filesystem.stream.PartitionCommitPredicate.PredicateContext;
+import static org.apache.flink.connector.file.table.stream.PartitionCommitPredicate.PredicateContext;
/**
* Partition commit trigger by creation time and processing time service. It'll commit the partition
* predicated to be committable by {@link PartitionCommitPredicate}
*/
+@Internal
public class ProcTimeCommitTrigger implements PartitionCommitTrigger {
private static final ListStateDescriptor<Map<String, Long>> PENDING_PARTITIONS_STATE_DESC =
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/StreamingFileWriter.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/StreamingFileWriter.java
similarity index 94%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/StreamingFileWriter.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/StreamingFileWriter.java
index cff3dc7..eec6907 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/StreamingFileWriter.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/StreamingFileWriter.java
@@ -16,8 +16,9 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem.stream;
+package org.apache.flink.connector.file.table.stream;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.fs.Path;
import org.apache.flink.runtime.state.StateInitializationContext;
@@ -35,10 +36,11 @@ import java.util.NavigableMap;
import java.util.Set;
import java.util.TreeMap;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_POLICY_KIND;
-import static org.apache.flink.table.filesystem.stream.PartitionCommitPredicate.PredicateContext;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_POLICY_KIND;
+import static org.apache.flink.connector.file.table.stream.PartitionCommitPredicate.PredicateContext;
/** Writer for emitting {@link PartitionCommitInfo} to downstream. */
+@Internal
public class StreamingFileWriter<IN> extends AbstractStreamingWriter<IN, PartitionCommitInfo> {
private static final long serialVersionUID = 2L;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/StreamingSink.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/StreamingSink.java
similarity index 86%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/StreamingSink.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/StreamingSink.java
index 72eb2fa..ee15753 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/StreamingSink.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/StreamingSink.java
@@ -16,11 +16,22 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem.stream;
+package org.apache.flink.connector.file.table.stream;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeinfo.Types;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.table.FileSystemFactory;
+import org.apache.flink.connector.file.table.TableMetaStoreFactory;
+import org.apache.flink.connector.file.table.stream.compact.CompactBucketWriter;
+import org.apache.flink.connector.file.table.stream.compact.CompactCoordinator;
+import org.apache.flink.connector.file.table.stream.compact.CompactFileWriter;
+import org.apache.flink.connector.file.table.stream.compact.CompactMessages.CoordinatorInput;
+import org.apache.flink.connector.file.table.stream.compact.CompactMessages.CoordinatorOutput;
+import org.apache.flink.connector.file.table.stream.compact.CompactOperator;
+import org.apache.flink.connector.file.table.stream.compact.CompactReader;
+import org.apache.flink.connector.file.table.stream.compact.CompactWriter;
import org.apache.flink.core.fs.FileSystem;
import org.apache.flink.core.fs.Path;
import org.apache.flink.streaming.api.datastream.DataStream;
@@ -30,25 +41,16 @@ import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
import org.apache.flink.streaming.api.functions.sink.filesystem.BucketWriter;
import org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink;
import org.apache.flink.table.catalog.ObjectIdentifier;
-import org.apache.flink.table.filesystem.FileSystemFactory;
-import org.apache.flink.table.filesystem.TableMetaStoreFactory;
-import org.apache.flink.table.filesystem.stream.compact.CompactBucketWriter;
-import org.apache.flink.table.filesystem.stream.compact.CompactCoordinator;
-import org.apache.flink.table.filesystem.stream.compact.CompactFileWriter;
-import org.apache.flink.table.filesystem.stream.compact.CompactMessages.CoordinatorInput;
-import org.apache.flink.table.filesystem.stream.compact.CompactMessages.CoordinatorOutput;
-import org.apache.flink.table.filesystem.stream.compact.CompactOperator;
-import org.apache.flink.table.filesystem.stream.compact.CompactReader;
-import org.apache.flink.table.filesystem.stream.compact.CompactWriter;
import org.apache.flink.util.function.SupplierWithException;
import java.io.IOException;
import java.io.Serializable;
import java.util.List;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_POLICY_KIND;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_POLICY_KIND;
/** Helper for creating streaming file sink. */
+@Internal
public class StreamingSink {
private StreamingSink() {}
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/TaskTracker.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/TaskTracker.java
similarity index 93%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/TaskTracker.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/TaskTracker.java
index 55eb8eb..379431a 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/TaskTracker.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/TaskTracker.java
@@ -16,7 +16,9 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem.stream;
+package org.apache.flink.connector.file.table.stream;
+
+import org.apache.flink.annotation.Internal;
import java.util.HashSet;
import java.util.Set;
@@ -26,6 +28,7 @@ import java.util.TreeMap;
* Track the upstream tasks to determine whether all the upstream data of a checkpoint has been
* received.
*/
+@Internal
public class TaskTracker {
private final int numberOfTasks;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/compact/CompactBucketWriter.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/compact/CompactBucketWriter.java
similarity index 96%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/compact/CompactBucketWriter.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/compact/CompactBucketWriter.java
index 9e7ca7f..e01c279 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/compact/CompactBucketWriter.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/compact/CompactBucketWriter.java
@@ -16,8 +16,9 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem.stream.compact;
+package org.apache.flink.connector.file.table.stream.compact;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.streaming.api.functions.sink.filesystem.BucketWriter;
import org.apache.flink.streaming.api.functions.sink.filesystem.InProgressFileWriter;
import org.apache.flink.util.function.SupplierWithException;
@@ -25,6 +26,7 @@ import org.apache.flink.util.function.SupplierWithException;
import java.io.IOException;
/** The {@link CompactWriter} to delegate {@link BucketWriter}. */
+@Internal
public class CompactBucketWriter<T> implements CompactWriter<T> {
private final BucketWriter<T, String> bucketWriter;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/compact/CompactBulkReader.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/compact/CompactBulkReader.java
similarity index 96%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/compact/CompactBulkReader.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/compact/CompactBulkReader.java
index 866bd8d..b85e36c 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/compact/CompactBulkReader.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/compact/CompactBulkReader.java
@@ -16,8 +16,9 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem.stream.compact;
+package org.apache.flink.connector.file.table.stream.compact;
+import org.apache.flink.annotation.Internal;
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;
@@ -27,6 +28,7 @@ import java.io.IOException;
import java.util.UUID;
/** The {@link CompactReader} to delegate {@link CompactBulkReader}. */
+@Internal
public class CompactBulkReader<T> implements CompactReader<T> {
private final BulkFormat.Reader<T> reader;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/compact/CompactContext.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/compact/CompactContext.java
similarity index 95%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/compact/CompactContext.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/compact/CompactContext.java
index 4f81544..42d1aaa 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/compact/CompactContext.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/compact/CompactContext.java
@@ -16,13 +16,15 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem.stream.compact;
+package org.apache.flink.connector.file.table.stream.compact;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.fs.FileSystem;
import org.apache.flink.core.fs.Path;
/** Context for {@link CompactReader} and {@link CompactWriter}. */
+@Internal
public interface CompactContext {
static CompactContext create(
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/compact/CompactCoordinator.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/compact/CompactCoordinator.java
similarity index 91%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/compact/CompactCoordinator.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/compact/CompactCoordinator.java
index 1bf982a..b4ae2a8 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/compact/CompactCoordinator.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/compact/CompactCoordinator.java
@@ -16,8 +16,9 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem.stream.compact;
+package org.apache.flink.connector.file.table.stream.compact;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.state.ListState;
import org.apache.flink.api.common.state.ListStateDescriptor;
import org.apache.flink.api.common.typeutils.base.ListSerializer;
@@ -25,6 +26,14 @@ import org.apache.flink.api.common.typeutils.base.LongSerializer;
import org.apache.flink.api.common.typeutils.base.MapSerializer;
import org.apache.flink.api.common.typeutils.base.StringSerializer;
import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer;
+import org.apache.flink.connector.file.table.BinPacking;
+import org.apache.flink.connector.file.table.stream.TaskTracker;
+import org.apache.flink.connector.file.table.stream.compact.CompactMessages.CompactionUnit;
+import org.apache.flink.connector.file.table.stream.compact.CompactMessages.CoordinatorInput;
+import org.apache.flink.connector.file.table.stream.compact.CompactMessages.CoordinatorOutput;
+import org.apache.flink.connector.file.table.stream.compact.CompactMessages.EndCheckpoint;
+import org.apache.flink.connector.file.table.stream.compact.CompactMessages.EndCompaction;
+import org.apache.flink.connector.file.table.stream.compact.CompactMessages.InputFile;
import org.apache.flink.core.fs.FileSystem;
import org.apache.flink.core.fs.Path;
import org.apache.flink.runtime.state.StateInitializationContext;
@@ -32,14 +41,6 @@ import org.apache.flink.runtime.state.StateSnapshotContext;
import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.table.filesystem.stream.TaskTracker;
-import org.apache.flink.table.filesystem.stream.compact.CompactMessages.CompactionUnit;
-import org.apache.flink.table.filesystem.stream.compact.CompactMessages.CoordinatorInput;
-import org.apache.flink.table.filesystem.stream.compact.CompactMessages.CoordinatorOutput;
-import org.apache.flink.table.filesystem.stream.compact.CompactMessages.EndCheckpoint;
-import org.apache.flink.table.filesystem.stream.compact.CompactMessages.EndCompaction;
-import org.apache.flink.table.filesystem.stream.compact.CompactMessages.InputFile;
-import org.apache.flink.table.runtime.util.BinPacking;
import org.apache.flink.util.function.SupplierWithException;
import org.slf4j.Logger;
@@ -71,6 +72,7 @@ import java.util.function.Function;
* <p>STATE: This operator stores input files in state, after the checkpoint completes successfully,
* input files are taken out from the state for coordination.
*/
+@Internal
public class CompactCoordinator extends AbstractStreamOperator<CoordinatorOutput>
implements OneInputStreamOperator<CoordinatorInput, CoordinatorOutput> {
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/compact/CompactFileWriter.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/compact/CompactFileWriter.java
similarity index 80%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/compact/CompactFileWriter.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/compact/CompactFileWriter.java
index d13e9813..1864c34 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/compact/CompactFileWriter.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/compact/CompactFileWriter.java
@@ -16,18 +16,20 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem.stream.compact;
+package org.apache.flink.connector.file.table.stream.compact;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.file.table.stream.AbstractStreamingWriter;
+import org.apache.flink.connector.file.table.stream.compact.CompactMessages.CoordinatorInput;
+import org.apache.flink.connector.file.table.stream.compact.CompactMessages.EndCheckpoint;
+import org.apache.flink.connector.file.table.stream.compact.CompactMessages.InputFile;
import org.apache.flink.core.fs.Path;
import org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.table.filesystem.stream.AbstractStreamingWriter;
-import org.apache.flink.table.filesystem.stream.compact.CompactMessages.EndCheckpoint;
-import org.apache.flink.table.filesystem.stream.compact.CompactMessages.InputFile;
/** Writer for emitting {@link InputFile} and {@link EndCheckpoint} to downstream. */
-public class CompactFileWriter<T>
- extends AbstractStreamingWriter<T, CompactMessages.CoordinatorInput> {
+@Internal
+public class CompactFileWriter<T> extends AbstractStreamingWriter<T, CoordinatorInput> {
private static final long serialVersionUID = 1L;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/compact/CompactMessages.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/compact/CompactMessages.java
similarity index 97%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/compact/CompactMessages.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/compact/CompactMessages.java
index d44dd03..fc905cd 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/compact/CompactMessages.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/compact/CompactMessages.java
@@ -16,8 +16,9 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem.stream.compact;
+package org.apache.flink.connector.file.table.stream.compact;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.core.fs.Path;
import java.io.Serializable;
@@ -37,6 +38,7 @@ import java.util.stream.Collectors;
* <p>Because the end message is a kind of barrier of record messages, they can only be transmitted
* in the way of full broadcast in the link from coordinator to compact operator.
*/
+@Internal
public class CompactMessages {
private CompactMessages() {}
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/compact/CompactOperator.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/compact/CompactOperator.java
similarity index 96%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/compact/CompactOperator.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/compact/CompactOperator.java
index d31e155..71e7c71 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/compact/CompactOperator.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/compact/CompactOperator.java
@@ -16,8 +16,9 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem.stream.compact;
+package org.apache.flink.connector.file.table.stream.compact;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.state.ListState;
import org.apache.flink.api.common.state.ListStateDescriptor;
import org.apache.flink.api.common.typeutils.base.ListSerializer;
@@ -26,6 +27,10 @@ import org.apache.flink.api.common.typeutils.base.MapSerializer;
import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.connector.file.table.stream.PartitionCommitInfo;
+import org.apache.flink.connector.file.table.stream.compact.CompactMessages.CompactionUnit;
+import org.apache.flink.connector.file.table.stream.compact.CompactMessages.CoordinatorOutput;
+import org.apache.flink.connector.file.table.stream.compact.CompactMessages.EndCompaction;
import org.apache.flink.core.fs.FSDataInputStream;
import org.apache.flink.core.fs.FileSystem;
import org.apache.flink.core.fs.Path;
@@ -38,10 +43,6 @@ import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
import org.apache.flink.streaming.api.operators.BoundedOneInput;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.table.filesystem.stream.PartitionCommitInfo;
-import org.apache.flink.table.filesystem.stream.compact.CompactMessages.CompactionUnit;
-import org.apache.flink.table.filesystem.stream.compact.CompactMessages.CoordinatorOutput;
-import org.apache.flink.table.filesystem.stream.compact.CompactMessages.EndCompaction;
import org.apache.flink.util.IOUtils;
import org.apache.flink.util.Preconditions;
import org.apache.flink.util.function.SupplierWithException;
@@ -66,6 +67,7 @@ import java.util.TreeMap;
* successfully, We can ensure that these files will not be used again and they can be deleted from
* the file system.
*/
+@Internal
public class CompactOperator<T> extends AbstractStreamOperator<PartitionCommitInfo>
implements OneInputStreamOperator<CoordinatorOutput, PartitionCommitInfo>, BoundedOneInput {
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/compact/CompactReader.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/compact/CompactReader.java
similarity index 91%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/compact/CompactReader.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/compact/CompactReader.java
index c396461..672bfba 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/compact/CompactReader.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/compact/CompactReader.java
@@ -16,13 +16,16 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem.stream.compact;
+package org.apache.flink.connector.file.table.stream.compact;
+
+import org.apache.flink.annotation.Internal;
import java.io.Closeable;
import java.io.IOException;
import java.io.Serializable;
/** Reader for compaction. */
+@Internal
public interface CompactReader<T> extends Closeable {
/** Read a record, return null if no more record. */
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/compact/CompactWriter.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/compact/CompactWriter.java
similarity index 91%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/compact/CompactWriter.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/compact/CompactWriter.java
index 33e3aae..cc108d5 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/compact/CompactWriter.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/compact/CompactWriter.java
@@ -16,12 +16,15 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem.stream.compact;
+package org.apache.flink.connector.file.table.stream.compact;
+
+import org.apache.flink.annotation.Internal;
import java.io.IOException;
import java.io.Serializable;
/** Writer for compaction. */
+@Internal
public interface CompactWriter<T> {
void write(T record) throws IOException;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/compact/FileInputFormatCompactReader.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/compact/FileInputFormatCompactReader.java
similarity index 95%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/compact/FileInputFormatCompactReader.java
rename to flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/compact/FileInputFormatCompactReader.java
index 15310e1..d6e7a7a 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/compact/FileInputFormatCompactReader.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/compact/FileInputFormatCompactReader.java
@@ -16,14 +16,16 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem.stream.compact;
+package org.apache.flink.connector.file.table.stream.compact;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.io.FileInputFormat;
import org.apache.flink.core.fs.FileInputSplit;
import java.io.IOException;
/** The {@link CompactReader} to delegate {@link FileInputFormat}. */
+@Internal
public class FileInputFormatCompactReader<T> implements CompactReader<T> {
private final FileInputFormat<T> format;
diff --git a/flink-table/flink-table-runtime/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-connectors/flink-connector-files/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
similarity index 92%
rename from flink-table/flink-table-runtime/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory
rename to flink-connectors/flink-connector-files/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
index 0d3371a..d1cf812 100644
--- a/flink-table/flink-table-runtime/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory
+++ b/flink-connectors/flink-connector-files/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
@@ -13,4 +13,4 @@
# See the License for the specific language governing permissions and
# limitations under the License.
-org.apache.flink.formats.testcsv.TestCsvFileSystemFormatFactory
+org.apache.flink.connector.file.table.FileSystemTableFactory
diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/util/BinPackingTest.java b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/BinPackingTest.java
similarity index 85%
rename from flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/util/BinPackingTest.java
rename to flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/BinPackingTest.java
index 0f150c1..7ac9d52 100644
--- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/util/BinPackingTest.java
+++ b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/BinPackingTest.java
@@ -7,17 +7,16 @@
* "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
+ * 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.
+ * 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.runtime.util;
+package org.apache.flink.connector.file.table;
import org.junit.Assert;
import org.junit.Test;
diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/EnrichedRowDataTest.java b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/EnrichedRowDataTest.java
similarity index 98%
rename from flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/EnrichedRowDataTest.java
rename to flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/EnrichedRowDataTest.java
index ea3efdb..d790e26 100644
--- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/EnrichedRowDataTest.java
+++ b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/EnrichedRowDataTest.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.table.data.GenericRowData;
import org.apache.flink.table.data.RowData;
diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/FileSystemCommitterTest.java b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/FileSystemCommitterTest.java
similarity index 99%
rename from flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/FileSystemCommitterTest.java
rename to flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/FileSystemCommitterTest.java
index a784050..d7fc75e 100644
--- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/FileSystemCommitterTest.java
+++ b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/FileSystemCommitterTest.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.core.fs.FileSystem;
import org.apache.flink.core.fs.Path;
diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/FileSystemOutputFormatTest.java b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/FileSystemOutputFormatTest.java
similarity index 99%
rename from flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/FileSystemOutputFormatTest.java
rename to flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/FileSystemOutputFormatTest.java
index 057695b..efb12be 100644
--- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/FileSystemOutputFormatTest.java
+++ b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/FileSystemOutputFormatTest.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.api.java.io.TextOutputFormat;
import org.apache.flink.core.fs.Path;
diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/LimitableBulkFormatTest.java b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/LimitableBulkFormatTest.java
similarity index 99%
rename from flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/LimitableBulkFormatTest.java
rename to flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/LimitableBulkFormatTest.java
index 11b461f..c692664 100644
--- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/LimitableBulkFormatTest.java
+++ b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/LimitableBulkFormatTest.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeinfo.Types;
diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/PartitionWriterTest.java b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/PartitionWriterTest.java
similarity index 98%
rename from flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/PartitionWriterTest.java
rename to flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/PartitionWriterTest.java
index d62ef2a..440a924 100644
--- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/PartitionWriterTest.java
+++ b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/PartitionWriterTest.java
@@ -16,13 +16,13 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.api.common.io.OutputFormat;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.table.PartitionWriter.Context;
import org.apache.flink.core.fs.FileSystem;
import org.apache.flink.core.fs.Path;
-import org.apache.flink.table.filesystem.PartitionWriter.Context;
import org.apache.flink.table.utils.LegacyRowResource;
import org.apache.flink.types.Row;
diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/RowPartitionComputerTest.java b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/RowPartitionComputerTest.java
similarity index 98%
rename from flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/RowPartitionComputerTest.java
rename to flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/RowPartitionComputerTest.java
index 9b6c836..7a006a4 100644
--- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/RowPartitionComputerTest.java
+++ b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/RowPartitionComputerTest.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.types.Row;
diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/stream/StreamingFileWriterTest.java b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/stream/StreamingFileWriterTest.java
similarity index 97%
rename from flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/stream/StreamingFileWriterTest.java
rename to flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/stream/StreamingFileWriterTest.java
index ef98667..7684f6ef 100644
--- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/stream/StreamingFileWriterTest.java
+++ b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/stream/StreamingFileWriterTest.java
@@ -16,10 +16,11 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem.stream;
+package org.apache.flink.connector.file.table.stream;
import org.apache.flink.api.common.serialization.Encoder;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.table.FileSystemTableSink;
import org.apache.flink.core.fs.Path;
import org.apache.flink.core.io.SimpleVersionedSerializer;
import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
@@ -34,7 +35,6 @@ import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
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.filesystem.FileSystemTableSink;
import org.junit.Assert;
import org.junit.Before;
@@ -55,10 +55,10 @@ import java.util.Collections;
import java.util.Date;
import java.util.List;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_DELAY;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_POLICY_KIND;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_TRIGGER;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_WATERMARK_TIME_ZONE;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_DELAY;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_POLICY_KIND;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_TRIGGER;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_WATERMARK_TIME_ZONE;
/** Test for {@link StreamingFileWriter}. */
public class StreamingFileWriterTest {
diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/stream/compact/AbstractCompactTestBase.java b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/stream/compact/AbstractCompactTestBase.java
similarity index 96%
rename from flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/stream/compact/AbstractCompactTestBase.java
rename to flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/stream/compact/AbstractCompactTestBase.java
index 4518d16..55e6e82 100644
--- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/stream/compact/AbstractCompactTestBase.java
+++ b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/stream/compact/AbstractCompactTestBase.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem.stream.compact;
+package org.apache.flink.connector.file.table.stream.compact;
import org.apache.flink.core.fs.Path;
diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/stream/compact/CompactCoordinatorTest.java b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/stream/compact/CompactCoordinatorTest.java
similarity index 90%
rename from flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/stream/compact/CompactCoordinatorTest.java
rename to flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/stream/compact/CompactCoordinatorTest.java
index d000554..a17ba0e 100644
--- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/stream/compact/CompactCoordinatorTest.java
+++ b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/stream/compact/CompactCoordinatorTest.java
@@ -16,17 +16,17 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem.stream.compact;
-
+package org.apache.flink.connector.file.table.stream.compact;
+
+import org.apache.flink.connector.file.table.stream.compact.CompactMessages.CompactionUnit;
+import org.apache.flink.connector.file.table.stream.compact.CompactMessages.CoordinatorInput;
+import org.apache.flink.connector.file.table.stream.compact.CompactMessages.CoordinatorOutput;
+import org.apache.flink.connector.file.table.stream.compact.CompactMessages.EndCheckpoint;
+import org.apache.flink.connector.file.table.stream.compact.CompactMessages.EndCompaction;
+import org.apache.flink.connector.file.table.stream.compact.CompactMessages.InputFile;
import org.apache.flink.core.fs.Path;
import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.apache.flink.table.filesystem.stream.compact.CompactMessages.CompactionUnit;
-import org.apache.flink.table.filesystem.stream.compact.CompactMessages.CoordinatorInput;
-import org.apache.flink.table.filesystem.stream.compact.CompactMessages.CoordinatorOutput;
-import org.apache.flink.table.filesystem.stream.compact.CompactMessages.EndCheckpoint;
-import org.apache.flink.table.filesystem.stream.compact.CompactMessages.EndCompaction;
-import org.apache.flink.table.filesystem.stream.compact.CompactMessages.InputFile;
import org.apache.flink.util.function.ThrowingConsumer;
import org.junit.Assert;
diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/stream/compact/CompactFileWriterTest.java b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/stream/compact/CompactFileWriterTest.java
similarity index 89%
rename from flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/stream/compact/CompactFileWriterTest.java
rename to flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/stream/compact/CompactFileWriterTest.java
index 9c5fd89..3fe95a4 100644
--- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/stream/compact/CompactFileWriterTest.java
+++ b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/stream/compact/CompactFileWriterTest.java
@@ -16,23 +16,23 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem.stream.compact;
+package org.apache.flink.connector.file.table.stream.compact;
import org.apache.flink.api.common.serialization.SimpleStringEncoder;
+import org.apache.flink.connector.file.table.stream.compact.CompactMessages.EndCheckpoint;
import org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink;
import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
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.filesystem.stream.compact.CompactMessages.EndCheckpoint;
import org.junit.Assert;
import org.junit.Test;
import java.util.List;
-import static org.apache.flink.table.filesystem.stream.compact.CompactMessages.CoordinatorInput;
-import static org.apache.flink.table.filesystem.stream.compact.CompactMessages.InputFile;
+import static org.apache.flink.connector.file.table.stream.compact.CompactMessages.CoordinatorInput;
+import static org.apache.flink.connector.file.table.stream.compact.CompactMessages.InputFile;
/** Test for {@link CompactFileWriter}. */
public class CompactFileWriterTest extends AbstractCompactTestBase {
diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/stream/compact/CompactOperatorTest.java b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/stream/compact/CompactOperatorTest.java
similarity index 96%
rename from flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/stream/compact/CompactOperatorTest.java
rename to flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/stream/compact/CompactOperatorTest.java
index 74b2b71..28e3234 100644
--- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/stream/compact/CompactOperatorTest.java
+++ b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/stream/compact/CompactOperatorTest.java
@@ -16,17 +16,17 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem.stream.compact;
+package org.apache.flink.connector.file.table.stream.compact;
+import org.apache.flink.connector.file.table.stream.PartitionCommitInfo;
+import org.apache.flink.connector.file.table.stream.compact.CompactMessages.CompactionUnit;
+import org.apache.flink.connector.file.table.stream.compact.CompactMessages.CoordinatorOutput;
+import org.apache.flink.connector.file.table.stream.compact.CompactMessages.EndCompaction;
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.runtime.checkpoint.OperatorSubtaskState;
import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.apache.flink.table.filesystem.stream.PartitionCommitInfo;
-import org.apache.flink.table.filesystem.stream.compact.CompactMessages.CompactionUnit;
-import org.apache.flink.table.filesystem.stream.compact.CompactMessages.CoordinatorOutput;
-import org.apache.flink.table.filesystem.stream.compact.CompactMessages.EndCompaction;
import org.apache.flink.util.FileUtils;
import org.apache.flink.util.function.ThrowingConsumer;
diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/stream/compact/TestByteFormat.java b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/stream/compact/TestByteFormat.java
similarity index 96%
rename from flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/stream/compact/TestByteFormat.java
rename to flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/stream/compact/TestByteFormat.java
index 4f9991a..b9eb38e 100644
--- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/stream/compact/TestByteFormat.java
+++ b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/stream/compact/TestByteFormat.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem.stream.compact;
+package org.apache.flink.connector.file.table.stream.compact;
import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
import org.apache.flink.api.common.typeinfo.TypeInformation;
@@ -30,7 +30,6 @@ import org.apache.flink.core.fs.FSDataInputStream;
import java.io.IOException;
/** Test byte stream format. */
-@SuppressWarnings("serial")
public class TestByteFormat extends SimpleStreamFormat<Byte> {
@Override
diff --git a/flink-connectors/flink-connector-hive/pom.xml b/flink-connectors/flink-connector-hive/pom.xml
index c7fa016..7b022ab 100644
--- a/flink-connectors/flink-connector-hive/pom.xml
+++ b/flink-connectors/flink-connector-hive/pom.xml
@@ -183,7 +183,7 @@ under the License.
<dependency>
<groupId>org.apache.flink</groupId>
- <artifactId>flink-orc_${scala.binary.version}</artifactId>
+ <artifactId>flink-orc</artifactId>
<version>${project.version}</version>
<exclusions>
<exclusion>
@@ -195,7 +195,7 @@ under the License.
<dependency>
<groupId>org.apache.flink</groupId>
- <artifactId>flink-orc-nohive_${scala.binary.version}</artifactId>
+ <artifactId>flink-orc-nohive</artifactId>
<version>${project.version}</version>
<exclusions>
<exclusion>
@@ -207,7 +207,7 @@ under the License.
<dependency>
<groupId>org.apache.flink</groupId>
- <artifactId>flink-parquet_${scala.binary.version}</artifactId>
+ <artifactId>flink-parquet</artifactId>
<version>${project.version}</version>
</dependency>
@@ -928,10 +928,10 @@ under the License.
<artifactSet>
<includes combine.children="append">
<include>org.apache.flink:flink-hadoop-fs</include>
- <include>org.apache.flink:flink-orc_${scala.binary.version}</include>
- <include>org.apache.flink:flink-orc-nohive_${scala.binary.version}</include>
+ <include>org.apache.flink:flink-orc</include>
+ <include>org.apache.flink:flink-orc-nohive</include>
<include>org.apache.flink:flink-hadoop-compatibility_${scala.binary.version}</include>
- <include>org.apache.flink:flink-parquet_${scala.binary.version}</include>
+ <include>org.apache.flink:flink-parquet</include>
<include>org.apache.flink:flink-hadoop-bulk</include>
<include>org.apache.parquet:parquet-hadoop</include>
<include>org.apache.parquet:parquet-format</include>
@@ -966,14 +966,14 @@ under the License.
</relocations>
<filters>
<filter>
- <artifact>org.apache.flink:flink-orc_${scala.binary.version}</artifact>
+ <artifact>org.apache.flink:flink-orc</artifact>
<excludes>
<exclude>META-INF/services/org.apache.flink.table.factories.Factory</exclude>
<exclude>META-INF/services/org.apache.flink.table.factories.TableFactory</exclude>
</excludes>
</filter>
<filter>
- <artifact>org.apache.flink:flink-parquet_${scala.binary.version}</artifact>
+ <artifact>org.apache.flink:flink-parquet</artifact>
<excludes>
<exclude>META-INF/services/org.apache.flink.table.factories.Factory</exclude>
<exclude>META-INF/services/org.apache.flink.table.factories.TableFactory</exclude>
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/ContinuousHiveSplitEnumerator.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/ContinuousHiveSplitEnumerator.java
index d5884d1..6a44cd8 100644
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/ContinuousHiveSplitEnumerator.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/ContinuousHiveSplitEnumerator.java
@@ -26,10 +26,10 @@ import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.connector.file.src.FileSourceSplit;
import org.apache.flink.connector.file.src.PendingSplitsCheckpoint;
import org.apache.flink.connector.file.src.assigners.FileSplitAssigner;
+import org.apache.flink.connector.file.table.ContinuousPartitionFetcher;
import org.apache.flink.connectors.hive.read.HiveContinuousPartitionContext;
import org.apache.flink.connectors.hive.read.HiveSourceSplit;
import org.apache.flink.table.catalog.ObjectPath;
-import org.apache.flink.table.filesystem.ContinuousPartitionFetcher;
import org.apache.hadoop.hive.metastore.api.Partition;
import org.apache.hadoop.mapred.JobConf;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/FileSystemLookupFunction.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/FileSystemLookupFunction.java
similarity index 97%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/FileSystemLookupFunction.java
rename to flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/FileSystemLookupFunction.java
index 1cc9e64..2c3b530 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/FileSystemLookupFunction.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/FileSystemLookupFunction.java
@@ -16,11 +16,13 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connectors.hive;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.connector.file.table.PartitionFetcher;
+import org.apache.flink.connector.file.table.PartitionReader;
import org.apache.flink.table.data.GenericRowData;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.functions.FunctionContext;
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HadoopFileSystemFactory.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HadoopFileSystemFactory.java
index a9e2a07..a12d986 100644
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HadoopFileSystemFactory.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HadoopFileSystemFactory.java
@@ -18,9 +18,9 @@
package org.apache.flink.connectors.hive;
+import org.apache.flink.connector.file.table.FileSystemFactory;
import org.apache.flink.core.fs.FileSystem;
import org.apache.flink.runtime.fs.hdfs.HadoopFileSystem;
-import org.apache.flink.table.filesystem.FileSystemFactory;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.JobConf;
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveDynamicTableFactory.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveDynamicTableFactory.java
index 763c9cb..9bc90dd 100644
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveDynamicTableFactory.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveDynamicTableFactory.java
@@ -21,6 +21,7 @@ package org.apache.flink.connectors.hive;
import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.file.table.FileSystemConnectorOptions;
import org.apache.flink.connectors.hive.util.JobConfUtils;
import org.apache.flink.table.catalog.CatalogTable;
import org.apache.flink.table.catalog.hive.HiveCatalog;
@@ -29,7 +30,6 @@ import org.apache.flink.table.connector.source.DynamicTableSource;
import org.apache.flink.table.factories.DynamicTableSinkFactory;
import org.apache.flink.table.factories.DynamicTableSourceFactory;
import org.apache.flink.table.factories.FactoryUtil;
-import org.apache.flink.table.filesystem.FileSystemConnectorOptions;
import org.apache.flink.util.Preconditions;
import org.apache.hadoop.hive.conf.HiveConf;
@@ -37,8 +37,8 @@ import org.apache.hadoop.mapred.JobConf;
import java.util.Set;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.STREAMING_SOURCE_ENABLE;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.STREAMING_SOURCE_PARTITION_INCLUDE;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.STREAMING_SOURCE_ENABLE;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.STREAMING_SOURCE_PARTITION_INCLUDE;
/** A dynamic table factory implementation for Hive catalog. */
public class HiveDynamicTableFactory implements DynamicTableSourceFactory, DynamicTableSinkFactory {
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveLookupTableSource.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveLookupTableSource.java
index bfafb66..2ddaaa5 100644
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveLookupTableSource.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveLookupTableSource.java
@@ -21,6 +21,8 @@ package org.apache.flink.connectors.hive;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.file.table.PartitionFetcher;
+import org.apache.flink.connector.file.table.PartitionReader;
import org.apache.flink.connectors.hive.read.HiveInputFormatPartitionReader;
import org.apache.flink.connectors.hive.read.HivePartitionFetcherContextBase;
import org.apache.flink.connectors.hive.util.HivePartitionUtils;
@@ -31,9 +33,6 @@ import org.apache.flink.table.catalog.hive.client.HiveShim;
import org.apache.flink.table.connector.source.LookupTableSource;
import org.apache.flink.table.connector.source.TableFunctionProvider;
import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.filesystem.FileSystemLookupFunction;
-import org.apache.flink.table.filesystem.PartitionFetcher;
-import org.apache.flink.table.filesystem.PartitionReader;
import org.apache.flink.table.functions.TableFunction;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.logical.RowType;
@@ -50,10 +49,10 @@ import java.util.ArrayList;
import java.util.List;
import java.util.Optional;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.LOOKUP_JOIN_CACHE_TTL;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.STREAMING_SOURCE_CONSUME_START_OFFSET;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.STREAMING_SOURCE_MONITOR_INTERVAL;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.STREAMING_SOURCE_PARTITION_INCLUDE;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.LOOKUP_JOIN_CACHE_TTL;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.STREAMING_SOURCE_CONSUME_START_OFFSET;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.STREAMING_SOURCE_MONITOR_INTERVAL;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.STREAMING_SOURCE_PARTITION_INCLUDE;
/**
* Hive Table Source that has lookup ability.
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveRowDataPartitionComputer.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveRowDataPartitionComputer.java
index 325d7b2..74f7976 100644
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveRowDataPartitionComputer.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveRowDataPartitionComputer.java
@@ -18,10 +18,10 @@
package org.apache.flink.connectors.hive;
+import org.apache.flink.connector.file.table.RowDataPartitionComputer;
import org.apache.flink.table.catalog.hive.client.HiveShim;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.data.util.DataFormatConverters;
-import org.apache.flink.table.filesystem.RowDataPartitionComputer;
import org.apache.flink.table.functions.hive.conversion.HiveInspectors;
import org.apache.flink.table.functions.hive.conversion.HiveObjectConversion;
import org.apache.flink.table.types.DataType;
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveRowPartitionComputer.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveRowPartitionComputer.java
index 4ce3061..3fe46bf 100644
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveRowPartitionComputer.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveRowPartitionComputer.java
@@ -18,8 +18,8 @@
package org.apache.flink.connectors.hive;
+import org.apache.flink.connector.file.table.RowPartitionComputer;
import org.apache.flink.table.catalog.hive.client.HiveShim;
-import org.apache.flink.table.filesystem.RowPartitionComputer;
import org.apache.flink.table.functions.hive.conversion.HiveInspectors;
import org.apache.flink.table.functions.hive.conversion.HiveObjectConversion;
import org.apache.flink.table.types.DataType;
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSource.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSource.java
index 7d8bbd1..61e4820 100644
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSource.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSource.java
@@ -28,12 +28,12 @@ import org.apache.flink.connector.file.src.PendingSplitsCheckpoint;
import org.apache.flink.connector.file.src.assigners.FileSplitAssigner;
import org.apache.flink.connector.file.src.enumerate.FileEnumerator;
import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.connector.file.table.ContinuousPartitionFetcher;
import org.apache.flink.connectors.hive.read.HiveSourceSplit;
import org.apache.flink.core.fs.Path;
import org.apache.flink.core.io.SimpleVersionedSerializer;
import org.apache.flink.table.catalog.ObjectPath;
import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.filesystem.ContinuousPartitionFetcher;
import org.apache.flink.util.Preconditions;
import org.apache.hadoop.hive.metastore.api.Partition;
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSourceBuilder.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSourceBuilder.java
index 4b42073..944631f 100644
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSourceBuilder.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSourceBuilder.java
@@ -25,6 +25,9 @@ import org.apache.flink.connector.file.src.ContinuousEnumerationSettings;
import org.apache.flink.connector.file.src.assigners.FileSplitAssigner;
import org.apache.flink.connector.file.src.assigners.SimpleSplitAssigner;
import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.connector.file.table.ContinuousPartitionFetcher;
+import org.apache.flink.connector.file.table.FileSystemConnectorOptions;
+import org.apache.flink.connector.file.table.LimitableBulkFormat;
import org.apache.flink.connectors.hive.read.HiveContinuousPartitionFetcher;
import org.apache.flink.connectors.hive.read.HiveInputFormat;
import org.apache.flink.connectors.hive.read.HiveSourceSplit;
@@ -41,9 +44,6 @@ import org.apache.flink.table.catalog.hive.client.HiveShim;
import org.apache.flink.table.catalog.hive.client.HiveShimLoader;
import org.apache.flink.table.catalog.hive.util.HiveTableUtil;
import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.filesystem.ContinuousPartitionFetcher;
-import org.apache.flink.table.filesystem.FileSystemConnectorOptions;
-import org.apache.flink.table.filesystem.LimitableBulkFormat;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.util.Preconditions;
@@ -65,11 +65,11 @@ import java.util.List;
import java.util.Map;
import static org.apache.flink.connector.file.src.FileSource.DEFAULT_SPLIT_ASSIGNER;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.STREAMING_SOURCE_ENABLE;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.STREAMING_SOURCE_MONITOR_INTERVAL;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.STREAMING_SOURCE_PARTITION_INCLUDE;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.STREAMING_SOURCE_PARTITION_ORDER;
import static org.apache.flink.table.catalog.hive.util.HiveTableUtil.checkAcidTable;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.STREAMING_SOURCE_ENABLE;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.STREAMING_SOURCE_MONITOR_INTERVAL;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.STREAMING_SOURCE_PARTITION_INCLUDE;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.STREAMING_SOURCE_PARTITION_ORDER;
import static org.apache.flink.util.Preconditions.checkArgument;
/** Builder to build {@link HiveSource} instances. */
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableMetaStoreFactory.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableMetaStoreFactory.java
index c379b66..22ec69b 100644
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableMetaStoreFactory.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableMetaStoreFactory.java
@@ -18,12 +18,12 @@
package org.apache.flink.connectors.hive;
+import org.apache.flink.connector.file.table.TableMetaStoreFactory;
import org.apache.flink.connectors.hive.util.HiveConfUtils;
import org.apache.flink.core.fs.Path;
import org.apache.flink.table.catalog.hive.client.HiveMetastoreClientFactory;
import org.apache.flink.table.catalog.hive.client.HiveMetastoreClientWrapper;
import org.apache.flink.table.catalog.hive.util.HiveTableUtil;
-import org.apache.flink.table.filesystem.TableMetaStoreFactory;
import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
import org.apache.hadoop.hive.metastore.api.Partition;
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSink.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSink.java
index 6e66c32..bab1ece 100644
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSink.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSink.java
@@ -22,6 +22,13 @@ import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.functions.MapFunction;
import org.apache.flink.api.common.serialization.BulkWriter;
import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.file.table.FileSystemConnectorOptions;
+import org.apache.flink.connector.file.table.FileSystemOutputFormat;
+import org.apache.flink.connector.file.table.FileSystemTableSink;
+import org.apache.flink.connector.file.table.FileSystemTableSink.TableBucketAssigner;
+import org.apache.flink.connector.file.table.stream.PartitionCommitInfo;
+import org.apache.flink.connector.file.table.stream.StreamingSink;
+import org.apache.flink.connector.file.table.stream.compact.CompactReader;
import org.apache.flink.connectors.hive.read.HiveCompactReaderFactory;
import org.apache.flink.connectors.hive.util.HiveConfUtils;
import org.apache.flink.connectors.hive.util.JobConfUtils;
@@ -55,13 +62,6 @@ import org.apache.flink.table.connector.sink.DynamicTableSink;
import org.apache.flink.table.connector.sink.abilities.SupportsOverwrite;
import org.apache.flink.table.connector.sink.abilities.SupportsPartitioning;
import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.filesystem.FileSystemConnectorOptions;
-import org.apache.flink.table.filesystem.FileSystemOutputFormat;
-import org.apache.flink.table.filesystem.FileSystemTableSink;
-import org.apache.flink.table.filesystem.FileSystemTableSink.TableBucketAssigner;
-import org.apache.flink.table.filesystem.stream.PartitionCommitInfo;
-import org.apache.flink.table.filesystem.stream.StreamingSink;
-import org.apache.flink.table.filesystem.stream.compact.CompactReader;
import org.apache.flink.table.types.logical.LogicalType;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.table.utils.TableSchemaUtils;
@@ -95,11 +95,11 @@ import java.util.Optional;
import java.util.Properties;
import java.util.UUID;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.SINK_ROLLING_POLICY_CHECK_INTERVAL;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.SINK_ROLLING_POLICY_FILE_SIZE;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.SINK_ROLLING_POLICY_ROLLOVER_INTERVAL;
+import static org.apache.flink.connector.file.table.stream.compact.CompactOperator.convertToUncompacted;
import static org.apache.flink.table.catalog.hive.util.HiveTableUtil.checkAcidTable;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.SINK_ROLLING_POLICY_CHECK_INTERVAL;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.SINK_ROLLING_POLICY_FILE_SIZE;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.SINK_ROLLING_POLICY_ROLLOVER_INTERVAL;
-import static org.apache.flink.table.filesystem.stream.compact.CompactOperator.convertToUncompacted;
/** Table sink to write to Hive tables. */
public class HiveTableSink implements DynamicTableSink, SupportsPartitioning, SupportsOverwrite {
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSource.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSource.java
index 370ccdc..e56b518 100644
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSource.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSource.java
@@ -25,6 +25,7 @@ import org.apache.flink.api.common.typeutils.base.LongSerializer;
import org.apache.flink.api.common.typeutils.base.StringSerializer;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.file.table.ContinuousPartitionFetcher;
import org.apache.flink.connectors.hive.read.HiveContinuousPartitionContext;
import org.apache.flink.connectors.hive.read.HivePartitionFetcherContextBase;
import org.apache.flink.connectors.hive.util.HivePartitionUtils;
@@ -45,7 +46,6 @@ import org.apache.flink.table.connector.source.abilities.SupportsLimitPushDown;
import org.apache.flink.table.connector.source.abilities.SupportsPartitionPushDown;
import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown;
import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.filesystem.ContinuousPartitionFetcher;
import org.apache.flink.table.types.DataType;
import org.apache.flink.util.Preconditions;
@@ -63,10 +63,10 @@ import java.util.List;
import java.util.Map;
import java.util.Optional;
+import static org.apache.flink.connector.file.table.DefaultPartTimeExtractor.toMills;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.STREAMING_SOURCE_CONSUME_START_OFFSET;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.STREAMING_SOURCE_ENABLE;
import static org.apache.flink.connectors.hive.util.HivePartitionUtils.getAllPartitions;
-import static org.apache.flink.table.filesystem.DefaultPartTimeExtractor.toMills;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.STREAMING_SOURCE_CONSUME_START_OFFSET;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.STREAMING_SOURCE_ENABLE;
/** A TableSource implementation to read data from Hive tables. */
public class HiveTableSource
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveCompactReaderFactory.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveCompactReaderFactory.java
index 21b3d4b..a9cd90c 100644
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveCompactReaderFactory.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveCompactReaderFactory.java
@@ -19,6 +19,9 @@
package org.apache.flink.connectors.hive.read;
import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.connector.file.table.stream.compact.CompactBulkReader;
+import org.apache.flink.connector.file.table.stream.compact.CompactContext;
+import org.apache.flink.connector.file.table.stream.compact.CompactReader;
import org.apache.flink.connectors.hive.CachedSerializedValue;
import org.apache.flink.connectors.hive.FlinkHiveException;
import org.apache.flink.connectors.hive.HiveTablePartition;
@@ -30,9 +33,6 @@ import org.apache.flink.table.catalog.CatalogTable;
import org.apache.flink.table.catalog.hive.client.HiveShim;
import org.apache.flink.table.catalog.hive.client.HiveShimLoader;
import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.filesystem.stream.compact.CompactBulkReader;
-import org.apache.flink.table.filesystem.stream.compact.CompactContext;
-import org.apache.flink.table.filesystem.stream.compact.CompactReader;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.logical.RowType;
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveContinuousPartitionContext.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveContinuousPartitionContext.java
index ba2faa2..688ea9f 100644
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveContinuousPartitionContext.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveContinuousPartitionContext.java
@@ -19,7 +19,7 @@
package org.apache.flink.connectors.hive.read;
import org.apache.flink.annotation.Internal;
-import org.apache.flink.table.filesystem.ContinuousPartitionFetcher;
+import org.apache.flink.connector.file.table.ContinuousPartitionFetcher;
/**
* Context for Hive continuous partition fetcher.
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveContinuousPartitionFetcher.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveContinuousPartitionFetcher.java
index e73473f..64241909 100644
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveContinuousPartitionFetcher.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveContinuousPartitionFetcher.java
@@ -20,9 +20,9 @@ package org.apache.flink.connectors.hive.read;
import org.apache.flink.annotation.Internal;
import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.table.filesystem.ContinuousPartitionFetcher;
-import org.apache.flink.table.filesystem.PartitionFetcher;
-import org.apache.flink.table.filesystem.PartitionFetcher.Context.ComparablePartitionValue;
+import org.apache.flink.connector.file.table.ContinuousPartitionFetcher;
+import org.apache.flink.connector.file.table.PartitionFetcher;
+import org.apache.flink.connector.file.table.PartitionFetcher.Context.ComparablePartitionValue;
import org.apache.hadoop.hive.metastore.api.Partition;
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveInputFormat.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveInputFormat.java
index 6ab0dc9..3b93bdf 100644
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveInputFormat.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveInputFormat.java
@@ -22,6 +22,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.connector.file.src.reader.BulkFormat;
import org.apache.flink.connector.file.src.util.ArrayResultIterator;
+import org.apache.flink.connector.file.table.PartitionFieldExtractor;
import org.apache.flink.connectors.hive.HiveTablePartition;
import org.apache.flink.connectors.hive.JobConfWrapper;
import org.apache.flink.connectors.hive.util.HivePartitionUtils;
@@ -34,7 +35,6 @@ import org.apache.flink.table.catalog.hive.client.HiveShim;
import org.apache.flink.table.catalog.hive.client.HiveShimLoader;
import org.apache.flink.table.catalog.hive.util.HiveTypeUtil;
import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.filesystem.PartitionFieldExtractor;
import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
import org.apache.flink.table.runtime.typeutils.RowDataSerializer;
import org.apache.flink.table.types.DataType;
@@ -55,7 +55,7 @@ import java.util.List;
import java.util.stream.Collectors;
import static org.apache.flink.connector.file.src.util.CheckpointedPosition.NO_OFFSET;
-import static org.apache.flink.table.data.vector.VectorizedColumnBatch.DEFAULT_SIZE;
+import static org.apache.flink.table.data.columnar.vector.VectorizedColumnBatch.DEFAULT_SIZE;
/**
* A BulkFormat implementation for HiveSource. This implementation delegates reading to other
@@ -139,6 +139,7 @@ public class HiveInputFormat implements BulkFormat<RowData, HiveSourceSplit> {
return ParquetColumnarRowInputFormat.createPartitionedFormat(
jobConfWrapper.conf(),
producedRowType,
+ InternalTypeInfo.of(producedRowType),
partitionKeys,
partitionFieldExtractor,
DEFAULT_SIZE,
@@ -171,7 +172,8 @@ public class HiveInputFormat implements BulkFormat<RowData, HiveSourceSplit> {
partitionFieldExtractor,
computeSelectedFields(),
Collections.emptyList(),
- DEFAULT_SIZE)
+ DEFAULT_SIZE,
+ InternalTypeInfo::of)
: OrcColumnarRowInputFormat.createPartitionedFormat(
OrcShim.createShim(hiveVersion),
jobConfWrapper.conf(),
@@ -180,7 +182,8 @@ public class HiveInputFormat implements BulkFormat<RowData, HiveSourceSplit> {
partitionFieldExtractor,
computeSelectedFields(),
Collections.emptyList(),
- DEFAULT_SIZE);
+ DEFAULT_SIZE,
+ InternalTypeInfo::of);
}
private boolean useOrcVectorizedRead(HiveTablePartition partition) {
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveInputFormatPartitionReader.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveInputFormatPartitionReader.java
index b9eb704..c70b20d 100644
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveInputFormatPartitionReader.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveInputFormatPartitionReader.java
@@ -18,11 +18,11 @@
package org.apache.flink.connectors.hive.read;
+import org.apache.flink.connector.file.table.PartitionReader;
import org.apache.flink.connectors.hive.HiveTablePartition;
import org.apache.flink.connectors.hive.JobConfWrapper;
import org.apache.flink.table.catalog.ObjectPath;
import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.filesystem.PartitionReader;
import org.apache.flink.table.types.DataType;
import org.apache.hadoop.mapred.JobConf;
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HivePartitionContext.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HivePartitionContext.java
index 6ad1cd9..51131f3 100644
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HivePartitionContext.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HivePartitionContext.java
@@ -19,8 +19,8 @@
package org.apache.flink.connectors.hive.read;
import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.file.table.PartitionFetcher;
import org.apache.flink.connectors.hive.HiveTablePartition;
-import org.apache.flink.table.filesystem.PartitionFetcher;
/**
* Context for Hive partition fetcher.
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HivePartitionFetcherContextBase.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HivePartitionFetcherContextBase.java
index dc517a4..cffd406 100644
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HivePartitionFetcherContextBase.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HivePartitionFetcherContextBase.java
@@ -19,6 +19,8 @@
package org.apache.flink.connectors.hive.read;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.table.FileSystemConnectorOptions.PartitionOrder;
+import org.apache.flink.connector.file.table.PartitionTimeExtractor;
import org.apache.flink.connectors.hive.HiveTablePartition;
import org.apache.flink.connectors.hive.JobConfWrapper;
import org.apache.flink.connectors.hive.util.HiveConfUtils;
@@ -27,8 +29,6 @@ import org.apache.flink.table.catalog.hive.client.HiveMetastoreClientWrapper;
import org.apache.flink.table.catalog.hive.client.HiveShim;
import org.apache.flink.table.catalog.hive.util.HiveReflectionUtils;
import org.apache.flink.table.data.TimestampData;
-import org.apache.flink.table.filesystem.FileSystemConnectorOptions.PartitionOrder;
-import org.apache.flink.table.filesystem.PartitionTimeExtractor;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.utils.PartitionPathUtils;
@@ -48,11 +48,11 @@ import java.util.Map;
import java.util.Properties;
import java.util.stream.Collectors;
-import static org.apache.flink.table.filesystem.DefaultPartTimeExtractor.toMills;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.PARTITION_TIME_EXTRACTOR_CLASS;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.PARTITION_TIME_EXTRACTOR_KIND;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.PARTITION_TIME_EXTRACTOR_TIMESTAMP_PATTERN;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.STREAMING_SOURCE_PARTITION_ORDER;
+import static org.apache.flink.connector.file.table.DefaultPartTimeExtractor.toMills;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.PARTITION_TIME_EXTRACTOR_CLASS;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.PARTITION_TIME_EXTRACTOR_KIND;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.PARTITION_TIME_EXTRACTOR_TIMESTAMP_PATTERN;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.STREAMING_SOURCE_PARTITION_ORDER;
/** Base class for table partition fetcher context. */
public abstract class HivePartitionFetcherContextBase<P> implements HivePartitionContext<P> {
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveVectorizedOrcSplitReader.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveVectorizedOrcSplitReader.java
index cfde044..3c26995 100644
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveVectorizedOrcSplitReader.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveVectorizedOrcSplitReader.java
@@ -38,7 +38,7 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Map;
-import static org.apache.flink.table.data.vector.VectorizedColumnBatch.DEFAULT_SIZE;
+import static org.apache.flink.table.data.columnar.vector.VectorizedColumnBatch.DEFAULT_SIZE;
/** Orc {@link SplitReader} to read files using {@link OrcColumnarRowSplitReader}. */
public class HiveVectorizedOrcSplitReader implements SplitReader {
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveVectorizedParquetSplitReader.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveVectorizedParquetSplitReader.java
index c0fd8e8..48dfaf4 100644
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveVectorizedParquetSplitReader.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveVectorizedParquetSplitReader.java
@@ -36,7 +36,7 @@ import org.apache.hadoop.mapred.JobConf;
import java.io.IOException;
import java.util.Map;
-import static org.apache.flink.table.data.vector.VectorizedColumnBatch.DEFAULT_SIZE;
+import static org.apache.flink.table.data.columnar.vector.VectorizedColumnBatch.DEFAULT_SIZE;
/** Orc {@link SplitReader} to read files using {@link ParquetColumnarRowSplitReader}. */
public class HiveVectorizedParquetSplitReader implements SplitReader {
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/write/HiveOutputFormatFactory.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/write/HiveOutputFormatFactory.java
index 99480ee..a9f929e 100644
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/write/HiveOutputFormatFactory.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/write/HiveOutputFormatFactory.java
@@ -19,9 +19,9 @@
package org.apache.flink.connectors.hive.write;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.table.OutputFormatFactory;
import org.apache.flink.core.fs.Path;
import org.apache.flink.runtime.fs.hdfs.HadoopFileSystem;
-import org.apache.flink.table.filesystem.OutputFormatFactory;
import org.apache.flink.types.Row;
import org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter;
diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDynamicTableFactoryTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDynamicTableFactoryTest.java
index f7d9dae..6f327ab 100644
--- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDynamicTableFactoryTest.java
+++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDynamicTableFactoryTest.java
@@ -19,6 +19,7 @@
package org.apache.flink.connectors.hive;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.table.FileSystemConnectorOptions.PartitionOrder;
import org.apache.flink.table.api.EnvironmentSettings;
import org.apache.flink.table.api.SqlDialect;
import org.apache.flink.table.api.TableEnvironment;
@@ -32,8 +33,6 @@ import org.apache.flink.table.connector.source.DynamicTableSource;
import org.apache.flink.table.factories.DynamicTableSinkFactory;
import org.apache.flink.table.factories.DynamicTableSourceFactory;
import org.apache.flink.table.factories.FactoryUtil;
-import org.apache.flink.table.filesystem.FileSystemConnectorOptions.PartitionOrder;
-import org.apache.flink.table.filesystem.FileSystemLookupFunction;
import org.apache.flink.util.ExceptionUtils;
import org.apache.hadoop.io.Text;
@@ -45,14 +44,14 @@ import org.junit.Test;
import java.time.Duration;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.LOOKUP_JOIN_CACHE_TTL;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.PARTITION_TIME_EXTRACTOR_CLASS;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.PARTITION_TIME_EXTRACTOR_KIND;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.STREAMING_SOURCE_CONSUME_START_OFFSET;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.STREAMING_SOURCE_ENABLE;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.STREAMING_SOURCE_MONITOR_INTERVAL;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.STREAMING_SOURCE_PARTITION_INCLUDE;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.STREAMING_SOURCE_PARTITION_ORDER;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.LOOKUP_JOIN_CACHE_TTL;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.PARTITION_TIME_EXTRACTOR_CLASS;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.PARTITION_TIME_EXTRACTOR_KIND;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.STREAMING_SOURCE_CONSUME_START_OFFSET;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.STREAMING_SOURCE_ENABLE;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.STREAMING_SOURCE_MONITOR_INTERVAL;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.STREAMING_SOURCE_PARTITION_INCLUDE;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.STREAMING_SOURCE_PARTITION_ORDER;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveLookupJoinITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveLookupJoinITCase.java
index e82444e..de491c0 100644
--- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveLookupJoinITCase.java
+++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveLookupJoinITCase.java
@@ -19,6 +19,9 @@
package org.apache.flink.connectors.hive;
import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.connector.file.table.FileSystemConnectorOptions;
+import org.apache.flink.connector.file.table.PartitionFetcher;
+import org.apache.flink.connector.file.table.PartitionReader;
import org.apache.flink.table.api.EnvironmentSettings;
import org.apache.flink.table.api.SqlDialect;
import org.apache.flink.table.api.TableEnvironment;
@@ -32,10 +35,6 @@ import org.apache.flink.table.data.GenericRowData;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.factories.DynamicTableSourceFactory;
import org.apache.flink.table.factories.FactoryUtil;
-import org.apache.flink.table.filesystem.FileSystemConnectorOptions;
-import org.apache.flink.table.filesystem.FileSystemLookupFunction;
-import org.apache.flink.table.filesystem.PartitionFetcher;
-import org.apache.flink.table.filesystem.PartitionReader;
import org.apache.flink.table.planner.factories.utils.TestCollectionTableFactory;
import org.apache.flink.table.runtime.typeutils.InternalSerializers;
import org.apache.flink.types.Row;
@@ -51,12 +50,12 @@ import java.util.Arrays;
import java.util.Comparator;
import java.util.List;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.PARTITION_TIME_EXTRACTOR_KIND;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.PARTITION_TIME_EXTRACTOR_TIMESTAMP_PATTERN;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.STREAMING_SOURCE_ENABLE;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.STREAMING_SOURCE_MONITOR_INTERVAL;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.STREAMING_SOURCE_PARTITION_INCLUDE;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.STREAMING_SOURCE_PARTITION_ORDER;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.PARTITION_TIME_EXTRACTOR_KIND;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.PARTITION_TIME_EXTRACTOR_TIMESTAMP_PATTERN;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.STREAMING_SOURCE_ENABLE;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.STREAMING_SOURCE_MONITOR_INTERVAL;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.STREAMING_SOURCE_PARTITION_INCLUDE;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.STREAMING_SOURCE_PARTITION_ORDER;
import static org.junit.Assert.assertEquals;
/** Test lookup join of hive tables. */
diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSinkITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSinkITCase.java
index 0cb2335..0cd9ff2 100644
--- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSinkITCase.java
+++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSinkITCase.java
@@ -62,12 +62,12 @@ import java.util.Map;
import java.util.Set;
import java.util.function.Consumer;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.PARTITION_TIME_EXTRACTOR_TIMESTAMP_PATTERN;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_DELAY;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_POLICY_CLASS;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_POLICY_KIND;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_SUCCESS_FILE_NAME;
import static org.apache.flink.table.api.Expressions.$;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.PARTITION_TIME_EXTRACTOR_TIMESTAMP_PATTERN;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_DELAY;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_POLICY_CLASS;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_POLICY_KIND;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_SUCCESS_FILE_NAME;
import static org.apache.flink.table.planner.utils.TableTestUtil.readFromResource;
import static org.apache.flink.table.planner.utils.TableTestUtil.replaceStageId;
import static org.apache.flink.table.planner.utils.TableTestUtil.replaceStreamNodeId;
diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTemporalJoinITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTemporalJoinITCase.java
index 7245198..5885e43 100644
--- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTemporalJoinITCase.java
+++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTemporalJoinITCase.java
@@ -18,6 +18,7 @@
package org.apache.flink.connectors.hive;
+import org.apache.flink.connector.file.table.FileSystemConnectorOptions;
import org.apache.flink.table.HiveVersionTestUtil;
import org.apache.flink.table.api.EnvironmentSettings;
import org.apache.flink.table.api.SqlDialect;
@@ -27,7 +28,6 @@ import org.apache.flink.table.api.ValidationException;
import org.apache.flink.table.catalog.CatalogTest;
import org.apache.flink.table.catalog.hive.HiveCatalog;
import org.apache.flink.table.catalog.hive.HiveTestUtils;
-import org.apache.flink.table.filesystem.FileSystemConnectorOptions;
import org.apache.flink.table.planner.factories.utils.TestCollectionTableFactory;
import org.apache.flink.table.planner.utils.TableTestBase;
import org.apache.flink.types.Row;
diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/PartitionMonitorTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/PartitionMonitorTest.java
index 7c5d857..6380b74 100644
--- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/PartitionMonitorTest.java
+++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/PartitionMonitorTest.java
@@ -21,10 +21,10 @@ package org.apache.flink.connectors.hive;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.table.ContinuousPartitionFetcher;
+import org.apache.flink.connector.file.table.PartitionFetcher;
import org.apache.flink.connectors.hive.read.HiveContinuousPartitionContext;
import org.apache.flink.table.catalog.ObjectPath;
-import org.apache.flink.table.filesystem.ContinuousPartitionFetcher;
-import org.apache.flink.table.filesystem.PartitionFetcher;
import org.apache.hadoop.hive.metastore.api.Partition;
import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TestCustomCommitPolicy.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TestCustomCommitPolicy.java
index 5e96c29..118af6c 100644
--- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TestCustomCommitPolicy.java
+++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TestCustomCommitPolicy.java
@@ -18,7 +18,7 @@
package org.apache.flink.connectors.hive;
-import org.apache.flink.table.filesystem.PartitionCommitPolicy;
+import org.apache.flink.connector.file.table.PartitionCommitPolicy;
import java.util.HashSet;
import java.util.Set;
diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/read/HivePartitionFetcherTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/read/HivePartitionFetcherTest.java
index c63852c9..76c8111 100644
--- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/read/HivePartitionFetcherTest.java
+++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/read/HivePartitionFetcherTest.java
@@ -18,6 +18,7 @@
package org.apache.flink.connectors.hive.read;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.table.FileSystemConnectorOptions.PartitionOrder;
import org.apache.flink.connectors.hive.JobConfWrapper;
import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.api.TableSchema;
@@ -28,7 +29,6 @@ import org.apache.flink.table.catalog.hive.HiveCatalog;
import org.apache.flink.table.catalog.hive.HiveTestUtils;
import org.apache.flink.table.catalog.hive.client.HiveShim;
import org.apache.flink.table.catalog.hive.client.HiveShimLoader;
-import org.apache.flink.table.filesystem.FileSystemConnectorOptions.PartitionOrder;
import org.apache.flink.table.types.DataType;
import org.apache.hadoop.fs.FileSystem;
@@ -44,7 +44,7 @@ import java.util.List;
import java.util.Map;
import java.util.Optional;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.STREAMING_SOURCE_PARTITION_ORDER;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.STREAMING_SOURCE_PARTITION_ORDER;
import static org.junit.Assert.assertEquals;
/** Tests for hive partition fetch implementations. */
diff --git a/flink-dist/pom.xml b/flink-dist/pom.xml
index ffe067c..f5f4779 100644
--- a/flink-dist/pom.xml
+++ b/flink-dist/pom.xml
@@ -343,6 +343,13 @@ under the License.
<dependency>
<groupId>org.apache.flink</groupId>
+ <artifactId>flink-connector-files</artifactId>
+ <version>${project.version}</version>
+ <scope>provided</scope>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.flink</groupId>
<artifactId>flink-json</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
diff --git a/flink-dist/src/main/assemblies/bin.xml b/flink-dist/src/main/assemblies/bin.xml
index 949b4d8..bf27897 100644
--- a/flink-dist/src/main/assemblies/bin.xml
+++ b/flink-dist/src/main/assemblies/bin.xml
@@ -87,7 +87,14 @@ under the License.
<fileMode>0644</fileMode>
</file>
- <!-- Built-in formats -->
+ <!-- Built-in connectors and formats -->
+ <file>
+ <source>../flink-connectors/flink-connector-files/target/flink-connector-files-${project.version}.jar</source>
+ <outputDirectory>lib/</outputDirectory>
+ <destName>flink-connector-files-${project.version}.jar</destName>
+ <fileMode>0644</fileMode>
+ </file>
+
<file>
<source>../flink-formats/flink-csv/target/flink-csv-${project.version}.jar</source>
<outputDirectory>lib/</outputDirectory>
diff --git a/flink-examples/flink-examples-table/pom.xml b/flink-examples/flink-examples-table/pom.xml
index 4427851..99200a5 100644
--- a/flink-examples/flink-examples-table/pom.xml
+++ b/flink-examples/flink-examples-table/pom.xml
@@ -55,6 +55,11 @@ under the License.
<!-- Table connectors and formats -->
<dependency>
<groupId>org.apache.flink</groupId>
+ <artifactId>flink-connector-files</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
<artifactId>flink-csv</artifactId>
<version>${project.version}</version>
</dependency>
diff --git a/flink-formats/flink-avro-confluent-registry/pom.xml b/flink-formats/flink-avro-confluent-registry/pom.xml
index 548998c..59fc1d1 100644
--- a/flink-formats/flink-avro-confluent-registry/pom.xml
+++ b/flink-formats/flink-avro-confluent-registry/pom.xml
@@ -84,11 +84,23 @@ under the License.
<version>${project.version}</version>
<scope>provided</scope>
</dependency>
+
+ <!-- Table ecosystem and filesystem connector -->
+
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-common</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
+ <optional>true</optional>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-connector-files</artifactId>
+ <version>${project.version}</version>
+ <scope>provided</scope>
+ <optional>true</optional>
</dependency>
<!-- test dependencies -->
diff --git a/flink-formats/flink-avro/pom.xml b/flink-formats/flink-avro/pom.xml
index a1088d6..a6e51b3 100644
--- a/flink-formats/flink-avro/pom.xml
+++ b/flink-formats/flink-avro/pom.xml
@@ -45,9 +45,8 @@ under the License.
<scope>provided</scope>
</dependency>
- <!-- Table ecosystem -->
+ <!-- Table ecosystem and filesystem connector -->
- <!-- Projects depending on this project won't depend on flink-table-*. -->
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-common</artifactId>
@@ -56,6 +55,14 @@ under the License.
<optional>true</optional>
</dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-connector-files</artifactId>
+ <version>${project.version}</version>
+ <scope>provided</scope>
+ <optional>true</optional>
+ </dependency>
+
<!-- Avro -->
<dependency>
diff --git a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroFileFormatFactory.java b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroFileFormatFactory.java
index d309a6b..7b9d8c0 100644
--- a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroFileFormatFactory.java
+++ b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroFileFormatFactory.java
@@ -22,13 +22,13 @@ import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.serialization.BulkWriter;
import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.file.table.factories.BulkWriterFormatFactory;
import org.apache.flink.core.fs.FSDataOutputStream;
import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter;
import org.apache.flink.table.connector.ChangelogMode;
import org.apache.flink.table.connector.format.EncodingFormat;
import org.apache.flink.table.connector.sink.DynamicTableSink;
import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.factories.BulkWriterFormatFactory;
import org.apache.flink.table.factories.DynamicTableFactory;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.logical.RowType;
diff --git a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroFileSystemFormatFactory.java b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroFileSystemFormatFactory.java
index 8f47c73..205ae2f 100644
--- a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroFileSystemFormatFactory.java
+++ b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroFileSystemFormatFactory.java
@@ -21,12 +21,12 @@ package org.apache.flink.formats.avro;
import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.io.InputFormat;
import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.connector.file.table.factories.FileSystemFormatFactory;
import org.apache.flink.core.fs.FileInputSplit;
import org.apache.flink.core.fs.Path;
import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter;
import org.apache.flink.table.data.GenericRowData;
import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.factories.FileSystemFormatFactory;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.table.utils.PartitionPathUtils;
diff --git a/flink-formats/flink-csv/pom.xml b/flink-formats/flink-csv/pom.xml
index 297a5e9..244dbae 100644
--- a/flink-formats/flink-csv/pom.xml
+++ b/flink-formats/flink-csv/pom.xml
@@ -57,11 +57,22 @@ under the License.
<scope>provided</scope>
</dependency>
+ <!-- Table ecosystem and filesystem connector -->
+
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-common</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
+ <optional>true</optional>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-connector-files</artifactId>
+ <version>${project.version}</version>
+ <scope>provided</scope>
+ <optional>true</optional>
</dependency>
<!-- test dependencies -->
diff --git a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFileSystemFormatFactory.java b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFileSystemFormatFactory.java
index 5e33966..74b2c21 100644
--- a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFileSystemFormatFactory.java
+++ b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFileSystemFormatFactory.java
@@ -21,11 +21,11 @@ package org.apache.flink.formats.csv;
import org.apache.flink.api.common.io.InputFormat;
import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.file.table.factories.FileSystemFormatFactory;
import org.apache.flink.core.fs.FileInputSplit;
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.factories.FileSystemFormatFactory;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.table.utils.PartitionPathUtils;
diff --git a/flink-formats/flink-json/pom.xml b/flink-formats/flink-json/pom.xml
index fca8d9b..0e60961 100644
--- a/flink-formats/flink-json/pom.xml
+++ b/flink-formats/flink-json/pom.xml
@@ -57,11 +57,22 @@ under the License.
<scope>provided</scope>
</dependency>
+ <!-- Table ecosystem and filesystem connector -->
+
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-common</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
+ <optional>true</optional>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-connector-files</artifactId>
+ <version>${project.version}</version>
+ <scope>provided</scope>
+ <optional>true</optional>
</dependency>
<!-- test dependencies -->
diff --git a/flink-formats/flink-orc-nohive/pom.xml b/flink-formats/flink-orc-nohive/pom.xml
index d6142e1..a619d61 100644
--- a/flink-formats/flink-orc-nohive/pom.xml
+++ b/flink-formats/flink-orc-nohive/pom.xml
@@ -29,27 +29,36 @@ under the License.
<relativePath>..</relativePath>
</parent>
- <artifactId>flink-orc-nohive_${scala.binary.version}</artifactId>
+ <artifactId>flink-orc-nohive</artifactId>
<name>Flink : Formats : Orc nohive</name>
<packaging>jar</packaging>
<dependencies>
- <!-- Core -->
+ <!-- Table ecosystem and filesystem connector -->
<dependency>
<groupId>org.apache.flink</groupId>
- <artifactId>flink-table-runtime_${scala.binary.version}</artifactId>
+ <artifactId>flink-table-common</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
+ <optional>true</optional>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-connector-files</artifactId>
+ <version>${project.version}</version>
+ <scope>provided</scope>
+ <optional>true</optional>
</dependency>
<!-- Flink ORC -->
<dependency>
<groupId>org.apache.flink</groupId>
- <artifactId>flink-orc_${scala.binary.version}</artifactId>
+ <artifactId>flink-orc</artifactId>
<version>${project.version}</version>
<exclusions>
<exclusion>
@@ -78,7 +87,14 @@ under the License.
<dependency>
<groupId>org.apache.flink</groupId>
- <artifactId>flink-orc_${scala.binary.version}</artifactId>
+ <artifactId>flink-table-planner_${scala.binary.version}</artifactId>
+ <version>${project.version}</version>
+ <scope>test</scope>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-orc</artifactId>
<version>${project.version}</version>
<type>test-jar</type>
<scope>test</scope>
diff --git a/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/OrcNoHiveColumnarRowInputFormat.java b/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/OrcNoHiveColumnarRowInputFormat.java
index b186faf..a55fc04 100644
--- a/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/OrcNoHiveColumnarRowInputFormat.java
+++ b/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/OrcNoHiveColumnarRowInputFormat.java
@@ -18,14 +18,16 @@
package org.apache.flink.orc.nohive;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.connector.file.src.FileSourceSplit;
+import org.apache.flink.connector.file.table.PartitionFieldExtractor;
import org.apache.flink.orc.OrcColumnarRowInputFormat;
import org.apache.flink.orc.OrcFilters;
import org.apache.flink.orc.nohive.shim.OrcNoHiveShim;
import org.apache.flink.orc.vector.ColumnBatchFactory;
-import org.apache.flink.table.data.vector.ColumnVector;
-import org.apache.flink.table.data.vector.VectorizedColumnBatch;
-import org.apache.flink.table.filesystem.PartitionFieldExtractor;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.columnar.vector.ColumnVector;
+import org.apache.flink.table.data.columnar.vector.VectorizedColumnBatch;
import org.apache.flink.table.types.logical.LogicalType;
import org.apache.flink.table.types.logical.RowType;
@@ -34,6 +36,7 @@ import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
import java.util.Arrays;
import java.util.List;
+import java.util.function.Function;
import java.util.stream.Collectors;
import static org.apache.flink.orc.OrcSplitReaderUtil.convertToOrcTypeWithPart;
@@ -58,7 +61,12 @@ public class OrcNoHiveColumnarRowInputFormat {
PartitionFieldExtractor<SplitT> extractor,
int[] selectedFields,
List<OrcFilters.Predicate> conjunctPredicates,
- int batchSize) {
+ int batchSize,
+ Function<RowType, TypeInformation<RowData>> rowTypeInfoFactory) {
+ // TODO FLINK-25113 all this partition keys code should be pruned from the orc format,
+ // because now FileSystemTableSource uses FileInfoExtractorBulkFormat for reading partition
+ // keys.
+
String[] tableFieldNames = tableType.getFieldNames().toArray(new String[0]);
LogicalType[] tableFieldTypes = tableType.getChildren().toArray(new LogicalType[0]);
List<String> orcFieldNames = getNonPartNames(tableFieldNames, partitionKeys);
@@ -92,9 +100,10 @@ public class OrcNoHiveColumnarRowInputFormat {
conjunctPredicates,
batchSize,
batchGenerator,
- new RowType(
- Arrays.stream(selectedFields)
- .mapToObj(i -> tableType.getFields().get(i))
- .collect(Collectors.toList())));
+ rowTypeInfoFactory.apply(
+ new RowType(
+ Arrays.stream(selectedFields)
+ .mapToObj(i -> tableType.getFields().get(i))
+ .collect(Collectors.toList()))));
}
}
diff --git a/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/OrcNoHiveSplitReaderUtil.java b/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/OrcNoHiveSplitReaderUtil.java
index 3114966..8bfad42 100644
--- a/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/OrcNoHiveSplitReaderUtil.java
+++ b/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/OrcNoHiveSplitReaderUtil.java
@@ -23,8 +23,8 @@ import org.apache.flink.orc.OrcColumnarRowSplitReader;
import org.apache.flink.orc.OrcFilters;
import org.apache.flink.orc.OrcSplitReader;
import org.apache.flink.orc.nohive.shim.OrcNoHiveShim;
-import org.apache.flink.table.data.vector.ColumnVector;
-import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.flink.table.data.columnar.vector.ColumnVector;
+import org.apache.flink.table.data.columnar.vector.VectorizedColumnBatch;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.logical.LogicalType;
diff --git a/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/vector/AbstractOrcNoHiveVector.java b/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/vector/AbstractOrcNoHiveVector.java
index 5bc6f75..ca4ac85 100644
--- a/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/vector/AbstractOrcNoHiveVector.java
+++ b/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/vector/AbstractOrcNoHiveVector.java
@@ -40,7 +40,7 @@ import static org.apache.flink.table.utils.DateTimeUtils.toInternal;
/** This column vector is used to adapt hive's ColumnVector to Flink's ColumnVector. */
public abstract class AbstractOrcNoHiveVector
- implements org.apache.flink.table.data.vector.ColumnVector {
+ implements org.apache.flink.table.data.columnar.vector.ColumnVector {
private ColumnVector orcVector;
@@ -53,7 +53,7 @@ public abstract class AbstractOrcNoHiveVector
return !orcVector.noNulls && orcVector.isNull[orcVector.isRepeating ? 0 : i];
}
- public static org.apache.flink.table.data.vector.ColumnVector createFlinkVector(
+ public static org.apache.flink.table.data.columnar.vector.ColumnVector createFlinkVector(
ColumnVector vector) {
if (vector instanceof LongColumnVector) {
return new OrcNoHiveLongVector((LongColumnVector) vector);
@@ -72,8 +72,8 @@ public abstract class AbstractOrcNoHiveVector
}
/** Create flink vector by hive vector from constant. */
- public static org.apache.flink.table.data.vector.ColumnVector createFlinkVectorFromConstant(
- LogicalType type, Object value, int batchSize) {
+ public static org.apache.flink.table.data.columnar.vector.ColumnVector
+ createFlinkVectorFromConstant(LogicalType type, Object value, int batchSize) {
return createFlinkVector(createHiveVectorFromConstant(type, value, batchSize));
}
diff --git a/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/vector/OrcNoHiveBytesVector.java b/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/vector/OrcNoHiveBytesVector.java
index 30298b3..32f2dc8 100644
--- a/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/vector/OrcNoHiveBytesVector.java
+++ b/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/vector/OrcNoHiveBytesVector.java
@@ -22,7 +22,7 @@ import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
/** This column vector is used to adapt hive's BytesColumnVector to Flink's BytesColumnVector. */
public class OrcNoHiveBytesVector extends AbstractOrcNoHiveVector
- implements org.apache.flink.table.data.vector.BytesColumnVector {
+ implements org.apache.flink.table.data.columnar.vector.BytesColumnVector {
private BytesColumnVector vector;
diff --git a/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/vector/OrcNoHiveDecimalVector.java b/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/vector/OrcNoHiveDecimalVector.java
index d337618..a8a2bae 100644
--- a/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/vector/OrcNoHiveDecimalVector.java
+++ b/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/vector/OrcNoHiveDecimalVector.java
@@ -28,7 +28,7 @@ import java.math.BigDecimal;
* This column vector is used to adapt hive's DecimalColumnVector to Flink's DecimalColumnVector.
*/
public class OrcNoHiveDecimalVector extends AbstractOrcNoHiveVector
- implements org.apache.flink.table.data.vector.DecimalColumnVector {
+ implements org.apache.flink.table.data.columnar.vector.DecimalColumnVector {
private DecimalColumnVector vector;
diff --git a/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/vector/OrcNoHiveDoubleVector.java b/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/vector/OrcNoHiveDoubleVector.java
index a3911d4..66e4446 100644
--- a/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/vector/OrcNoHiveDoubleVector.java
+++ b/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/vector/OrcNoHiveDoubleVector.java
@@ -25,8 +25,8 @@ import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
* ColumnVector.
*/
public class OrcNoHiveDoubleVector extends AbstractOrcNoHiveVector
- implements org.apache.flink.table.data.vector.DoubleColumnVector,
- org.apache.flink.table.data.vector.FloatColumnVector {
+ implements org.apache.flink.table.data.columnar.vector.DoubleColumnVector,
+ org.apache.flink.table.data.columnar.vector.FloatColumnVector {
private DoubleColumnVector vector;
diff --git a/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/vector/OrcNoHiveLongVector.java b/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/vector/OrcNoHiveLongVector.java
index da69cad..a6517a0 100644
--- a/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/vector/OrcNoHiveLongVector.java
+++ b/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/vector/OrcNoHiveLongVector.java
@@ -25,11 +25,11 @@ import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
* and long ColumnVector.
*/
public class OrcNoHiveLongVector extends AbstractOrcNoHiveVector
- implements org.apache.flink.table.data.vector.LongColumnVector,
- org.apache.flink.table.data.vector.BooleanColumnVector,
- org.apache.flink.table.data.vector.ByteColumnVector,
- org.apache.flink.table.data.vector.ShortColumnVector,
- org.apache.flink.table.data.vector.IntColumnVector {
+ implements org.apache.flink.table.data.columnar.vector.LongColumnVector,
+ org.apache.flink.table.data.columnar.vector.BooleanColumnVector,
+ org.apache.flink.table.data.columnar.vector.ByteColumnVector,
+ org.apache.flink.table.data.columnar.vector.ShortColumnVector,
+ org.apache.flink.table.data.columnar.vector.IntColumnVector {
private LongColumnVector vector;
diff --git a/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/vector/OrcNoHiveTimestampVector.java b/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/vector/OrcNoHiveTimestampVector.java
index e6aef5a..33bd6cf 100644
--- a/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/vector/OrcNoHiveTimestampVector.java
+++ b/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/vector/OrcNoHiveTimestampVector.java
@@ -29,7 +29,7 @@ import java.sql.Timestamp;
* TimestampColumnVector.
*/
public class OrcNoHiveTimestampVector extends AbstractOrcNoHiveVector
- implements org.apache.flink.table.data.vector.TimestampColumnVector {
+ implements org.apache.flink.table.data.columnar.vector.TimestampColumnVector {
private TimestampColumnVector vector;
diff --git a/flink-formats/flink-orc/pom.xml b/flink-formats/flink-orc/pom.xml
index 5d03612..d30a21f 100644
--- a/flink-formats/flink-orc/pom.xml
+++ b/flink-formats/flink-orc/pom.xml
@@ -29,7 +29,7 @@ under the License.
<relativePath>..</relativePath>
</parent>
- <artifactId>flink-orc_${scala.binary.version}</artifactId>
+ <artifactId>flink-orc</artifactId>
<name>Flink : Formats : Orc</name>
<packaging>jar</packaging>
@@ -45,13 +45,22 @@ under the License.
<scope>provided</scope>
</dependency>
- <!-- Table ecosystem -->
+ <!-- Table ecosystem and filesystem connector -->
<dependency>
<groupId>org.apache.flink</groupId>
- <artifactId>flink-table-runtime_${scala.binary.version}</artifactId>
+ <artifactId>flink-table-common</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
+ <optional>true</optional>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-connector-files</artifactId>
+ <version>${project.version}</version>
+ <scope>provided</scope>
+ <optional>true</optional>
</dependency>
<!-- ORC -->
diff --git a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/OrcColumnarRowInputFormat.java b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/OrcColumnarRowInputFormat.java
index bb12d51..d719a1c 100644
--- a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/OrcColumnarRowInputFormat.java
+++ b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/OrcColumnarRowInputFormat.java
@@ -21,16 +21,15 @@ package org.apache.flink.orc;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.connector.file.src.FileSourceSplit;
import org.apache.flink.connector.file.src.util.Pool;
+import org.apache.flink.connector.file.table.ColumnarRowIterator;
+import org.apache.flink.connector.file.table.PartitionFieldExtractor;
import org.apache.flink.orc.shim.OrcShim;
import org.apache.flink.orc.vector.ColumnBatchFactory;
import org.apache.flink.orc.vector.OrcVectorizedBatchWrapper;
-import org.apache.flink.table.data.ColumnarRowData;
import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.data.vector.ColumnVector;
-import org.apache.flink.table.data.vector.VectorizedColumnBatch;
-import org.apache.flink.table.filesystem.ColumnarRowIterator;
-import org.apache.flink.table.filesystem.PartitionFieldExtractor;
-import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.data.columnar.ColumnarRowData;
+import org.apache.flink.table.data.columnar.vector.ColumnVector;
+import org.apache.flink.table.data.columnar.vector.VectorizedColumnBatch;
import org.apache.flink.table.types.logical.LogicalType;
import org.apache.flink.table.types.logical.RowType;
@@ -40,6 +39,7 @@ import org.apache.orc.TypeDescription;
import java.util.Arrays;
import java.util.List;
+import java.util.function.Function;
import java.util.stream.Collectors;
import static org.apache.flink.orc.OrcSplitReaderUtil.convertToOrcTypeWithPart;
@@ -61,7 +61,7 @@ public class OrcColumnarRowInputFormat<BatchT, SplitT extends FileSourceSplit>
private static final long serialVersionUID = 1L;
private final ColumnBatchFactory<BatchT, SplitT> batchFactory;
- private final RowType projectedOutputType;
+ private final TypeInformation<RowData> producedTypeInfo;
public OrcColumnarRowInputFormat(
final OrcShim<BatchT> shim,
@@ -71,10 +71,10 @@ public class OrcColumnarRowInputFormat<BatchT, SplitT extends FileSourceSplit>
final List<OrcFilters.Predicate> conjunctPredicates,
final int batchSize,
final ColumnBatchFactory<BatchT, SplitT> batchFactory,
- final RowType projectedOutputType) {
+ TypeInformation<RowData> producedTypeInfo) {
super(shim, hadoopConfig, schema, selectedFields, conjunctPredicates, batchSize);
this.batchFactory = batchFactory;
- this.projectedOutputType = projectedOutputType;
+ this.producedTypeInfo = producedTypeInfo;
}
@Override
@@ -91,7 +91,7 @@ public class OrcColumnarRowInputFormat<BatchT, SplitT extends FileSourceSplit>
@Override
public TypeInformation<RowData> getProducedType() {
- return InternalTypeInfo.of(projectedOutputType);
+ return this.producedTypeInfo;
}
// ------------------------------------------------------------------------
@@ -138,7 +138,12 @@ public class OrcColumnarRowInputFormat<BatchT, SplitT extends FileSourceSplit>
PartitionFieldExtractor<SplitT> extractor,
int[] selectedFields,
List<OrcFilters.Predicate> conjunctPredicates,
- int batchSize) {
+ int batchSize,
+ Function<RowType, TypeInformation<RowData>> rowTypeInfoFactory) {
+ // TODO FLINK-25113 all this partition keys code should be pruned from the orc format,
+ // because now FileSystemTableSource uses FileInfoExtractorBulkFormat for reading partition
+ // keys.
+
String[] tableFieldNames = tableType.getFieldNames().toArray(new String[0]);
LogicalType[] tableFieldTypes = tableType.getChildren().toArray(new LogicalType[0]);
List<String> orcFieldNames = getNonPartNames(tableFieldNames, partitionKeys);
@@ -172,9 +177,10 @@ public class OrcColumnarRowInputFormat<BatchT, SplitT extends FileSourceSplit>
conjunctPredicates,
batchSize,
batchGenerator,
- new RowType(
- Arrays.stream(selectedFields)
- .mapToObj(i -> tableType.getFields().get(i))
- .collect(Collectors.toList())));
+ rowTypeInfoFactory.apply(
+ new RowType(
+ Arrays.stream(selectedFields)
+ .mapToObj(i -> tableType.getFields().get(i))
+ .collect(Collectors.toList()))));
}
}
diff --git a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/OrcColumnarRowSplitReader.java b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/OrcColumnarRowSplitReader.java
index b472ae3..8b55050 100644
--- a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/OrcColumnarRowSplitReader.java
+++ b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/OrcColumnarRowSplitReader.java
@@ -20,9 +20,9 @@ package org.apache.flink.orc;
import org.apache.flink.core.fs.Path;
import org.apache.flink.orc.shim.OrcShim;
-import org.apache.flink.table.data.ColumnarRowData;
import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.flink.table.data.columnar.ColumnarRowData;
+import org.apache.flink.table.data.columnar.vector.VectorizedColumnBatch;
import org.apache.hadoop.conf.Configuration;
import org.apache.orc.TypeDescription;
diff --git a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/OrcFileFormatFactory.java b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/OrcFileFormatFactory.java
index 6f7eb27..1e0bf42 100644
--- a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/OrcFileFormatFactory.java
+++ b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/OrcFileFormatFactory.java
@@ -23,22 +23,22 @@ import org.apache.flink.configuration.ConfigOption;
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.connector.file.table.factories.BulkReaderFormatFactory;
+import org.apache.flink.connector.file.table.factories.BulkWriterFormatFactory;
+import org.apache.flink.connector.file.table.format.BulkDecodingFormat;
import org.apache.flink.orc.shim.OrcShim;
import org.apache.flink.orc.vector.RowDataVectorizer;
import org.apache.flink.orc.writer.OrcBulkWriterFactory;
import org.apache.flink.table.connector.ChangelogMode;
import org.apache.flink.table.connector.Projection;
-import org.apache.flink.table.connector.format.BulkDecodingFormat;
import org.apache.flink.table.connector.format.EncodingFormat;
import org.apache.flink.table.connector.format.ProjectableDecodingFormat;
import org.apache.flink.table.connector.sink.DynamicTableSink;
import org.apache.flink.table.connector.source.DynamicTableSource;
import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.flink.table.data.columnar.vector.VectorizedColumnBatch;
import org.apache.flink.table.expressions.Expression;
import org.apache.flink.table.expressions.ResolvedExpression;
-import org.apache.flink.table.factories.BulkReaderFormatFactory;
-import org.apache.flink.table.factories.BulkWriterFormatFactory;
import org.apache.flink.table.factories.DynamicTableFactory;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.logical.LogicalType;
@@ -154,7 +154,8 @@ public class OrcFileFormatFactory implements BulkReaderFormatFactory, BulkWriter
null,
Projection.of(projections).toTopLevelIndexes(),
orcPredicates,
- VectorizedColumnBatch.DEFAULT_SIZE);
+ VectorizedColumnBatch.DEFAULT_SIZE,
+ sourceContext::createTypeInformation);
}
@Override
diff --git a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/OrcSplitReaderUtil.java b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/OrcSplitReaderUtil.java
index b9f1e7d..dcceedb 100644
--- a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/OrcSplitReaderUtil.java
+++ b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/OrcSplitReaderUtil.java
@@ -22,8 +22,8 @@ import org.apache.flink.core.fs.Path;
import org.apache.flink.orc.OrcColumnarRowSplitReader.ColumnBatchGenerator;
import org.apache.flink.orc.shim.OrcShim;
import org.apache.flink.table.api.DataTypes;
-import org.apache.flink.table.data.vector.ColumnVector;
-import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.flink.table.data.columnar.vector.ColumnVector;
+import org.apache.flink.table.data.columnar.vector.VectorizedColumnBatch;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.logical.ArrayType;
import org.apache.flink.table.types.logical.CharType;
diff --git a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/AbstractOrcColumnVector.java b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/AbstractOrcColumnVector.java
index 994fd4c..0239024 100644
--- a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/AbstractOrcColumnVector.java
+++ b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/AbstractOrcColumnVector.java
@@ -45,7 +45,7 @@ import static org.apache.flink.table.utils.DateTimeUtils.toInternal;
/** This column vector is used to adapt hive's ColumnVector to Flink's ColumnVector. */
public abstract class AbstractOrcColumnVector
- implements org.apache.flink.table.data.vector.ColumnVector {
+ implements org.apache.flink.table.data.columnar.vector.ColumnVector {
private ColumnVector vector;
@@ -58,7 +58,7 @@ public abstract class AbstractOrcColumnVector
return !vector.noNulls && vector.isNull[vector.isRepeating ? 0 : i];
}
- public static org.apache.flink.table.data.vector.ColumnVector createFlinkVector(
+ public static org.apache.flink.table.data.columnar.vector.ColumnVector createFlinkVector(
ColumnVector vector, LogicalType logicalType) {
if (vector instanceof LongColumnVector) {
if (logicalType.getTypeRoot() == LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE) {
@@ -87,8 +87,8 @@ public abstract class AbstractOrcColumnVector
}
/** Create flink vector by hive vector from constant. */
- public static org.apache.flink.table.data.vector.ColumnVector createFlinkVectorFromConstant(
- LogicalType type, Object value, int batchSize) {
+ public static org.apache.flink.table.data.columnar.vector.ColumnVector
+ createFlinkVectorFromConstant(LogicalType type, Object value, int batchSize) {
return createFlinkVector(createHiveVectorFromConstant(type, value, batchSize), type);
}
diff --git a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/ColumnBatchFactory.java b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/ColumnBatchFactory.java
index 23c4c95..90d7fea 100644
--- a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/ColumnBatchFactory.java
+++ b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/ColumnBatchFactory.java
@@ -19,7 +19,7 @@
package org.apache.flink.orc.vector;
import org.apache.flink.connector.file.src.FileSourceSplit;
-import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.flink.table.data.columnar.vector.VectorizedColumnBatch;
import java.io.Serializable;
diff --git a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcArrayColumnVector.java b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcArrayColumnVector.java
index 9d68a42..1272971 100644
--- a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcArrayColumnVector.java
+++ b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcArrayColumnVector.java
@@ -19,15 +19,15 @@
package org.apache.flink.orc.vector;
import org.apache.flink.table.data.ArrayData;
-import org.apache.flink.table.data.ColumnarArrayData;
-import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.columnar.ColumnarArrayData;
+import org.apache.flink.table.data.columnar.vector.ColumnVector;
import org.apache.flink.table.types.logical.ArrayType;
import org.apache.hadoop.hive.ql.exec.vector.ListColumnVector;
/** This column vector is used to adapt hive's ListColumnVector to Flink's ArrayColumnVector. */
public class OrcArrayColumnVector extends AbstractOrcColumnVector
- implements org.apache.flink.table.data.vector.ArrayColumnVector {
+ implements org.apache.flink.table.data.columnar.vector.ArrayColumnVector {
private final ListColumnVector hiveVector;
private final ColumnVector flinkVector;
diff --git a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcBytesColumnVector.java b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcBytesColumnVector.java
index 272b105..77065e3 100644
--- a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcBytesColumnVector.java
+++ b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcBytesColumnVector.java
@@ -22,7 +22,7 @@ import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
/** This column vector is used to adapt hive's BytesColumnVector to Flink's BytesColumnVector. */
public class OrcBytesColumnVector extends AbstractOrcColumnVector
- implements org.apache.flink.table.data.vector.BytesColumnVector {
+ implements org.apache.flink.table.data.columnar.vector.BytesColumnVector {
private BytesColumnVector vector;
diff --git a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcDecimalColumnVector.java b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcDecimalColumnVector.java
index 3590e70..4117828 100644
--- a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcDecimalColumnVector.java
+++ b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcDecimalColumnVector.java
@@ -28,7 +28,7 @@ import java.math.BigDecimal;
* This column vector is used to adapt hive's DecimalColumnVector to Flink's DecimalColumnVector.
*/
public class OrcDecimalColumnVector extends AbstractOrcColumnVector
- implements org.apache.flink.table.data.vector.DecimalColumnVector {
+ implements org.apache.flink.table.data.columnar.vector.DecimalColumnVector {
private DecimalColumnVector vector;
diff --git a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcDoubleColumnVector.java b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcDoubleColumnVector.java
index 14fc432..bc01ad2 100644
--- a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcDoubleColumnVector.java
+++ b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcDoubleColumnVector.java
@@ -25,8 +25,8 @@ import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
* ColumnVector.
*/
public class OrcDoubleColumnVector extends AbstractOrcColumnVector
- implements org.apache.flink.table.data.vector.DoubleColumnVector,
- org.apache.flink.table.data.vector.FloatColumnVector {
+ implements org.apache.flink.table.data.columnar.vector.DoubleColumnVector,
+ org.apache.flink.table.data.columnar.vector.FloatColumnVector {
private DoubleColumnVector vector;
diff --git a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcLegacyTimestampColumnVector.java b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcLegacyTimestampColumnVector.java
index 3fccbac..cdb8125 100644
--- a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcLegacyTimestampColumnVector.java
+++ b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcLegacyTimestampColumnVector.java
@@ -31,7 +31,7 @@ import java.time.LocalDateTime;
* LongColumnVector.
*/
public class OrcLegacyTimestampColumnVector extends AbstractOrcColumnVector
- implements org.apache.flink.table.data.vector.TimestampColumnVector {
+ implements org.apache.flink.table.data.columnar.vector.TimestampColumnVector {
private final LongColumnVector hiveVector;
diff --git a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcLongColumnVector.java b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcLongColumnVector.java
index 97855be..e2a5993 100644
--- a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcLongColumnVector.java
+++ b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcLongColumnVector.java
@@ -25,11 +25,11 @@ import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
* and long ColumnVector.
*/
public class OrcLongColumnVector extends AbstractOrcColumnVector
- implements org.apache.flink.table.data.vector.LongColumnVector,
- org.apache.flink.table.data.vector.BooleanColumnVector,
- org.apache.flink.table.data.vector.ByteColumnVector,
- org.apache.flink.table.data.vector.ShortColumnVector,
- org.apache.flink.table.data.vector.IntColumnVector {
+ implements org.apache.flink.table.data.columnar.vector.LongColumnVector,
+ org.apache.flink.table.data.columnar.vector.BooleanColumnVector,
+ org.apache.flink.table.data.columnar.vector.ByteColumnVector,
+ org.apache.flink.table.data.columnar.vector.ShortColumnVector,
+ org.apache.flink.table.data.columnar.vector.IntColumnVector {
private LongColumnVector vector;
diff --git a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcMapColumnVector.java b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcMapColumnVector.java
index 1856c07..814f53c 100644
--- a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcMapColumnVector.java
+++ b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcMapColumnVector.java
@@ -18,16 +18,16 @@
package org.apache.flink.orc.vector;
-import org.apache.flink.table.data.ColumnarMapData;
import org.apache.flink.table.data.MapData;
-import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.columnar.ColumnarMapData;
+import org.apache.flink.table.data.columnar.vector.ColumnVector;
import org.apache.flink.table.types.logical.MapType;
import org.apache.hadoop.hive.ql.exec.vector.MapColumnVector;
/** This column vector is used to adapt hive's MapColumnVector to Flink's MapColumnVector. */
public class OrcMapColumnVector extends AbstractOrcColumnVector
- implements org.apache.flink.table.data.vector.MapColumnVector {
+ implements org.apache.flink.table.data.columnar.vector.MapColumnVector {
private final MapColumnVector hiveVector;
private final ColumnVector keyFlinkVector;
diff --git a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcRowColumnVector.java b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcRowColumnVector.java
index 22bfbb1..668ec70 100644
--- a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcRowColumnVector.java
+++ b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcRowColumnVector.java
@@ -18,16 +18,16 @@
package org.apache.flink.orc.vector;
-import org.apache.flink.table.data.ColumnarRowData;
-import org.apache.flink.table.data.vector.ColumnVector;
-import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.flink.table.data.columnar.ColumnarRowData;
+import org.apache.flink.table.data.columnar.vector.ColumnVector;
+import org.apache.flink.table.data.columnar.vector.VectorizedColumnBatch;
import org.apache.flink.table.types.logical.RowType;
import org.apache.hadoop.hive.ql.exec.vector.StructColumnVector;
/** This column vector is used to adapt hive's StructColumnVector to Flink's RowColumnVector. */
public class OrcRowColumnVector extends AbstractOrcColumnVector
- implements org.apache.flink.table.data.vector.RowColumnVector {
+ implements org.apache.flink.table.data.columnar.vector.RowColumnVector {
private final ColumnarRowData columnarRowData;
diff --git a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcTimestampColumnVector.java b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcTimestampColumnVector.java
index ae8abfb..b06d91f 100644
--- a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcTimestampColumnVector.java
+++ b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/OrcTimestampColumnVector.java
@@ -31,7 +31,7 @@ import java.time.LocalDateTime;
* TimestampColumnVector.
*/
public class OrcTimestampColumnVector extends AbstractOrcColumnVector
- implements org.apache.flink.table.data.vector.TimestampColumnVector {
+ implements org.apache.flink.table.data.columnar.vector.TimestampColumnVector {
private TimestampColumnVector vector;
diff --git a/flink-formats/flink-orc/src/test/java/org/apache/flink/orc/OrcColumnarRowInputFormatTest.java b/flink-formats/flink-orc/src/test/java/org/apache/flink/orc/OrcColumnarRowInputFormatTest.java
index be32054..85c58bb 100644
--- a/flink-formats/flink-orc/src/test/java/org/apache/flink/orc/OrcColumnarRowInputFormatTest.java
+++ b/flink-formats/flink-orc/src/test/java/org/apache/flink/orc/OrcColumnarRowInputFormatTest.java
@@ -23,6 +23,7 @@ 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.connector.file.src.util.Utils;
+import org.apache.flink.connector.file.table.PartitionFieldExtractor;
import org.apache.flink.core.fs.FileStatus;
import org.apache.flink.core.fs.Path;
import org.apache.flink.orc.OrcFilters.Between;
@@ -33,7 +34,7 @@ import org.apache.flink.orc.shim.OrcShim;
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.filesystem.PartitionFieldExtractor;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
import org.apache.flink.table.types.logical.DecimalType;
import org.apache.flink.table.types.logical.LogicalType;
import org.apache.flink.table.types.logical.RowType;
@@ -354,7 +355,8 @@ public class OrcColumnarRowInputFormatTest {
PartitionFieldExtractor.forFileSystem(""),
selectedFields,
conjunctPredicates,
- BATCH_SIZE);
+ BATCH_SIZE,
+ InternalTypeInfo::of);
}
protected OrcColumnarRowInputFormat<?, FileSourceSplit> createPartitionFormat(
@@ -367,7 +369,8 @@ public class OrcColumnarRowInputFormatTest {
PartitionFieldExtractor.forFileSystem(""),
selectedFields,
new ArrayList<>(),
- BATCH_SIZE);
+ BATCH_SIZE,
+ InternalTypeInfo::of);
}
private BulkFormat.Reader<RowData> createReader(
diff --git a/flink-formats/flink-parquet/pom.xml b/flink-formats/flink-parquet/pom.xml
index e41986c..35bda88 100644
--- a/flink-formats/flink-parquet/pom.xml
+++ b/flink-formats/flink-parquet/pom.xml
@@ -29,7 +29,7 @@ under the License.
<relativePath>..</relativePath>
</parent>
- <artifactId>flink-parquet_${scala.binary.version}</artifactId>
+ <artifactId>flink-parquet</artifactId>
<name>Flink : Formats : Parquet</name>
<packaging>jar</packaging>
@@ -45,11 +45,19 @@ under the License.
<scope>provided</scope>
</dependency>
- <!-- Table ecosystem -->
+ <!-- Table ecosystem and filesystem connector -->
<dependency>
<groupId>org.apache.flink</groupId>
- <artifactId>flink-table-runtime_${scala.binary.version}</artifactId>
+ <artifactId>flink-table-common</artifactId>
+ <version>${project.version}</version>
+ <scope>provided</scope>
+ <optional>true</optional>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-connector-files</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
<optional>true</optional>
diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetColumnarRowInputFormat.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetColumnarRowInputFormat.java
index 438ab3a..c899597 100644
--- a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetColumnarRowInputFormat.java
+++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetColumnarRowInputFormat.java
@@ -21,17 +21,16 @@ package org.apache.flink.formats.parquet;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.connector.file.src.FileSourceSplit;
import org.apache.flink.connector.file.src.util.Pool;
+import org.apache.flink.connector.file.table.ColumnarRowIterator;
+import org.apache.flink.connector.file.table.PartitionFieldExtractor;
import org.apache.flink.core.fs.Path;
import org.apache.flink.formats.parquet.utils.SerializableConfiguration;
import org.apache.flink.formats.parquet.vector.ColumnBatchFactory;
-import org.apache.flink.table.data.ColumnarRowData;
import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.data.vector.ColumnVector;
-import org.apache.flink.table.data.vector.VectorizedColumnBatch;
-import org.apache.flink.table.data.vector.writable.WritableColumnVector;
-import org.apache.flink.table.filesystem.ColumnarRowIterator;
-import org.apache.flink.table.filesystem.PartitionFieldExtractor;
-import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.data.columnar.ColumnarRowData;
+import org.apache.flink.table.data.columnar.vector.ColumnVector;
+import org.apache.flink.table.data.columnar.vector.VectorizedColumnBatch;
+import org.apache.flink.table.data.columnar.vector.writable.WritableColumnVector;
import org.apache.flink.table.types.logical.RowType;
import org.apache.hadoop.conf.Configuration;
@@ -50,19 +49,20 @@ public class ParquetColumnarRowInputFormat<SplitT extends FileSourceSplit>
private static final long serialVersionUID = 1L;
- private final RowType producedType;
+ private final TypeInformation<RowData> producedTypeInfo;
/** Constructor to create parquet format without extra fields. */
- public ParquetColumnarRowInputFormat(
+ ParquetColumnarRowInputFormat(
Configuration hadoopConfig,
RowType projectedType,
+ TypeInformation<RowData> producedTypeInfo,
int batchSize,
boolean isUtcTimestamp,
boolean isCaseSensitive) {
this(
hadoopConfig,
projectedType,
- projectedType,
+ producedTypeInfo,
ColumnBatchFactory.withoutExtraFields(),
batchSize,
isUtcTimestamp,
@@ -73,13 +73,14 @@ public class ParquetColumnarRowInputFormat<SplitT extends FileSourceSplit>
* Constructor to create parquet format with extra fields created by {@link ColumnBatchFactory}.
*
* @param projectedType the projected row type for parquet format, excludes extra fields.
- * @param producedType the produced row type for this input format, includes extra fields.
+ * @param producedTypeInfo the produced row type info for this input format, includes extra
+ * fields.
* @param batchFactory factory for creating column batch, can cram in extra fields.
*/
- public ParquetColumnarRowInputFormat(
+ ParquetColumnarRowInputFormat(
Configuration hadoopConfig,
RowType projectedType,
- RowType producedType,
+ TypeInformation<RowData> producedTypeInfo,
ColumnBatchFactory<SplitT> batchFactory,
int batchSize,
boolean isUtcTimestamp,
@@ -91,7 +92,7 @@ public class ParquetColumnarRowInputFormat<SplitT extends FileSourceSplit>
batchSize,
isUtcTimestamp,
isCaseSensitive);
- this.producedType = producedType;
+ this.producedTypeInfo = producedTypeInfo;
}
@Override
@@ -113,7 +114,7 @@ public class ParquetColumnarRowInputFormat<SplitT extends FileSourceSplit>
@Override
public TypeInformation<RowData> getProducedType() {
- return InternalTypeInfo.of(producedType);
+ return producedTypeInfo;
}
private static class ColumnarRowReaderBatch extends ParquetReaderBatch<RowData> {
@@ -144,11 +145,16 @@ public class ParquetColumnarRowInputFormat<SplitT extends FileSourceSplit>
ParquetColumnarRowInputFormat<SplitT> createPartitionedFormat(
Configuration hadoopConfig,
RowType producedRowType,
+ TypeInformation<RowData> producedTypeInfo,
List<String> partitionKeys,
PartitionFieldExtractor<SplitT> extractor,
int batchSize,
boolean isUtcTimestamp,
boolean isCaseSensitive) {
+ // TODO FLINK-25113 all this partition keys code should be pruned from the parquet format,
+ // because now FileSystemTableSource uses FileInfoExtractorBulkFormat for reading partition
+ // keys.
+
RowType projectedRowType =
new RowType(
producedRowType.getFields().stream()
@@ -178,7 +184,7 @@ public class ParquetColumnarRowInputFormat<SplitT extends FileSourceSplit>
return new ParquetColumnarRowInputFormat<>(
hadoopConfig,
projectedRowType,
- producedRowType,
+ producedTypeInfo,
factory,
batchSize,
isUtcTimestamp,
diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetFileFormatFactory.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetFileFormatFactory.java
index 2321d7b..fb906b1 100644
--- a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetFileFormatFactory.java
+++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetFileFormatFactory.java
@@ -23,18 +23,18 @@ import org.apache.flink.configuration.ConfigOption;
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.connector.file.table.factories.BulkReaderFormatFactory;
+import org.apache.flink.connector.file.table.factories.BulkWriterFormatFactory;
+import org.apache.flink.connector.file.table.format.BulkDecodingFormat;
import org.apache.flink.formats.parquet.row.ParquetRowDataBuilder;
import org.apache.flink.table.connector.ChangelogMode;
import org.apache.flink.table.connector.Projection;
-import org.apache.flink.table.connector.format.BulkDecodingFormat;
import org.apache.flink.table.connector.format.EncodingFormat;
import org.apache.flink.table.connector.format.ProjectableDecodingFormat;
import org.apache.flink.table.connector.sink.DynamicTableSink;
import org.apache.flink.table.connector.source.DynamicTableSource;
import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.data.vector.VectorizedColumnBatch;
-import org.apache.flink.table.factories.BulkReaderFormatFactory;
-import org.apache.flink.table.factories.BulkWriterFormatFactory;
+import org.apache.flink.table.data.columnar.vector.VectorizedColumnBatch;
import org.apache.flink.table.factories.DynamicTableFactory;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.logical.RowType;
@@ -130,6 +130,7 @@ public class ParquetFileFormatFactory implements BulkReaderFormatFactory, BulkWr
return ParquetColumnarRowInputFormat.createPartitionedFormat(
getParquetConfiguration(formatOptions),
(RowType) Projection.of(projections).project(producedDataType).getLogicalType(),
+ sourceContext.createTypeInformation(producedDataType),
Collections.emptyList(),
null,
VectorizedColumnBatch.DEFAULT_SIZE,
diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetVectorizedInputFormat.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetVectorizedInputFormat.java
index 5246334..a639458 100644
--- a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetVectorizedInputFormat.java
+++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetVectorizedInputFormat.java
@@ -31,9 +31,9 @@ import org.apache.flink.formats.parquet.vector.ColumnBatchFactory;
import org.apache.flink.formats.parquet.vector.ParquetDecimalVector;
import org.apache.flink.formats.parquet.vector.reader.AbstractColumnReader;
import org.apache.flink.formats.parquet.vector.reader.ColumnReader;
-import org.apache.flink.table.data.vector.ColumnVector;
-import org.apache.flink.table.data.vector.VectorizedColumnBatch;
-import org.apache.flink.table.data.vector.writable.WritableColumnVector;
+import org.apache.flink.table.data.columnar.vector.ColumnVector;
+import org.apache.flink.table.data.columnar.vector.VectorizedColumnBatch;
+import org.apache.flink.table.data.columnar.vector.writable.WritableColumnVector;
import org.apache.flink.table.types.logical.LogicalType;
import org.apache.flink.table.types.logical.LogicalTypeRoot;
import org.apache.flink.table.types.logical.RowType;
diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ColumnBatchFactory.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ColumnBatchFactory.java
index 9acc2c3..025d676 100644
--- a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ColumnBatchFactory.java
+++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ColumnBatchFactory.java
@@ -19,8 +19,8 @@
package org.apache.flink.formats.parquet.vector;
import org.apache.flink.connector.file.src.FileSourceSplit;
-import org.apache.flink.table.data.vector.ColumnVector;
-import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.flink.table.data.columnar.vector.ColumnVector;
+import org.apache.flink.table.data.columnar.vector.VectorizedColumnBatch;
import java.io.Serializable;
diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetColumnarRowSplitReader.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetColumnarRowSplitReader.java
index 5268e3f..4504739 100644
--- a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetColumnarRowSplitReader.java
+++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetColumnarRowSplitReader.java
@@ -20,10 +20,10 @@ package org.apache.flink.formats.parquet.vector;
import org.apache.flink.formats.parquet.vector.reader.AbstractColumnReader;
import org.apache.flink.formats.parquet.vector.reader.ColumnReader;
-import org.apache.flink.table.data.ColumnarRowData;
-import org.apache.flink.table.data.vector.ColumnVector;
-import org.apache.flink.table.data.vector.VectorizedColumnBatch;
-import org.apache.flink.table.data.vector.writable.WritableColumnVector;
+import org.apache.flink.table.data.columnar.ColumnarRowData;
+import org.apache.flink.table.data.columnar.vector.ColumnVector;
+import org.apache.flink.table.data.columnar.vector.VectorizedColumnBatch;
+import org.apache.flink.table.data.columnar.vector.writable.WritableColumnVector;
import org.apache.flink.table.types.logical.LogicalType;
import org.apache.flink.table.types.logical.LogicalTypeRoot;
import org.apache.flink.util.FlinkRuntimeException;
diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetDecimalVector.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetDecimalVector.java
index 6ca1d95..a47b159 100644
--- a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetDecimalVector.java
+++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetDecimalVector.java
@@ -20,11 +20,11 @@ package org.apache.flink.formats.parquet.vector;
import org.apache.flink.formats.parquet.utils.ParquetSchemaConverter;
import org.apache.flink.table.data.DecimalData;
-import org.apache.flink.table.data.vector.BytesColumnVector;
-import org.apache.flink.table.data.vector.ColumnVector;
-import org.apache.flink.table.data.vector.DecimalColumnVector;
-import org.apache.flink.table.data.vector.IntColumnVector;
-import org.apache.flink.table.data.vector.LongColumnVector;
+import org.apache.flink.table.data.columnar.vector.BytesColumnVector;
+import org.apache.flink.table.data.columnar.vector.ColumnVector;
+import org.apache.flink.table.data.columnar.vector.DecimalColumnVector;
+import org.apache.flink.table.data.columnar.vector.IntColumnVector;
+import org.apache.flink.table.data.columnar.vector.LongColumnVector;
/**
* Parquet write decimal as int32 and int64 and binary, this class wrap the real vector to provide
diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetDictionary.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetDictionary.java
index b4b2f98..2a09811 100644
--- a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetDictionary.java
+++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetDictionary.java
@@ -19,7 +19,7 @@
package org.apache.flink.formats.parquet.vector;
import org.apache.flink.table.data.TimestampData;
-import org.apache.flink.table.data.vector.Dictionary;
+import org.apache.flink.table.data.columnar.vector.Dictionary;
import static org.apache.flink.formats.parquet.vector.reader.TimestampColumnReader.decodeInt96ToTimestamp;
diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetSplitReaderUtil.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetSplitReaderUtil.java
index 8634f5f..e00eb07 100644
--- a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetSplitReaderUtil.java
+++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetSplitReaderUtil.java
@@ -33,18 +33,18 @@ import org.apache.flink.formats.parquet.vector.reader.ShortColumnReader;
import org.apache.flink.formats.parquet.vector.reader.TimestampColumnReader;
import org.apache.flink.table.data.DecimalData;
import org.apache.flink.table.data.TimestampData;
-import org.apache.flink.table.data.vector.ColumnVector;
-import org.apache.flink.table.data.vector.VectorizedColumnBatch;
-import org.apache.flink.table.data.vector.heap.HeapBooleanVector;
-import org.apache.flink.table.data.vector.heap.HeapByteVector;
-import org.apache.flink.table.data.vector.heap.HeapBytesVector;
-import org.apache.flink.table.data.vector.heap.HeapDoubleVector;
-import org.apache.flink.table.data.vector.heap.HeapFloatVector;
-import org.apache.flink.table.data.vector.heap.HeapIntVector;
-import org.apache.flink.table.data.vector.heap.HeapLongVector;
-import org.apache.flink.table.data.vector.heap.HeapShortVector;
-import org.apache.flink.table.data.vector.heap.HeapTimestampVector;
-import org.apache.flink.table.data.vector.writable.WritableColumnVector;
+import org.apache.flink.table.data.columnar.vector.ColumnVector;
+import org.apache.flink.table.data.columnar.vector.VectorizedColumnBatch;
+import org.apache.flink.table.data.columnar.vector.heap.HeapBooleanVector;
+import org.apache.flink.table.data.columnar.vector.heap.HeapByteVector;
+import org.apache.flink.table.data.columnar.vector.heap.HeapBytesVector;
+import org.apache.flink.table.data.columnar.vector.heap.HeapDoubleVector;
+import org.apache.flink.table.data.columnar.vector.heap.HeapFloatVector;
+import org.apache.flink.table.data.columnar.vector.heap.HeapIntVector;
+import org.apache.flink.table.data.columnar.vector.heap.HeapLongVector;
+import org.apache.flink.table.data.columnar.vector.heap.HeapShortVector;
+import org.apache.flink.table.data.columnar.vector.heap.HeapTimestampVector;
+import org.apache.flink.table.data.columnar.vector.writable.WritableColumnVector;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.logical.BigIntType;
import org.apache.flink.table.types.logical.DecimalType;
diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/AbstractColumnReader.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/AbstractColumnReader.java
index 4f8258c..8f4382a 100644
--- a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/AbstractColumnReader.java
+++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/AbstractColumnReader.java
@@ -18,8 +18,8 @@
package org.apache.flink.formats.parquet.vector.reader;
import org.apache.flink.formats.parquet.vector.ParquetDictionary;
-import org.apache.flink.table.data.vector.writable.WritableColumnVector;
-import org.apache.flink.table.data.vector.writable.WritableIntVector;
+import org.apache.flink.table.data.columnar.vector.writable.WritableColumnVector;
+import org.apache.flink.table.data.columnar.vector.writable.WritableIntVector;
import org.apache.parquet.Preconditions;
import org.apache.parquet.bytes.ByteBufferInputStream;
diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/BooleanColumnReader.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/BooleanColumnReader.java
index cae0a7c..e9e5e0d 100644
--- a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/BooleanColumnReader.java
+++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/BooleanColumnReader.java
@@ -17,8 +17,8 @@
package org.apache.flink.formats.parquet.vector.reader;
-import org.apache.flink.table.data.vector.writable.WritableBooleanVector;
-import org.apache.flink.table.data.vector.writable.WritableIntVector;
+import org.apache.flink.table.data.columnar.vector.writable.WritableBooleanVector;
+import org.apache.flink.table.data.columnar.vector.writable.WritableIntVector;
import org.apache.parquet.column.ColumnDescriptor;
import org.apache.parquet.column.page.PageReader;
diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/ByteColumnReader.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/ByteColumnReader.java
index a67aa57..b0de98f 100644
--- a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/ByteColumnReader.java
+++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/ByteColumnReader.java
@@ -17,8 +17,8 @@
package org.apache.flink.formats.parquet.vector.reader;
-import org.apache.flink.table.data.vector.writable.WritableByteVector;
-import org.apache.flink.table.data.vector.writable.WritableIntVector;
+import org.apache.flink.table.data.columnar.vector.writable.WritableByteVector;
+import org.apache.flink.table.data.columnar.vector.writable.WritableIntVector;
import org.apache.parquet.column.ColumnDescriptor;
import org.apache.parquet.column.page.PageReader;
diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/BytesColumnReader.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/BytesColumnReader.java
index 37d6b88..2cbdbda 100644
--- a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/BytesColumnReader.java
+++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/BytesColumnReader.java
@@ -17,8 +17,8 @@
package org.apache.flink.formats.parquet.vector.reader;
-import org.apache.flink.table.data.vector.writable.WritableBytesVector;
-import org.apache.flink.table.data.vector.writable.WritableIntVector;
+import org.apache.flink.table.data.columnar.vector.writable.WritableBytesVector;
+import org.apache.flink.table.data.columnar.vector.writable.WritableIntVector;
import org.apache.parquet.column.ColumnDescriptor;
import org.apache.parquet.column.page.PageReader;
diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/ColumnReader.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/ColumnReader.java
index 8486d77..53b63f2 100644
--- a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/ColumnReader.java
+++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/ColumnReader.java
@@ -17,7 +17,7 @@
package org.apache.flink.formats.parquet.vector.reader;
-import org.apache.flink.table.data.vector.writable.WritableColumnVector;
+import org.apache.flink.table.data.columnar.vector.writable.WritableColumnVector;
import java.io.IOException;
diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/DoubleColumnReader.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/DoubleColumnReader.java
index b7d8c23..aec8519 100644
--- a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/DoubleColumnReader.java
+++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/DoubleColumnReader.java
@@ -17,8 +17,8 @@
package org.apache.flink.formats.parquet.vector.reader;
-import org.apache.flink.table.data.vector.writable.WritableDoubleVector;
-import org.apache.flink.table.data.vector.writable.WritableIntVector;
+import org.apache.flink.table.data.columnar.vector.writable.WritableDoubleVector;
+import org.apache.flink.table.data.columnar.vector.writable.WritableIntVector;
import org.apache.parquet.column.ColumnDescriptor;
import org.apache.parquet.column.page.PageReader;
diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/FixedLenBytesColumnReader.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/FixedLenBytesColumnReader.java
index 9b12067..e82f68a 100644
--- a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/FixedLenBytesColumnReader.java
+++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/FixedLenBytesColumnReader.java
@@ -18,10 +18,10 @@
package org.apache.flink.formats.parquet.vector.reader;
import org.apache.flink.formats.parquet.utils.ParquetSchemaConverter;
-import org.apache.flink.table.data.vector.writable.WritableBytesVector;
-import org.apache.flink.table.data.vector.writable.WritableColumnVector;
-import org.apache.flink.table.data.vector.writable.WritableIntVector;
-import org.apache.flink.table.data.vector.writable.WritableLongVector;
+import org.apache.flink.table.data.columnar.vector.writable.WritableBytesVector;
+import org.apache.flink.table.data.columnar.vector.writable.WritableColumnVector;
+import org.apache.flink.table.data.columnar.vector.writable.WritableIntVector;
+import org.apache.flink.table.data.columnar.vector.writable.WritableLongVector;
import org.apache.parquet.column.ColumnDescriptor;
import org.apache.parquet.column.page.PageReader;
diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/FloatColumnReader.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/FloatColumnReader.java
index 9606a15..840c609 100644
--- a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/FloatColumnReader.java
+++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/FloatColumnReader.java
@@ -17,8 +17,8 @@
package org.apache.flink.formats.parquet.vector.reader;
-import org.apache.flink.table.data.vector.writable.WritableFloatVector;
-import org.apache.flink.table.data.vector.writable.WritableIntVector;
+import org.apache.flink.table.data.columnar.vector.writable.WritableFloatVector;
+import org.apache.flink.table.data.columnar.vector.writable.WritableIntVector;
import org.apache.parquet.column.ColumnDescriptor;
import org.apache.parquet.column.page.PageReader;
diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/IntColumnReader.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/IntColumnReader.java
index 5682b59..dae8969 100644
--- a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/IntColumnReader.java
+++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/IntColumnReader.java
@@ -17,7 +17,7 @@
package org.apache.flink.formats.parquet.vector.reader;
-import org.apache.flink.table.data.vector.writable.WritableIntVector;
+import org.apache.flink.table.data.columnar.vector.writable.WritableIntVector;
import org.apache.parquet.column.ColumnDescriptor;
import org.apache.parquet.column.page.PageReader;
diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/LongColumnReader.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/LongColumnReader.java
index 4c22ac6..601978d 100644
--- a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/LongColumnReader.java
+++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/LongColumnReader.java
@@ -17,8 +17,8 @@
package org.apache.flink.formats.parquet.vector.reader;
-import org.apache.flink.table.data.vector.writable.WritableIntVector;
-import org.apache.flink.table.data.vector.writable.WritableLongVector;
+import org.apache.flink.table.data.columnar.vector.writable.WritableIntVector;
+import org.apache.flink.table.data.columnar.vector.writable.WritableLongVector;
import org.apache.parquet.column.ColumnDescriptor;
import org.apache.parquet.column.page.PageReader;
diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/RunLengthDecoder.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/RunLengthDecoder.java
index 0e7fae8..8410af0 100644
--- a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/RunLengthDecoder.java
+++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/RunLengthDecoder.java
@@ -17,8 +17,8 @@
package org.apache.flink.formats.parquet.vector.reader;
-import org.apache.flink.table.data.vector.writable.WritableColumnVector;
-import org.apache.flink.table.data.vector.writable.WritableIntVector;
+import org.apache.flink.table.data.columnar.vector.writable.WritableColumnVector;
+import org.apache.flink.table.data.columnar.vector.writable.WritableIntVector;
import org.apache.parquet.Preconditions;
import org.apache.parquet.bytes.ByteBufferInputStream;
diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/ShortColumnReader.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/ShortColumnReader.java
index 6e4546d..676ff79 100644
--- a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/ShortColumnReader.java
+++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/ShortColumnReader.java
@@ -17,8 +17,8 @@
package org.apache.flink.formats.parquet.vector.reader;
-import org.apache.flink.table.data.vector.writable.WritableIntVector;
-import org.apache.flink.table.data.vector.writable.WritableShortVector;
+import org.apache.flink.table.data.columnar.vector.writable.WritableIntVector;
+import org.apache.flink.table.data.columnar.vector.writable.WritableShortVector;
import org.apache.parquet.column.ColumnDescriptor;
import org.apache.parquet.column.page.PageReader;
diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/TimestampColumnReader.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/TimestampColumnReader.java
index bc8c7a7..7deec50 100644
--- a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/TimestampColumnReader.java
+++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/TimestampColumnReader.java
@@ -18,8 +18,8 @@
package org.apache.flink.formats.parquet.vector.reader;
import org.apache.flink.table.data.TimestampData;
-import org.apache.flink.table.data.vector.writable.WritableIntVector;
-import org.apache.flink.table.data.vector.writable.WritableTimestampVector;
+import org.apache.flink.table.data.columnar.vector.writable.WritableIntVector;
+import org.apache.flink.table.data.columnar.vector.writable.WritableTimestampVector;
import org.apache.parquet.Preconditions;
import org.apache.parquet.column.ColumnDescriptor;
diff --git a/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/ParquetColumnarRowInputFormatTest.java b/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/ParquetColumnarRowInputFormatTest.java
index a28c86a..e376952 100644
--- a/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/ParquetColumnarRowInputFormatTest.java
+++ b/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/ParquetColumnarRowInputFormatTest.java
@@ -21,11 +21,12 @@ package org.apache.flink.formats.parquet;
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.table.PartitionFieldExtractor;
import org.apache.flink.core.fs.FileStatus;
import org.apache.flink.core.fs.Path;
import org.apache.flink.table.data.DecimalData;
import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.filesystem.PartitionFieldExtractor;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
import org.apache.flink.table.types.logical.BigIntType;
import org.apache.flink.table.types.logical.BooleanType;
import org.apache.flink.table.types.logical.DateType;
@@ -237,6 +238,7 @@ public class ParquetColumnarRowInputFormatTest {
new ParquetColumnarRowInputFormat(
new Configuration(),
RowType.of(fieldTypes, new String[] {"f7", "f2", "f4"}),
+ null,
500,
false,
true);
@@ -278,6 +280,7 @@ public class ParquetColumnarRowInputFormatTest {
new Configuration(),
// f99 not exist in parquet file.
RowType.of(fieldTypes, new String[] {"f7", "f2", "f4", "f99"}),
+ null,
500,
false,
true);
@@ -415,6 +418,7 @@ public class ParquetColumnarRowInputFormatTest {
"f0", "f1", "f2", "f3", "f4", "f5", "f6", "f7", "f8", "f9",
"f10", "f11", "f12", "f13", "f14"
}),
+ null,
500,
false,
true);
@@ -578,6 +582,7 @@ public class ParquetColumnarRowInputFormatTest {
ParquetColumnarRowInputFormat.createPartitionedFormat(
new Configuration(),
producedType,
+ InternalTypeInfo.of(producedType),
partitionKeys,
PartitionFieldExtractor.forFileSystem("my_default_value"),
500,
diff --git a/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/vector/ParquetColumnarRowSplitReaderTest.java b/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/vector/ParquetColumnarRowSplitReaderTest.java
index b280e72..88d5b79 100644
--- a/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/vector/ParquetColumnarRowSplitReaderTest.java
+++ b/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/vector/ParquetColumnarRowSplitReaderTest.java
@@ -19,9 +19,9 @@
package org.apache.flink.formats.parquet.vector;
import org.apache.flink.core.fs.Path;
-import org.apache.flink.table.data.ColumnarRowData;
import org.apache.flink.table.data.DecimalData;
-import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.flink.table.data.columnar.ColumnarRowData;
+import org.apache.flink.table.data.columnar.vector.VectorizedColumnBatch;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.logical.BigIntType;
import org.apache.flink.table.types.logical.BooleanType;
diff --git a/flink-formats/flink-sql-orc/pom.xml b/flink-formats/flink-sql-orc/pom.xml
index 13c560a..dd26c3a 100644
--- a/flink-formats/flink-sql-orc/pom.xml
+++ b/flink-formats/flink-sql-orc/pom.xml
@@ -29,7 +29,7 @@ under the License.
<relativePath>..</relativePath>
</parent>
- <artifactId>flink-sql-orc_${scala.binary.version}</artifactId>
+ <artifactId>flink-sql-orc</artifactId>
<name>Flink : Formats : SQL Orc</name>
<packaging>jar</packaging>
@@ -37,7 +37,7 @@ under the License.
<dependencies>
<dependency>
<groupId>org.apache.flink</groupId>
- <artifactId>flink-orc_${scala.binary.version}</artifactId>
+ <artifactId>flink-orc</artifactId>
<version>${project.version}</version>
</dependency>
</dependencies>
@@ -57,7 +57,7 @@ under the License.
<configuration>
<artifactSet>
<includes>
- <include>org.apache.flink:flink-orc_${scala.binary.version}</include>
+ <include>org.apache.flink:flink-orc</include>
<include>org.apache.orc:orc-core</include>
<include>org.apache.orc:orc-shims</include>
<include>org.apache.hive:hive-storage-api</include>
diff --git a/flink-formats/flink-sql-parquet/pom.xml b/flink-formats/flink-sql-parquet/pom.xml
index 647ff9e..360eae2 100644
--- a/flink-formats/flink-sql-parquet/pom.xml
+++ b/flink-formats/flink-sql-parquet/pom.xml
@@ -29,7 +29,7 @@ under the License.
<relativePath>..</relativePath>
</parent>
- <artifactId>flink-sql-parquet_${scala.binary.version}</artifactId>
+ <artifactId>flink-sql-parquet</artifactId>
<name>Flink : Formats : SQL Parquet</name>
<packaging>jar</packaging>
@@ -37,7 +37,7 @@ under the License.
<dependencies>
<dependency>
<groupId>org.apache.flink</groupId>
- <artifactId>flink-parquet_${scala.binary.version}</artifactId>
+ <artifactId>flink-parquet</artifactId>
<version>${project.version}</version>
</dependency>
</dependencies>
@@ -57,7 +57,7 @@ under the License.
<configuration>
<artifactSet>
<includes>
- <include>org.apache.flink:flink-parquet_${scala.binary.version}</include>
+ <include>org.apache.flink:flink-parquet</include>
<include>org.apache.parquet:parquet-hadoop</include>
<include>org.apache.parquet:parquet-format</include>
<include>org.apache.parquet:parquet-column</include>
diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowReader.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowReader.java
index fde8f4f..3f9eb79e 100644
--- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowReader.java
+++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowReader.java
@@ -19,10 +19,10 @@
package org.apache.flink.table.runtime.arrow;
import org.apache.flink.annotation.Internal;
-import org.apache.flink.table.data.ColumnarRowData;
import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.data.vector.ColumnVector;
-import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.flink.table.data.columnar.ColumnarRowData;
+import org.apache.flink.table.data.columnar.vector.ColumnVector;
+import org.apache.flink.table.data.columnar.vector.VectorizedColumnBatch;
import org.apache.flink.util.Preconditions;
/** {@link ArrowReader} which read the underlying Arrow format data as {@link RowData}. */
diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java
index 01895ae..46448bf 100644
--- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java
+++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java
@@ -29,8 +29,8 @@ import org.apache.flink.table.api.internal.TableEnvironmentImpl;
import org.apache.flink.table.api.internal.TableImpl;
import org.apache.flink.table.data.ArrayData;
import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.columnar.vector.ColumnVector;
import org.apache.flink.table.data.util.DataFormatConverters;
-import org.apache.flink.table.data.vector.ColumnVector;
import org.apache.flink.table.operations.OutputConversionModifyOperation;
import org.apache.flink.table.runtime.arrow.sources.ArrowTableSource;
import org.apache.flink.table.runtime.arrow.vectors.ArrowArrayColumnVector;
diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowArrayColumnVector.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowArrayColumnVector.java
index c13606f..04017b3 100644
--- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowArrayColumnVector.java
+++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowArrayColumnVector.java
@@ -20,9 +20,9 @@ package org.apache.flink.table.runtime.arrow.vectors;
import org.apache.flink.annotation.Internal;
import org.apache.flink.table.data.ArrayData;
-import org.apache.flink.table.data.ColumnarArrayData;
-import org.apache.flink.table.data.vector.ArrayColumnVector;
-import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.columnar.ColumnarArrayData;
+import org.apache.flink.table.data.columnar.vector.ArrayColumnVector;
+import org.apache.flink.table.data.columnar.vector.ColumnVector;
import org.apache.flink.util.Preconditions;
import org.apache.arrow.vector.complex.ListVector;
diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowBigIntColumnVector.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowBigIntColumnVector.java
index 8c2354e..1d92988 100644
--- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowBigIntColumnVector.java
+++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowBigIntColumnVector.java
@@ -19,7 +19,7 @@
package org.apache.flink.table.runtime.arrow.vectors;
import org.apache.flink.annotation.Internal;
-import org.apache.flink.table.data.vector.LongColumnVector;
+import org.apache.flink.table.data.columnar.vector.LongColumnVector;
import org.apache.flink.util.Preconditions;
import org.apache.arrow.vector.BigIntVector;
diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowBooleanColumnVector.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowBooleanColumnVector.java
index b9f6e59..f8c6aa7 100644
--- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowBooleanColumnVector.java
+++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowBooleanColumnVector.java
@@ -19,7 +19,7 @@
package org.apache.flink.table.runtime.arrow.vectors;
import org.apache.flink.annotation.Internal;
-import org.apache.flink.table.data.vector.BooleanColumnVector;
+import org.apache.flink.table.data.columnar.vector.BooleanColumnVector;
import org.apache.flink.util.Preconditions;
import org.apache.arrow.vector.BitVector;
diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowDateColumnVector.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowDateColumnVector.java
index 715d9ba..294aba9 100644
--- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowDateColumnVector.java
+++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowDateColumnVector.java
@@ -19,7 +19,7 @@
package org.apache.flink.table.runtime.arrow.vectors;
import org.apache.flink.annotation.Internal;
-import org.apache.flink.table.data.vector.IntColumnVector;
+import org.apache.flink.table.data.columnar.vector.IntColumnVector;
import org.apache.flink.util.Preconditions;
import org.apache.arrow.vector.DateDayVector;
diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowDecimalColumnVector.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowDecimalColumnVector.java
index 5ea4429..217842d 100644
--- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowDecimalColumnVector.java
+++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowDecimalColumnVector.java
@@ -20,7 +20,7 @@ package org.apache.flink.table.runtime.arrow.vectors;
import org.apache.flink.annotation.Internal;
import org.apache.flink.table.data.DecimalData;
-import org.apache.flink.table.data.vector.DecimalColumnVector;
+import org.apache.flink.table.data.columnar.vector.DecimalColumnVector;
import org.apache.flink.util.Preconditions;
import org.apache.arrow.vector.DecimalVector;
diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowDoubleColumnVector.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowDoubleColumnVector.java
index 0ba2c11..aa7bcc2 100644
--- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowDoubleColumnVector.java
+++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowDoubleColumnVector.java
@@ -19,7 +19,7 @@
package org.apache.flink.table.runtime.arrow.vectors;
import org.apache.flink.annotation.Internal;
-import org.apache.flink.table.data.vector.DoubleColumnVector;
+import org.apache.flink.table.data.columnar.vector.DoubleColumnVector;
import org.apache.flink.util.Preconditions;
import org.apache.arrow.vector.Float8Vector;
diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowFloatColumnVector.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowFloatColumnVector.java
index 2dfe772..e7085f6 100644
--- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowFloatColumnVector.java
+++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowFloatColumnVector.java
@@ -19,7 +19,7 @@
package org.apache.flink.table.runtime.arrow.vectors;
import org.apache.flink.annotation.Internal;
-import org.apache.flink.table.data.vector.FloatColumnVector;
+import org.apache.flink.table.data.columnar.vector.FloatColumnVector;
import org.apache.flink.util.Preconditions;
import org.apache.arrow.vector.Float4Vector;
diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowIntColumnVector.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowIntColumnVector.java
index f9c90f3..1ceceb0 100644
--- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowIntColumnVector.java
+++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowIntColumnVector.java
@@ -19,7 +19,7 @@
package org.apache.flink.table.runtime.arrow.vectors;
import org.apache.flink.annotation.Internal;
-import org.apache.flink.table.data.vector.IntColumnVector;
+import org.apache.flink.table.data.columnar.vector.IntColumnVector;
import org.apache.flink.util.Preconditions;
import org.apache.arrow.vector.IntVector;
diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowRowColumnVector.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowRowColumnVector.java
index c828899..83aeba1 100644
--- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowRowColumnVector.java
+++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowRowColumnVector.java
@@ -19,10 +19,10 @@
package org.apache.flink.table.runtime.arrow.vectors;
import org.apache.flink.annotation.Internal;
-import org.apache.flink.table.data.ColumnarRowData;
-import org.apache.flink.table.data.vector.ColumnVector;
-import org.apache.flink.table.data.vector.RowColumnVector;
-import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.flink.table.data.columnar.ColumnarRowData;
+import org.apache.flink.table.data.columnar.vector.ColumnVector;
+import org.apache.flink.table.data.columnar.vector.RowColumnVector;
+import org.apache.flink.table.data.columnar.vector.VectorizedColumnBatch;
import org.apache.flink.util.Preconditions;
import org.apache.arrow.vector.complex.StructVector;
diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowSmallIntColumnVector.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowSmallIntColumnVector.java
index 4988ad3..85113d9 100644
--- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowSmallIntColumnVector.java
+++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowSmallIntColumnVector.java
@@ -19,7 +19,7 @@
package org.apache.flink.table.runtime.arrow.vectors;
import org.apache.flink.annotation.Internal;
-import org.apache.flink.table.data.vector.ShortColumnVector;
+import org.apache.flink.table.data.columnar.vector.ShortColumnVector;
import org.apache.flink.util.Preconditions;
import org.apache.arrow.vector.SmallIntVector;
diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowTimeColumnVector.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowTimeColumnVector.java
index f8f6668..d45f35b 100644
--- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowTimeColumnVector.java
+++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowTimeColumnVector.java
@@ -19,7 +19,7 @@
package org.apache.flink.table.runtime.arrow.vectors;
import org.apache.flink.annotation.Internal;
-import org.apache.flink.table.data.vector.IntColumnVector;
+import org.apache.flink.table.data.columnar.vector.IntColumnVector;
import org.apache.flink.util.Preconditions;
import org.apache.arrow.vector.TimeMicroVector;
diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowTimestampColumnVector.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowTimestampColumnVector.java
index 56bd628..297f21d 100644
--- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowTimestampColumnVector.java
+++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowTimestampColumnVector.java
@@ -20,7 +20,7 @@ package org.apache.flink.table.runtime.arrow.vectors;
import org.apache.flink.annotation.Internal;
import org.apache.flink.table.data.TimestampData;
-import org.apache.flink.table.data.vector.TimestampColumnVector;
+import org.apache.flink.table.data.columnar.vector.TimestampColumnVector;
import org.apache.flink.util.Preconditions;
import org.apache.arrow.vector.TimeStampMicroVector;
diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowTinyIntColumnVector.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowTinyIntColumnVector.java
index c73cee3..27d125b 100644
--- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowTinyIntColumnVector.java
+++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowTinyIntColumnVector.java
@@ -19,7 +19,7 @@
package org.apache.flink.table.runtime.arrow.vectors;
import org.apache.flink.annotation.Internal;
-import org.apache.flink.table.data.vector.ByteColumnVector;
+import org.apache.flink.table.data.columnar.vector.ByteColumnVector;
import org.apache.flink.util.Preconditions;
import org.apache.arrow.vector.TinyIntVector;
diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowVarBinaryColumnVector.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowVarBinaryColumnVector.java
index 85b63a1..7d102de 100644
--- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowVarBinaryColumnVector.java
+++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowVarBinaryColumnVector.java
@@ -19,7 +19,7 @@
package org.apache.flink.table.runtime.arrow.vectors;
import org.apache.flink.annotation.Internal;
-import org.apache.flink.table.data.vector.BytesColumnVector;
+import org.apache.flink.table.data.columnar.vector.BytesColumnVector;
import org.apache.flink.util.Preconditions;
import org.apache.arrow.vector.VarBinaryVector;
diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowVarCharColumnVector.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowVarCharColumnVector.java
index 5a90d61..5de401a 100644
--- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowVarCharColumnVector.java
+++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowVarCharColumnVector.java
@@ -19,7 +19,7 @@
package org.apache.flink.table.runtime.arrow.vectors;
import org.apache.flink.annotation.Internal;
-import org.apache.flink.table.data.vector.BytesColumnVector;
+import org.apache.flink.table.data.columnar.vector.BytesColumnVector;
import org.apache.flink.util.Preconditions;
import org.apache.arrow.vector.VarCharVector;
diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java
index 68442a3..225071b 100644
--- a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java
+++ b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java
@@ -21,7 +21,7 @@ package org.apache.flink.table.runtime.arrow;
import org.apache.flink.api.java.tuple.Tuple5;
import org.apache.flink.table.data.GenericRowData;
import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.columnar.vector.ColumnVector;
import org.apache.flink.table.runtime.arrow.vectors.ArrowArrayColumnVector;
import org.apache.flink.table.runtime.arrow.vectors.ArrowBigIntColumnVector;
import org.apache.flink.table.runtime.arrow.vectors.ArrowBooleanColumnVector;
diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvAppendTableSinkFactory.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvAppendTableSinkFactory.java
index 14526a4..7580afb 100644
--- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvAppendTableSinkFactory.java
+++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvAppendTableSinkFactory.java
@@ -19,7 +19,6 @@
package org.apache.flink.table.sinks;
import org.apache.flink.annotation.Internal;
-import org.apache.flink.connector.file.sink.FileSink;
import org.apache.flink.table.factories.StreamTableSinkFactory;
import org.apache.flink.types.Row;
@@ -29,7 +28,7 @@ import java.util.Map;
/**
* Factory base for creating configured instances of {@link CsvTableSink} in a stream environment.
*
- * @deprecated The legacy CSV connector has been replaced by {@link FileSink}. It is kept only to
+ * @deprecated The legacy CSV connector has been replaced by {@code FileSink}. It is kept only to
* support tests for the legacy connector stack.
*/
@Internal
diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvBatchTableSinkFactory.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvBatchTableSinkFactory.java
index f50bc12..651f176 100644
--- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvBatchTableSinkFactory.java
+++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvBatchTableSinkFactory.java
@@ -19,7 +19,6 @@
package org.apache.flink.table.sinks;
import org.apache.flink.annotation.Internal;
-import org.apache.flink.connector.file.sink.FileSink;
import org.apache.flink.table.factories.StreamTableSinkFactory;
import org.apache.flink.types.Row;
@@ -28,7 +27,7 @@ import java.util.Map;
/**
* Factory base for creating configured instances of {@link CsvTableSink} in a batch environment.
*
- * @deprecated The legacy CSV connector has been replaced by {@link FileSink}. It is kept only to
+ * @deprecated The legacy CSV connector has been replaced by {@code FileSink}. It is kept only to
* support tests for the legacy connector stack.
*/
@Internal
diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSink.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSink.java
index 2244e20..05ff4ed 100644
--- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSink.java
+++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSink.java
@@ -21,7 +21,6 @@ package org.apache.flink.table.sinks;
import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.functions.MapFunction;
import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.connector.file.sink.FileSink;
import org.apache.flink.core.fs.FileSystem;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.DataStreamSink;
@@ -37,7 +36,7 @@ import java.util.Arrays;
/**
* A simple {@link TableSink} to emit data as CSV files.
*
- * @deprecated The legacy CSV connector has been replaced by {@link FileSink}. It is kept only to
+ * @deprecated The legacy CSV connector has been replaced by {@code FileSink}. It is kept only to
* support tests for the legacy connector stack.
*/
@Internal
diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSinkFactoryBase.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSinkFactoryBase.java
index 6f7090b..4cbbfaf 100644
--- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSinkFactoryBase.java
+++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSinkFactoryBase.java
@@ -19,7 +19,6 @@
package org.apache.flink.table.sinks;
import org.apache.flink.annotation.Internal;
-import org.apache.flink.connector.file.sink.FileSink;
import org.apache.flink.core.fs.FileSystem;
import org.apache.flink.table.api.TableException;
import org.apache.flink.table.api.TableSchema;
@@ -58,7 +57,7 @@ import static org.apache.flink.table.sources.CsvTableSourceFactoryBase.getFieldL
/**
* Factory base for creating configured instances of {@link CsvTableSink}.
*
- * @deprecated The legacy CSV connector has been replaced by {@link FileSink}. It is kept only to
+ * @deprecated The legacy CSV connector has been replaced by {@code FileSink}. It is kept only to
* support tests for the legacy connector stack.
*/
@Internal
diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sources/CsvAppendTableSourceFactory.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sources/CsvAppendTableSourceFactory.java
index abe4dfb..39e7946 100644
--- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sources/CsvAppendTableSourceFactory.java
+++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sources/CsvAppendTableSourceFactory.java
@@ -19,7 +19,6 @@
package org.apache.flink.table.sources;
import org.apache.flink.annotation.Internal;
-import org.apache.flink.connector.file.src.FileSource;
import org.apache.flink.table.factories.StreamTableSourceFactory;
import org.apache.flink.types.Row;
@@ -29,7 +28,7 @@ import java.util.Map;
/**
* Factory for creating configured instances of {@link CsvTableSource} in a stream environment.
*
- * @deprecated The legacy CSV connector has been replaced by {@link FileSource}. It is kept only to
+ * @deprecated The legacy CSV connector has been replaced by {@code FileSource}. It is kept only to
* support tests for the legacy connector stack.
*/
@Internal
diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sources/CsvBatchTableSourceFactory.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sources/CsvBatchTableSourceFactory.java
index 0597fcd..357e771 100644
--- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sources/CsvBatchTableSourceFactory.java
+++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sources/CsvBatchTableSourceFactory.java
@@ -19,7 +19,6 @@
package org.apache.flink.table.sources;
import org.apache.flink.annotation.Internal;
-import org.apache.flink.connector.file.src.FileSource;
import org.apache.flink.table.factories.StreamTableSourceFactory;
import org.apache.flink.types.Row;
@@ -28,7 +27,7 @@ import java.util.Map;
/**
* Factory for creating configured instances of {@link CsvTableSource} in a batch environment.
*
- * @deprecated The legacy CSV connector has been replaced by {@link FileSource}. It is kept only to
+ * @deprecated The legacy CSV connector has been replaced by {@code FileSource}. It is kept only to
* support tests for the legacy connector stack.
*/
@Internal
diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sources/CsvTableSource.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sources/CsvTableSource.java
index 72371a8a..1e7b30b6 100644
--- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sources/CsvTableSource.java
+++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sources/CsvTableSource.java
@@ -23,7 +23,6 @@ import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.io.CsvInputFormat;
import org.apache.flink.api.java.io.RowCsvInputFormat;
import org.apache.flink.api.java.typeutils.RowTypeInfo;
-import org.apache.flink.connector.file.src.FileSource;
import org.apache.flink.core.fs.FileInputSplit;
import org.apache.flink.core.fs.Path;
import org.apache.flink.streaming.api.datastream.DataStream;
@@ -55,7 +54,7 @@ import static org.apache.flink.table.types.utils.TypeConversions.fromDataTypeToL
/**
* A {@link StreamTableSource} for simple CSV files with a (logically) unlimited number of fields.
*
- * @deprecated The legacy CSV connector has been replaced by {@link FileSource}. It is kept only to
+ * @deprecated The legacy CSV connector has been replaced by {@code FileSource}. It is kept only to
* support tests for the legacy connector stack.
*/
@Internal
diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sources/CsvTableSourceFactoryBase.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sources/CsvTableSourceFactoryBase.java
index 7278404..948c789 100644
--- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sources/CsvTableSourceFactoryBase.java
+++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sources/CsvTableSourceFactoryBase.java
@@ -19,7 +19,6 @@
package org.apache.flink.table.sources;
import org.apache.flink.annotation.Internal;
-import org.apache.flink.connector.file.src.FileSource;
import org.apache.flink.table.api.TableException;
import org.apache.flink.table.api.TableSchema;
import org.apache.flink.table.descriptors.DescriptorProperties;
@@ -61,7 +60,7 @@ import static org.apache.flink.table.descriptors.Schema.SCHEMA;
/**
* Factory base for creating configured instances of {@link CsvTableSource}.
*
- * @deprecated The legacy CSV connector has been replaced by {@link FileSource}. It is kept only to
+ * @deprecated The legacy CSV connector has been replaced by {@code FileSource}. It is kept only to
* support tests for the legacy connector stack.
*/
@Internal
diff --git a/flink-table/flink-table-common/pom.xml b/flink-table/flink-table-common/pom.xml
index 16f7fe1..17a7c5a 100644
--- a/flink-table/flink-table-common/pom.xml
+++ b/flink-table/flink-table-common/pom.xml
@@ -44,12 +44,6 @@ under the License.
<version>${project.version}</version>
</dependency>
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-connector-files</artifactId>
- <version>${project.version}</version>
- </dependency>
-
<!-- Used for structured types extraction. -->
<dependency>
<groupId>org.apache.flink</groupId>
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/ColumnarArrayData.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/ColumnarArrayData.java
similarity index 77%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/ColumnarArrayData.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/ColumnarArrayData.java
index a5df5c4..6cfc3b9 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/ColumnarArrayData.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/ColumnarArrayData.java
@@ -1,12 +1,13 @@
/*
- * 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
+ * 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
+ * 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,
@@ -15,27 +16,36 @@
* limitations under the License.
*/
-package org.apache.flink.table.data;
+package org.apache.flink.table.data.columnar;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+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.data.binary.TypedSetters;
-import org.apache.flink.table.data.vector.ArrayColumnVector;
-import org.apache.flink.table.data.vector.BooleanColumnVector;
-import org.apache.flink.table.data.vector.ByteColumnVector;
-import org.apache.flink.table.data.vector.BytesColumnVector;
-import org.apache.flink.table.data.vector.ColumnVector;
-import org.apache.flink.table.data.vector.DecimalColumnVector;
-import org.apache.flink.table.data.vector.DoubleColumnVector;
-import org.apache.flink.table.data.vector.FloatColumnVector;
-import org.apache.flink.table.data.vector.IntColumnVector;
-import org.apache.flink.table.data.vector.LongColumnVector;
-import org.apache.flink.table.data.vector.MapColumnVector;
-import org.apache.flink.table.data.vector.RowColumnVector;
-import org.apache.flink.table.data.vector.ShortColumnVector;
-import org.apache.flink.table.data.vector.TimestampColumnVector;
+import org.apache.flink.table.data.columnar.vector.ArrayColumnVector;
+import org.apache.flink.table.data.columnar.vector.BooleanColumnVector;
+import org.apache.flink.table.data.columnar.vector.ByteColumnVector;
+import org.apache.flink.table.data.columnar.vector.BytesColumnVector;
+import org.apache.flink.table.data.columnar.vector.ColumnVector;
+import org.apache.flink.table.data.columnar.vector.DecimalColumnVector;
+import org.apache.flink.table.data.columnar.vector.DoubleColumnVector;
+import org.apache.flink.table.data.columnar.vector.FloatColumnVector;
+import org.apache.flink.table.data.columnar.vector.IntColumnVector;
+import org.apache.flink.table.data.columnar.vector.LongColumnVector;
+import org.apache.flink.table.data.columnar.vector.MapColumnVector;
+import org.apache.flink.table.data.columnar.vector.RowColumnVector;
+import org.apache.flink.table.data.columnar.vector.ShortColumnVector;
+import org.apache.flink.table.data.columnar.vector.TimestampColumnVector;
import java.util.Arrays;
/** Columnar array to support access to vector column data. */
+@Internal
public final class ColumnarArrayData implements ArrayData, TypedSetters {
private final ColumnVector data;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/ColumnarMapData.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/ColumnarMapData.java
similarity index 69%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/ColumnarMapData.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/ColumnarMapData.java
index 2e3a4b1..b509fb3 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/ColumnarMapData.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/ColumnarMapData.java
@@ -1,12 +1,13 @@
/*
- * 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
+ * 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
+ * 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,
@@ -15,11 +16,15 @@
* limitations under the License.
*/
-package org.apache.flink.table.data;
+package org.apache.flink.table.data.columnar;
-import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.columnar.vector.ColumnVector;
/** Columnar map to support access to vector column data. */
+@Internal
public final class ColumnarMapData implements MapData {
private final ColumnVector keyColumnVector;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/ColumnarRowData.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/ColumnarRowData.java
similarity index 91%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/ColumnarRowData.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/ColumnarRowData.java
index a7dd5e5..32ac4e6 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/ColumnarRowData.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/ColumnarRowData.java
@@ -16,17 +16,26 @@
* limitations under the License.
*/
-package org.apache.flink.table.data;
-
+package org.apache.flink.table.data.columnar;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+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.data.binary.TypedSetters;
-import org.apache.flink.table.data.vector.BytesColumnVector.Bytes;
-import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.flink.table.data.columnar.vector.BytesColumnVector.Bytes;
+import org.apache.flink.table.data.columnar.vector.VectorizedColumnBatch;
import org.apache.flink.types.RowKind;
/**
* Columnar row to support access to vector column data. It is a row view in {@link
* VectorizedColumnBatch}.
*/
+@Internal
public final class ColumnarRowData implements RowData, TypedSetters {
private RowKind rowKind = RowKind.INSERT;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/ArrayColumnVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/ArrayColumnVector.java
similarity index 89%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/ArrayColumnVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/ArrayColumnVector.java
index 21c15d1..1126b36 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/ArrayColumnVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/ArrayColumnVector.java
@@ -16,11 +16,13 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector;
+package org.apache.flink.table.data.columnar.vector;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.table.data.ArrayData;
/** Array column vector. */
+@Internal
public interface ArrayColumnVector extends ColumnVector {
ArrayData getArray(int i);
}
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/BooleanColumnVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/BooleanColumnVector.java
similarity index 89%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/BooleanColumnVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/BooleanColumnVector.java
index 7704e1e..78119d5 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/BooleanColumnVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/BooleanColumnVector.java
@@ -16,9 +16,12 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector;
+package org.apache.flink.table.data.columnar.vector;
+
+import org.apache.flink.annotation.Internal;
/** Boolean column vector. */
+@Internal
public interface BooleanColumnVector extends ColumnVector {
boolean getBoolean(int i);
}
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/ByteColumnVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/ByteColumnVector.java
similarity index 89%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/ByteColumnVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/ByteColumnVector.java
index a8e0499..7e6c8eb 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/ByteColumnVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/ByteColumnVector.java
@@ -16,9 +16,12 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector;
+package org.apache.flink.table.data.columnar.vector;
+
+import org.apache.flink.annotation.Internal;
/** Byte column vector. */
+@Internal
public interface ByteColumnVector extends ColumnVector {
byte getByte(int i);
}
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/BytesColumnVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/BytesColumnVector.java
similarity index 93%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/BytesColumnVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/BytesColumnVector.java
index 377078c..f553255 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/BytesColumnVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/BytesColumnVector.java
@@ -16,12 +16,15 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector;
+package org.apache.flink.table.data.columnar.vector;
+
+import org.apache.flink.annotation.Internal;
/**
* Bytes column vector to get {@link Bytes}, it include original data and offset and length. The
* data in {@link Bytes} maybe reuse.
*/
+@Internal
public interface BytesColumnVector extends ColumnVector {
Bytes getBytes(int i);
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/ColumnVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/ColumnVector.java
similarity index 89%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/ColumnVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/ColumnVector.java
index 76e0176..a635184 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/ColumnVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/ColumnVector.java
@@ -16,9 +16,12 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector;
+package org.apache.flink.table.data.columnar.vector;
+
+import org.apache.flink.annotation.Internal;
/** Nullable column vector. Access data through specific subclasses. */
+@Internal
public interface ColumnVector {
boolean isNullAt(int i);
}
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/DecimalColumnVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/DecimalColumnVector.java
similarity index 90%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/DecimalColumnVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/DecimalColumnVector.java
index f8533f3..88f8cb5 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/DecimalColumnVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/DecimalColumnVector.java
@@ -16,11 +16,13 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector;
+package org.apache.flink.table.data.columnar.vector;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.table.data.DecimalData;
/** Decimal column vector. */
+@Internal
public interface DecimalColumnVector extends ColumnVector {
DecimalData getDecimal(int i, int precision, int scale);
}
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/Dictionary.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/Dictionary.java
similarity index 91%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/Dictionary.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/Dictionary.java
index ab64c9d..6b56aca 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/Dictionary.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/Dictionary.java
@@ -15,11 +15,13 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector;
+package org.apache.flink.table.data.columnar.vector;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.table.data.TimestampData;
/** The interface for dictionary in AbstractColumnVector to decode dictionary encoded values. */
+@Internal
public interface Dictionary {
int decodeToInt(int id);
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/DoubleColumnVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/DoubleColumnVector.java
similarity index 89%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/DoubleColumnVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/DoubleColumnVector.java
index cd6e0d8..10da1f8 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/DoubleColumnVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/DoubleColumnVector.java
@@ -16,9 +16,12 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector;
+package org.apache.flink.table.data.columnar.vector;
+
+import org.apache.flink.annotation.Internal;
/** Double column vector. */
+@Internal
public interface DoubleColumnVector extends ColumnVector {
double getDouble(int i);
}
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/FloatColumnVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/FloatColumnVector.java
similarity index 89%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/FloatColumnVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/FloatColumnVector.java
index 0419355..7125940 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/FloatColumnVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/FloatColumnVector.java
@@ -16,9 +16,12 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector;
+package org.apache.flink.table.data.columnar.vector;
+
+import org.apache.flink.annotation.Internal;
/** Float column vector. */
+@Internal
public interface FloatColumnVector extends ColumnVector {
float getFloat(int i);
}
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/IntColumnVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/IntColumnVector.java
similarity index 89%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/IntColumnVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/IntColumnVector.java
index 984dcc8..ffab6a0 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/IntColumnVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/IntColumnVector.java
@@ -16,9 +16,12 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector;
+package org.apache.flink.table.data.columnar.vector;
+
+import org.apache.flink.annotation.Internal;
/** Int column vector. */
+@Internal
public interface IntColumnVector extends ColumnVector {
int getInt(int i);
}
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/LongColumnVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/LongColumnVector.java
similarity index 89%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/LongColumnVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/LongColumnVector.java
index b1bd11f..0300e01 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/LongColumnVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/LongColumnVector.java
@@ -16,9 +16,12 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector;
+package org.apache.flink.table.data.columnar.vector;
+
+import org.apache.flink.annotation.Internal;
/** Long column vector. */
+@Internal
public interface LongColumnVector extends ColumnVector {
long getLong(int i);
}
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/MapColumnVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/MapColumnVector.java
similarity index 89%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/MapColumnVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/MapColumnVector.java
index 7fed5f8..1f63dc1 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/MapColumnVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/MapColumnVector.java
@@ -16,11 +16,13 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector;
+package org.apache.flink.table.data.columnar.vector;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.table.data.MapData;
/** Map column vector. */
+@Internal
public interface MapColumnVector extends ColumnVector {
MapData getMap(int i);
}
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/RowColumnVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/RowColumnVector.java
similarity index 84%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/RowColumnVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/RowColumnVector.java
index ae2e148..8486c2f 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/RowColumnVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/RowColumnVector.java
@@ -16,11 +16,13 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector;
+package org.apache.flink.table.data.columnar.vector;
-import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.columnar.ColumnarRowData;
/** Row column vector. */
+@Internal
public interface RowColumnVector extends ColumnVector {
ColumnarRowData getRow(int i);
}
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/ShortColumnVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/ShortColumnVector.java
similarity index 89%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/ShortColumnVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/ShortColumnVector.java
index 77d5236..049304e 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/ShortColumnVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/ShortColumnVector.java
@@ -16,9 +16,12 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector;
+package org.apache.flink.table.data.columnar.vector;
+
+import org.apache.flink.annotation.Internal;
/** Short column vector. */
+@Internal
public interface ShortColumnVector extends ColumnVector {
short getShort(int i);
}
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/TimestampColumnVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/TimestampColumnVector.java
similarity index 90%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/TimestampColumnVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/TimestampColumnVector.java
index 8f4a6c4..dec0a8b 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/TimestampColumnVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/TimestampColumnVector.java
@@ -16,11 +16,13 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector;
+package org.apache.flink.table.data.columnar.vector;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.table.data.TimestampData;
/** Timestamp column vector. */
+@Internal
public interface TimestampColumnVector extends ColumnVector {
TimestampData getTimestamp(int i, int precision);
}
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/VectorizedColumnBatch.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/VectorizedColumnBatch.java
similarity index 95%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/VectorizedColumnBatch.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/VectorizedColumnBatch.java
index 7df1a99..9a1428a 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/VectorizedColumnBatch.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/VectorizedColumnBatch.java
@@ -16,14 +16,15 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector;
+package org.apache.flink.table.data.columnar.vector;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.table.data.ArrayData;
import org.apache.flink.table.data.DecimalData;
import org.apache.flink.table.data.MapData;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.data.TimestampData;
-import org.apache.flink.table.data.vector.BytesColumnVector.Bytes;
+import org.apache.flink.table.data.columnar.vector.BytesColumnVector.Bytes;
import java.io.Serializable;
import java.nio.charset.StandardCharsets;
@@ -34,6 +35,7 @@ import java.nio.charset.StandardCharsets;
*
* <p>{@code VectorizedColumnBatch}s are influenced by Apache Hive VectorizedRowBatch.
*/
+@Internal
public class VectorizedColumnBatch implements Serializable {
private static final long serialVersionUID = 8180323238728166155L;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/heap/AbstractHeapVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/heap/AbstractHeapVector.java
similarity index 94%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/heap/AbstractHeapVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/heap/AbstractHeapVector.java
index ff4a5ed..734e33e 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/heap/AbstractHeapVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/heap/AbstractHeapVector.java
@@ -16,15 +16,17 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector.heap;
+package org.apache.flink.table.data.columnar.vector.heap;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.core.memory.MemoryUtils;
-import org.apache.flink.table.data.vector.writable.AbstractWritableVector;
+import org.apache.flink.table.data.columnar.vector.writable.AbstractWritableVector;
import java.nio.ByteOrder;
import java.util.Arrays;
/** Heap vector that nullable shared structure. */
+@Internal
public abstract class AbstractHeapVector extends AbstractWritableVector {
public static final boolean LITTLE_ENDIAN = ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/heap/HeapBooleanVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/heap/HeapBooleanVector.java
similarity index 90%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/heap/HeapBooleanVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/heap/HeapBooleanVector.java
index b4f8b38..2121eb4 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/heap/HeapBooleanVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/heap/HeapBooleanVector.java
@@ -16,13 +16,15 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector.heap;
+package org.apache.flink.table.data.columnar.vector.heap;
-import org.apache.flink.table.data.vector.writable.WritableBooleanVector;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.columnar.vector.writable.WritableBooleanVector;
import java.util.Arrays;
/** This class represents a nullable heap boolean column vector. */
+@Internal
public class HeapBooleanVector extends AbstractHeapVector implements WritableBooleanVector {
private static final long serialVersionUID = 4131239076731313596L;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/heap/HeapByteVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/heap/HeapByteVector.java
similarity index 89%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/heap/HeapByteVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/heap/HeapByteVector.java
index 47d5859..a2c4380 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/heap/HeapByteVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/heap/HeapByteVector.java
@@ -16,13 +16,15 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector.heap;
+package org.apache.flink.table.data.columnar.vector.heap;
-import org.apache.flink.table.data.vector.writable.WritableByteVector;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.columnar.vector.writable.WritableByteVector;
import java.util.Arrays;
/** This class represents a nullable byte column vector. */
+@Internal
public class HeapByteVector extends AbstractHeapVector implements WritableByteVector {
private static final long serialVersionUID = 7216045902943789034L;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/heap/HeapBytesVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/heap/HeapBytesVector.java
similarity index 95%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/heap/HeapBytesVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/heap/HeapBytesVector.java
index 2640eaf..95e553d 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/heap/HeapBytesVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/heap/HeapBytesVector.java
@@ -16,9 +16,10 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector.heap;
+package org.apache.flink.table.data.columnar.vector.heap;
-import org.apache.flink.table.data.vector.writable.WritableBytesVector;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.columnar.vector.writable.WritableBytesVector;
import java.util.Arrays;
@@ -35,6 +36,7 @@ import java.util.Arrays;
* first. You can mix "by value" and "by reference" in the same column vector, though that use is
* probably not typical.
*/
+@Internal
public class HeapBytesVector extends AbstractHeapVector implements WritableBytesVector {
private static final long serialVersionUID = -8529155738773478597L;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/heap/HeapDoubleVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/heap/HeapDoubleVector.java
similarity index 93%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/heap/HeapDoubleVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/heap/HeapDoubleVector.java
index ca347f4..cf153e7 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/heap/HeapDoubleVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/heap/HeapDoubleVector.java
@@ -16,9 +16,10 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector.heap;
+package org.apache.flink.table.data.columnar.vector.heap;
-import org.apache.flink.table.data.vector.writable.WritableDoubleVector;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.columnar.vector.writable.WritableDoubleVector;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
@@ -29,6 +30,7 @@ import java.util.Arrays;
* be used for operations on all floating point double types and as such will use a 64-bit double
* value to hold the biggest possible value.
*/
+@Internal
public class HeapDoubleVector extends AbstractHeapVector implements WritableDoubleVector {
private static final long serialVersionUID = 6193940154117411328L;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/heap/HeapFloatVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/heap/HeapFloatVector.java
similarity index 93%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/heap/HeapFloatVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/heap/HeapFloatVector.java
index 7550763..e7a1c75 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/heap/HeapFloatVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/heap/HeapFloatVector.java
@@ -16,9 +16,10 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector.heap;
+package org.apache.flink.table.data.columnar.vector.heap;
-import org.apache.flink.table.data.vector.writable.WritableFloatVector;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.columnar.vector.writable.WritableFloatVector;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
@@ -28,6 +29,7 @@ import java.util.Arrays;
* This class represents a nullable double precision floating point column vector. This class will
* be used for operations on all floating point float types.
*/
+@Internal
public class HeapFloatVector extends AbstractHeapVector implements WritableFloatVector {
private static final long serialVersionUID = 8928878923550041110L;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/heap/HeapIntVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/heap/HeapIntVector.java
similarity index 94%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/heap/HeapIntVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/heap/HeapIntVector.java
index adc282d..821e9bb 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/heap/HeapIntVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/heap/HeapIntVector.java
@@ -16,13 +16,15 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector.heap;
+package org.apache.flink.table.data.columnar.vector.heap;
-import org.apache.flink.table.data.vector.writable.WritableIntVector;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.columnar.vector.writable.WritableIntVector;
import java.util.Arrays;
/** This class represents a nullable int column vector. */
+@Internal
public class HeapIntVector extends AbstractHeapVector implements WritableIntVector {
private static final long serialVersionUID = -2749499358889718254L;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/heap/HeapLongVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/heap/HeapLongVector.java
similarity index 93%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/heap/HeapLongVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/heap/HeapLongVector.java
index 6d4a5fb..efd0468 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/heap/HeapLongVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/heap/HeapLongVector.java
@@ -16,13 +16,15 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector.heap;
+package org.apache.flink.table.data.columnar.vector.heap;
-import org.apache.flink.table.data.vector.writable.WritableLongVector;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.columnar.vector.writable.WritableLongVector;
import java.util.Arrays;
/** This class represents a nullable long column vector. */
+@Internal
public class HeapLongVector extends AbstractHeapVector implements WritableLongVector {
private static final long serialVersionUID = 8534925169458006397L;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/heap/HeapShortVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/heap/HeapShortVector.java
similarity index 89%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/heap/HeapShortVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/heap/HeapShortVector.java
index 165cd13..63ed593 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/heap/HeapShortVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/heap/HeapShortVector.java
@@ -16,13 +16,15 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector.heap;
+package org.apache.flink.table.data.columnar.vector.heap;
-import org.apache.flink.table.data.vector.writable.WritableShortVector;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.columnar.vector.writable.WritableShortVector;
import java.util.Arrays;
/** This class represents a nullable short column vector. */
+@Internal
public class HeapShortVector extends AbstractHeapVector implements WritableShortVector {
private static final long serialVersionUID = -8278486456144676292L;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/heap/HeapTimestampVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/heap/HeapTimestampVector.java
similarity index 91%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/heap/HeapTimestampVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/heap/HeapTimestampVector.java
index 30d0b76..81f6800 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/heap/HeapTimestampVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/heap/HeapTimestampVector.java
@@ -16,14 +16,16 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector.heap;
+package org.apache.flink.table.data.columnar.vector.heap;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.table.data.TimestampData;
-import org.apache.flink.table.data.vector.writable.WritableTimestampVector;
+import org.apache.flink.table.data.columnar.vector.writable.WritableTimestampVector;
import java.util.Arrays;
/** This class represents a nullable byte column vector. */
+@Internal
public class HeapTimestampVector extends AbstractHeapVector implements WritableTimestampVector {
private static final long serialVersionUID = 1L;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/writable/AbstractWritableVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/writable/AbstractWritableVector.java
similarity index 87%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/writable/AbstractWritableVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/writable/AbstractWritableVector.java
index 2b191bd..c3abe36 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/writable/AbstractWritableVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/writable/AbstractWritableVector.java
@@ -16,10 +16,11 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector.writable;
+package org.apache.flink.table.data.columnar.vector.writable;
-import org.apache.flink.table.data.vector.ColumnVector;
-import org.apache.flink.table.data.vector.Dictionary;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.columnar.vector.ColumnVector;
+import org.apache.flink.table.data.columnar.vector.Dictionary;
import java.io.Serializable;
@@ -27,6 +28,7 @@ import java.io.Serializable;
* Contains the shared structure for {@link ColumnVector}s, including NULL information and
* dictionary. NOTE: if there are some nulls, must set {@link #noNulls} to false.
*/
+@Internal
public abstract class AbstractWritableVector implements WritableColumnVector, Serializable {
private static final long serialVersionUID = 1L;
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/writable/WritableBooleanVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/writable/WritableBooleanVector.java
similarity index 85%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/writable/WritableBooleanVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/writable/WritableBooleanVector.java
index 7a47029..36b14b9 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/writable/WritableBooleanVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/writable/WritableBooleanVector.java
@@ -16,11 +16,13 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector.writable;
+package org.apache.flink.table.data.columnar.vector.writable;
-import org.apache.flink.table.data.vector.BooleanColumnVector;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.columnar.vector.BooleanColumnVector;
/** Writable {@link BooleanColumnVector}. */
+@Internal
public interface WritableBooleanVector extends WritableColumnVector, BooleanColumnVector {
/** Set boolean at rowId with the provided value. */
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/writable/WritableByteVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/writable/WritableByteVector.java
similarity index 85%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/writable/WritableByteVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/writable/WritableByteVector.java
index b735f14..2c0fd0f 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/writable/WritableByteVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/writable/WritableByteVector.java
@@ -16,11 +16,13 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector.writable;
+package org.apache.flink.table.data.columnar.vector.writable;
-import org.apache.flink.table.data.vector.ByteColumnVector;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.columnar.vector.ByteColumnVector;
/** Writable {@link ByteColumnVector}. */
+@Internal
public interface WritableByteVector extends WritableColumnVector, ByteColumnVector {
/** Set byte at rowId with the provided value. */
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/writable/WritableBytesVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/writable/WritableBytesVector.java
similarity index 87%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/writable/WritableBytesVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/writable/WritableBytesVector.java
index c662150..dd2dd09 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/writable/WritableBytesVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/writable/WritableBytesVector.java
@@ -16,11 +16,13 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector.writable;
+package org.apache.flink.table.data.columnar.vector.writable;
-import org.apache.flink.table.data.vector.BytesColumnVector;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.columnar.vector.BytesColumnVector;
/** Writable {@link BytesColumnVector}. */
+@Internal
public interface WritableBytesVector extends WritableColumnVector, BytesColumnVector {
/**
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/writable/WritableColumnVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/writable/WritableColumnVector.java
similarity index 88%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/writable/WritableColumnVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/writable/WritableColumnVector.java
index f51447e..12c6a6c 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/writable/WritableColumnVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/writable/WritableColumnVector.java
@@ -16,12 +16,14 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector.writable;
+package org.apache.flink.table.data.columnar.vector.writable;
-import org.apache.flink.table.data.vector.ColumnVector;
-import org.apache.flink.table.data.vector.Dictionary;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.columnar.vector.ColumnVector;
+import org.apache.flink.table.data.columnar.vector.Dictionary;
/** Writable {@link ColumnVector}. */
+@Internal
public interface WritableColumnVector extends ColumnVector {
/** Resets the column to default state. */
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/writable/WritableDoubleVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/writable/WritableDoubleVector.java
similarity index 88%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/writable/WritableDoubleVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/writable/WritableDoubleVector.java
index e46e310..4d182c0 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/writable/WritableDoubleVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/writable/WritableDoubleVector.java
@@ -16,11 +16,13 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector.writable;
+package org.apache.flink.table.data.columnar.vector.writable;
-import org.apache.flink.table.data.vector.DoubleColumnVector;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.columnar.vector.DoubleColumnVector;
/** Writable {@link DoubleColumnVector}. */
+@Internal
public interface WritableDoubleVector extends WritableColumnVector, DoubleColumnVector {
/** Set double at rowId with the provided value. */
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/writable/WritableFloatVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/writable/WritableFloatVector.java
similarity index 88%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/writable/WritableFloatVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/writable/WritableFloatVector.java
index 7fe441a..9545bca 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/writable/WritableFloatVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/writable/WritableFloatVector.java
@@ -16,11 +16,13 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector.writable;
+package org.apache.flink.table.data.columnar.vector.writable;
-import org.apache.flink.table.data.vector.FloatColumnVector;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.columnar.vector.FloatColumnVector;
/** Writable {@link FloatColumnVector}. */
+@Internal
public interface WritableFloatVector extends WritableColumnVector, FloatColumnVector {
/** Set float at rowId with the provided value. */
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/writable/WritableIntVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/writable/WritableIntVector.java
similarity index 90%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/writable/WritableIntVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/writable/WritableIntVector.java
index c7af1b0..3ec53b1 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/writable/WritableIntVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/writable/WritableIntVector.java
@@ -16,11 +16,13 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector.writable;
+package org.apache.flink.table.data.columnar.vector.writable;
-import org.apache.flink.table.data.vector.IntColumnVector;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.columnar.vector.IntColumnVector;
/** Writable {@link IntColumnVector}. */
+@Internal
public interface WritableIntVector extends WritableColumnVector, IntColumnVector {
/** Set int at rowId with the provided value. */
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/writable/WritableLongVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/writable/WritableLongVector.java
similarity index 88%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/writable/WritableLongVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/writable/WritableLongVector.java
index 91903e6..b3a2b44 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/writable/WritableLongVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/writable/WritableLongVector.java
@@ -16,11 +16,13 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector.writable;
+package org.apache.flink.table.data.columnar.vector.writable;
-import org.apache.flink.table.data.vector.LongColumnVector;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.columnar.vector.LongColumnVector;
/** Writable {@link LongColumnVector}. */
+@Internal
public interface WritableLongVector extends WritableColumnVector, LongColumnVector {
/** Set long at rowId with the provided value. */
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/writable/WritableShortVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/writable/WritableShortVector.java
similarity index 85%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/writable/WritableShortVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/writable/WritableShortVector.java
index bfe20d2..ec72993 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/writable/WritableShortVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/writable/WritableShortVector.java
@@ -16,11 +16,13 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector.writable;
+package org.apache.flink.table.data.columnar.vector.writable;
-import org.apache.flink.table.data.vector.ShortColumnVector;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.columnar.vector.ShortColumnVector;
/** Writable {@link ShortColumnVector}. */
+@Internal
public interface WritableShortVector extends WritableColumnVector, ShortColumnVector {
/** Set short at rowId with the provided value. */
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/writable/WritableTimestampVector.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/writable/WritableTimestampVector.java
similarity index 86%
rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/writable/WritableTimestampVector.java
rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/writable/WritableTimestampVector.java
index 70a7d87..42f57c3 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/vector/writable/WritableTimestampVector.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/vector/writable/WritableTimestampVector.java
@@ -16,12 +16,14 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector.writable;
+package org.apache.flink.table.data.columnar.vector.writable;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.table.data.TimestampData;
-import org.apache.flink.table.data.vector.TimestampColumnVector;
+import org.apache.flink.table.data.columnar.vector.TimestampColumnVector;
/** Writable {@link TimestampColumnVector}. */
+@Internal
public interface WritableTimestampVector extends WritableColumnVector, TimestampColumnVector {
/** Set {@link TimestampData} at rowId with the provided value. */
diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/descriptors/FileSystemValidator.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/descriptors/FileSystemValidator.java
index b490eaa..b8eb3b3 100644
--- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/descriptors/FileSystemValidator.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/descriptors/FileSystemValidator.java
@@ -19,13 +19,11 @@
package org.apache.flink.table.descriptors;
import org.apache.flink.annotation.Internal;
-import org.apache.flink.connector.file.sink.FileSink;
-import org.apache.flink.connector.file.src.FileSource;
/**
* Validator for the options of the filesystem connector.
*
- * @deprecated The legacy CSV connector has been replaced by {@link FileSource} / {@link FileSink}.
+ * @deprecated The legacy CSV connector has been replaced by {@code FileSource} / {@code FileSink}.
* It is kept only to support tests for the legacy connector stack.
*/
@Internal
diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/data/vector/ColumnVectorTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/data/columnar/vector/ColumnVectorTest.java
similarity index 89%
rename from flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/data/vector/ColumnVectorTest.java
rename to flink-table/flink-table-common/src/test/java/org/apache/flink/table/data/columnar/vector/ColumnVectorTest.java
index af62531..4b6bad9 100644
--- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/data/vector/ColumnVectorTest.java
+++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/data/columnar/vector/ColumnVectorTest.java
@@ -7,7 +7,7 @@
* "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
+ * 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,
@@ -16,19 +16,19 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector;
+package org.apache.flink.table.data.columnar.vector;
import org.apache.flink.table.data.TimestampData;
-import org.apache.flink.table.data.vector.heap.HeapBooleanVector;
-import org.apache.flink.table.data.vector.heap.HeapByteVector;
-import org.apache.flink.table.data.vector.heap.HeapBytesVector;
-import org.apache.flink.table.data.vector.heap.HeapDoubleVector;
-import org.apache.flink.table.data.vector.heap.HeapFloatVector;
-import org.apache.flink.table.data.vector.heap.HeapIntVector;
-import org.apache.flink.table.data.vector.heap.HeapLongVector;
-import org.apache.flink.table.data.vector.heap.HeapShortVector;
-import org.apache.flink.table.data.vector.heap.HeapTimestampVector;
-import org.apache.flink.table.data.vector.writable.WritableColumnVector;
+import org.apache.flink.table.data.columnar.vector.heap.HeapBooleanVector;
+import org.apache.flink.table.data.columnar.vector.heap.HeapByteVector;
+import org.apache.flink.table.data.columnar.vector.heap.HeapBytesVector;
+import org.apache.flink.table.data.columnar.vector.heap.HeapDoubleVector;
+import org.apache.flink.table.data.columnar.vector.heap.HeapFloatVector;
+import org.apache.flink.table.data.columnar.vector.heap.HeapIntVector;
+import org.apache.flink.table.data.columnar.vector.heap.HeapLongVector;
+import org.apache.flink.table.data.columnar.vector.heap.HeapShortVector;
+import org.apache.flink.table.data.columnar.vector.heap.HeapTimestampVector;
+import org.apache.flink.table.data.columnar.vector.writable.WritableColumnVector;
import org.junit.Test;
@@ -36,12 +36,12 @@ import java.nio.charset.StandardCharsets;
import java.util.stream.IntStream;
import java.util.stream.LongStream;
-import static org.apache.flink.table.data.vector.heap.AbstractHeapVector.BYTE_ARRAY_OFFSET;
-import static org.apache.flink.table.data.vector.heap.AbstractHeapVector.DOUBLE_ARRAY_OFFSET;
-import static org.apache.flink.table.data.vector.heap.AbstractHeapVector.FLOAT_ARRAY_OFFSET;
-import static org.apache.flink.table.data.vector.heap.AbstractHeapVector.INT_ARRAY_OFFSET;
-import static org.apache.flink.table.data.vector.heap.AbstractHeapVector.LONG_ARRAY_OFFSET;
-import static org.apache.flink.table.data.vector.heap.AbstractHeapVector.UNSAFE;
+import static org.apache.flink.table.data.columnar.vector.heap.AbstractHeapVector.BYTE_ARRAY_OFFSET;
+import static org.apache.flink.table.data.columnar.vector.heap.AbstractHeapVector.DOUBLE_ARRAY_OFFSET;
+import static org.apache.flink.table.data.columnar.vector.heap.AbstractHeapVector.FLOAT_ARRAY_OFFSET;
+import static org.apache.flink.table.data.columnar.vector.heap.AbstractHeapVector.INT_ARRAY_OFFSET;
+import static org.apache.flink.table.data.columnar.vector.heap.AbstractHeapVector.LONG_ARRAY_OFFSET;
+import static org.apache.flink.table.data.columnar.vector.heap.AbstractHeapVector.UNSAFE;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/data/vector/VectorizedColumnBatchTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/data/columnar/vector/VectorizedColumnBatchTest.java
similarity index 93%
rename from flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/data/vector/VectorizedColumnBatchTest.java
rename to flink-table/flink-table-common/src/test/java/org/apache/flink/table/data/columnar/vector/VectorizedColumnBatchTest.java
index 2f3fa99..6be0585 100644
--- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/data/vector/VectorizedColumnBatchTest.java
+++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/data/columnar/vector/VectorizedColumnBatchTest.java
@@ -16,21 +16,21 @@
* limitations under the License.
*/
-package org.apache.flink.table.data.vector;
+package org.apache.flink.table.data.columnar.vector;
import org.apache.flink.table.data.ArrayData;
-import org.apache.flink.table.data.ColumnarArrayData;
-import org.apache.flink.table.data.ColumnarRowData;
import org.apache.flink.table.data.DecimalData;
import org.apache.flink.table.data.TimestampData;
-import org.apache.flink.table.data.vector.heap.HeapBooleanVector;
-import org.apache.flink.table.data.vector.heap.HeapByteVector;
-import org.apache.flink.table.data.vector.heap.HeapBytesVector;
-import org.apache.flink.table.data.vector.heap.HeapDoubleVector;
-import org.apache.flink.table.data.vector.heap.HeapFloatVector;
-import org.apache.flink.table.data.vector.heap.HeapIntVector;
-import org.apache.flink.table.data.vector.heap.HeapLongVector;
-import org.apache.flink.table.data.vector.heap.HeapShortVector;
+import org.apache.flink.table.data.columnar.ColumnarArrayData;
+import org.apache.flink.table.data.columnar.ColumnarRowData;
+import org.apache.flink.table.data.columnar.vector.heap.HeapBooleanVector;
+import org.apache.flink.table.data.columnar.vector.heap.HeapByteVector;
+import org.apache.flink.table.data.columnar.vector.heap.HeapBytesVector;
+import org.apache.flink.table.data.columnar.vector.heap.HeapDoubleVector;
+import org.apache.flink.table.data.columnar.vector.heap.HeapFloatVector;
+import org.apache.flink.table.data.columnar.vector.heap.HeapIntVector;
+import org.apache.flink.table.data.columnar.vector.heap.HeapLongVector;
+import org.apache.flink.table.data.columnar.vector.heap.HeapShortVector;
import org.apache.flink.table.utils.DateTimeUtils;
import org.junit.Test;
diff --git a/flink-table/flink-table-planner/pom.xml b/flink-table/flink-table-planner/pom.xml
index 290b8db..26912eb 100644
--- a/flink-table/flink-table-planner/pom.xml
+++ b/flink-table/flink-table-planner/pom.xml
@@ -290,6 +290,14 @@ under the License.
<scope>test</scope>
</dependency>
+ <!-- For using the filesystem connector in tests -->
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-connector-files</artifactId>
+ <version>${project.version}</version>
+ <scope>test</scope>
+ </dependency>
+
<!-- utility to scan classpaths -->
<dependency>
<groupId>org.reflections</groupId>
diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/FileSystemTableFactoryTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/file/table/FileSystemTableFactoryTest.java
similarity index 98%
rename from flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/FileSystemTableFactoryTest.java
rename to flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/file/table/FileSystemTableFactoryTest.java
index 1917160..51612e8 100644
--- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/filesystem/FileSystemTableFactoryTest.java
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/file/table/FileSystemTableFactoryTest.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.api.ValidationException;
@@ -33,10 +33,10 @@ import org.junit.Test;
import java.util.HashMap;
import java.util.Map;
+import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_WATERMARK_TIME_ZONE;
import static org.apache.flink.core.testutils.FlinkMatchers.containsCause;
import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink;
import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource;
-import static org.apache.flink.table.filesystem.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_WATERMARK_TIME_ZONE;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/filesystem/FileSystemTableSinkTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/file/table/FileSystemTableSinkTest.java
similarity index 99%
rename from flink-table/flink-table-planner/src/test/java/org/apache/flink/table/filesystem/FileSystemTableSinkTest.java
rename to flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/file/table/FileSystemTableSinkTest.java
index abbd9bc..864bdfc 100644
--- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/filesystem/FileSystemTableSinkTest.java
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/file/table/FileSystemTableSinkTest.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.table.api.EnvironmentSettings;
import org.apache.flink.table.api.ExplainDetail;
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/filesystem/FileSystemTableSourceTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/file/table/FileSystemTableSourceTest.java
similarity index 98%
rename from flink-table/flink-table-planner/src/test/java/org/apache/flink/table/filesystem/FileSystemTableSourceTest.java
rename to flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/file/table/FileSystemTableSourceTest.java
index a90c30b..3b3b71a 100644
--- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/filesystem/FileSystemTableSourceTest.java
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/file/table/FileSystemTableSourceTest.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.table.filesystem;
+package org.apache.flink.connector.file.table;
import org.apache.flink.table.api.TableConfig;
import org.apache.flink.table.api.TableEnvironment;
diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/formats/testcsv/TestCsvDeserializationSchema.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/formats/testcsv/TestCsvDeserializationSchema.java
similarity index 100%
rename from flink-table/flink-table-runtime/src/test/java/org/apache/flink/formats/testcsv/TestCsvDeserializationSchema.java
rename to flink-table/flink-table-planner/src/test/java/org/apache/flink/formats/testcsv/TestCsvDeserializationSchema.java
diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/formats/testcsv/TestCsvFormatFactory.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/formats/testcsv/TestCsvFormatFactory.java
similarity index 100%
rename from flink-table/flink-table-runtime/src/test/java/org/apache/flink/formats/testcsv/TestCsvFormatFactory.java
rename to flink-table/flink-table-planner/src/test/java/org/apache/flink/formats/testcsv/TestCsvFormatFactory.java
diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/formats/testcsv/TestCsvSerializationSchema.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/formats/testcsv/TestCsvSerializationSchema.java
similarity index 100%
rename from flink-table/flink-table-runtime/src/test/java/org/apache/flink/formats/testcsv/TestCsvSerializationSchema.java
rename to flink-table/flink-table-planner/src/test/java/org/apache/flink/formats/testcsv/TestCsvSerializationSchema.java
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestFileFactory.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestFileFactory.java
index 44e7bc2..8949160 100644
--- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestFileFactory.java
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestFileFactory.java
@@ -27,6 +27,7 @@ import org.apache.flink.configuration.Configuration;
import org.apache.flink.connector.file.sink.FileSink;
import org.apache.flink.connector.file.src.FileSource;
import org.apache.flink.connector.file.src.reader.SimpleStreamFormat;
+import org.apache.flink.connector.file.table.FileSystemConnectorOptions;
import org.apache.flink.core.fs.FSDataInputStream;
import org.apache.flink.core.fs.Path;
import org.apache.flink.streaming.api.datastream.DataStream;
@@ -43,7 +44,6 @@ import org.apache.flink.table.data.RowData;
import org.apache.flink.table.data.StringData;
import org.apache.flink.table.factories.DynamicTableSinkFactory;
import org.apache.flink.table.factories.DynamicTableSourceFactory;
-import org.apache.flink.table.filesystem.FileSystemConnectorOptions;
import java.io.BufferedReader;
import java.io.IOException;
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/CompactionITCaseBase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/CompactionITCaseBase.java
index 4ef25d3..22baf80 100644
--- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/CompactionITCaseBase.java
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/CompactionITCaseBase.java
@@ -42,7 +42,7 @@ import java.util.ArrayList;
import java.util.Comparator;
import java.util.List;
-import static org.apache.flink.table.filesystem.stream.compact.CompactOperator.COMPACTED_PREFIX;
+import static org.apache.flink.connector.file.table.stream.compact.CompactOperator.COMPACTED_PREFIX;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
diff --git a/flink-table/flink-table-planner/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-table/flink-table-planner/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory
index 7f96306..12deb1d 100644
--- a/flink-table/flink-table-planner/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory
+++ b/flink-table/flink-table-planner/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory
@@ -13,6 +13,7 @@
# See the License for the specific language governing permissions and
# limitations under the License.
+org.apache.flink.formats.testcsv.TestCsvFormatFactory
org.apache.flink.table.planner.factories.TestValuesTableFactory
org.apache.flink.table.planner.factories.TestFileFactory
org.apache.flink.table.planner.factories.TableFactoryHarness$Factory
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/filesystem/FileSystemTableSourceTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/connector/file/table/FileSystemTableSourceTest.xml
similarity index 100%
rename from flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/filesystem/FileSystemTableSourceTest.xml
rename to flink-table/flink-table-planner/src/test/resources/org/apache/flink/connector/file/table/FileSystemTableSourceTest.xml
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala
index 136b339..bdba353 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala
@@ -22,6 +22,7 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo.{INT_TYPE_INFO, STRING
import org.apache.flink.api.common.typeinfo.TypeInformation
import org.apache.flink.api.java.typeutils.RowTypeInfo
import org.apache.flink.configuration.Configuration
+import org.apache.flink.connector.file.table.FileSystemConnectorOptions
import org.apache.flink.streaming.api.datastream.{DataStream, DataStreamSink}
import org.apache.flink.streaming.api.functions.sink.RichSinkFunction
import org.apache.flink.table.api.config.ExecutionConfigOptions
@@ -31,7 +32,6 @@ import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR
import org.apache.flink.table.descriptors.DescriptorProperties
import org.apache.flink.table.descriptors.Schema.SCHEMA
import org.apache.flink.table.factories.TableSinkFactory
-import org.apache.flink.table.filesystem.FileSystemConnectorOptions
import org.apache.flink.table.planner.runtime.batch.sql.PartitionableSinkITCase.{type4, type_int_string, _}
import org.apache.flink.table.planner.runtime.utils.BatchTestBase
import org.apache.flink.table.planner.runtime.utils.BatchTestBase.row
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/FsStreamingSinkITCaseBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/FsStreamingSinkITCaseBase.scala
index d7806ce..6ab1c62 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/FsStreamingSinkITCaseBase.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/FsStreamingSinkITCaseBase.scala
@@ -24,8 +24,8 @@ import org.apache.flink.streaming.api.functions.source.SourceFunction
import org.apache.flink.streaming.api.scala.DataStream
import org.apache.flink.streaming.api.watermark.Watermark
import org.apache.flink.table.api.Expressions.$
-import org.apache.flink.table.filesystem.DefaultPartTimeExtractor.{toLocalDateTime, toMills}
-import org.apache.flink.table.filesystem.FileSystemConnectorOptions._
+import org.apache.flink.connector.file.table.DefaultPartTimeExtractor.{toLocalDateTime, toMills}
+import org.apache.flink.connector.file.table.FileSystemConnectorOptions._
import org.apache.flink.table.planner.runtime.utils.{StreamingTestBase, TestSinkUtil}
import org.apache.flink.types.Row
import org.apache.flink.util.CollectionUtil
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/typeutils/ArrayDataSerializer.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/typeutils/ArrayDataSerializer.java
index f36c56a..078f01a 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/typeutils/ArrayDataSerializer.java
+++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/typeutils/ArrayDataSerializer.java
@@ -29,10 +29,10 @@ 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.ArrayData;
-import org.apache.flink.table.data.ColumnarArrayData;
import org.apache.flink.table.data.GenericArrayData;
import org.apache.flink.table.data.binary.BinaryArrayData;
import org.apache.flink.table.data.binary.BinarySegmentUtils;
+import org.apache.flink.table.data.columnar.ColumnarArrayData;
import org.apache.flink.table.data.writer.BinaryArrayWriter;
import org.apache.flink.table.data.writer.BinaryWriter;
import org.apache.flink.table.types.logical.LogicalType;
diff --git a/flink-table/flink-table-runtime/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-table/flink-table-runtime/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
index b59b493..9f8c122 100644
--- a/flink-table/flink-table-runtime/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
+++ b/flink-table/flink-table-runtime/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
@@ -13,5 +13,4 @@
# See the License for the specific language governing permissions and
# limitations under the License.
-org.apache.flink.table.filesystem.FileSystemTableFactory
org.apache.flink.formats.raw.RawFormatFactory
diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/typeutils/ArrayDataSerializerTest.java b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/typeutils/ArrayDataSerializerTest.java
index b2a2786..558bbbb 100644
--- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/typeutils/ArrayDataSerializerTest.java
+++ b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/typeutils/ArrayDataSerializerTest.java
@@ -21,11 +21,11 @@ package org.apache.flink.table.runtime.typeutils;
import org.apache.flink.api.common.typeutils.SerializerTestBase;
import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.data.ArrayData;
-import org.apache.flink.table.data.ColumnarArrayData;
import org.apache.flink.table.data.GenericArrayData;
import org.apache.flink.table.data.StringData;
import org.apache.flink.table.data.binary.BinaryArrayData;
-import org.apache.flink.table.data.vector.heap.HeapBytesVector;
+import org.apache.flink.table.data.columnar.ColumnarArrayData;
+import org.apache.flink.table.data.columnar.vector.heap.HeapBytesVector;
import org.apache.flink.table.data.writer.BinaryArrayWriter;
import org.apache.flink.testutils.DeeplyEqualsChecker;
diff --git a/flink-table/flink-table-uber/pom.xml b/flink-table/flink-table-uber/pom.xml
index edc94c8f..9b985df 100644
--- a/flink-table/flink-table-uber/pom.xml
+++ b/flink-table/flink-table-uber/pom.xml
@@ -115,9 +115,6 @@ under the License.
<include>org.apache.flink:flink-table-planner_${scala.binary.version}</include>
<include>org.apache.flink:flink-table-runtime_${scala.binary.version}</include>
<include>org.apache.flink:flink-cep</include>
- <include>org.apache.flink:flink-connector-files</include>
- <include>org.apache.flink:flink-connector-base</include>
- <include>org.apache.flink:flink-file-sink-common</include>
</includes>
</artifactSet>
<relocations>