From a36fb68416d51d33e7b7ebb5c41e89d09f4df1ed Mon Sep 17 00:00:00 2001 From: sunxiaojian Date: Mon, 8 Jan 2024 14:00:48 +0800 Subject: [PATCH] Supports iceberg sink #6198 --- docs/en/connector-v2/sink/Iceberg.md | 182 +++++++ docs/en/connector-v2/source/Iceberg.md | 40 +- plugin-mapping.properties | 1 + pom.xml | 3 +- release-note.md | 1 + .../connector-cdc/connector-cdc-base/pom.xml | 14 + .../connector-iceberg/pom.xml | 72 ++- .../iceberg/IcebergCatalogFactory.java | 88 ---- .../iceberg/IcebergCatalogLoader.java | 124 +++++ .../seatunnel/iceberg/IcebergTableLoader.java | 66 ++- .../iceberg/catalog/IcebergCatalog.java | 263 ++++++++++ .../catalog/IcebergCatalogFactory.java | 54 ++ .../iceberg/config/CommonConfig.java | 87 ++-- .../seatunnel/iceberg/config/SinkConfig.java | 145 ++++++ .../iceberg/config/SourceConfig.java | 8 +- .../iceberg/data/IcebergTypeMapper.java | 57 +++ .../seatunnel/iceberg/data/RowConverter.java | 482 ++++++++++++++++++ .../seatunnel/iceberg/sink/IcebergSink.java | 214 ++++++++ .../iceberg/sink/IcebergSinkFactory.java | 112 ++++ .../iceberg/sink/IcebergSinkWriter.java | 160 ++++++ .../commit/IcebergAggregatedCommitInfo.java | 30 ++ .../commit/IcebergAggregatedCommitter.java | 71 +++ .../sink/commit/IcebergCommitInfo.java | 32 ++ .../sink/commit/IcebergFilesCommitter.java | 91 ++++ .../iceberg/sink/schema/ISchemaChange.java | 20 + .../iceberg/sink/schema/SchemaAddColumn.java | 48 ++ .../sink/schema/SchemaChangeColumn.java | 37 ++ .../sink/schema/SchemaChangeWrapper.java | 72 +++ .../sink/schema/SchemaDeleteColumn.java | 30 ++ .../sink/schema/SchemaModifyColumn.java | 39 ++ .../iceberg/sink/state/IcebergSinkState.java | 37 ++ .../sink/writer/BaseDeltaTaskWriter.java | 126 +++++ .../iceberg/sink/writer/IcebergRecord.java | 87 ++++ .../sink/writer/IcebergRecordWriter.java | 189 +++++++ .../sink/writer/IcebergWriterFactory.java | 197 +++++++ .../sink/writer/PartitionedAppendWriter.java | 56 ++ .../sink/writer/PartitionedDeltaWriter.java | 95 ++++ .../iceberg/sink/writer/RecordProjection.java | 196 +++++++ .../iceberg/sink/writer/RecordWriter.java | 40 ++ .../sink/writer/UnpartitionedDeltaWriter.java | 68 +++ .../iceberg/sink/writer/WriteResult.java | 41 ++ .../iceberg/source/IcebergSource.java | 55 +- .../iceberg/source/IcebergSourceFactory.java | 56 +- .../enumerator/AbstractSplitEnumerator.java | 8 +- .../IcebergBatchSplitEnumerator.java | 7 +- .../IcebergStreamSplitEnumerator.java | 7 +- .../reader/IcebergFileScanTaskReader.java | 6 +- .../source/reader/IcebergSourceReader.java | 9 +- .../seatunnel/iceberg/utils/SchemaUtils.java | 324 ++++++++++++ .../iceberg/TestIcebergMetastore.java | 20 +- .../iceberg/catalog/IcebergCatalogTest.java | 172 +++++++ .../connector-iceberg-e2e/pom.xml | 54 ++ .../connector/iceberg/IcebergSinkCDCIT.java | 459 +++++++++++++++++ .../e2e/connector/iceberg/IcebergSinkIT.java | 169 ++++++ .../connector/iceberg/IcebergSourceIT.java | 21 +- .../src/test/resources/ddl/inventory.sql | 95 ++++ .../src/test/resources/ddl/mysql_cdc.sql | 186 +++++++ .../resources/iceberg/fake_to_iceberg.conf | 74 +++ .../resources/iceberg/iceberg_source.conf | 6 +- .../iceberg/mysql_cdc_to_iceberg.conf | 64 +++ ...ysql_cdc_to_iceberg_for_schema_change.conf | 68 +++ .../test/resources/mysql/server-gtids/my.cnf | 65 +++ .../src/test/resources/mysql/setup.sql | 27 + .../iceberg/hadoop3/IcebergSourceIT.java | 21 +- .../resources/iceberg/iceberg_source.conf | 6 +- .../seatunnel-hadoop3-3.1.4-uber/pom.xml | 1 - 66 files changed, 5524 insertions(+), 231 deletions(-) create mode 100644 docs/en/connector-v2/sink/Iceberg.md delete mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/IcebergCatalogFactory.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/IcebergCatalogLoader.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/catalog/IcebergCatalog.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/catalog/IcebergCatalogFactory.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/config/SinkConfig.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/RowConverter.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSink.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSinkFactory.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSinkWriter.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/commit/IcebergAggregatedCommitInfo.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/commit/IcebergAggregatedCommitter.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/commit/IcebergCommitInfo.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/commit/IcebergFilesCommitter.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/schema/ISchemaChange.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/schema/SchemaAddColumn.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/schema/SchemaChangeColumn.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/schema/SchemaChangeWrapper.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/schema/SchemaDeleteColumn.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/schema/SchemaModifyColumn.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/state/IcebergSinkState.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/BaseDeltaTaskWriter.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergRecord.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergRecordWriter.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergWriterFactory.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/PartitionedAppendWriter.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/PartitionedDeltaWriter.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/RecordProjection.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/RecordWriter.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/UnpartitionedDeltaWriter.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/WriteResult.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/utils/SchemaUtils.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/test/java/org/apache/seatunnel/connectors/seatunnel/iceberg/catalog/IcebergCatalogTest.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/java/org/apache/seatunnel/e2e/connector/iceberg/IcebergSinkCDCIT.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/java/org/apache/seatunnel/e2e/connector/iceberg/IcebergSinkIT.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/ddl/inventory.sql create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/ddl/mysql_cdc.sql create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/iceberg/fake_to_iceberg.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/iceberg/mysql_cdc_to_iceberg.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/iceberg/mysql_cdc_to_iceberg_for_schema_change.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/mysql/server-gtids/my.cnf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/mysql/setup.sql diff --git a/docs/en/connector-v2/sink/Iceberg.md b/docs/en/connector-v2/sink/Iceberg.md new file mode 100644 index 000000000000..dc73f491bcd5 --- /dev/null +++ b/docs/en/connector-v2/sink/Iceberg.md @@ -0,0 +1,182 @@ +# Apache Iceberg + +> Apache Iceberg sink connector + +## Support Iceberg Version + +- 1.4.2 + +## Support Those Engines + +> Spark
+> Flink
+> SeaTunnel Zeta
+ +## Description + +Sink connector for Apache Iceberg. It can support cdc mode 、auto create table and table schema evolution. + +## Supported DataSource Info + +| Datasource | Dependent | Maven | +|------------|-----------|---------------------------------------------------------------------------| +| Iceberg | hive-exec | [Download](https://mvnrepository.com/artifact/org.apache.hive/hive-exec) | +| Iceberg | libfb303 | [Download](https://mvnrepository.com/artifact/org.apache.thrift/libfb303) | + +## Database Dependency + +> In order to be compatible with different versions of Hadoop and Hive, the scope of hive-exec in the project pom file are provided, so if you use the Flink engine, first you may need to add the following Jar packages to /lib directory, if you are using the Spark engine and integrated with Hadoop, then you do not need to add the following Jar packages. + +``` +hive-exec-xxx.jar +libfb303-xxx.jar +``` + +> Some versions of the hive-exec package do not have libfb303-xxx.jar, so you also need to manually import the Jar package. + +## Data Type Mapping + +| SeaTunnel Data type | Iceberg Data type | +|---------------------|-------------------| +| BOOLEAN | BOOLEAN | +| INT | INTEGER | +| BIGINT | LONG | +| FLOAT | FLOAT | +| DOUBLE | DOUBLE | +| DATE | DATE | +| TIME | TIME | +| TIMESTAMP | TIMESTAMP | +| STRING | STRING | +| BYTES | FIXED
BINARY | +| DECIMAL | DECIMAL | +| ROW | STRUCT | +| ARRAY | LIST | +| MAP | MAP | + +## Sink Options + +| Name | Type | Required | Default | Description | +|----------------------------------------|---------|----------|------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| catalog_name | string | yes | default | User-specified catalog name. default is `default` | +| namespace | string | yes | default | The iceberg database name in the backend catalog. default is `default` | +| table | string | yes | - | The iceberg table name in the backend catalog. | +| iceberg.catalog.config | map | yes | - | Specify the properties for initializing the Iceberg catalog, which can be referenced in this file:"https://github.com/apache/iceberg/blob/main/core/src/main/java/org/apache/iceberg/CatalogProperties.java" | +| hadoop.config | map | no | - | Properties passed through to the Hadoop configuration | +| iceberg.hadoop-conf-path | string | no | - | The specified loading paths for the 'core-site.xml', 'hdfs-site.xml', 'hive-site.xml' files. | +| case_sensitive | boolean | no | false | If data columns where selected via schema [config], controls whether the match to the schema will be done with case sensitivity. | +| iceberg.table.write-props | map | no | - | Properties passed through to Iceberg writer initialization, these take precedence, such as 'write.format.default', 'write.target-file-size-bytes', and other settings, can be found with specific parameters at 'https://github.com/apache/iceberg/blob/main/core/src/main/java/org/apache/iceberg/TableProperties.java'. | +| iceberg.table.auto-create-props | map | no | - | Configuration specified by Iceberg during automatic table creation. | +| iceberg.table.schema-evolution-enabled | boolean | no | false | Setting to true enables Iceberg tables to support schema evolution during the synchronization process | +| iceberg.table.primary-keys | string | no | - | Default comma-separated list of columns that identify a row in tables (primary key) | +| iceberg.table.partition-keys | string | no | - | Default comma-separated list of partition fields to use when creating tables | +| iceberg.table.upsert-mode-enabled | boolean | no | false | Set to `true` to enable upsert mode, default is `false` | +| schema_save_mode | Enum | no | CREATE_SCHEMA_WHEN_NOT_EXIST | the schema save mode, please refer to `schema_save_mode` below | +| data_save_mode | Enum | no | APPEND_DATA | the data save mode, please refer to `data_save_mode` below | + +## Task Example + +### Simple: + +```hocon +env { + parallelism = 1 + job.mode = "STREAMING" + checkpoint.interval = 5000 +} + +source { + MySQL-CDC { + result_table_name = "customers_mysql_cdc_iceberg" + server-id = 5652 + username = "st_user" + password = "seatunnel" + table-names = ["mysql_cdc.mysql_cdc_e2e_source_table"] + base-url = "jdbc:mysql://mysql_cdc_e2e:3306/mysql_cdc" + } +} + +transform { +} + +sink { + Iceberg { + catalog_name="seatunnel_test" + iceberg.catalog.config={ + "type"="hadoop" + "warehouse"="file:///tmp/seatunnel/iceberg/hadoop-sink/" + } + namespace="seatunnel_namespace" + table="iceberg_sink_table" + iceberg.table.write-props={ + write.format.default="parquet" + write.target-file-size-bytes=536870912 + } + iceberg.table.primary-keys="id" + iceberg.table.partition-keys="f_datetime" + iceberg.table.upsert-mode-enabled=true + iceberg.table.schema-evolution-enabled=true + case_sensitive=true + } +} +``` + +### Hive Catalog: + +```hocon +sink { + Iceberg { + catalog_name="seatunnel_test" + iceberg.catalog.config={ + type = "hive" + uri = "thrift://localhost:9083" + warehouse = "hdfs://your_cluster//tmp/seatunnel/iceberg/" + } + namespace="seatunnel_namespace" + table="iceberg_sink_table" + iceberg.table.write-props={ + write.format.default="parquet" + write.target-file-size-bytes=536870912 + } + iceberg.table.primary-keys="id" + iceberg.table.partition-keys="f_datetime" + iceberg.table.upsert-mode-enabled=true + iceberg.table.schema-evolution-enabled=true + case_sensitive=true + } +} +``` + +### Hadoop catalog: + +```hocon +sink { + Iceberg { + catalog_name="seatunnel_test" + iceberg.catalog.config={ + type = "hadoop" + warehouse = "hdfs://your_cluster/tmp/seatunnel/iceberg/" + } + namespace="seatunnel_namespace" + table="iceberg_sink_table" + iceberg.table.write-props={ + write.format.default="parquet" + write.target-file-size-bytes=536870912 + } + iceberg.table.primary-keys="id" + iceberg.table.partition-keys="f_datetime" + iceberg.table.upsert-mode-enabled=true + iceberg.table.schema-evolution-enabled=true + case_sensitive=true + } +} + +``` + +## Changelog + +### 2.3.4-SNAPSHOT 2024-01-18 + +- Add Iceberg Sink Connector + +### next version + diff --git a/docs/en/connector-v2/source/Iceberg.md b/docs/en/connector-v2/source/Iceberg.md index a01833f066bb..8fb296467a88 100644 --- a/docs/en/connector-v2/source/Iceberg.md +++ b/docs/en/connector-v2/source/Iceberg.md @@ -4,7 +4,7 @@ ## Support Iceberg Version -- 0.14.0 +- 1.4.2 ## Support Those Engines @@ -34,18 +34,16 @@ Source connector for Apache Iceberg. It can support batch and stream mode. ## Supported DataSource Info -| Datasource | Dependent | Maven | -|------------|---------------------|---------------------------------------------------------------------------| -| Iceberg | flink-shaded-hadoop | [Download](https://mvnrepository.com/search?q=flink-shaded-hadoop-) | -| Iceberg | hive-exec | [Download](https://mvnrepository.com/artifact/org.apache.hive/hive-exec) | -| Iceberg | libfb303 | [Download](https://mvnrepository.com/artifact/org.apache.thrift/libfb303) | +| Datasource | Dependent | Maven | +|------------|-----------|---------------------------------------------------------------------------| +| Iceberg | hive-exec | [Download](https://mvnrepository.com/artifact/org.apache.hive/hive-exec) | +| Iceberg | libfb303 | [Download](https://mvnrepository.com/artifact/org.apache.thrift/libfb303) | ## Database Dependency -> In order to be compatible with different versions of Hadoop and Hive, the scope of hive-exec and flink-shaded-hadoop-2 in the project pom file are provided, so if you use the Flink engine, first you may need to add the following Jar packages to /lib directory, if you are using the Spark engine and integrated with Hadoop, then you do not need to add the following Jar packages. +> In order to be compatible with different versions of Hadoop and Hive, the scope of hive-exec in the project pom file are provided, so if you use the Flink engine, first you may need to add the following Jar packages to /lib directory, if you are using the Spark engine and integrated with Hadoop, then you do not need to add the following Jar packages. ``` -flink-shaded-hadoop-x-xxx.jar hive-exec-xxx.jar libfb303-xxx.jar ``` @@ -76,11 +74,11 @@ libfb303-xxx.jar | Name | Type | Required | Default | Description | |--------------------------|---------|----------|----------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| | catalog_name | string | yes | - | User-specified catalog name. | -| catalog_type | string | yes | - | The optional values are: hive(The hive metastore catalog),hadoop(The hadoop catalog) | -| uri | string | no | - | The Hive metastore’s thrift URI. | -| warehouse | string | yes | - | The location to store metadata files and data files. | | namespace | string | yes | - | The iceberg database name in the backend catalog. | | table | string | yes | - | The iceberg table name in the backend catalog. | +| iceberg.catalog.config | map | yes | - | Specify the properties for initializing the Iceberg catalog, which can be referenced in this file:"https://github.com/apache/iceberg/blob/main/core/src/main/java/org/apache/iceberg/CatalogProperties.java" | +| hadoop.config | map | no | - | Properties passed through to the Hadoop configuration | +| iceberg.hadoop-conf-path | string | no | - | The specified loading paths for the 'core-site.xml', 'hdfs-site.xml', 'hive-site.xml' files. | | schema | config | no | - | Use projection to select data columns and columns order. | | case_sensitive | boolean | no | false | If data columns where selected via schema [config], controls whether the match to the schema will be done with case sensitivity. | | start_snapshot_timestamp | long | no | - | Instructs this scan to look for changes starting from the most recent snapshot for the table as of the timestamp.
timestamp – the timestamp in millis since the Unix epoch | @@ -123,8 +121,10 @@ source { } } catalog_name = "seatunnel" - catalog_type = "hadoop" - warehouse = "file:///tmp/seatunnel/iceberg/hadoop/" + iceberg.catalog.config={ + type = "hadoop" + warehouse = "file:///tmp/seatunnel/iceberg/hadoop/" + } namespace = "database1" table = "source" result_table_name = "iceberg" @@ -147,9 +147,13 @@ sink { source { Iceberg { catalog_name = "seatunnel" + iceberg.catalog.config={ + type = "hive" + uri = "thrift://localhost:9083" + warehouse = "hdfs://your_cluster//tmp/seatunnel/iceberg/" + } catalog_type = "hive" - uri = "thrift://localhost:9083" - warehouse = "hdfs://your_cluster//tmp/seatunnel/iceberg/" + namespace = "your_iceberg_database" table = "your_iceberg_table" } @@ -162,8 +166,10 @@ source { source { Iceberg { catalog_name = "seatunnel" - catalog_type = "hadoop" - warehouse = "hdfs://your_cluster/tmp/seatunnel/iceberg/" + iceberg.catalog.config={ + type = "hadoop" + warehouse = "hdfs://your_cluster/tmp/seatunnel/iceberg/" + } namespace = "your_iceberg_database" table = "your_iceberg_table" diff --git a/plugin-mapping.properties b/plugin-mapping.properties index c5ccae20a222..9e3769550556 100644 --- a/plugin-mapping.properties +++ b/plugin-mapping.properties @@ -72,6 +72,7 @@ seatunnel.sink.Sentry = connector-sentry seatunnel.source.MongoDB = connector-mongodb seatunnel.sink.MongoDB = connector-mongodb seatunnel.source.Iceberg = connector-iceberg +seatunnel.sink.Iceberg = connector-iceberg seatunnel.source.InfluxDB = connector-influxdb seatunnel.source.S3File = connector-file-s3 seatunnel.sink.S3File = connector-file-s3 diff --git a/pom.xml b/pom.xml index 274fe05c7eba..3c9865273fb1 100644 --- a/pom.xml +++ b/pom.xml @@ -70,7 +70,6 @@ 1.2.17 1.2.3 1.2 - 0.13.1 1.13.6 1.15.3 2.4.0 @@ -87,6 +86,7 @@ 2.13.3 1.18.24 1.20 + 1.11.1 false false false @@ -105,7 +105,6 @@ 2.11.0 4.4 3.3.0 - 1.20 1.8.0 provided provided diff --git a/release-note.md b/release-note.md index 831018d27329..ac84ca90aca5 100644 --- a/release-note.md +++ b/release-note.md @@ -187,6 +187,7 @@ - [Transform-V2] Add support CatalogTable for FilterFieldTransform (#4422) - [Transform-V2] Add catalog support for SQL Transform plugin (#4819) - [Connector-V2] [Assert] Support check the precision and scale of Decimal type (#6110) +- [Connector-V2] [Iceberg] Support iceberg sink #6198 ### Zeta(ST-Engine) diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/pom.xml b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/pom.xml index a4f082a3eae2..d9947ec324cc 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/pom.xml +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/pom.xml @@ -69,8 +69,18 @@ org.glassfish.jersey.core * + + + com.github.luben + zstd-jni + + + com.github.luben + zstd-jni + 1.5.5-5 + @@ -84,6 +94,10 @@ io.debezium debezium-embedded + + com.github.luben + zstd-jni + com.zaxxer HikariCP diff --git a/seatunnel-connectors-v2/connector-iceberg/pom.xml b/seatunnel-connectors-v2/connector-iceberg/pom.xml index d1689e8ef9e2..309900b09c77 100644 --- a/seatunnel-connectors-v2/connector-iceberg/pom.xml +++ b/seatunnel-connectors-v2/connector-iceberg/pom.xml @@ -30,12 +30,23 @@ SeaTunnel : Connectors V2 : Iceberg - 0.14.0 - 1.12.3 + 1.4.2 + 1.13.1 1.11.3 2.3.9 + connector-iceberg + + + + com.github.luben + zstd-jni + 1.5.5-5 + + + + org.apache.seatunnel @@ -48,6 +59,13 @@ iceberg-core ${iceberg.version} + + + org.apache.iceberg + iceberg-common + ${iceberg.version} + + org.apache.iceberg iceberg-api @@ -119,13 +137,15 @@ - org.apache.flink - flink-shaded-hadoop-2 + org.apache.seatunnel + seatunnel-hadoop3-3.1.4-uber + ${project.version} + optional provided org.apache.avro - * + avro @@ -186,10 +206,50 @@ junit junit - ${junit4.version} + 4.13.2 test + + + + org.apache.maven.plugins + maven-shade-plugin + + + + shade + + package + + + + org.apache.avro + + ${seatunnel.shade.package}.${connector.name}.org.apache.avro + + + org.apache.orc + ${seatunnel.shade.package}.${connector.name}.org.apache.orc + + + org.apache.parquet + + ${seatunnel.shade.package}.${connector.name}.org.apache.parquet + + + shaded.parquet + + ${seatunnel.shade.package}.${connector.name}.shaded.parquet + + + + + + + + + diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/IcebergCatalogFactory.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/IcebergCatalogFactory.java deleted file mode 100644 index 83a406897657..000000000000 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/IcebergCatalogFactory.java +++ /dev/null @@ -1,88 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.iceberg; - -import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; -import org.apache.seatunnel.connectors.seatunnel.iceberg.config.IcebergCatalogType; -import org.apache.seatunnel.connectors.seatunnel.iceberg.exception.IcebergConnectorException; - -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.CatalogUtil; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.hadoop.HadoopCatalog; -import org.apache.iceberg.hadoop.SerializableConfiguration; -import org.apache.iceberg.hive.HiveCatalog; - -import lombok.NonNull; - -import java.io.Serializable; -import java.util.HashMap; -import java.util.Map; - -public class IcebergCatalogFactory implements Serializable { - - private static final long serialVersionUID = -6003040601422350869L; - - private final String catalogName; - private final IcebergCatalogType catalogType; - private final String warehouse; - private final String uri; - - public IcebergCatalogFactory( - @NonNull String catalogName, - @NonNull IcebergCatalogType catalogType, - @NonNull String warehouse, - String uri) { - this.catalogName = catalogName; - this.catalogType = catalogType; - this.warehouse = warehouse; - this.uri = uri; - } - - public Catalog create() { - Configuration conf = new Configuration(); - SerializableConfiguration serializableConf = new SerializableConfiguration(conf); - Map properties = new HashMap<>(); - properties.put(CatalogProperties.WAREHOUSE_LOCATION, warehouse); - - switch (catalogType) { - case HADOOP: - return hadoop(catalogName, serializableConf, properties); - case HIVE: - properties.put(CatalogProperties.URI, uri); - return hive(catalogName, serializableConf, properties); - default: - throw new IcebergConnectorException( - CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, - String.format("Unsupported catalogType: %s", catalogType)); - } - } - - private static Catalog hadoop( - String catalogName, SerializableConfiguration conf, Map properties) { - return CatalogUtil.loadCatalog( - HadoopCatalog.class.getName(), catalogName, properties, conf.get()); - } - - private static Catalog hive( - String catalogName, SerializableConfiguration conf, Map properties) { - return CatalogUtil.loadCatalog( - HiveCatalog.class.getName(), catalogName, properties, conf.get()); - } -} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/IcebergCatalogLoader.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/IcebergCatalogLoader.java new file mode 100644 index 000000000000..63596c88beb2 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/IcebergCatalogLoader.java @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.iceberg; + +import org.apache.seatunnel.shade.com.google.common.collect.ImmutableList; + +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.CommonConfig; + +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.common.DynClasses; +import org.apache.iceberg.common.DynMethods; + +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.io.Serializable; +import java.lang.reflect.InvocationTargetException; +import java.net.URL; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.List; + +@Slf4j +public class IcebergCatalogLoader implements Serializable { + + private static final long serialVersionUID = -6003040601422350869L; + private static final List HADOOP_CONF_FILES = + ImmutableList.of("core-site.xml", "hdfs-site.xml", "hive-site.xml"); + private CommonConfig config; + + public IcebergCatalogLoader(CommonConfig config) { + this.config = config; + } + + public Catalog loadCatalog() { + // When using the seatunel engine, set the current class loader to prevent loading failures + Thread.currentThread().setContextClassLoader(IcebergCatalogLoader.class.getClassLoader()); + return CatalogUtil.buildIcebergCatalog( + config.getCatalogName(), config.getCatalogProps(), loadHadoopConfig(config)); + } + + /** + * Loading Hadoop configuration through reflection + * + * @param config + * @return + */ + private Object loadHadoopConfig(CommonConfig config) { + Class configClass = + DynClasses.builder() + .impl("org.apache.hadoop.hdfs.HdfsConfiguration") + .orNull() + .build(); + if (configClass == null) { + configClass = + DynClasses.builder() + .impl("org.apache.hadoop.conf.Configuration") + .orNull() + .build(); + } + + if (configClass == null) { + log.info("Hadoop not found on classpath, not creating Hadoop config"); + return null; + } + + try { + Object result = configClass.getDeclaredConstructor().newInstance(); + DynMethods.BoundMethod addResourceMethod = + DynMethods.builder("addResource").impl(configClass, URL.class).build(result); + DynMethods.BoundMethod setMethod = + DynMethods.builder("set") + .impl(configClass, String.class, String.class) + .build(result); + + // load any config files in the specified config directory + String hadoopConfPath = config.getHadoopConfPath(); + if (hadoopConfPath != null) { + HADOOP_CONF_FILES.forEach( + confFile -> { + Path path = Paths.get(hadoopConfPath, confFile); + if (Files.exists(path)) { + try { + addResourceMethod.invoke(path.toUri().toURL()); + } catch (IOException e) { + log.warn( + "Error adding Hadoop resource {}, resource was not added", + path, + e); + } + } + }); + } + config.getHadoopProps().forEach(setMethod::invoke); + log.info("Hadoop config initialized: {}", configClass.getName()); + return result; + } catch (InstantiationException + | IllegalAccessException + | NoSuchMethodException + | InvocationTargetException e) { + log.warn( + "Hadoop found on classpath but could not create config, proceeding without config", + e); + } + return null; + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/IcebergTableLoader.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/IcebergTableLoader.java index 554dd0bd7599..5e3960474022 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/IcebergTableLoader.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/IcebergTableLoader.java @@ -17,46 +17,59 @@ package org.apache.seatunnel.connectors.seatunnel.iceberg; -import org.apache.seatunnel.connectors.seatunnel.iceberg.config.SourceConfig; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.CommonConfig; +import org.apache.commons.lang3.StringUtils; import org.apache.iceberg.CachingCatalog; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; +import com.google.common.annotations.VisibleForTesting; import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; import java.io.Closeable; import java.io.IOException; import java.io.Serializable; +import java.util.Objects; -import static org.apache.seatunnel.shade.com.google.common.base.Preconditions.checkArgument; - +@Slf4j public class IcebergTableLoader implements Closeable, Serializable { private static final long serialVersionUID = 9061073826700804273L; - private final IcebergCatalogFactory icebergCatalogFactory; + private final IcebergCatalogLoader icebergCatalogFactory; private final String tableIdentifierStr; - - private Catalog catalog; + private transient Catalog catalog; public IcebergTableLoader( - @NonNull IcebergCatalogFactory icebergCatalogFactory, + @NonNull IcebergCatalogLoader icebergCatalogFactory, @NonNull TableIdentifier tableIdentifier) { this.icebergCatalogFactory = icebergCatalogFactory; this.tableIdentifierStr = tableIdentifier.toString(); } - public void open() { - catalog = CachingCatalog.wrap(icebergCatalogFactory.create()); + public Catalog getCatalog() { + return catalog; + } + + public TableIdentifier getTableIdentifier() { + return TableIdentifier.parse(tableIdentifierStr); + } + + public IcebergTableLoader open() { + catalog = CachingCatalog.wrap(icebergCatalogFactory.loadCatalog()); + return this; } public Table loadTable() { TableIdentifier tableIdentifier = TableIdentifier.parse(tableIdentifierStr); - checkArgument( - catalog.tableExists(tableIdentifier), "Illegal source table: " + tableIdentifier); + if (catalog == null) { + open(); + } return catalog.loadTable(tableIdentifier); } @@ -67,16 +80,27 @@ public void close() throws IOException { } } - public static IcebergTableLoader create(SourceConfig sourceConfig) { - IcebergCatalogFactory catalogFactory = - new IcebergCatalogFactory( - sourceConfig.getCatalogName(), - sourceConfig.getCatalogType(), - sourceConfig.getWarehouse(), - sourceConfig.getUri()); + @VisibleForTesting + public static IcebergTableLoader create(CommonConfig config) { + return create(config, null); + } + + public static IcebergTableLoader create(CommonConfig config, CatalogTable catalogTable) { + IcebergCatalogLoader catalogFactory = new IcebergCatalogLoader(config); + String table; + if (Objects.nonNull(catalogTable) + && StringUtils.isNotEmpty(catalogTable.getTableId().getTableName())) { + log.info( + "Config table name is empty, use catalog table name: {}", + catalogTable.getTableId().getTableName()); + table = catalogTable.getTableId().getTableName(); + } else if (StringUtils.isNotEmpty(config.getTable())) { + // for test in sink + table = config.getTable(); + } else { + throw new IllegalArgumentException("Table name is empty"); + } return new IcebergTableLoader( - catalogFactory, - TableIdentifier.of( - Namespace.of(sourceConfig.getNamespace()), sourceConfig.getTable())); + catalogFactory, TableIdentifier.of(Namespace.of(config.getNamespace()), table)); } } diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/catalog/IcebergCatalog.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/catalog/IcebergCatalog.java new file mode 100644 index 000000000000..4a4b9b898419 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/catalog/IcebergCatalog.java @@ -0,0 +1,263 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.iceberg.catalog; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.table.catalog.Catalog; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.PhysicalColumn; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.catalog.exception.CatalogException; +import org.apache.seatunnel.api.table.catalog.exception.DatabaseAlreadyExistException; +import org.apache.seatunnel.api.table.catalog.exception.DatabaseNotExistException; +import org.apache.seatunnel.api.table.catalog.exception.TableAlreadyExistException; +import org.apache.seatunnel.api.table.catalog.exception.TableNotExistException; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.seatunnel.iceberg.IcebergCatalogLoader; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.CommonConfig; +import org.apache.seatunnel.connectors.seatunnel.iceberg.utils.SchemaUtils; + +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SupportsNamespaces; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.types.Types; + +import lombok.extern.slf4j.Slf4j; + +import java.io.Closeable; +import java.io.IOException; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.seatunnel.connectors.seatunnel.iceberg.utils.SchemaUtils.toIcebergTableIdentifier; +import static org.apache.seatunnel.connectors.seatunnel.iceberg.utils.SchemaUtils.toTablePath; + +@Slf4j +public class IcebergCatalog implements Catalog { + private String catalogName; + private ReadonlyConfig readonlyConfig; + private IcebergCatalogLoader icebergCatalogLoader; + private org.apache.iceberg.catalog.Catalog catalog; + + public IcebergCatalog(String catalogName, ReadonlyConfig readonlyConfig) { + this.readonlyConfig = readonlyConfig; + this.catalogName = catalogName; + this.icebergCatalogLoader = new IcebergCatalogLoader(new CommonConfig(readonlyConfig)); + } + + @Override + public String name() { + return this.catalogName; + } + + @Override + public void open() throws CatalogException { + this.catalog = icebergCatalogLoader.loadCatalog(); + } + + @Override + public void close() throws CatalogException { + if (catalog != null && catalog instanceof Closeable) { + try { + ((Closeable) catalog).close(); + } catch (IOException e) { + log.error("Error while closing IcebergCatalog.", e); + throw new CatalogException(e); + } + } + } + + @Override + public String getDefaultDatabase() throws CatalogException { + return "default"; + } + + @Override + public boolean databaseExists(String databaseName) throws CatalogException { + if (catalog instanceof SupportsNamespaces) { + boolean exists = + ((SupportsNamespaces) catalog).namespaceExists(Namespace.of(databaseName)); + log.info("Database {} existence status: {}", databaseName, exists); + return exists; + } else { + throw new UnsupportedOperationException( + "catalog not implements SupportsNamespaces so can't check database exists"); + } + } + + @Override + public List listDatabases() throws CatalogException { + if (catalog instanceof SupportsNamespaces) { + List databases = + ((SupportsNamespaces) catalog) + .listNamespaces().stream() + .map(Namespace::toString) + .collect(Collectors.toList()); + log.info("Fetched {} namespaces.", databases.size()); + return databases; + } else { + throw new UnsupportedOperationException( + "catalog not implements SupportsNamespaces so can't list databases"); + } + } + + @Override + public List listTables(String databaseName) + throws CatalogException, DatabaseNotExistException { + List tables = + catalog.listTables(Namespace.of(databaseName)).stream() + .map(tableIdentifier -> toTablePath(tableIdentifier).getTableName()) + .collect(Collectors.toList()); + log.info("Fetched {} tables.", tables.size()); + return tables; + } + + @Override + public boolean tableExists(TablePath tablePath) throws CatalogException { + return catalog.tableExists(toIcebergTableIdentifier(tablePath)); + } + + @Override + public CatalogTable getTable(TablePath tablePath) + throws CatalogException, TableNotExistException { + TableIdentifier icebergTableIdentifier = toIcebergTableIdentifier(tablePath); + try { + CatalogTable catalogTable = + toCatalogTable(catalog.loadTable(icebergTableIdentifier), tablePath); + log.info("Fetched table details for: {}", tablePath); + return catalogTable; + } catch (NoSuchTableException e) { + throw new TableNotExistException("Table not exist", tablePath, e); + } + } + + @Override + public void createTable(TablePath tablePath, CatalogTable table, boolean ignoreIfExists) + throws TableAlreadyExistException, DatabaseNotExistException, CatalogException { + log.info("Creating table at path: {}", tablePath); + SchemaUtils.autoCreateTable(catalog, tablePath, table, readonlyConfig); + } + + @Override + public void dropTable(TablePath tablePath, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + if (ignoreIfNotExists) { + if (!tableExists(tablePath)) { + log.info( + "Attempted to drop table at path: {}. The table does not exist, but proceeding as 'ignoreIfNotExists' is set to true.", + tablePath); + return; + } + } + catalog.dropTable(toIcebergTableIdentifier(tablePath), true); + log.info("Dropped table at path: {}", tablePath); + } + + @Override + public void createDatabase(TablePath tablePath, boolean ignoreIfExists) + throws DatabaseAlreadyExistException, CatalogException { + // Do nothing + } + + @Override + public void dropDatabase(TablePath tablePath, boolean ignoreIfNotExists) + throws DatabaseNotExistException, CatalogException { + // Do nothing + } + + @Override + public boolean isExistsData(TablePath tablePath) { + if (!tableExists(tablePath)) { + throw new TableNotExistException("table not exist", tablePath); + } + TableIdentifier icebergTableIdentifier = toIcebergTableIdentifier(tablePath); + Snapshot snapshot = catalog.loadTable(icebergTableIdentifier).currentSnapshot(); + if (snapshot != null) { + String total = snapshot.summary().getOrDefault("total-records", null); + return total != null && !total.equals("0"); + } + return false; + } + + @Override + public void executeSql(TablePath tablePath, String sql) { + throw new UnsupportedOperationException("Does not support executing custom SQL"); + } + + public void truncateTable(TablePath tablePath, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + if (!tableExists(tablePath)) { + throw new TableNotExistException("table not exist", tablePath); + } + TableIdentifier icebergTableIdentifier = toIcebergTableIdentifier(tablePath); + catalog.loadTable(icebergTableIdentifier) + .newDelete() + .deleteFromRowFilter(org.apache.iceberg.expressions.Expressions.alwaysTrue()) + .commit(); + log.info("Truncated table at path: {}", tablePath); + } + + public CatalogTable toCatalogTable(Table icebergTable, TablePath tablePath) { + List columns = icebergTable.schema().columns(); + TableSchema.Builder builder = TableSchema.builder(); + columns.stream() + .forEach( + nestedField -> { + String name = nestedField.name(); + SeaTunnelDataType seaTunnelType = + SchemaUtils.toSeaTunnelType(name, nestedField.type()); + PhysicalColumn physicalColumn = + PhysicalColumn.of( + name, + seaTunnelType, + null, + true, + null, + nestedField.doc()); + builder.column(physicalColumn); + }); + + List partitionKeys = + icebergTable.spec().fields().stream() + .map(PartitionField::name) + .collect(Collectors.toList()); + + return CatalogTable.of( + org.apache.seatunnel.api.table.catalog.TableIdentifier.of( + catalogName, tablePath.getDatabaseName(), tablePath.getTableName()), + builder.build(), + icebergTable.properties(), + partitionKeys, + null, + catalogName); + } + + public Schema toIcebergSchema(TableSchema tableSchema) { + // Generate struct type + SeaTunnelRowType rowType = tableSchema.toPhysicalRowDataType(); + Types.StructType structType = SchemaUtils.toIcebergType(rowType).asStructType(); + return new Schema(structType.fields()); + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/catalog/IcebergCatalogFactory.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/catalog/IcebergCatalogFactory.java new file mode 100644 index 000000000000..125969906835 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/catalog/IcebergCatalogFactory.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.iceberg.catalog; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.table.catalog.Catalog; +import org.apache.seatunnel.api.table.factory.CatalogFactory; +import org.apache.seatunnel.api.table.factory.Factory; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.CommonConfig; + +import com.google.auto.service.AutoService; + +import static org.apache.seatunnel.connectors.seatunnel.iceberg.config.CommonConfig.KEY_CASE_SENSITIVE; + +@AutoService(Factory.class) +public class IcebergCatalogFactory implements CatalogFactory { + @Override + public Catalog createCatalog(String catalogName, ReadonlyConfig options) { + return new IcebergCatalog(catalogName, options); + } + + @Override + public String factoryIdentifier() { + return "Iceberg"; + } + + @Override + public OptionRule optionRule() { + return OptionRule.builder() + .required( + CommonConfig.KEY_CATALOG_NAME, + CommonConfig.KEY_NAMESPACE, + CommonConfig.KEY_TABLE, + CommonConfig.CATALOG_PROPS) + .optional(CommonConfig.HADOOP_PROPS, KEY_CASE_SENSITIVE) + .build(); + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/config/CommonConfig.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/config/CommonConfig.java index 2f893da092bf..a7503e6e3062 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/config/CommonConfig.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/config/CommonConfig.java @@ -17,19 +17,18 @@ package org.apache.seatunnel.connectors.seatunnel.iceberg.config; -import org.apache.seatunnel.shade.com.typesafe.config.Config; - import org.apache.seatunnel.api.configuration.Option; import org.apache.seatunnel.api.configuration.Options; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.common.config.ConfigRuntimeException; import lombok.Getter; import lombok.ToString; import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; -import static org.apache.seatunnel.connectors.seatunnel.iceberg.config.IcebergCatalogType.HADOOP; -import static org.apache.seatunnel.connectors.seatunnel.iceberg.config.IcebergCatalogType.HIVE; -import static org.apache.seatunnel.shade.com.google.common.base.Preconditions.checkArgument; import static org.apache.seatunnel.shade.com.google.common.base.Preconditions.checkNotNull; @Getter @@ -40,19 +39,13 @@ public class CommonConfig implements Serializable { public static final Option KEY_CATALOG_NAME = Options.key("catalog_name") .stringType() - .noDefaultValue() + .defaultValue("default") .withDescription(" the iceberg catalog name"); - public static final Option KEY_CATALOG_TYPE = - Options.key("catalog_type") - .enumType(IcebergCatalogType.class) - .noDefaultValue() - .withDescription(" the iceberg catalog type"); - public static final Option KEY_NAMESPACE = Options.key("namespace") .stringType() - .noDefaultValue() + .defaultValue("default") .withDescription(" the iceberg namespace"); public static final Option KEY_TABLE = @@ -61,17 +54,25 @@ public class CommonConfig implements Serializable { .noDefaultValue() .withDescription(" the iceberg table"); - public static final Option KEY_URI = - Options.key("uri") - .stringType() + public static final Option> CATALOG_PROPS = + Options.key("iceberg.catalog.config") + .mapType() .noDefaultValue() - .withDescription(" the iceberg server uri"); + .withDescription( + "Specify the properties for initializing the Iceberg catalog, which can be referenced in this file:'https://github.com/apache/iceberg/blob/main/core/src/main/java/org/apache/iceberg/CatalogProperties.java'"); - public static final Option KEY_WAREHOUSE = - Options.key("warehouse") + public static final Option> HADOOP_PROPS = + Options.key("hadoop.config") + .mapType() + .defaultValue(new HashMap<>()) + .withDescription("Properties passed through to the Hadoop configuration"); + + public static final Option HADOOP_CONF_PATH_PROP = + Options.key("iceberg.hadoop-conf-path") .stringType() - .noDefaultValue() - .withDescription(" the iceberg warehouse"); + .defaultValue(null) + .withDescription( + "The specified loading paths for the 'core-site.xml', 'hdfs-site.xml', 'hive-site.xml' files."); public static final Option KEY_CASE_SENSITIVE = Options.key("case_sensitive") @@ -80,35 +81,39 @@ public class CommonConfig implements Serializable { .withDescription(" the iceberg case_sensitive"); private String catalogName; - private IcebergCatalogType catalogType; - private String uri; - private String warehouse; private String namespace; private String table; private boolean caseSensitive; - public CommonConfig(Config pluginConfig) { - String catalogType = checkArgumentNotNull(pluginConfig.getString(KEY_CATALOG_TYPE.key())); - checkArgument( - HADOOP.getType().equals(catalogType) || HIVE.getType().equals(catalogType), - "Illegal catalogType: " + catalogType); - - this.catalogType = IcebergCatalogType.valueOf(catalogType.toUpperCase()); - this.catalogName = checkArgumentNotNull(pluginConfig.getString(KEY_CATALOG_NAME.key())); - if (pluginConfig.hasPath(KEY_URI.key())) { - this.uri = checkArgumentNotNull(pluginConfig.getString(KEY_URI.key())); - } - this.warehouse = checkArgumentNotNull(pluginConfig.getString(KEY_WAREHOUSE.key())); - this.namespace = checkArgumentNotNull(pluginConfig.getString(KEY_NAMESPACE.key())); - this.table = checkArgumentNotNull(pluginConfig.getString(KEY_TABLE.key())); - - if (pluginConfig.hasPath(KEY_CASE_SENSITIVE.key())) { - this.caseSensitive = pluginConfig.getBoolean(KEY_CASE_SENSITIVE.key()); + private Map catalogProps; + private Map hadoopProps; + private String hadoopConfPath; + + public CommonConfig(ReadonlyConfig pluginConfig) { + this.catalogName = checkArgumentNotNull(pluginConfig.get(KEY_CATALOG_NAME)); + this.namespace = pluginConfig.get(KEY_NAMESPACE); + this.table = pluginConfig.get(KEY_TABLE); + this.catalogProps = pluginConfig.get(CATALOG_PROPS); + this.hadoopProps = pluginConfig.get(HADOOP_PROPS); + this.hadoopConfPath = pluginConfig.get(HADOOP_CONF_PATH_PROP); + if (pluginConfig.toConfig().hasPath(KEY_CASE_SENSITIVE.key())) { + this.caseSensitive = pluginConfig.get(KEY_CASE_SENSITIVE); } + validate(); } protected T checkArgumentNotNull(T argument) { checkNotNull(argument); return argument; } + + private void validate() { + checkState(!catalogProps.isEmpty(), "Must specify iceberg catalog config"); + } + + private void checkState(boolean condition, String msg) { + if (!condition) { + throw new ConfigRuntimeException(msg); + } + } } diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/config/SinkConfig.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/config/SinkConfig.java new file mode 100644 index 000000000000..de9c74344f0f --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/config/SinkConfig.java @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.iceberg.config; + +import org.apache.seatunnel.shade.com.google.common.annotations.VisibleForTesting; +import org.apache.seatunnel.shade.com.google.common.collect.ImmutableList; + +import org.apache.seatunnel.api.configuration.Option; +import org.apache.seatunnel.api.configuration.Options; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.sink.DataSaveMode; +import org.apache.seatunnel.api.sink.SchemaSaveMode; + +import lombok.Getter; +import lombok.Setter; +import lombok.ToString; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static java.util.stream.Collectors.toList; + +@Getter +@Setter +@ToString +public class SinkConfig extends CommonConfig { + + public static final int SCHEMA_UPDATE_RETRIES = 2; // 3 total attempts + public static final int CREATE_TABLE_RETRIES = 2; // 3 total attempts + + private static final String ID_COLUMNS = "id-columns"; + private static final String PARTITION_BY = "partition-by"; + + public static final Option> TABLE_PROPS = + Options.key("iceberg.table.config") + .mapType() + .defaultValue(new HashMap<>()) + .withDescription("Iceberg table configs"); + + public static final Option> WRITE_PROPS = + Options.key("iceberg.table.write-props") + .mapType() + .defaultValue(new HashMap<>()) + .withDescription( + "Properties passed through to Iceberg writer initialization, these take precedence, such as 'write.format.default', 'write.target-file-size-bytes', and other settings, can be found with specific parameters at 'https://github.com/apache/iceberg/blob/main/core/src/main/java/org/apache/iceberg/TableProperties.java'."); + + public static final Option> AUTO_CREATE_PROPS = + Options.key("iceberg.table.auto-create-props") + .mapType() + .defaultValue(new HashMap<>()) + .withDescription( + "Configuration specified by Iceberg during automatic table creation."); + + public static final Option TABLE_SCHEMA_EVOLUTION_ENABLED_PROP = + Options.key("iceberg.table.schema-evolution-enabled") + .booleanType() + .defaultValue(false) + .withDescription( + "Setting to true enables Iceberg tables to support schema evolution during the synchronization process"); + + public static final Option TABLE_PRIMARY_KEYS = + Options.key("iceberg.table.primary-keys") + .stringType() + .noDefaultValue() + .withDescription( + "Default comma-separated list of columns that identify a row in tables (primary key)"); + + public static final Option TABLE_DEFAULT_PARTITION_KEYS = + Options.key("iceberg.table.partition-keys") + .stringType() + .noDefaultValue() + .withDescription( + "Default comma-separated list of partition fields to use when creating tables."); + + public static final Option TABLE_UPSERT_MODE_ENABLED_PROP = + Options.key("iceberg.table.upsert-mode-enabled") + .booleanType() + .defaultValue(false) + .withDescription("Set to `true` to enable upsert mode, default is `false`"); + + public static final Option SCHEMA_SAVE_MODE = + Options.key("schema_save_mode") + .enumType(SchemaSaveMode.class) + .defaultValue(SchemaSaveMode.CREATE_SCHEMA_WHEN_NOT_EXIST) + .withDescription("schema save mode"); + + public static final Option DATA_SAVE_MODE = + Options.key("data_save_mode") + .enumType(DataSaveMode.class) + .defaultValue(DataSaveMode.APPEND_DATA) + .withDescription("data save mode"); + + @VisibleForTesting private static final String COMMA_NO_PARENS_REGEX = ",(?![^()]*+\\))"; + + private final ReadonlyConfig readonlyConfig; + private Map autoCreateProps; + private Map writeProps; + private List primaryKeys; + private List partitionKeys; + + private boolean upsertModeEnabled; + private boolean tableSchemaEvolutionEnabled; + private SchemaSaveMode schemaSaveMode; + private DataSaveMode dataSaveMode; + + public SinkConfig(ReadonlyConfig readonlyConfig) { + super(readonlyConfig); + this.readonlyConfig = readonlyConfig; + this.autoCreateProps = readonlyConfig.get(AUTO_CREATE_PROPS); + this.writeProps = readonlyConfig.get(WRITE_PROPS); + this.primaryKeys = stringToList(readonlyConfig.get(TABLE_PRIMARY_KEYS), ","); + this.partitionKeys = stringToList(readonlyConfig.get(TABLE_DEFAULT_PARTITION_KEYS), ","); + this.upsertModeEnabled = readonlyConfig.get(TABLE_UPSERT_MODE_ENABLED_PROP); + this.tableSchemaEvolutionEnabled = readonlyConfig.get(TABLE_SCHEMA_EVOLUTION_ENABLED_PROP); + this.schemaSaveMode = readonlyConfig.get(SCHEMA_SAVE_MODE); + this.dataSaveMode = readonlyConfig.get(DATA_SAVE_MODE); + } + + @VisibleForTesting + public static List stringToList(String value, String regex) { + if (value == null || value.isEmpty()) { + return ImmutableList.of(); + } + return Arrays.stream(value.split(regex)).map(String::trim).collect(toList()); + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/config/SourceConfig.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/config/SourceConfig.java index fc9394d07794..56d0a4509df9 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/config/SourceConfig.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/config/SourceConfig.java @@ -23,6 +23,7 @@ import org.apache.seatunnel.api.configuration.Option; import org.apache.seatunnel.api.configuration.Options; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.connectors.seatunnel.iceberg.source.enumerator.scan.IcebergStreamScanStrategy; import org.apache.iceberg.expressions.Expression; @@ -86,8 +87,9 @@ public class SourceConfig extends CommonConfig { private Integer splitLookback; private Long splitOpenFileCost; - public SourceConfig(Config pluginConfig) { - super(pluginConfig); + public SourceConfig(ReadonlyConfig readonlyConfig) { + super(readonlyConfig); + Config pluginConfig = readonlyConfig.toConfig(); if (pluginConfig.hasPath(KEY_START_SNAPSHOT_TIMESTAMP.key())) { this.startSnapshotTimestamp = pluginConfig.getLong(KEY_START_SNAPSHOT_TIMESTAMP.key()); } @@ -110,7 +112,7 @@ public SourceConfig(Config pluginConfig) { } } - public static SourceConfig loadConfig(Config pluginConfig) { + public static SourceConfig loadConfig(ReadonlyConfig pluginConfig) { return new SourceConfig(pluginConfig); } } diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/IcebergTypeMapper.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/IcebergTypeMapper.java index e8a9e9752887..0b6ed2ccc555 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/IcebergTypeMapper.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/IcebergTypeMapper.java @@ -36,6 +36,7 @@ import java.util.List; public class IcebergTypeMapper { + private static int fieldId = 1; public static SeaTunnelDataType mapping(String field, @NonNull Type icebergType) { switch (icebergType.typeId()) { @@ -110,4 +111,60 @@ private static ArrayType mappingListType(String field, Types.ListType listType) private static MapType mappingMapType(String field, Types.MapType mapType) { return new MapType(mapping(field, mapType.keyType()), mapping(field, mapType.valueType())); } + + public static Type toIcebergType(SeaTunnelDataType dataType) { + switch (dataType.getSqlType()) { + case BOOLEAN: + return Types.BooleanType.get(); + case BYTES: + return Types.BinaryType.get(); + case SMALLINT: + case TINYINT: + case INT: + return Types.IntegerType.get(); + case BIGINT: + return Types.LongType.get(); + case FLOAT: + return Types.FloatType.get(); + case DOUBLE: + return Types.DoubleType.get(); + case DECIMAL: + DecimalType decimalType = (DecimalType) dataType; + return Types.DecimalType.of(decimalType.getPrecision(), decimalType.getScale()); + case ARRAY: + ArrayType arrayType = (ArrayType) dataType; + // converter elementType + Type elementType = toIcebergType(arrayType.getElementType()); + return Types.ListType.ofOptional(nextId(), elementType); + case MAP: + org.apache.seatunnel.api.table.type.MapType mapType = + (org.apache.seatunnel.api.table.type.MapType) dataType; + Type keyType = toIcebergType(mapType.getKeyType()); + Type valueType = toIcebergType(mapType.getValueType()); + return Types.MapType.ofOptional(nextId(), nextId(), keyType, valueType); + case ROW: + SeaTunnelRowType seaTunnelRowType = (SeaTunnelRowType) dataType; + List structFields = new ArrayList<>(); + for (int i = 0; i < seaTunnelRowType.getFieldNames().length; i++) { + String field = seaTunnelRowType.getFieldName(i); + SeaTunnelDataType fieldType = seaTunnelRowType.getFieldType(i); + structFields.add( + Types.NestedField.of(nextId(), true, field, toIcebergType(fieldType))); + } + return Types.StructType.of(structFields); + case DATE: + return Types.DateType.get(); + case TIME: + return Types.TimeType.get(); + case TIMESTAMP: + return Types.TimestampType.withZone(); + case STRING: + default: + return Types.StringType.get(); + } + } + + private static int nextId() { + return fieldId++; + } } diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/RowConverter.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/RowConverter.java new file mode 100644 index 000000000000..3dc453f80309 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/RowConverter.java @@ -0,0 +1,482 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.iceberg.data; + +import org.apache.seatunnel.shade.com.google.common.base.Preconditions; +import org.apache.seatunnel.shade.com.google.common.collect.Maps; + +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.SinkConfig; +import org.apache.seatunnel.connectors.seatunnel.iceberg.sink.schema.SchemaChangeWrapper; +import org.apache.seatunnel.connectors.seatunnel.iceberg.utils.SchemaUtils; + +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.mapping.MappedField; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.mapping.NameMappingParser; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.DateTimeUtil; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.math.BigDecimal; +import java.math.RoundingMode; +import java.nio.ByteBuffer; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; +import java.time.format.DateTimeFormatterBuilder; +import java.time.format.DateTimeParseException; +import java.time.temporal.Temporal; +import java.util.Arrays; +import java.util.Base64; +import java.util.Date; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.UUID; + +import static java.util.stream.Collectors.toList; + +public class RowConverter { + private static final ObjectMapper MAPPER = new ObjectMapper(); + + private static final DateTimeFormatter OFFSET_TS_FMT = + new DateTimeFormatterBuilder() + .append(DateTimeFormatter.ISO_LOCAL_DATE_TIME) + .appendOffset("+HHmm", "Z") + .toFormatter(); + + private final Schema tableSchema; + private final NameMapping nameMapping; + private final SinkConfig config; + private final Map> structNames = Maps.newHashMap(); + + public RowConverter(Table table, SinkConfig config) { + this.tableSchema = table.schema(); + this.nameMapping = createNameMapping(table); + this.config = config; + } + + private NameMapping createNameMapping(Table table) { + String nameMappingString = table.properties().get(TableProperties.DEFAULT_NAME_MAPPING); + return nameMappingString != null ? NameMappingParser.fromJson(nameMappingString) : null; + } + + public Record convert(Object row, SeaTunnelDataType rowType) { + return convertStructValue(row, rowType, tableSchema.asStruct(), -1, null); + } + + public Record convert(Object row, SeaTunnelDataType rowType, SchemaChangeWrapper wrapper) { + return convertStructValue(row, rowType, tableSchema.asStruct(), -1, wrapper); + } + + protected GenericRecord convertStructValue( + Object value, + SeaTunnelDataType fromType, + Types.StructType schema, + int parentFieldId, + SchemaChangeWrapper wrapper) { + switch (fromType.getSqlType()) { + case ROW: + return convertToStruct( + (SeaTunnelRow) value, + (SeaTunnelRowType) fromType, + schema, + parentFieldId, + wrapper); + default: + throw new IllegalArgumentException( + "Cannot convert to struct: " + fromType.getSqlType().name()); + } + } + + /** + * Convert RowType + * + * @param row + * @param fromType + * @param schema + * @param structFieldId + * @return + */ + private GenericRecord convertToStruct( + SeaTunnelRow row, + SeaTunnelRowType fromType, + Types.StructType schema, + int structFieldId, + SchemaChangeWrapper wrapper) { + GenericRecord result = GenericRecord.create(schema); + String[] filedNames = fromType.getFieldNames(); + for (int i = 0; i < filedNames.length; i++) { + String recordField = filedNames[i]; + SeaTunnelDataType dataType = fromType.getFieldType(i); + Types.NestedField tableField = lookupStructField(recordField, schema, structFieldId); + // add column + if (Objects.isNull(tableField)) { + if (config.isTableSchemaEvolutionEnabled() && Objects.nonNull(wrapper)) { + // add the column if schema evolution is on + String parentFieldName = + structFieldId < 0 ? null : tableSchema.findColumnName(structFieldId); + Type type = SchemaUtils.toIcebergType(dataType); + wrapper.addColumn(parentFieldName, recordField, type); + } + continue; + } + result.setField( + tableField.name(), + convertValue( + row.getField(i), + fromType.getFieldType(i), + tableField.type(), + tableField.fieldId(), + wrapper)); + } + return result; + } + + public Object convertValue( + Object value, + SeaTunnelDataType fromType, + Type type, + int fieldId, + SchemaChangeWrapper wrapper) { + if (value == null) { + return null; + } + switch (type.typeId()) { + case STRUCT: + return convertStructValue(value, fromType, type.asStructType(), fieldId, wrapper); + case LIST: + return convertListValue(value, fromType, type.asListType(), wrapper); + case MAP: + return convertMapValue(value, fromType, type.asMapType(), wrapper); + case INTEGER: + return convertInt(value); + case LONG: + return convertLong(value); + case FLOAT: + return convertFloat(value); + case DOUBLE: + return convertDouble(value); + case DECIMAL: + return convertDecimal(value, (Types.DecimalType) type); + case BOOLEAN: + return convertBoolean(value); + case STRING: + return convertString(value); + case UUID: + return convertUUID(value); + case BINARY: + case FIXED: + return convertBase64Binary(value); + case DATE: + return convertDateValue(value); + case TIME: + return convertTimeValue(value); + case TIMESTAMP: + return convertTimestampValue(value, (Types.TimestampType) type); + } + throw new UnsupportedOperationException("Unsupported type: " + type.typeId()); + } + + private Types.NestedField lookupStructField( + String fieldName, Types.StructType schema, int structFieldId) { + if (nameMapping == null) { + return config.isCaseSensitive() + ? schema.caseInsensitiveField(fieldName) + : schema.field(fieldName); + } + + return structNames + .computeIfAbsent(structFieldId, notUsed -> createStructNameMap(schema)) + .get(fieldName); + } + + private Map createStructNameMap(Types.StructType schema) { + Map map = Maps.newHashMap(); + schema.fields() + .forEach( + col -> { + MappedField mappedField = nameMapping.find(col.fieldId()); + if (mappedField != null && !mappedField.names().isEmpty()) { + mappedField.names().forEach(name -> map.put(name, col)); + } else { + map.put(col.name(), col); + } + }); + return map; + } + + protected List convertListValue( + Object value, + SeaTunnelDataType fromType, + Types.ListType type, + SchemaChangeWrapper wrapper) { + Preconditions.checkArgument(value.getClass().isArray()); + Object[] list = (Object[]) value; + return Arrays.stream(list) + .map( + element -> { + int fieldId = type.fields().get(0).fieldId(); + return convertValue( + element, fromType, type.elementType(), fieldId, wrapper); + }) + .collect(toList()); + } + + protected Map convertMapValue( + Object value, + SeaTunnelDataType fromType, + Types.MapType type, + SchemaChangeWrapper wrapper) { + Preconditions.checkArgument(value instanceof Map); + Map map = (Map) value; + Map result = Maps.newHashMap(); + map.forEach( + (k, v) -> { + int keyFieldId = type.fields().get(0).fieldId(); + int valueFieldId = type.fields().get(1).fieldId(); + result.put( + convertValue(k, fromType, type.keyType(), keyFieldId, wrapper), + convertValue(v, fromType, type.valueType(), valueFieldId, wrapper)); + }); + return result; + } + + protected int convertInt(Object value) { + if (value instanceof Number) { + return ((Number) value).intValue(); + } else if (value instanceof String) { + return Integer.parseInt((String) value); + } + throw new IllegalArgumentException("Cannot convert to int: " + value.getClass().getName()); + } + + protected long convertLong(Object value) { + if (value instanceof Number) { + return ((Number) value).longValue(); + } else if (value instanceof String) { + return Long.parseLong((String) value); + } + throw new IllegalArgumentException("Cannot convert to long: " + value.getClass().getName()); + } + + protected float convertFloat(Object value) { + if (value instanceof Number) { + return ((Number) value).floatValue(); + } else if (value instanceof String) { + return Float.parseFloat((String) value); + } + throw new IllegalArgumentException( + "Cannot convert to float: " + value.getClass().getName()); + } + + protected double convertDouble(Object value) { + if (value instanceof Number) { + return ((Number) value).doubleValue(); + } else if (value instanceof String) { + return Double.parseDouble((String) value); + } + throw new IllegalArgumentException( + "Cannot convert to double: " + value.getClass().getName()); + } + + protected BigDecimal convertDecimal(Object value, Types.DecimalType type) { + BigDecimal bigDecimal; + if (value instanceof BigDecimal) { + bigDecimal = (BigDecimal) value; + } else if (value instanceof Number) { + Number num = (Number) value; + Double dbl = num.doubleValue(); + if (dbl.equals(Math.floor(dbl))) { + bigDecimal = BigDecimal.valueOf(num.longValue()); + } else { + bigDecimal = BigDecimal.valueOf(dbl); + } + } else if (value instanceof String) { + bigDecimal = new BigDecimal((String) value); + } else { + throw new IllegalArgumentException( + "Cannot convert to BigDecimal: " + value.getClass().getName()); + } + return bigDecimal.setScale(type.scale(), RoundingMode.HALF_UP); + } + + protected boolean convertBoolean(Object value) { + if (value instanceof Boolean) { + return (boolean) value; + } else if (value instanceof String) { + return Boolean.parseBoolean((String) value); + } + throw new IllegalArgumentException( + "Cannot convert to boolean: " + value.getClass().getName()); + } + + protected String convertString(Object value) { + try { + if (value instanceof String) { + return (String) value; + } else if (value instanceof Number || value instanceof Boolean) { + return value.toString(); + } else if (value instanceof Map || value instanceof List) { + return MAPPER.writeValueAsString(value); + } else { + return MAPPER.writeValueAsString(value); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + protected UUID convertUUID(Object value) { + if (value instanceof String) { + return UUID.fromString((String) value); + } else if (value instanceof UUID) { + return (UUID) value; + } + throw new IllegalArgumentException("Cannot convert to UUID: " + value.getClass().getName()); + } + + protected ByteBuffer convertBase64Binary(Object value) { + if (value instanceof String) { + return ByteBuffer.wrap(Base64.getDecoder().decode((String) value)); + } else if (value instanceof byte[]) { + return ByteBuffer.wrap((byte[]) value); + } else if (value instanceof ByteBuffer) { + return (ByteBuffer) value; + } + throw new IllegalArgumentException( + "Cannot convert to binary: " + value.getClass().getName()); + } + + protected LocalDate convertDateValue(Object value) { + if (value instanceof Number) { + int days = ((Number) value).intValue(); + return DateTimeUtil.dateFromDays(days); + } else if (value instanceof String) { + return LocalDate.parse((String) value); + } else if (value instanceof LocalDate) { + return (LocalDate) value; + } else if (value instanceof Date) { + int days = (int) (((Date) value).getTime() / 1000 / 60 / 60 / 24); + return DateTimeUtil.dateFromDays(days); + } + throw new RuntimeException("Cannot convert date: " + value); + } + + protected LocalTime convertTimeValue(Object value) { + if (value instanceof Number) { + long millis = ((Number) value).longValue(); + return DateTimeUtil.timeFromMicros(millis * 1000); + } else if (value instanceof String) { + return LocalTime.parse((String) value); + } else if (value instanceof LocalTime) { + return (LocalTime) value; + } else if (value instanceof Date) { + long millis = ((Date) value).getTime(); + return DateTimeUtil.timeFromMicros(millis * 1000); + } + throw new RuntimeException("Cannot convert time: " + value); + } + + protected Temporal convertTimestampValue(Object value, Types.TimestampType type) { + if (type.shouldAdjustToUTC()) { + return convertOffsetDateTime(value); + } + return convertLocalDateTime(value); + } + + private OffsetDateTime convertOffsetDateTime(Object value) { + if (value instanceof Number) { + long millis = ((Number) value).longValue(); + return DateTimeUtil.timestamptzFromMicros(millis * 1000); + } else if (value instanceof String) { + return parseOffsetDateTime((String) value); + } else if (value instanceof OffsetDateTime) { + return (OffsetDateTime) value; + } else if (value instanceof LocalDateTime) { + return ((LocalDateTime) value).atOffset(ZoneOffset.UTC); + } else if (value instanceof Date) { + return DateTimeUtil.timestamptzFromMicros(((Date) value).getTime() * 1000); + } + throw new RuntimeException( + "Cannot convert timestamptz: " + value + ", type: " + value.getClass()); + } + + private OffsetDateTime parseOffsetDateTime(String str) { + String tsStr = ensureTimestampFormat(str); + try { + return OFFSET_TS_FMT.parse(tsStr, OffsetDateTime::from); + } catch (DateTimeParseException e) { + return LocalDateTime.parse(tsStr, DateTimeFormatter.ISO_LOCAL_DATE_TIME) + .atOffset(ZoneOffset.UTC); + } + } + + private LocalDateTime convertLocalDateTime(Object value) { + if (value instanceof Number) { + long millis = ((Number) value).longValue(); + return DateTimeUtil.timestampFromMicros(millis * 1000); + } else if (value instanceof String) { + return parseLocalDateTime((String) value); + } else if (value instanceof LocalDateTime) { + return (LocalDateTime) value; + } else if (value instanceof OffsetDateTime) { + return ((OffsetDateTime) value).toLocalDateTime(); + } else if (value instanceof Date) { + return DateTimeUtil.timestampFromMicros(((Date) value).getTime() * 1000); + } + throw new RuntimeException( + "Cannot convert timestamp: " + value + ", type: " + value.getClass()); + } + + private LocalDateTime parseLocalDateTime(String str) { + String tsStr = ensureTimestampFormat(str); + try { + return LocalDateTime.parse(tsStr, DateTimeFormatter.ISO_LOCAL_DATE_TIME); + } catch (DateTimeParseException e) { + return OFFSET_TS_FMT.parse(tsStr, OffsetDateTime::from).toLocalDateTime(); + } + } + + private String ensureTimestampFormat(String str) { + String result = str; + if (result.charAt(10) == ' ') { + result = result.substring(0, 10) + 'T' + result.substring(11); + } + if (result.length() > 22 && result.charAt(19) == '+' && result.charAt(22) == ':') { + result = result.substring(0, 19) + result.substring(19).replace(":", ""); + } + return result; + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSink.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSink.java new file mode 100644 index 000000000000..06131bc9f759 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSink.java @@ -0,0 +1,214 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.iceberg.sink; + +import org.apache.seatunnel.api.common.SeaTunnelAPIErrorCode; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.serialization.DefaultSerializer; +import org.apache.seatunnel.api.serialization.Serializer; +import org.apache.seatunnel.api.sink.DefaultSaveModeHandler; +import org.apache.seatunnel.api.sink.SaveModeHandler; +import org.apache.seatunnel.api.sink.SeaTunnelSink; +import org.apache.seatunnel.api.sink.SinkAggregatedCommitter; +import org.apache.seatunnel.api.sink.SinkWriter; +import org.apache.seatunnel.api.sink.SupportMultiTableSink; +import org.apache.seatunnel.api.sink.SupportSaveMode; +import org.apache.seatunnel.api.table.catalog.Catalog; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.ConstraintKey; +import org.apache.seatunnel.api.table.catalog.PhysicalColumn; +import org.apache.seatunnel.api.table.catalog.PrimaryKey; +import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.factory.CatalogFactory; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.common.constants.PluginType; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.SinkConfig; +import org.apache.seatunnel.connectors.seatunnel.iceberg.exception.IcebergConnectorException; +import org.apache.seatunnel.connectors.seatunnel.iceberg.sink.commit.IcebergAggregatedCommitInfo; +import org.apache.seatunnel.connectors.seatunnel.iceberg.sink.commit.IcebergAggregatedCommitter; +import org.apache.seatunnel.connectors.seatunnel.iceberg.sink.commit.IcebergCommitInfo; +import org.apache.seatunnel.connectors.seatunnel.iceberg.sink.state.IcebergSinkState; + +import com.google.auto.service.AutoService; + +import java.io.IOException; +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import java.util.stream.Collectors; + +import static org.apache.seatunnel.api.table.factory.FactoryUtil.discoverFactory; + +@AutoService(SeaTunnelSink.class) +public class IcebergSink + implements SeaTunnelSink< + SeaTunnelRow, + IcebergSinkState, + IcebergCommitInfo, + IcebergAggregatedCommitInfo>, + SupportSaveMode, + SupportMultiTableSink { + private static String PLUGIN_NAME = "Iceberg"; + private SinkConfig config; + private ReadonlyConfig readonlyConfig; + private CatalogTable catalogTable; + + public IcebergSink(ReadonlyConfig pluginConfig, CatalogTable catalogTable) { + this.readonlyConfig = pluginConfig; + this.config = new SinkConfig(pluginConfig); + this.catalogTable = convertLowerCaseCatalogTable(catalogTable); + // Reset primary keys if need + if (config.getPrimaryKeys().isEmpty() + && Objects.nonNull(this.catalogTable.getTableSchema().getPrimaryKey())) { + this.config.setPrimaryKeys( + this.catalogTable.getTableSchema().getPrimaryKey().getColumnNames()); + } + // reset partition keys if need + if (config.getPartitionKeys().isEmpty() + && Objects.nonNull(this.catalogTable.getPartitionKeys())) { + this.config.setPartitionKeys(this.catalogTable.getPartitionKeys()); + } + } + + @Override + public String getPluginName() { + return PLUGIN_NAME; + } + + @Override + public SinkWriter createWriter( + SinkWriter.Context context) throws IOException { + return IcebergSinkWriter.of(config, catalogTable); + } + + @Override + public SinkWriter restoreWriter( + SinkWriter.Context context, List states) throws IOException { + return IcebergSinkWriter.of(config, catalogTable, states); + } + + @Override + public Optional> + createAggregatedCommitter() throws IOException { + return Optional.of(new IcebergAggregatedCommitter(config, catalogTable)); + } + + @Override + public Optional> getAggregatedCommitInfoSerializer() { + return Optional.of(new DefaultSerializer<>()); + } + + @Override + public Optional> getCommitInfoSerializer() { + return Optional.of(new DefaultSerializer<>()); + } + + @Override + public Optional getSaveModeHandler() { + CatalogFactory catalogFactory = + discoverFactory( + Thread.currentThread().getContextClassLoader(), + CatalogFactory.class, + "Iceberg"); + if (catalogFactory == null) { + throw new IcebergConnectorException( + SeaTunnelAPIErrorCode.CONFIG_VALIDATION_FAILED, + String.format( + "PluginName: %s, PluginType: %s, Message: %s", + getPluginName(), PluginType.SINK, "Cannot find Doris catalog factory")); + } + Catalog catalog = + catalogFactory.createCatalog(catalogFactory.factoryIdentifier(), readonlyConfig); + catalog.open(); + return Optional.of( + new DefaultSaveModeHandler( + config.getSchemaSaveMode(), + config.getDataSaveMode(), + catalog, + catalogTable, + null)); + } + + private CatalogTable convertLowerCaseCatalogTable(CatalogTable catalogTable) { + TableSchema tableSchema = catalogTable.getTableSchema(); + TableSchema.Builder builder = TableSchema.builder(); + tableSchema + .getColumns() + .forEach( + column -> { + PhysicalColumn physicalColumn = + PhysicalColumn.of( + column.getName(), + column.getDataType(), + column.getColumnLength(), + column.isNullable(), + column.getDefaultValue(), + column.getComment()); + builder.column(physicalColumn); + }); + // set primary + if (Objects.nonNull(tableSchema.getPrimaryKey())) { + PrimaryKey newPrimaryKey = + PrimaryKey.of( + tableSchema.getPrimaryKey().getPrimaryKey(), + tableSchema.getPrimaryKey().getColumnNames().stream() + .map(String::toLowerCase) + .collect(Collectors.toList())); + builder.primaryKey(newPrimaryKey); + } + + if (Objects.nonNull(tableSchema.getConstraintKeys())) { + tableSchema + .getConstraintKeys() + .forEach( + constraintKey -> { + ConstraintKey newConstraintKey = + ConstraintKey.of( + constraintKey.getConstraintType(), + constraintKey.getConstraintName(), + constraintKey.getColumnNames() != null + ? constraintKey.getColumnNames().stream() + .map( + constraintKeyColumn -> + ConstraintKey + .ConstraintKeyColumn + .of( + constraintKeyColumn + .getColumnName() + != null + ? constraintKeyColumn + .getColumnName() + .toLowerCase() + : null, + constraintKeyColumn + .getSortType())) + .collect(Collectors.toList()) + : null); + builder.constraintKey(newConstraintKey); + }); + } + + return CatalogTable.of( + catalogTable.getTableId(), + builder.build(), + catalogTable.getOptions(), + catalogTable.getPartitionKeys(), + catalogTable.getComment(), + catalogTable.getCatalogName()); + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSinkFactory.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSinkFactory.java new file mode 100644 index 000000000000..3c30c38e0daa --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSinkFactory.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.iceberg.sink; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.TableIdentifier; +import org.apache.seatunnel.api.table.connector.TableSink; +import org.apache.seatunnel.api.table.factory.Factory; +import org.apache.seatunnel.api.table.factory.TableSinkFactory; +import org.apache.seatunnel.api.table.factory.TableSinkFactoryContext; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.CommonConfig; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.SinkConfig; + +import org.apache.commons.lang3.StringUtils; + +import com.google.auto.service.AutoService; + +@AutoService(Factory.class) +public class IcebergSinkFactory implements TableSinkFactory { + + public static final String REPLACE_TABLE_NAME_KEY = "${table_name}"; + + public static final String REPLACE_SCHEMA_NAME_KEY = "${schema_name}"; + + public static final String REPLACE_DATABASE_NAME_KEY = "${database_name}"; + + @Override + public String factoryIdentifier() { + return "Iceberg"; + } + + @Override + public OptionRule optionRule() { + return OptionRule.builder() + .required( + CommonConfig.KEY_CATALOG_NAME, + SinkConfig.KEY_NAMESPACE, + SinkConfig.KEY_TABLE, + SinkConfig.CATALOG_PROPS) + .optional( + SinkConfig.TABLE_PROPS, + SinkConfig.HADOOP_PROPS, + SinkConfig.WRITE_PROPS, + SinkConfig.AUTO_CREATE_PROPS, + SinkConfig.TABLE_PRIMARY_KEYS, + SinkConfig.TABLE_DEFAULT_PARTITION_KEYS, + SinkConfig.TABLE_UPSERT_MODE_ENABLED_PROP, + SinkConfig.TABLE_SCHEMA_EVOLUTION_ENABLED_PROP) + .build(); + } + + @Override + public TableSink createSink(TableSinkFactoryContext context) { + ReadonlyConfig config = context.getOptions(); + CatalogTable catalogTable = + renameCatalogTable(new SinkConfig(config), context.getCatalogTable()); + return () -> new IcebergSink(config, catalogTable); + } + + private CatalogTable renameCatalogTable(SinkConfig sinkConfig, CatalogTable catalogTable) { + TableIdentifier tableId = catalogTable.getTableId(); + String tableName; + String namespace; + if (StringUtils.isNotEmpty(sinkConfig.getTable())) { + tableName = replaceName(sinkConfig.getTable(), tableId); + } else { + tableName = tableId.getTableName(); + } + + if (StringUtils.isNotEmpty(sinkConfig.getNamespace())) { + namespace = replaceName(sinkConfig.getNamespace(), tableId); + } else { + namespace = tableId.getSchemaName(); + } + + TableIdentifier newTableId = + TableIdentifier.of( + tableId.getCatalogName(), namespace, tableId.getSchemaName(), tableName); + + return CatalogTable.of(newTableId, catalogTable); + } + + private String replaceName(String original, TableIdentifier tableId) { + if (tableId.getTableName() != null) { + original = original.replace(REPLACE_TABLE_NAME_KEY, tableId.getTableName()); + } + if (tableId.getSchemaName() != null) { + original = original.replace(REPLACE_SCHEMA_NAME_KEY, tableId.getSchemaName()); + } + if (tableId.getDatabaseName() != null) { + original = original.replace(REPLACE_DATABASE_NAME_KEY, tableId.getDatabaseName()); + } + return original; + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSinkWriter.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSinkWriter.java new file mode 100644 index 000000000000..aed6522ca877 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSinkWriter.java @@ -0,0 +1,160 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.iceberg.sink; + +import org.apache.seatunnel.api.sink.SinkWriter; +import org.apache.seatunnel.api.sink.SupportMultiTableSinkWriter; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.event.SchemaChangeEvent; +import org.apache.seatunnel.api.table.event.handler.DataTypeChangeEventDispatcher; +import org.apache.seatunnel.api.table.event.handler.DataTypeChangeEventHandler; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.seatunnel.iceberg.IcebergTableLoader; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.SinkConfig; +import org.apache.seatunnel.connectors.seatunnel.iceberg.sink.commit.IcebergCommitInfo; +import org.apache.seatunnel.connectors.seatunnel.iceberg.sink.commit.IcebergFilesCommitter; +import org.apache.seatunnel.connectors.seatunnel.iceberg.sink.state.IcebergSinkState; +import org.apache.seatunnel.connectors.seatunnel.iceberg.sink.writer.IcebergWriterFactory; +import org.apache.seatunnel.connectors.seatunnel.iceberg.sink.writer.RecordWriter; +import org.apache.seatunnel.connectors.seatunnel.iceberg.sink.writer.WriteResult; + +import org.apache.commons.lang3.StringUtils; + +import com.google.common.collect.Lists; +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import java.util.UUID; + +/** Iceberg sink writer */ +@Slf4j +public class IcebergSinkWriter + implements SinkWriter, + SupportMultiTableSinkWriter { + private SeaTunnelRowType rowType; + private SinkConfig config; + private IcebergTableLoader icebergTableLoader; + private RecordWriter writer; + private IcebergFilesCommitter filesCommitter; + private List results = Lists.newArrayList(); + private String commitUser = UUID.randomUUID().toString(); + private long checkpointId; + + private final DataTypeChangeEventHandler dataTypeChangeEventHandler; + + public IcebergSinkWriter( + IcebergTableLoader icebergTableLoader, + SinkConfig config, + SeaTunnelRowType seaTunnelRowType, + List states) { + this.config = config; + this.icebergTableLoader = icebergTableLoader; + this.rowType = seaTunnelRowType; + this.filesCommitter = IcebergFilesCommitter.of(config, icebergTableLoader); + this.dataTypeChangeEventHandler = new DataTypeChangeEventDispatcher(); + tryCreateRecordWriter(); + if (Objects.nonNull(states) && !states.isEmpty()) { + this.commitUser = states.get(0).getCommitUser(); + this.checkpointId = states.get(0).getCheckpointId(); + preCommit(states); + } + } + + private void preCommit(List states) { + states.forEach( + icebergSinkState -> { + filesCommitter.doCommit(icebergSinkState.getWriteResults()); + }); + } + + private void tryCreateRecordWriter() { + if (this.writer == null) { + IcebergWriterFactory icebergWriterFactory = + new IcebergWriterFactory(icebergTableLoader, config); + this.writer = icebergWriterFactory.createWriter(this.rowType); + } + } + + public static IcebergSinkWriter of(SinkConfig config, CatalogTable catalogTable) { + return of(config, catalogTable, null); + } + + public static IcebergSinkWriter of( + SinkConfig config, CatalogTable catalogTable, List states) { + IcebergTableLoader icebergTableLoader = + IcebergTableLoader.create(config, catalogTable).open(); + return new IcebergSinkWriter( + icebergTableLoader, config, catalogTable.getSeaTunnelRowType(), states); + } + + @Override + public void write(SeaTunnelRow element) throws IOException { + tryCreateRecordWriter(); + writer.write(element, rowType); + } + + @Override + public Optional prepareCommit() throws IOException { + List writeResults = writer.complete(); + IcebergCommitInfo icebergCommitInfo = new IcebergCommitInfo(writeResults); + this.results.addAll(writeResults); + return Optional.of(icebergCommitInfo); + } + + @Override + public void applySchemaChange(SchemaChangeEvent event) throws IOException { + // Waiting cdc connector support schema change event + if (config.isTableSchemaEvolutionEnabled()) { + log.info("changed rowType before: {}", fieldsInfo(rowType)); + this.rowType = dataTypeChangeEventHandler.reset(rowType).apply(event); + log.info("changed rowType after: {}", fieldsInfo(rowType)); + writer.applySchemaChange(this.rowType, event); + } + } + + @Override + public List snapshotState(long checkpointId) throws IOException { + IcebergSinkState icebergSinkState = new IcebergSinkState(results, commitUser, checkpointId); + results.clear(); + return Collections.singletonList(icebergSinkState); + } + + @Override + public void abortPrepare() {} + + @Override + public void close() throws IOException {} + + private String fieldsInfo(SeaTunnelRowType seaTunnelRowType) { + String[] fieldsInfo = new String[seaTunnelRowType.getTotalFields()]; + for (int i = 0; i < seaTunnelRowType.getTotalFields(); i++) { + fieldsInfo[i] = + String.format( + "%s<%s>", + seaTunnelRowType.getFieldName(i), seaTunnelRowType.getFieldType(i)); + } + return StringUtils.join(fieldsInfo, ", "); + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/commit/IcebergAggregatedCommitInfo.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/commit/IcebergAggregatedCommitInfo.java new file mode 100644 index 000000000000..5ba75d3d3604 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/commit/IcebergAggregatedCommitInfo.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.iceberg.sink.commit; + +import lombok.AllArgsConstructor; +import lombok.Data; + +import java.io.Serializable; +import java.util.List; + +@Data +@AllArgsConstructor +public class IcebergAggregatedCommitInfo implements Serializable { + List commitInfos; +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/commit/IcebergAggregatedCommitter.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/commit/IcebergAggregatedCommitter.java new file mode 100644 index 000000000000..dcd1e6201f2b --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/commit/IcebergAggregatedCommitter.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.iceberg.sink.commit; + +import org.apache.seatunnel.api.sink.SinkAggregatedCommitter; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.connectors.seatunnel.iceberg.IcebergTableLoader; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.SinkConfig; + +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; + +/** Iceberg aggregated committer */ +@Slf4j +public class IcebergAggregatedCommitter + implements SinkAggregatedCommitter { + + private final IcebergFilesCommitter filesCommitter; + + public IcebergAggregatedCommitter(SinkConfig config, CatalogTable catalogTable) { + IcebergTableLoader tableLoader = IcebergTableLoader.create(config, catalogTable).open(); + this.filesCommitter = IcebergFilesCommitter.of(config, tableLoader); + } + + @Override + public List commit( + List aggregatedCommitInfo) throws IOException { + for (IcebergAggregatedCommitInfo commitInfo : aggregatedCommitInfo) { + commitFiles(commitInfo.commitInfos); + } + return Collections.emptyList(); + } + + private void commitFiles(List commitInfos) { + for (IcebergCommitInfo icebergCommitInfo : commitInfos) { + if (icebergCommitInfo.getResults() == null) { + continue; + } + filesCommitter.doCommit(icebergCommitInfo.getResults()); + } + } + + @Override + public IcebergAggregatedCommitInfo combine(List commitInfos) { + return new IcebergAggregatedCommitInfo(commitInfos); + } + + @Override + public void abort(List aggregatedCommitInfo) throws Exception {} + + @Override + public void close() throws IOException {} +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/commit/IcebergCommitInfo.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/commit/IcebergCommitInfo.java new file mode 100644 index 000000000000..1b4affc0b9e6 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/commit/IcebergCommitInfo.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.iceberg.sink.commit; + +import org.apache.seatunnel.connectors.seatunnel.iceberg.sink.writer.WriteResult; + +import lombok.AllArgsConstructor; +import lombok.Data; + +import java.io.Serializable; +import java.util.List; + +@Data +@AllArgsConstructor +public class IcebergCommitInfo implements Serializable { + private List results; +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/commit/IcebergFilesCommitter.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/commit/IcebergFilesCommitter.java new file mode 100644 index 000000000000..07363d69e1a0 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/commit/IcebergFilesCommitter.java @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.iceberg.sink.commit; + +import org.apache.seatunnel.connectors.seatunnel.iceberg.IcebergTableLoader; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.SinkConfig; +import org.apache.seatunnel.connectors.seatunnel.iceberg.sink.writer.WriteResult; + +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; + +import lombok.extern.slf4j.Slf4j; + +import java.io.Serializable; +import java.util.List; + +import static java.util.stream.Collectors.toList; + +@Slf4j +public class IcebergFilesCommitter implements Serializable { + private IcebergTableLoader icebergTableLoader; + private boolean caseSensitive; + + private IcebergFilesCommitter(SinkConfig config, IcebergTableLoader icebergTableLoader) { + this.icebergTableLoader = icebergTableLoader; + this.caseSensitive = config.isCaseSensitive(); + } + + public static IcebergFilesCommitter of( + SinkConfig config, IcebergTableLoader icebergTableLoader) { + return new IcebergFilesCommitter(config, icebergTableLoader); + } + + public void doCommit(List results) { + TableIdentifier tableIdentifier = icebergTableLoader.getTableIdentifier(); + Table table = icebergTableLoader.loadTable(); + log.info("do commit table : " + table.toString()); + commit(tableIdentifier, table, results); + } + + private void commit(TableIdentifier tableIdentifier, Table table, List results) { + List dataFiles = + results.stream() + .filter(payload -> payload.getDataFiles() != null) + .flatMap(payload -> payload.getDataFiles().stream()) + .filter(dataFile -> dataFile.recordCount() > 0) + .collect(toList()); + + List deleteFiles = + results.stream() + .filter(payload -> payload.getDeleteFiles() != null) + .flatMap(payload -> payload.getDeleteFiles().stream()) + .filter(deleteFile -> deleteFile.recordCount() > 0) + .collect(toList()); + + if (dataFiles.isEmpty() && deleteFiles.isEmpty()) { + log.info(String.format("Nothing to commit to table %s, skipping", tableIdentifier)); + } else { + if (deleteFiles.isEmpty()) { + AppendFiles append = table.newAppend(); + dataFiles.forEach(append::appendFile); + append.commit(); + } else { + RowDelta delta = table.newRowDelta(); + delta.caseSensitive(caseSensitive); + dataFiles.forEach(delta::addRows); + deleteFiles.forEach(delta::addDeletes); + delta.commit(); + } + } + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/schema/ISchemaChange.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/schema/ISchemaChange.java new file mode 100644 index 000000000000..047b726ac0e4 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/schema/ISchemaChange.java @@ -0,0 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.iceberg.sink.schema; + +public interface ISchemaChange {} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/schema/SchemaAddColumn.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/schema/SchemaAddColumn.java new file mode 100644 index 000000000000..c36b5bcf22c7 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/schema/SchemaAddColumn.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.iceberg.sink.schema; + +import org.apache.iceberg.types.Type; + +public class SchemaAddColumn implements ISchemaChange { + private final String parentName; + private final String name; + private final Type type; + + public SchemaAddColumn(String parentName, String name, Type type) { + this.parentName = parentName; + this.name = name; + this.type = type; + } + + public String parentName() { + return parentName; + } + + public String name() { + return name; + } + + public String key() { + return parentName == null ? name : parentName + "." + name; + } + + public Type type() { + return type; + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/schema/SchemaChangeColumn.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/schema/SchemaChangeColumn.java new file mode 100644 index 000000000000..efa1a6cf53d1 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/schema/SchemaChangeColumn.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.iceberg.sink.schema; + +/** Rename column name */ +public class SchemaChangeColumn implements ISchemaChange { + private final String oldName; + private final String newName; + + public SchemaChangeColumn(String oldName, String newName) { + this.oldName = oldName; + this.newName = newName; + } + + public String oldName() { + return oldName; + } + + public String newName() { + return newName; + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/schema/SchemaChangeWrapper.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/schema/SchemaChangeWrapper.java new file mode 100644 index 000000000000..2391920bbe1d --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/schema/SchemaChangeWrapper.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.iceberg.sink.schema; + +import org.apache.seatunnel.shade.com.google.common.collect.Maps; + +import org.apache.iceberg.types.Type; + +import java.util.Collection; +import java.util.Map; + +public class SchemaChangeWrapper { + private final Map addColumns = Maps.newHashMap(); + private final Map deleteColumns = Maps.newHashMap(); + private final Map modifyColumns = Maps.newHashMap(); + private final Map changeColumns = Maps.newHashMap(); + + public Collection addColumns() { + return addColumns.values(); + } + + public Collection modifyColumns() { + return modifyColumns.values(); + } + + public Collection deleteColumns() { + return deleteColumns.values(); + } + + public Collection changeColumns() { + return changeColumns.values(); + } + + public boolean empty() { + return addColumns.isEmpty() + && modifyColumns.isEmpty() + && deleteColumns.isEmpty() + && changeColumns.isEmpty(); + } + + public void addColumn(String parentName, String name, Type type) { + SchemaAddColumn addCol = new SchemaAddColumn(parentName, name, type); + addColumns.put(addCol.key(), addCol); + } + + public void modifyColumn(String name, Type.PrimitiveType type) { + modifyColumns.put(name, new SchemaModifyColumn(name, type)); + } + + public void deleteColumn(String name) { + deleteColumns.put(name, new SchemaDeleteColumn(name)); + } + + public void changeColumn(String oldName, String newName) { + changeColumns.put(newName, new SchemaChangeColumn(oldName, newName)); + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/schema/SchemaDeleteColumn.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/schema/SchemaDeleteColumn.java new file mode 100644 index 000000000000..cc4b3bdee138 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/schema/SchemaDeleteColumn.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.iceberg.sink.schema; + +public class SchemaDeleteColumn implements ISchemaChange { + private final String name; + + public SchemaDeleteColumn(String name) { + this.name = name; + } + + public String name() { + return name; + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/schema/SchemaModifyColumn.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/schema/SchemaModifyColumn.java new file mode 100644 index 000000000000..0f0351e72312 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/schema/SchemaModifyColumn.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.iceberg.sink.schema; + +import org.apache.iceberg.types.Type; + +/** Modify column type */ +public class SchemaModifyColumn implements ISchemaChange { + private final String name; + private final Type.PrimitiveType type; + + public SchemaModifyColumn(String name, Type.PrimitiveType type) { + this.name = name; + this.type = type; + } + + public String name() { + return name; + } + + public Type.PrimitiveType type() { + return type; + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/state/IcebergSinkState.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/state/IcebergSinkState.java new file mode 100644 index 000000000000..1cd69130eb78 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/state/IcebergSinkState.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.iceberg.sink.state; + +import org.apache.seatunnel.connectors.seatunnel.iceberg.sink.writer.WriteResult; + +import lombok.AllArgsConstructor; +import lombok.Data; + +import java.io.Serializable; +import java.util.List; + +@Data +@AllArgsConstructor +public class IcebergSinkState implements Serializable { + private static final long serialVersionUID = 1L; + private List writeResults; + private String commitUser; + private long checkpointId; +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/BaseDeltaTaskWriter.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/BaseDeltaTaskWriter.java new file mode 100644 index 000000000000..c79e8d705c38 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/BaseDeltaTaskWriter.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.iceberg.sink.writer; + +import org.apache.seatunnel.shade.com.google.common.collect.Sets; + +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.data.InternalRecordWrapper; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.io.BaseTaskWriter; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.types.TypeUtil; + +import java.io.IOException; +import java.util.Set; + +abstract class BaseDeltaTaskWriter extends BaseTaskWriter { + + private final Schema schema; + private final Schema deleteSchema; + + private final InternalRecordWrapper wrapper; + private final InternalRecordWrapper keyWrapper; + private final RecordProjection keyProjection; + + private final boolean upsert; + + BaseDeltaTaskWriter( + PartitionSpec spec, + FileFormat format, + FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, + FileIO io, + long targetFileSize, + Schema schema, + Set identifierFieldIds, + boolean upsert) { + super(spec, format, appenderFactory, fileFactory, io, targetFileSize); + this.schema = schema; + this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(identifierFieldIds)); + this.wrapper = new InternalRecordWrapper(schema.asStruct()); + this.keyWrapper = new InternalRecordWrapper(deleteSchema.asStruct()); + this.keyProjection = RecordProjection.create(schema, deleteSchema); + this.upsert = upsert; + } + + abstract RowDataDeltaWriter route(IcebergRecord row); + + InternalRecordWrapper wrapper() { + return wrapper; + } + + @Override + public void write(Record record) throws IOException { + + if (!(record instanceof IcebergRecord)) { + throw new RuntimeException(); + } + IcebergRecord row = (IcebergRecord) record; + RowDataDeltaWriter writer = route(row); + switch (row.getRowKind()) { + case INSERT: + case UPDATE_AFTER: + if (upsert) { + writer.deleteKey(keyProjection.wrap(row)); + } + writer.write(row); + break; + case UPDATE_BEFORE: + if (upsert) { + break; + } + writer.delete(row); + break; + case DELETE: + if (upsert) { + writer.deleteKey(keyProjection.wrap(row)); + } else { + writer.delete(row); + } + break; + + default: + throw new UnsupportedOperationException("Unknown row kind: " + row.getRowKind()); + } + } + + class RowDataDeltaWriter extends BaseEqualityDeltaWriter { + RowDataDeltaWriter(PartitionKey partition) { + super(partition, schema, deleteSchema); + } + + @Override + protected StructLike asStructLike(Record data) { + return wrapper.wrap(data); + } + + @Override + protected StructLike asStructLikeKey(Record data) { + return keyWrapper.wrap(data); + } + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergRecord.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergRecord.java new file mode 100644 index 000000000000..163bf118c768 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergRecord.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.iceberg.sink.writer; + +import org.apache.seatunnel.api.table.type.RowKind; + +import org.apache.iceberg.data.Record; +import org.apache.iceberg.types.Types; + +import java.util.Map; + +public class IcebergRecord implements Record { + + private final Record delegate; + private final RowKind rowKind; + + public IcebergRecord(Record delegate, RowKind rowKind) { + this.delegate = delegate; + this.rowKind = rowKind; + } + + public RowKind getRowKind() { + return rowKind; + } + + @Override + public Types.StructType struct() { + return delegate.struct(); + } + + @Override + public Object getField(String name) { + return delegate.getField(name); + } + + @Override + public void setField(String name, Object value) { + delegate.setField(name, value); + } + + @Override + public Object get(int pos) { + return delegate.get(pos); + } + + @Override + public Record copy() { + return new IcebergRecord(delegate.copy(), rowKind); + } + + @Override + public Record copy(Map overwriteValues) { + return new IcebergRecord(delegate.copy(overwriteValues), rowKind); + } + + @Override + public int size() { + return delegate.size(); + } + + @Override + public T get(int pos, Class javaClass) { + return delegate.get(pos, javaClass); + } + + @Override + public void set(int pos, T value) { + delegate.set(pos, value); + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergRecordWriter.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergRecordWriter.java new file mode 100644 index 000000000000..ad3f3c483c6a --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergRecordWriter.java @@ -0,0 +1,189 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.iceberg.sink.writer; + +import org.apache.seatunnel.shade.com.google.common.collect.Lists; + +import org.apache.seatunnel.api.table.catalog.Column; +import org.apache.seatunnel.api.table.event.AlterTableAddColumnEvent; +import org.apache.seatunnel.api.table.event.AlterTableChangeColumnEvent; +import org.apache.seatunnel.api.table.event.AlterTableDropColumnEvent; +import org.apache.seatunnel.api.table.event.AlterTableModifyColumnEvent; +import org.apache.seatunnel.api.table.event.SchemaChangeEvent; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.SinkConfig; +import org.apache.seatunnel.connectors.seatunnel.iceberg.data.RowConverter; +import org.apache.seatunnel.connectors.seatunnel.iceberg.sink.schema.SchemaChangeWrapper; +import org.apache.seatunnel.connectors.seatunnel.iceberg.utils.SchemaUtils; + +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Arrays; +import java.util.List; + +@Slf4j +public class IcebergRecordWriter implements RecordWriter { + private final Table table; + private final SinkConfig config; + private final List writerResults; + private TaskWriter writer; + private RowConverter recordConverter; + private IcebergWriterFactory writerFactory; + + public IcebergRecordWriter(Table table, IcebergWriterFactory writerFactory, SinkConfig config) { + this.config = config; + this.table = table; + this.writerResults = Lists.newArrayList(); + this.recordConverter = new RowConverter(table, config); + this.writerFactory = writerFactory; + this.writer = createTaskWriter(); + } + + private TaskWriter createTaskWriter() { + return writerFactory.createTaskWriter(table, config); + } + + @Override + public void write(SeaTunnelRow seaTunnelRow, SeaTunnelRowType rowType) { + SchemaChangeWrapper updates = new SchemaChangeWrapper(); + Record record = recordConverter.convert(seaTunnelRow, rowType, updates); + if (!updates.empty()) { + // Apply for schema update + applySchemaUpdate(updates); + // convert the row again, this time using the new table schema + record = recordConverter.convert(seaTunnelRow, rowType); + } + IcebergRecord icebergRecord = new IcebergRecord(record, seaTunnelRow.getRowKind()); + try { + this.writer.write(icebergRecord); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public void applySchemaChange(SeaTunnelRowType afterRowType, SchemaChangeEvent event) { + log.info("Apply schema change start."); + SchemaChangeWrapper updates = new SchemaChangeWrapper(); + Schema schema = table.schema(); + if (event instanceof AlterTableDropColumnEvent) { + AlterTableDropColumnEvent dropColumnEvent = (AlterTableDropColumnEvent) event; + updates.deleteColumn(dropColumnEvent.getColumn()); + } else if (event instanceof AlterTableAddColumnEvent) { + // Change during data consumption process + } else if (event instanceof AlterTableChangeColumnEvent) { + AlterTableChangeColumnEvent changeColumnEvent = (AlterTableChangeColumnEvent) event; + changeColumn( + schema, + changeColumnEvent.getColumn(), + changeColumnEvent.getOldColumn(), + updates); + } else if (event instanceof AlterTableModifyColumnEvent) { + AlterTableModifyColumnEvent modifyColumnEvent = (AlterTableModifyColumnEvent) event; + modifyColumn(schema, modifyColumnEvent.getColumn(), updates); + } + if (!updates.empty()) { + applySchemaUpdate(updates); + } + log.info("Apply schema change end."); + } + + private void changeColumn( + Schema schema, Column column, String oldColumn, SchemaChangeWrapper updates) { + Types.NestedField nestedField = schema.findField(oldColumn); + if (nestedField != null) { + updates.changeColumn(oldColumn, column.getName()); + } + } + + private static void modifyColumn( + Schema schema, Column changeColumn, SchemaChangeWrapper updates) { + Types.NestedField nestedField = schema.findField(changeColumn.getName()); + if (nestedField != null) { + Type type = SchemaUtils.toIcebergType(changeColumn.getDataType()); + if (nestedField.type() != type && type.isPrimitiveType()) { + updates.modifyColumn(changeColumn.getName(), (Type.PrimitiveType) type); + } else { + log.warn("Only PrimitiveType updates are supported, {}", type); + } + } + } + + /** + * apply schema update + * + * @param updates + * @return + */ + private void applySchemaUpdate(SchemaChangeWrapper updates) { + // complete the current file + flush(); + // apply the schema updates, this will refresh the table + SchemaUtils.applySchemaUpdates(table, updates); + // initialize a new writer with the new schema + resetWriter(); + } + + @Override + public List complete() { + flush(); + List result = Lists.newArrayList(writerResults); + writerResults.clear(); + resetWriter(); + return result; + } + + /** Reset record writer */ + private void resetWriter() { + this.writer = createTaskWriter(); + this.recordConverter = new RowConverter(table, config); + } + + private void flush() { + if (writer == null) { + return; + } + org.apache.iceberg.io.WriteResult writeResult; + try { + writeResult = writer.complete(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + writerResults.add( + new WriteResult( + Arrays.asList(writeResult.dataFiles()), + Arrays.asList(writeResult.deleteFiles()), + table.spec().partitionType())); + writer = null; + } + + @Override + public void close() {} +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergWriterFactory.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergWriterFactory.java new file mode 100644 index 000000000000..6aa729b1153f --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergWriterFactory.java @@ -0,0 +1,197 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.iceberg.sink.writer; + +import org.apache.seatunnel.shade.com.google.common.collect.Maps; +import org.apache.seatunnel.shade.com.google.common.collect.Sets; +import org.apache.seatunnel.shade.com.google.common.primitives.Ints; + +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.seatunnel.iceberg.IcebergTableLoader; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.SinkConfig; +import org.apache.seatunnel.connectors.seatunnel.iceberg.utils.SchemaUtils; + +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.io.UnpartitionedWriter; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.PropertyUtil; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import lombok.extern.slf4j.Slf4j; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; + +import static java.util.stream.Collectors.toSet; +import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; +import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT; +import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES; +import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT; + +@Slf4j +public class IcebergWriterFactory { + private static final Logger LOG = LoggerFactory.getLogger(IcebergWriterFactory.class); + private final IcebergTableLoader tableLoader; + private final SinkConfig config; + + public IcebergWriterFactory(IcebergTableLoader tableLoader, SinkConfig config) { + this.tableLoader = tableLoader; + this.config = config; + } + + public RecordWriter createWriter(SeaTunnelRowType rowType) { + Table table; + try { + table = tableLoader.loadTable(); + } catch (NoSuchTableException exception) { + // for e2e test , Normally, IcebergCatalog should be used to create a table + switch (config.getSchemaSaveMode()) { + case CREATE_SCHEMA_WHEN_NOT_EXIST: + table = + SchemaUtils.autoCreateTable( + tableLoader.getCatalog(), + tableLoader.getTableIdentifier(), + config, + rowType); + break; + default: + throw exception; + } + } + return new IcebergRecordWriter(table, this, config); + } + + public TaskWriter createTaskWriter(Table table, SinkConfig config) { + Map tableProps = Maps.newHashMap(table.properties()); + tableProps.putAll(config.getWriteProps()); + + String formatStr = + tableProps.getOrDefault(DEFAULT_FILE_FORMAT, DEFAULT_FILE_FORMAT_DEFAULT); + FileFormat format = FileFormat.valueOf(formatStr.toUpperCase()); + + long targetFileSize = + PropertyUtil.propertyAsLong( + tableProps, + WRITE_TARGET_FILE_SIZE_BYTES, + WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT); + + Set identifierFieldIds = table.schema().identifierFieldIds(); + + // override the identifier fields if the config is set + List idCols = config.getPrimaryKeys(); + if (!idCols.isEmpty()) { + identifierFieldIds = + idCols.stream() + .map(colName -> table.schema().findField(colName).fieldId()) + .collect(toSet()); + } + + FileAppenderFactory appenderFactory; + if (identifierFieldIds == null || identifierFieldIds.isEmpty()) { + appenderFactory = + new GenericAppenderFactory(table.schema(), table.spec(), null, null, null) + .setAll(tableProps); + } else { + appenderFactory = + new GenericAppenderFactory( + table.schema(), + table.spec(), + Ints.toArray(identifierFieldIds), + TypeUtil.select( + table.schema(), Sets.newHashSet(identifierFieldIds)), + null) + .setAll(tableProps); + } + + // (partition ID + task ID + operation ID) must be unique + OutputFileFactory fileFactory = + OutputFileFactory.builderFor(table, 1, System.currentTimeMillis()) + .defaultSpec(table.spec()) + .operationId(UUID.randomUUID().toString()) + .format(format) + .build(); + + TaskWriter writer; + if (table.spec().isUnpartitioned()) { + if (identifierFieldIds.isEmpty() && !config.isUpsertModeEnabled()) { + // No delta writer + writer = + new UnpartitionedWriter<>( + table.spec(), + format, + appenderFactory, + fileFactory, + table.io(), + targetFileSize); + } else { + // Delta writer + writer = + new UnpartitionedDeltaWriter( + table.spec(), + format, + appenderFactory, + fileFactory, + table.io(), + targetFileSize, + table.schema(), + identifierFieldIds, + config.isUpsertModeEnabled()); + } + } else { + if (identifierFieldIds.isEmpty() && !config.isUpsertModeEnabled()) { + // No delta writer + writer = + new PartitionedAppendWriter( + table.spec(), + format, + appenderFactory, + fileFactory, + table.io(), + targetFileSize, + table.schema()); + } else { + // Delta writer + writer = + new PartitionedDeltaWriter( + table.spec(), + format, + appenderFactory, + fileFactory, + table.io(), + targetFileSize, + table.schema(), + identifierFieldIds, + config.isUpsertModeEnabled()); + } + } + return writer; + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/PartitionedAppendWriter.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/PartitionedAppendWriter.java new file mode 100644 index 000000000000..a7dbcd5c290a --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/PartitionedAppendWriter.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.iceberg.sink.writer; + +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.InternalRecordWrapper; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.io.PartitionedFanoutWriter; + +public class PartitionedAppendWriter extends PartitionedFanoutWriter { + + private final PartitionKey partitionKey; + private final InternalRecordWrapper wrapper; + + public PartitionedAppendWriter( + PartitionSpec spec, + FileFormat format, + FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, + FileIO io, + long targetFileSize, + Schema schema) { + super(spec, format, appenderFactory, fileFactory, io, targetFileSize); + this.partitionKey = new PartitionKey(spec, schema); + this.wrapper = new InternalRecordWrapper(schema.asStruct()); + } + + @Override + protected PartitionKey partition(Record row) { + partitionKey.partition(wrapper.wrap(row)); + return partitionKey; + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/PartitionedDeltaWriter.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/PartitionedDeltaWriter.java new file mode 100644 index 000000000000..df90b3bbd4c1 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/PartitionedDeltaWriter.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.seatunnel.connectors.seatunnel.iceberg.sink.writer; + +import org.apache.seatunnel.shade.com.google.common.collect.Maps; + +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.util.Tasks; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Map; +import java.util.Set; + +public class PartitionedDeltaWriter extends BaseDeltaTaskWriter { + private final PartitionKey partitionKey; + + private final Map writers = Maps.newHashMap(); + + public PartitionedDeltaWriter( + PartitionSpec spec, + FileFormat format, + FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, + FileIO io, + long targetFileSize, + Schema schema, + Set identifierFieldIds, + boolean upsertMode) { + super( + spec, + format, + appenderFactory, + fileFactory, + io, + targetFileSize, + schema, + identifierFieldIds, + upsertMode); + this.partitionKey = new PartitionKey(spec, schema); + } + + @Override + RowDataDeltaWriter route(IcebergRecord row) { + partitionKey.partition(wrapper().wrap(row)); + + RowDataDeltaWriter writer = writers.get(partitionKey); + if (writer == null) { + // NOTICE: we need to copy a new partition key here, in case of messing up the keys in + // writers. + PartitionKey copiedKey = partitionKey.copy(); + writer = new RowDataDeltaWriter(copiedKey); + writers.put(copiedKey, writer); + } + + return writer; + } + + @Override + public void close() { + try { + Tasks.foreach(writers.values()) + .throwFailureWhenFinished() + .noRetry() + .run(RowDataDeltaWriter::close, IOException.class); + + writers.clear(); + } catch (IOException e) { + throw new UncheckedIOException("Failed to close equality delta writer", e); + } + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/RecordProjection.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/RecordProjection.java new file mode 100644 index 000000000000..f2c68aed7103 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/RecordProjection.java @@ -0,0 +1,196 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.seatunnel.connectors.seatunnel.iceberg.sink.writer; + +import org.apache.seatunnel.shade.com.google.common.base.Preconditions; + +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.types.Types.ListType; +import org.apache.iceberg.types.Types.MapType; +import org.apache.iceberg.types.Types.NestedField; +import org.apache.iceberg.types.Types.StructType; + +import java.util.List; +import java.util.Map; + +/** + * This is modified from {@link org.apache.iceberg.util.StructProjection} to support record types. + */ +public class RecordProjection implements Record { + + public static RecordProjection create(Schema dataSchema, Schema projectedSchema) { + return new RecordProjection(dataSchema.asStruct(), projectedSchema.asStruct()); + } + + private final StructType type; + private final int[] positionMap; + private final RecordProjection[] nestedProjections; + private IcebergRecord record; + + private RecordProjection(StructType structType, StructType projection) { + this(structType, projection, false); + } + + @SuppressWarnings("checkstyle:CyclomaticComplexity") + private RecordProjection(StructType structType, StructType projection, boolean allowMissing) { + this.type = projection; + this.positionMap = new int[projection.fields().size()]; + this.nestedProjections = new RecordProjection[projection.fields().size()]; + + // set up the projection positions and any nested projections that are needed + List dataFields = structType.fields(); + for (int pos = 0; pos < positionMap.length; pos += 1) { + NestedField projectedField = projection.fields().get(pos); + + boolean found = false; + for (int i = 0; !found && i < dataFields.size(); i += 1) { + NestedField dataField = dataFields.get(i); + if (projectedField.fieldId() == dataField.fieldId()) { + found = true; + positionMap[pos] = i; + switch (projectedField.type().typeId()) { + case STRUCT: + nestedProjections[pos] = + new RecordProjection( + dataField.type().asStructType(), + projectedField.type().asStructType()); + break; + case MAP: + MapType projectedMap = projectedField.type().asMapType(); + MapType originalMap = dataField.type().asMapType(); + + boolean keyProjectable = + !projectedMap.keyType().isNestedType() + || projectedMap.keyType().equals(originalMap.keyType()); + boolean valueProjectable = + !projectedMap.valueType().isNestedType() + || projectedMap + .valueType() + .equals(originalMap.valueType()); + Preconditions.checkArgument( + keyProjectable && valueProjectable, + "Cannot project a partial map key or value struct. Trying to project %s out of %s", + projectedField, + dataField); + + nestedProjections[pos] = null; + break; + case LIST: + ListType projectedList = projectedField.type().asListType(); + ListType originalList = dataField.type().asListType(); + + boolean elementProjectable = + !projectedList.elementType().isNestedType() + || projectedList + .elementType() + .equals(originalList.elementType()); + Preconditions.checkArgument( + elementProjectable, + "Cannot project a partial list element struct. Trying to project %s out of %s", + projectedField, + dataField); + + nestedProjections[pos] = null; + break; + default: + nestedProjections[pos] = null; + } + } + } + + if (!found && projectedField.isOptional() && allowMissing) { + positionMap[pos] = -1; + nestedProjections[pos] = null; + } else if (!found) { + throw new IllegalArgumentException( + String.format("Cannot find field %s in %s", projectedField, structType)); + } + } + } + + public RecordProjection wrap(IcebergRecord newRecord) { + this.record = newRecord; + return this; + } + + @Override + public int size() { + return type.fields().size(); + } + + @Override + public T get(int pos, Class javaClass) { + // struct can be null if wrap is not called first before the get call + // or if a null struct is wrapped. + if (record == null) { + return null; + } + + int recordPos = positionMap[pos]; + if (nestedProjections[pos] != null) { + IcebergRecord nestedStruct = record.get(recordPos, IcebergRecord.class); + if (nestedStruct == null) { + return null; + } + return javaClass.cast(nestedProjections[pos].wrap(nestedStruct)); + } + + if (recordPos != -1) { + return record.get(recordPos, javaClass); + } else { + return null; + } + } + + @Override + public void set(int pos, T value) { + throw new UnsupportedOperationException(); + } + + @Override + public StructType struct() { + return type; + } + + @Override + public Object getField(String name) { + throw new UnsupportedOperationException(); + } + + @Override + public void setField(String name, Object value) { + throw new UnsupportedOperationException(); + } + + @Override + public Object get(int pos) { + return get(pos, Object.class); + } + + @Override + public Record copy() { + throw new UnsupportedOperationException(); + } + + @Override + public Record copy(Map overwriteValues) { + throw new UnsupportedOperationException(); + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/RecordWriter.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/RecordWriter.java new file mode 100644 index 000000000000..8c093db98663 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/RecordWriter.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.seatunnel.connectors.seatunnel.iceberg.sink.writer; + +import org.apache.seatunnel.shade.com.google.common.collect.ImmutableList; + +import org.apache.seatunnel.api.table.event.SchemaChangeEvent; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; + +import java.util.List; + +public interface RecordWriter extends Cloneable { + + void write(SeaTunnelRow seaTunnelRow, SeaTunnelRowType rowType); + + void applySchemaChange(SeaTunnelRowType afterRowType, SchemaChangeEvent event); + + default List complete() { + return ImmutableList.of(); + } + + default void close() {} +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/UnpartitionedDeltaWriter.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/UnpartitionedDeltaWriter.java new file mode 100644 index 000000000000..3b3700ae2bd8 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/UnpartitionedDeltaWriter.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.iceberg.sink.writer; + +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFileFactory; + +import java.io.IOException; +import java.util.Set; + +public class UnpartitionedDeltaWriter extends BaseDeltaTaskWriter { + private final RowDataDeltaWriter writer; + + public UnpartitionedDeltaWriter( + PartitionSpec spec, + FileFormat format, + FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, + FileIO io, + long targetFileSize, + Schema schema, + Set identifierFieldIds, + boolean upsertMode) { + super( + spec, + format, + appenderFactory, + fileFactory, + io, + targetFileSize, + schema, + identifierFieldIds, + upsertMode); + this.writer = new RowDataDeltaWriter(null); + } + + @Override + RowDataDeltaWriter route(IcebergRecord row) { + return writer; + } + + @Override + public void close() throws IOException { + writer.close(); + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/WriteResult.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/WriteResult.java new file mode 100644 index 000000000000..c695eef1e22e --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/WriteResult.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.iceberg.sink.writer; + +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.types.Types.StructType; + +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; + +import java.io.Serializable; +import java.util.List; + +/** Write result */ +@Data +@NoArgsConstructor +@AllArgsConstructor +public class WriteResult implements Serializable { + private List dataFiles; + private List deleteFiles; + private StructType partitionStruct; +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/IcebergSource.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/IcebergSource.java index 9be5303fb820..3d1cde13a999 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/IcebergSource.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/IcebergSource.java @@ -20,13 +20,14 @@ import org.apache.seatunnel.shade.com.typesafe.config.Config; import org.apache.seatunnel.api.common.JobContext; -import org.apache.seatunnel.api.common.PrepareFailException; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.source.SourceReader; import org.apache.seatunnel.api.source.SourceSplitEnumerator; import org.apache.seatunnel.api.source.SupportColumnProjection; import org.apache.seatunnel.api.source.SupportParallelism; +import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; @@ -37,13 +38,13 @@ import org.apache.seatunnel.common.constants.JobMode; import org.apache.seatunnel.connectors.seatunnel.iceberg.IcebergTableLoader; import org.apache.seatunnel.connectors.seatunnel.iceberg.config.SourceConfig; -import org.apache.seatunnel.connectors.seatunnel.iceberg.data.IcebergTypeMapper; import org.apache.seatunnel.connectors.seatunnel.iceberg.source.enumerator.IcebergBatchSplitEnumerator; import org.apache.seatunnel.connectors.seatunnel.iceberg.source.enumerator.IcebergSplitEnumeratorState; import org.apache.seatunnel.connectors.seatunnel.iceberg.source.enumerator.IcebergStreamSplitEnumerator; import org.apache.seatunnel.connectors.seatunnel.iceberg.source.enumerator.scan.IcebergScanContext; import org.apache.seatunnel.connectors.seatunnel.iceberg.source.reader.IcebergSourceReader; import org.apache.seatunnel.connectors.seatunnel.iceberg.source.split.IcebergFileScanTaskSplit; +import org.apache.seatunnel.connectors.seatunnel.iceberg.utils.SchemaUtils; import org.apache.iceberg.Schema; import org.apache.iceberg.types.Types; @@ -52,6 +53,7 @@ import lombok.SneakyThrows; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import static org.apache.seatunnel.shade.com.google.common.base.Preconditions.checkArgument; @@ -70,23 +72,30 @@ public class IcebergSource private Schema projectedSchema; private SeaTunnelRowType seaTunnelRowType; private JobContext jobContext; + private CatalogTable catalogTable; + + public IcebergSource(ReadonlyConfig config, CatalogTable catalogTable) { + this.sourceConfig = SourceConfig.loadConfig(config); + this.tableSchema = loadIcebergSchema(sourceConfig); + this.seaTunnelRowType = loadSeaTunnelRowType(tableSchema, config.toConfig()); + this.projectedSchema = tableSchema.select(seaTunnelRowType.getFieldNames()); + this.catalogTable = catalogTable; + } @Override - public String getPluginName() { - return "Iceberg"; + public List getProducedCatalogTables() { + return Collections.singletonList(catalogTable); } @Override - public void prepare(Config pluginConfig) throws PrepareFailException { - this.sourceConfig = SourceConfig.loadConfig(pluginConfig); - this.tableSchema = loadIcebergSchema(sourceConfig); - this.seaTunnelRowType = loadSeaTunnelRowType(tableSchema, pluginConfig); - this.projectedSchema = tableSchema.select(seaTunnelRowType.getFieldNames()); + public String getPluginName() { + return "Iceberg"; } @SneakyThrows private Schema loadIcebergSchema(SourceConfig sourceConfig) { - try (IcebergTableLoader icebergTableLoader = IcebergTableLoader.create(sourceConfig)) { + try (IcebergTableLoader icebergTableLoader = + IcebergTableLoader.create(sourceConfig, catalogTable)) { icebergTableLoader.open(); return icebergTableLoader.loadTable().schema(); } @@ -97,7 +106,7 @@ private SeaTunnelRowType loadSeaTunnelRowType(Schema tableSchema, Config pluginC List> columnDataTypes = new ArrayList<>(tableSchema.columns().size()); for (Types.NestedField column : tableSchema.columns()) { columnNames.add(column.name()); - columnDataTypes.add(IcebergTypeMapper.mapping(column.name(), column.type())); + columnDataTypes.add(SchemaUtils.toSeaTunnelType(column.name(), column.type())); } SeaTunnelRowType originalRowType = new SeaTunnelRowType( @@ -138,16 +147,16 @@ public void setJobContext(JobContext jobContext) { this.jobContext = jobContext; } - @Override - public SeaTunnelDataType getProducedType() { - return seaTunnelRowType; - } - @Override public SourceReader createReader( SourceReader.Context readerContext) { return new IcebergSourceReader( - readerContext, seaTunnelRowType, tableSchema, projectedSchema, sourceConfig); + readerContext, + seaTunnelRowType, + tableSchema, + projectedSchema, + sourceConfig, + catalogTable); } @Override @@ -159,13 +168,15 @@ public SourceReader createReader( enumeratorContext, IcebergScanContext.scanContext(sourceConfig, projectedSchema), sourceConfig, - null); + null, + catalogTable); } return new IcebergStreamSplitEnumerator( enumeratorContext, IcebergScanContext.streamScanContext(sourceConfig, projectedSchema), sourceConfig, - null); + null, + catalogTable); } @Override @@ -178,12 +189,14 @@ public SourceReader createReader( enumeratorContext, IcebergScanContext.scanContext(sourceConfig, projectedSchema), sourceConfig, - checkpointState); + checkpointState, + catalogTable); } return new IcebergStreamSplitEnumerator( enumeratorContext, IcebergScanContext.streamScanContext(sourceConfig, projectedSchema), sourceConfig, - checkpointState); + checkpointState, + catalogTable); } } diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/IcebergSourceFactory.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/IcebergSourceFactory.java index 1318ab8a9d72..6e7c05c9ab1b 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/IcebergSourceFactory.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/IcebergSourceFactory.java @@ -17,22 +17,31 @@ package org.apache.seatunnel.connectors.seatunnel.iceberg.source; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.source.SourceSplit; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.TableIdentifier; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; +import org.apache.seatunnel.api.table.connector.TableSource; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSourceFactory; +import org.apache.seatunnel.api.table.factory.TableSourceFactoryContext; +import org.apache.seatunnel.connectors.seatunnel.iceberg.catalog.IcebergCatalog; +import org.apache.seatunnel.connectors.seatunnel.iceberg.catalog.IcebergCatalogFactory; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.CommonConfig; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.SinkConfig; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.SourceConfig; import com.google.auto.service.AutoService; +import lombok.extern.slf4j.Slf4j; + +import java.io.Serializable; import static org.apache.seatunnel.connectors.seatunnel.iceberg.config.CommonConfig.KEY_CASE_SENSITIVE; -import static org.apache.seatunnel.connectors.seatunnel.iceberg.config.CommonConfig.KEY_CATALOG_NAME; -import static org.apache.seatunnel.connectors.seatunnel.iceberg.config.CommonConfig.KEY_CATALOG_TYPE; -import static org.apache.seatunnel.connectors.seatunnel.iceberg.config.CommonConfig.KEY_NAMESPACE; -import static org.apache.seatunnel.connectors.seatunnel.iceberg.config.CommonConfig.KEY_TABLE; -import static org.apache.seatunnel.connectors.seatunnel.iceberg.config.CommonConfig.KEY_URI; -import static org.apache.seatunnel.connectors.seatunnel.iceberg.config.CommonConfig.KEY_WAREHOUSE; -import static org.apache.seatunnel.connectors.seatunnel.iceberg.config.IcebergCatalogType.HIVE; import static org.apache.seatunnel.connectors.seatunnel.iceberg.config.SourceConfig.KEY_END_SNAPSHOT_ID; import static org.apache.seatunnel.connectors.seatunnel.iceberg.config.SourceConfig.KEY_START_SNAPSHOT_ID; import static org.apache.seatunnel.connectors.seatunnel.iceberg.config.SourceConfig.KEY_START_SNAPSHOT_TIMESTAMP; @@ -40,6 +49,7 @@ import static org.apache.seatunnel.connectors.seatunnel.iceberg.config.SourceConfig.KEY_USE_SNAPSHOT_ID; import static org.apache.seatunnel.connectors.seatunnel.iceberg.config.SourceConfig.KEY_USE_SNAPSHOT_TIMESTAMP; +@Slf4j @AutoService(Factory.class) public class IcebergSourceFactory implements TableSourceFactory { @@ -52,8 +62,10 @@ public String factoryIdentifier() { public OptionRule optionRule() { return OptionRule.builder() .required( - KEY_CATALOG_NAME, KEY_CATALOG_TYPE, KEY_WAREHOUSE, KEY_NAMESPACE, KEY_TABLE) - .conditional(KEY_CATALOG_TYPE, HIVE, KEY_URI) + CommonConfig.KEY_CATALOG_NAME, + SinkConfig.KEY_NAMESPACE, + SinkConfig.KEY_TABLE, + SinkConfig.CATALOG_PROPS) .optional( TableSchemaOptions.SCHEMA, KEY_CASE_SENSITIVE, @@ -66,6 +78,32 @@ public OptionRule optionRule() { .build(); } + @Override + public + TableSource createSource(TableSourceFactoryContext context) { + ReadonlyConfig options = context.getOptions(); + SourceConfig config = new SourceConfig(options); + TablePath tablePath = TablePath.of(config.getNamespace(), config.getTable()); + CatalogTable catalogTable; + if (options.get(TableSchemaOptions.SCHEMA) != null) { + catalogTable = CatalogTableUtil.buildWithConfig(factoryIdentifier(), options); + TableIdentifier tableIdentifier = + TableIdentifier.of(catalogTable.getCatalogName(), tablePath); + CatalogTable table = CatalogTable.of(tableIdentifier, catalogTable); + return () -> (SeaTunnelSource) new IcebergSource(options, table); + } else { + // build iceberg catalog + IcebergCatalogFactory icebergCatalogFactory = new IcebergCatalogFactory(); + IcebergCatalog catalog = + (IcebergCatalog) + icebergCatalogFactory.createCatalog(factoryIdentifier(), options); + catalog.open(); + catalogTable = catalog.getTable(tablePath); + return () -> + (SeaTunnelSource) new IcebergSource(options, catalogTable); + } + } + @Override public Class getSourceClass() { return IcebergSource.class; diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/AbstractSplitEnumerator.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/AbstractSplitEnumerator.java index 26a971cc2e1d..ef3beacd26e7 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/AbstractSplitEnumerator.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/AbstractSplitEnumerator.java @@ -18,6 +18,7 @@ package org.apache.seatunnel.connectors.seatunnel.iceberg.source.enumerator; import org.apache.seatunnel.api.source.SourceSplitEnumerator; +import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.connectors.seatunnel.iceberg.IcebergTableLoader; import org.apache.seatunnel.connectors.seatunnel.iceberg.config.SourceConfig; import org.apache.seatunnel.connectors.seatunnel.iceberg.source.split.IcebergFileScanTaskSplit; @@ -47,19 +48,22 @@ public abstract class AbstractSplitEnumerator protected IcebergTableLoader icebergTableLoader; @Getter private volatile boolean isOpen = false; + private CatalogTable catalogTable; public AbstractSplitEnumerator( @NonNull SourceSplitEnumerator.Context context, @NonNull SourceConfig sourceConfig, - @NonNull Map> pendingSplits) { + @NonNull Map> pendingSplits, + CatalogTable catalogTable) { this.context = context; this.sourceConfig = sourceConfig; this.pendingSplits = new HashMap<>(pendingSplits); + this.catalogTable = catalogTable; } @Override public void open() { - icebergTableLoader = IcebergTableLoader.create(sourceConfig); + icebergTableLoader = IcebergTableLoader.create(sourceConfig, catalogTable); icebergTableLoader.open(); isOpen = true; } diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/IcebergBatchSplitEnumerator.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/IcebergBatchSplitEnumerator.java index a7378da17d54..b0adfb011c1b 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/IcebergBatchSplitEnumerator.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/IcebergBatchSplitEnumerator.java @@ -18,6 +18,7 @@ package org.apache.seatunnel.connectors.seatunnel.iceberg.source.enumerator; import org.apache.seatunnel.api.source.SourceSplitEnumerator; +import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.connectors.seatunnel.iceberg.config.SourceConfig; import org.apache.seatunnel.connectors.seatunnel.iceberg.source.enumerator.scan.IcebergScanContext; import org.apache.seatunnel.connectors.seatunnel.iceberg.source.enumerator.scan.IcebergScanSplitPlanner; @@ -41,11 +42,13 @@ public IcebergBatchSplitEnumerator( @NonNull SourceSplitEnumerator.Context context, @NonNull IcebergScanContext icebergScanContext, @NonNull SourceConfig sourceConfig, - IcebergSplitEnumeratorState restoreState) { + IcebergSplitEnumeratorState restoreState, + CatalogTable catalogTable) { super( context, sourceConfig, - restoreState != null ? restoreState.getPendingSplits() : Collections.emptyMap()); + restoreState != null ? restoreState.getPendingSplits() : Collections.emptyMap(), + catalogTable); this.icebergScanContext = icebergScanContext; } diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/IcebergStreamSplitEnumerator.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/IcebergStreamSplitEnumerator.java index 59bffc22e0f8..266985a07751 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/IcebergStreamSplitEnumerator.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/IcebergStreamSplitEnumerator.java @@ -18,6 +18,7 @@ package org.apache.seatunnel.connectors.seatunnel.iceberg.source.enumerator; import org.apache.seatunnel.api.source.SourceSplitEnumerator; +import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.connectors.seatunnel.iceberg.config.SourceConfig; import org.apache.seatunnel.connectors.seatunnel.iceberg.source.enumerator.scan.IcebergScanContext; import org.apache.seatunnel.connectors.seatunnel.iceberg.source.enumerator.scan.IcebergScanSplitPlanner; @@ -43,11 +44,13 @@ public IcebergStreamSplitEnumerator( @NonNull SourceSplitEnumerator.Context context, @NonNull IcebergScanContext icebergScanContext, @NonNull SourceConfig sourceConfig, - IcebergSplitEnumeratorState restoreState) { + IcebergSplitEnumeratorState restoreState, + CatalogTable catalogTable) { super( context, sourceConfig, - restoreState != null ? restoreState.getPendingSplits() : Collections.emptyMap()); + restoreState != null ? restoreState.getPendingSplits() : Collections.emptyMap(), + catalogTable); this.icebergScanContext = icebergScanContext; this.enumeratorPosition = new AtomicReference<>(); if (restoreState != null) { diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/reader/IcebergFileScanTaskReader.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/reader/IcebergFileScanTaskReader.java index c1d2138ee17d..e178dc481ed8 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/reader/IcebergFileScanTaskReader.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/reader/IcebergFileScanTaskReader.java @@ -32,7 +32,6 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.data.avro.DataReader; import org.apache.iceberg.data.orc.GenericOrcReader; -import org.apache.iceberg.data.parquet.GenericParquetReaders; import org.apache.iceberg.expressions.Evaluator; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; @@ -52,6 +51,8 @@ import java.io.Closeable; import java.util.Map; +import static org.apache.iceberg.data.parquet.GenericParquetReaders.buildReader; + @Builder public class IcebergFileScanTaskReader implements Closeable { @@ -130,8 +131,7 @@ private CloseableIterable openFile(FileScanTask task, Schema fileProject .project(fileProjection) .createReaderFunc( fileSchema -> - GenericParquetReaders.buildReader( - fileProjection, fileSchema, partition)) + buildReader(fileProjection, fileSchema, partition)) .split(task.start(), task.length()) .filter(task.residual()); if (reuseContainers) { diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/reader/IcebergSourceReader.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/reader/IcebergSourceReader.java index 22427048603c..ebc75bae9b8e 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/reader/IcebergSourceReader.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/reader/IcebergSourceReader.java @@ -20,6 +20,7 @@ import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.Collector; import org.apache.seatunnel.api.source.SourceReader; +import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.connectors.seatunnel.iceberg.IcebergTableLoader; @@ -58,14 +59,18 @@ public class IcebergSourceReader implements SourceReader(); + this.catalogTable = catalogTable; this.deserializer = new DefaultDeserializer(seaTunnelRowType, projectedSchema); this.tableSchema = tableSchema; this.projectedSchema = projectedSchema; @@ -74,7 +79,7 @@ public IcebergSourceReader( @Override public void open() { - icebergTableLoader = IcebergTableLoader.create(sourceConfig); + icebergTableLoader = IcebergTableLoader.create(sourceConfig, catalogTable); icebergTableLoader.open(); icebergFileScanTaskSplitReader = diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/utils/SchemaUtils.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/utils/SchemaUtils.java new file mode 100644 index 000000000000..f6503caa3858 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/utils/SchemaUtils.java @@ -0,0 +1,324 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.iceberg.utils; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.catalog.exception.CatalogException; +import org.apache.seatunnel.api.table.catalog.exception.DatabaseNotExistException; +import org.apache.seatunnel.api.table.catalog.exception.TableAlreadyExistException; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.api.table.type.SqlType; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.SinkConfig; +import org.apache.seatunnel.connectors.seatunnel.iceberg.data.IcebergTypeMapper; +import org.apache.seatunnel.connectors.seatunnel.iceberg.sink.schema.SchemaAddColumn; +import org.apache.seatunnel.connectors.seatunnel.iceberg.sink.schema.SchemaChangeColumn; +import org.apache.seatunnel.connectors.seatunnel.iceberg.sink.schema.SchemaChangeWrapper; +import org.apache.seatunnel.connectors.seatunnel.iceberg.sink.schema.SchemaDeleteColumn; +import org.apache.seatunnel.connectors.seatunnel.iceberg.sink.schema.SchemaModifyColumn; + +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.UpdateSchema; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Pair; +import org.apache.iceberg.util.Tasks; + +import org.jetbrains.annotations.NotNull; + +import lombok.extern.slf4j.Slf4j; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import static java.util.stream.Collectors.toList; + +@Slf4j +public class SchemaUtils { + private static final Pattern TRANSFORM_REGEX = Pattern.compile("(\\w+)\\((.+)\\)"); + + private SchemaUtils() {} + + public static Type.PrimitiveType needsDataTypeUpdate( + Type currentIcebergType, SeaTunnelDataType dataType) { + if (currentIcebergType.typeId() == Type.TypeID.FLOAT + && dataType.getSqlType() == SqlType.DOUBLE) { + return Types.DoubleType.get(); + } + if (currentIcebergType.typeId() == Type.TypeID.INTEGER + && dataType.getSqlType() == SqlType.BIGINT) { + return Types.LongType.get(); + } + return null; + } + + public static void applySchemaUpdates(Table table, SchemaChangeWrapper wrapper) { + if (wrapper == null || wrapper.empty()) { + // no updates to apply + return; + } + Tasks.range(1) + .retry(SinkConfig.SCHEMA_UPDATE_RETRIES) + .run(notUsed -> commitSchemaUpdates(table, wrapper)); + } + + public static Table autoCreateTable( + Catalog catalog, TablePath tablePath, CatalogTable table, ReadonlyConfig readonlyConfig) + throws TableAlreadyExistException, DatabaseNotExistException, CatalogException { + TableSchema tableSchema = table.getTableSchema(); + // Convert to iceberg schema + Schema schema = toIcebergSchema(tableSchema.toPhysicalRowDataType()); + // Convert sink config + SinkConfig config = new SinkConfig(readonlyConfig); + // build auto create table + Map options = new HashMap<>(table.getOptions()); + options.put(TableProperties.FORMAT_VERSION, "2"); + // override + options.putAll(config.getAutoCreateProps()); + return createTable(catalog, toIcebergTableIdentifier(tablePath), config, schema, options); + } + + /** + * For local test + * + * @param catalog + * @param tableIdentifier + * @param config + * @param rowType + * @return + */ + public static Table autoCreateTable( + Catalog catalog, + TableIdentifier tableIdentifier, + SinkConfig config, + SeaTunnelRowType rowType) { + // Generate struct type + Schema schema = toIcebergSchema(rowType); + return createTable(catalog, tableIdentifier, config, schema, config.getAutoCreateProps()); + } + + private static Table createTable( + Catalog catalog, + TableIdentifier tableIdentifier, + SinkConfig config, + Schema schema, + Map autoCreateProps) { + + List partitionBy = config.getPartitionKeys(); + PartitionSpec spec; + try { + spec = SchemaUtils.createPartitionSpec(schema, partitionBy); + } catch (Exception e) { + log.error( + "Unable to create partition spec {}, table {} will be unpartitioned", + partitionBy, + tableIdentifier, + e); + spec = PartitionSpec.unpartitioned(); + } + PartitionSpec partitionSpec = spec; + AtomicReference result = new AtomicReference<>(); + Tasks.range(1) + .retry(SinkConfig.CREATE_TABLE_RETRIES) + .run( + notUsed -> { + Table table = + catalog.createTable( + tableIdentifier, + schema, + partitionSpec, + autoCreateProps); + result.set(table); + }); + return result.get(); + } + + @NotNull private static Schema toIcebergSchema(SeaTunnelRowType rowType) { + Types.StructType structType = SchemaUtils.toIcebergType(rowType).asStructType(); + return new Schema(structType.fields()); + } + + public static TableIdentifier toIcebergTableIdentifierFromCatalogTable( + CatalogTable catalogTable) { + org.apache.seatunnel.api.table.catalog.TableIdentifier tableIdentifier = + catalogTable.getTableId(); + return TableIdentifier.of( + tableIdentifier.getDatabaseName(), tableIdentifier.getTableName()); + } + + public static TableIdentifier toIcebergTableIdentifier(TablePath tablePath) { + return TableIdentifier.of(tablePath.getDatabaseName(), tablePath.getTableName()); + } + + public static TablePath toTablePath(TableIdentifier tableIdentifier) { + return TablePath.of(tableIdentifier.namespace().toString(), tableIdentifier.name()); + } + + /** + * Commit table schema updates + * + * @param table + * @param wrapper + */ + private static void commitSchemaUpdates(Table table, SchemaChangeWrapper wrapper) { + // get the latest schema in case another process updated it + table.refresh(); + // filter out columns that have already been added + List addColumns = + wrapper.addColumns().stream() + .filter(addCol -> !columnExists(table.schema(), addCol)) + .collect(toList()); + + // filter out columns that have the updated type + List modifyColumns = + wrapper.modifyColumns().stream() + .filter(updateType -> !typeMatches(table.schema(), updateType)) + .collect(toList()); + + // filter out columns that have the updated type + List deleteColumns = + wrapper.deleteColumns().stream() + .filter(deleteColumn -> findColumns(table.schema(), deleteColumn)) + .collect(toList()); + + // Rename column name + List changeColumns = wrapper.changeColumns().stream().collect(toList()); + + if (addColumns.isEmpty() + && modifyColumns.isEmpty() + && deleteColumns.isEmpty() + && changeColumns.isEmpty()) { + // no updates to apply + log.info("Schema for table {} already up-to-date", table.name()); + return; + } + + // apply the updates + UpdateSchema updateSchema = table.updateSchema(); + addColumns.forEach( + update -> + updateSchema.addColumn(update.parentName(), update.name(), update.type())); + modifyColumns.forEach(update -> updateSchema.updateColumn(update.name(), update.type())); + deleteColumns.forEach(delete -> updateSchema.deleteColumn(delete.name())); + changeColumns.forEach( + changeColumn -> + updateSchema.renameColumn(changeColumn.oldName(), changeColumn.newName())); + updateSchema.commit(); + log.info("Schema for table {} updated with new columns", table.name()); + } + + private static boolean columnExists(org.apache.iceberg.Schema schema, SchemaAddColumn update) { + Types.StructType struct = + update.parentName() == null + ? schema.asStruct() + : schema.findType(update.parentName()).asStructType(); + return struct.field(update.name()) != null; + } + + private static boolean typeMatches( + org.apache.iceberg.Schema schema, SchemaModifyColumn update) { + return schema.findType(update.name()).typeId() == update.type().typeId(); + } + + private static boolean findColumns( + org.apache.iceberg.Schema schema, SchemaDeleteColumn deleteColumn) { + return schema.findField(deleteColumn.name()) != null; + } + + public static SeaTunnelDataType toSeaTunnelType(String fieldName, Type type) { + return IcebergTypeMapper.mapping(fieldName, type); + } + + public static Type toIcebergType(SeaTunnelDataType rowType) { + return IcebergTypeMapper.toIcebergType(rowType); + } + + public static PartitionSpec createPartitionSpec( + org.apache.iceberg.Schema schema, List partitionBy) { + if (partitionBy.isEmpty()) { + return PartitionSpec.unpartitioned(); + } + PartitionSpec.Builder specBuilder = PartitionSpec.builderFor(schema); + partitionBy.forEach( + partitionField -> { + Matcher matcher = TRANSFORM_REGEX.matcher(partitionField); + if (matcher.matches()) { + String transform = matcher.group(1); + switch (transform) { + case "year": + case "years": + specBuilder.year(matcher.group(2)); + break; + case "month": + case "months": + specBuilder.month(matcher.group(2)); + break; + case "day": + case "days": + specBuilder.day(matcher.group(2)); + break; + case "hour": + case "hours": + specBuilder.hour(matcher.group(2)); + break; + case "bucket": + { + Pair args = transformArgPair(matcher.group(2)); + specBuilder.bucket(args.first(), args.second()); + break; + } + case "truncate": + { + Pair args = transformArgPair(matcher.group(2)); + specBuilder.truncate(args.first(), args.second()); + break; + } + default: + throw new UnsupportedOperationException( + "Unsupported transform: " + transform); + } + } else { + specBuilder.identity(partitionField); + } + }); + return specBuilder.build(); + } + + private static Pair transformArgPair(String argsStr) { + String[] parts = argsStr.split(","); + if (parts.length != 2) { + throw new IllegalArgumentException( + "Invalid argument " + argsStr + ", should have 2 parts"); + } + return Pair.of(parts[0].trim(), Integer.parseInt(parts[1].trim())); + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/test/java/org/apache/seatunnel/connectors/seatunnel/iceberg/TestIcebergMetastore.java b/seatunnel-connectors-v2/connector-iceberg/src/test/java/org/apache/seatunnel/connectors/seatunnel/iceberg/TestIcebergMetastore.java index 7280199a3832..3bbd1ea48a62 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/test/java/org/apache/seatunnel/connectors/seatunnel/iceberg/TestIcebergMetastore.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/test/java/org/apache/seatunnel/connectors/seatunnel/iceberg/TestIcebergMetastore.java @@ -17,6 +17,10 @@ package org.apache.seatunnel.connectors.seatunnel.iceberg; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.CommonConfig; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.SinkConfig; + import org.apache.hadoop.hive.conf.HiveConf; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.hive.HiveCatalog; @@ -29,6 +33,8 @@ import org.junit.jupiter.api.Test; import java.io.File; +import java.util.HashMap; +import java.util.Map; import static org.apache.seatunnel.connectors.seatunnel.iceberg.config.IcebergCatalogType.HIVE; @@ -50,11 +56,19 @@ public void testUseHiveMetastore() { String warehousePath = "/tmp/seatunnel/iceberg/hive/"; new File(warehousePath).mkdirs(); + Map configs = new HashMap<>(); + Map catalogProps = new HashMap<>(); + catalogProps.put("type", HIVE.getType()); + catalogProps.put("warehouse", "file://" + warehousePath); + catalogProps.put("uri", METASTORE_URI); + + configs.put(CommonConfig.KEY_CATALOG_NAME.key(), "seatunnel"); + configs.put(CommonConfig.CATALOG_PROPS.key(), catalogProps); + HiveCatalog catalog = (HiveCatalog) - new IcebergCatalogFactory( - "seatunnel", HIVE, "file://" + warehousePath, METASTORE_URI) - .create(); + new IcebergCatalogLoader(new SinkConfig(ReadonlyConfig.fromMap(configs))) + .loadCatalog(); catalog.createNamespace(Namespace.of("test_database")); Assertions.assertTrue(catalog.namespaceExists(Namespace.of("test_database"))); } diff --git a/seatunnel-connectors-v2/connector-iceberg/src/test/java/org/apache/seatunnel/connectors/seatunnel/iceberg/catalog/IcebergCatalogTest.java b/seatunnel-connectors-v2/connector-iceberg/src/test/java/org/apache/seatunnel/connectors/seatunnel/iceberg/catalog/IcebergCatalogTest.java new file mode 100644 index 000000000000..242be3cb049e --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/test/java/org/apache/seatunnel/connectors/seatunnel/iceberg/catalog/IcebergCatalogTest.java @@ -0,0 +1,172 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.iceberg.catalog; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.PhysicalColumn; +import org.apache.seatunnel.api.table.catalog.TableIdentifier; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.type.BasicType; +import org.apache.seatunnel.api.table.type.DecimalType; +import org.apache.seatunnel.api.table.type.PrimitiveByteArrayType; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.CommonConfig; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.IcebergCatalogType; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.SinkConfig; + +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.MethodOrderer; +import org.junit.jupiter.api.Order; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestMethodOrder; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.apache.seatunnel.api.table.type.BasicType.STRING_TYPE; +import static org.apache.seatunnel.api.table.type.LocalTimeType.LOCAL_DATE_TIME_TYPE; +import static org.apache.seatunnel.api.table.type.LocalTimeType.LOCAL_DATE_TYPE; +import static org.apache.seatunnel.connectors.seatunnel.iceberg.config.IcebergCatalogType.HADOOP; + +@TestMethodOrder(MethodOrderer.OrderAnnotation.class) +@Disabled +class IcebergCatalogTest { + private static final String CATALOG_NAME = "seatunnel"; + private static final IcebergCatalogType CATALOG_TYPE = HADOOP; + private static final String CATALOG_DIR = "/tmp/seatunnel/iceberg/hadoop-test/"; + private static final String WAREHOUSE = "file://" + CATALOG_DIR; + + private static IcebergCatalog icebergCatalog; + + private static String databaseName = "default"; + private static String tableName = "tbl6"; + + private TablePath tablePath = TablePath.of(databaseName, null, tableName); + private TableIdentifier tableIdentifier = + TableIdentifier.of(CATALOG_NAME, databaseName, null, tableName); + + @BeforeAll + static void setUpBeforeClass() throws Exception { + Map configs = new HashMap<>(); + // build catalog props + Map catalogProps = new HashMap<>(); + catalogProps.put("type", CATALOG_TYPE.getType()); + catalogProps.put("warehouse", WAREHOUSE); + configs.put(CommonConfig.KEY_CATALOG_NAME.key(), CATALOG_NAME); + configs.put(CommonConfig.CATALOG_PROPS.key(), catalogProps); + configs.put(SinkConfig.TABLE_DEFAULT_PARTITION_KEYS.key(), "dt_col"); + icebergCatalog = new IcebergCatalog(CATALOG_NAME, ReadonlyConfig.fromMap(configs)); + icebergCatalog.open(); + } + + @AfterAll + static void tearDownAfterClass() throws Exception { + icebergCatalog.close(); + } + + @Test + @Order(1) + void getDefaultDatabase() { + Assertions.assertEquals(icebergCatalog.getDefaultDatabase(), databaseName); + } + + @Test + @Order(2) + void createTable() { + CatalogTable catalogTable = buildAllTypesTable(tableIdentifier); + icebergCatalog.createTable(tablePath, catalogTable, true); + Assertions.assertTrue(icebergCatalog.tableExists(tablePath)); + } + + @Test + @Order(3) + void databaseExists() { + Assertions.assertTrue(icebergCatalog.databaseExists(databaseName)); + Assertions.assertFalse(icebergCatalog.databaseExists("sssss")); + } + + @Test + @Order(4) + void listDatabases() { + icebergCatalog.listDatabases().forEach(System.out::println); + Assertions.assertTrue(icebergCatalog.listDatabases().contains(databaseName)); + } + + @Test + @Order(5) + void listTables() { + Assertions.assertTrue(icebergCatalog.listTables(databaseName).contains(tableName)); + } + + @Test + @Order(6) + void tableExists() { + Assertions.assertTrue(icebergCatalog.tableExists(tablePath)); + Assertions.assertFalse(icebergCatalog.tableExists(TablePath.of(databaseName, "ssssss"))); + } + + @Test + @Order(7) + void getTable() { + CatalogTable table = icebergCatalog.getTable(tablePath); + CatalogTable templateTable = buildAllTypesTable(tableIdentifier); + Assertions.assertEquals(table.toString(), templateTable.toString()); + } + + @Test + @Order(8) + void dropTable() { + icebergCatalog.dropTable(tablePath, false); + Assertions.assertFalse(icebergCatalog.tableExists(tablePath)); + } + + CatalogTable buildAllTypesTable(TableIdentifier tableIdentifier) { + TableSchema.Builder builder = TableSchema.builder(); + builder.column(PhysicalColumn.of("id", BasicType.INT_TYPE, null, true, null, null)); + builder.column( + PhysicalColumn.of("boolean_col", BasicType.BOOLEAN_TYPE, null, true, null, null)); + builder.column( + PhysicalColumn.of("integer_col", BasicType.INT_TYPE, null, true, null, null)); + builder.column(PhysicalColumn.of("long_col", BasicType.LONG_TYPE, null, true, null, null)); + builder.column( + PhysicalColumn.of("float_col", BasicType.FLOAT_TYPE, null, true, null, null)); + builder.column( + PhysicalColumn.of("double_col", BasicType.DOUBLE_TYPE, null, true, null, null)); + builder.column(PhysicalColumn.of("date_col", LOCAL_DATE_TYPE, null, true, null, null)); + builder.column( + PhysicalColumn.of("timestamp_col", LOCAL_DATE_TIME_TYPE, null, true, null, null)); + builder.column(PhysicalColumn.of("string_col", STRING_TYPE, null, true, null, null)); + builder.column( + PhysicalColumn.of( + "binary_col", PrimitiveByteArrayType.INSTANCE, null, true, null, null)); + builder.column( + PhysicalColumn.of("decimal_col", new DecimalType(38, 18), null, true, null, null)); + builder.column(PhysicalColumn.of("dt_col", STRING_TYPE, null, true, null, null)); + + TableSchema schema = builder.build(); + HashMap options = new HashMap<>(); + options.put("write.parquet.compression-codec", "zstd"); + return CatalogTable.of( + tableIdentifier, schema, options, Collections.singletonList("dt_col"), "null"); + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/pom.xml index 50754d6d46b3..11c147432d03 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/pom.xml @@ -25,7 +25,24 @@ connector-iceberg-e2e SeaTunnel : E2E : Connector V2 : Iceberg + + + + org.apache.seatunnel + connector-jdbc + ${project.version} + pom + import + + + + + + org.apache.seatunnel + connector-fake + ${project.version} + org.apache.seatunnel connector-iceberg @@ -51,6 +68,43 @@ optional test + + + + + org.apache.seatunnel + connector-jdbc + ${project.version} + test + + + + org.apache.seatunnel + connector-cdc-mysql + ${project.version} + test + + + + org.apache.seatunnel + connector-cdc-mysql + ${project.version} + test-jar + test + + + + org.testcontainers + mysql + ${testcontainer.version} + + + + mysql + mysql-connector-java + test + + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/java/org/apache/seatunnel/e2e/connector/iceberg/IcebergSinkCDCIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/java/org/apache/seatunnel/e2e/connector/iceberg/IcebergSinkCDCIT.java new file mode 100644 index 000000000000..afe6d3d43b76 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/java/org/apache/seatunnel/e2e/connector/iceberg/IcebergSinkCDCIT.java @@ -0,0 +1,459 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.e2e.connector.iceberg; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.common.utils.FileUtils; +import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.testutils.MySqlContainer; +import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.testutils.MySqlVersion; +import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.testutils.UniqueDatabase; +import org.apache.seatunnel.connectors.seatunnel.iceberg.IcebergTableLoader; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.CommonConfig; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.SourceConfig; +import org.apache.seatunnel.e2e.common.TestResource; +import org.apache.seatunnel.e2e.common.TestSuiteBase; +import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; +import org.apache.seatunnel.e2e.common.container.EngineType; +import org.apache.seatunnel.e2e.common.container.TestContainer; +import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; +import org.apache.seatunnel.e2e.common.junit.TestContainerExtension; + +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.IcebergGenerics; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.types.Types; + +import org.jetbrains.annotations.NotNull; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.condition.DisabledOnOs; +import org.junit.jupiter.api.condition.OS; +import org.testcontainers.containers.Container; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.lifecycle.Startables; +import org.testcontainers.utility.DockerLoggerFactory; + +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.stream.Stream; + +import static java.lang.Thread.sleep; +import static org.apache.seatunnel.connectors.seatunnel.iceberg.config.IcebergCatalogType.HADOOP; +import static org.awaitility.Awaitility.given; + +@Slf4j +@DisabledOnContainer( + value = {}, + type = {EngineType.SPARK}, + disabledReason = "Currently SPARK do not support cdc") +@DisabledOnOs(OS.WINDOWS) +public class IcebergSinkCDCIT extends TestSuiteBase implements TestResource { + + private static final String CATALOG_DIR = "/tmp/seatunnel/iceberg/hadoop-cdc-sink/"; + + private static final String NAMESPACE = "seatunnel_namespace"; + + // mysql + private static final String MYSQL_HOST = "mysql_cdc_e2e"; + private static final String MYSQL_USER_NAME = "st_user"; + private static final String MYSQL_USER_PASSWORD = "seatunnel"; + private static final String MYSQL_DATABASE = "mysql_cdc"; + private static final MySqlContainer MYSQL_CONTAINER = createMySqlContainer(MySqlVersion.V8_0); + + private final UniqueDatabase inventoryDatabase = + new UniqueDatabase( + MYSQL_CONTAINER, MYSQL_DATABASE, "mysqluser", "mysqlpw", MYSQL_DATABASE); + + private static MySqlContainer createMySqlContainer(MySqlVersion version) { + return new MySqlContainer(version) + .withConfigurationOverride("mysql/server-gtids/my.cnf") + .withSetupSQL("mysql/setup.sql") + .withNetwork(NETWORK) + .withNetworkAliases(MYSQL_HOST) + .withDatabaseName(MYSQL_DATABASE) + .withUsername(MYSQL_USER_NAME) + .withPassword(MYSQL_USER_PASSWORD) + .withLogConsumer( + new Slf4jLogConsumer(DockerLoggerFactory.getLogger("mysql-mysql-image"))); + } + + private String driverUrl() { + return "https://repo1.maven.org/maven2/com/mysql/mysql-connector-j/8.0.32/mysql-connector-j-8.0.32.jar"; + } + + private String zstdUrl() { + return "https://repo1.maven.org/maven2/com/github/luben/zstd-jni/1.5.5-5/zstd-jni-1.5.5-5.jar"; + } + + @TestContainerExtension + protected final ContainerExtendedFactory extendedFactory = + container -> { + container.execInContainer("sh", "-c", "mkdir -p " + CATALOG_DIR); + container.execInContainer("sh", "-c", "chmod -R 777 " + CATALOG_DIR); + Container.ExecResult extraCommandsZSTD = + container.execInContainer( + "sh", + "-c", + "mkdir -p /tmp/seatunnel/plugins/Iceberg/lib && cd /tmp/seatunnel/plugins/Iceberg/lib && wget " + + zstdUrl()); + Assertions.assertEquals( + 0, extraCommandsZSTD.getExitCode(), extraCommandsZSTD.getStderr()); + Container.ExecResult extraCommands = + container.execInContainer( + "sh", + "-c", + "mkdir -p /tmp/seatunnel/plugins/MySQL-CDC/lib && cd /tmp/seatunnel/plugins/MySQL-CDC/lib && wget " + + driverUrl()); + Assertions.assertEquals(0, extraCommands.getExitCode(), extraCommands.getStderr()); + }; + + private final String NAMESPACE_TAR = NAMESPACE + ".tar.gz"; + protected final ContainerExtendedFactory containerExtendedFactory = + new ContainerExtendedFactory() { + @Override + public void extend(GenericContainer container) + throws IOException, InterruptedException { + FileUtils.createNewDir(CATALOG_DIR); + container.execInContainer( + "sh", + "-c", + "cd " + + CATALOG_DIR + + " && tar -czvf " + + NAMESPACE_TAR + + " " + + NAMESPACE); + container.copyFileFromContainer( + CATALOG_DIR + NAMESPACE_TAR, CATALOG_DIR + NAMESPACE_TAR); + extractFiles(); + } + + private void extractFiles() { + ProcessBuilder processBuilder = new ProcessBuilder(); + processBuilder.command( + "sh", "-c", "cd " + CATALOG_DIR + " && tar -zxvf " + NAMESPACE_TAR); + try { + Process process = processBuilder.start(); + // 等待命令执行完成 + int exitCode = process.waitFor(); + if (exitCode == 0) { + log.info("Extract files successful."); + } else { + log.error("Extract files failed with exit code " + exitCode); + } + } catch (IOException | InterruptedException e) { + e.printStackTrace(); + } + } + }; + + private static final String SOURCE_TABLE = "mysql_cdc_e2e_source_table"; + + @BeforeAll + @Override + public void startUp() throws Exception { + log.info("The second stage: Starting Mysql containers..."); + Startables.deepStart(Stream.of(MYSQL_CONTAINER)).join(); + log.info("Mysql Containers are started"); + inventoryDatabase.createAndInitialize(); + log.info("Mysql ddl execution is complete"); + } + + @TestTemplate + public void testMysqlCdcCheckDataE2e(TestContainer container) + throws IOException, InterruptedException { + // Clear related content to ensure that multiple operations are not affected + clearTable(MYSQL_DATABASE, SOURCE_TABLE); + CompletableFuture.supplyAsync( + () -> { + try { + container.executeJob("/iceberg/mysql_cdc_to_iceberg.conf"); + } catch (Exception e) { + log.error("Commit task exception :" + e.getMessage()); + throw new RuntimeException(e); + } + return null; + }); + insertAndCheckData(container); + upsertAndCheckData(container); + } + + @TestTemplate + @Disabled + public void testMysqlCdcCheckSchemaChangeE2e(TestContainer container) + throws IOException, InterruptedException { + // Clear related content to ensure that multiple operations are not affected + clearTable(MYSQL_DATABASE, SOURCE_TABLE); + CompletableFuture.supplyAsync( + () -> { + try { + container.executeJob( + "/iceberg/mysql_cdc_to_iceberg_for_schema_change.conf"); + } catch (Exception e) { + log.error("Commit task exception :" + e.getMessage()); + throw new RuntimeException(e); + } + return null; + }); + initSourceTableData(MYSQL_DATABASE, SOURCE_TABLE); + alterSchemaAndCheckIcebergSchema(container); + } + + private void alterSchemaAndCheckIcebergSchema(TestContainer container) + throws InterruptedException, IOException { + String dropField = "f_binary"; + // Init table data + dropTableColumn(MYSQL_DATABASE, SOURCE_TABLE, dropField); + // Waiting 30s for source capture data + sleep(30000); + + // stream stage + given().ignoreExceptions() + .await() + .atMost(120000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + // copy iceberg to local + container.executeExtraCommands(containerExtendedFactory); + Schema schema = loadIcebergSchema(); + Types.NestedField nestedField = schema.findField(dropField); + Assertions.assertEquals(true, nestedField == null); + // for next test + addTableColumn(MYSQL_DATABASE, SOURCE_TABLE, dropField); + }); + } + + private void upsertAndCheckData(TestContainer container) + throws InterruptedException, IOException { + upsertDeleteSourceTable(MYSQL_DATABASE, SOURCE_TABLE); + // Waiting 30s for source capture data + sleep(30000); + + // stream stage + given().ignoreExceptions() + .await() + .atMost(120000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + // copy iceberg to local + container.executeExtraCommands(containerExtendedFactory); + List records = loadIcebergTable(); + Assertions.assertEquals(4, records.size()); + for (Record record : records) { + Integer id = (Integer) record.getField("id"); + Long f_bigint = (Long) record.getField("f_bigint"); + if (id == 3) { + Assertions.assertEquals(10000, f_bigint); + } + } + }); + } + + private void insertAndCheckData(TestContainer container) + throws InterruptedException, IOException { + // Init table data + initSourceTableData(MYSQL_DATABASE, SOURCE_TABLE); + // Waiting 30s for source capture data + sleep(30000); + + // stream stage + given().ignoreExceptions() + .await() + .atMost(60000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + // copy iceberg to local + container.executeExtraCommands(containerExtendedFactory); + Assertions.assertEquals(3, loadIcebergTable().size()); + }); + } + + private Schema loadIcebergSchema() { + IcebergTableLoader tableLoader = getTableLoader(); + Table table = tableLoader.loadTable(); + return table.schema(); + } + + private List loadIcebergTable() { + List results = new ArrayList<>(); + IcebergTableLoader tableLoader = getTableLoader(); + try { + Table table = tableLoader.loadTable(); + try (CloseableIterable records = IcebergGenerics.read(table).build()) { + for (Record record : records) { + results.add(record); + } + } catch (IOException e) { + e.printStackTrace(); + } + } catch (Exception ex) { + ex.printStackTrace(); + } + return results; + } + + @NotNull private static IcebergTableLoader getTableLoader() { + Map configs = new HashMap<>(); + Map catalogProps = new HashMap<>(); + catalogProps.put("type", HADOOP.getType()); + catalogProps.put("warehouse", "file://" + CATALOG_DIR); + configs.put(CommonConfig.KEY_CATALOG_NAME.key(), "seatunnel_test"); + configs.put(CommonConfig.KEY_NAMESPACE.key(), "seatunnel_namespace"); + configs.put(CommonConfig.KEY_TABLE.key(), "iceberg_sink_table"); + configs.put(CommonConfig.CATALOG_PROPS.key(), catalogProps); + IcebergTableLoader tableLoader = + IcebergTableLoader.create(new SourceConfig(ReadonlyConfig.fromMap(configs))); + tableLoader.open(); + return tableLoader; + } + + private void dropTableColumn(String database, String tableName, String dropField) { + executeSql("ALTER TABLE " + database + "." + tableName + " DROP COLUMN " + dropField); + } + + private void addTableColumn(String database, String tableName, String dropField) { + executeSql("ALTER TABLE " + database + "." + tableName + " ADD COLUMN " + dropField); + } + + private void clearTable(String database, String tableName) { + executeSql("truncate table " + database + "." + tableName); + } + + // Execute SQL + private void executeSql(String sql) { + try (Connection connection = getJdbcConnection()) { + connection.createStatement().execute(sql); + } catch (SQLException e) { + throw new RuntimeException(e); + } + } + + private Connection getJdbcConnection() throws SQLException { + return DriverManager.getConnection( + MYSQL_CONTAINER.getJdbcUrl(), + MYSQL_CONTAINER.getUsername(), + MYSQL_CONTAINER.getPassword()); + } + + @AfterAll + @Override + public void tearDown() throws Exception { + // close Container + if (MYSQL_CONTAINER != null) { + MYSQL_CONTAINER.close(); + } + } + + private void initSourceTableData(String database, String tableName) { + executeSql( + "INSERT INTO " + + database + + "." + + tableName + + " ( id, f_binary, f_blob, f_long_varbinary, f_longblob, f_tinyblob, f_varbinary, f_smallint,\n" + + " f_smallint_unsigned, f_mediumint, f_mediumint_unsigned, f_int, f_int_unsigned, f_integer,\n" + + " f_integer_unsigned, f_bigint, f_bigint_unsigned, f_numeric, f_decimal, f_float, f_double,\n" + + " f_double_precision, f_longtext, f_mediumtext, f_text, f_tinytext, f_varchar, f_date, f_datetime,\n" + + " f_timestamp, f_bit1, f_bit64, f_char, f_enum, f_mediumblob, f_long_varchar, f_real, f_time,\n" + + " f_tinyint, f_tinyint_unsigned, f_json, f_year )\n" + + "VALUES ( 1, 0x61626374000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000,\n" + + " 0x68656C6C6F, 0x18000000789C0BC9C82C5600A244859CFCBC7485B2C4A2A4CCBCC4A24A00697308D4, NULL,\n" + + " 0x74696E79626C6F62, 0x48656C6C6F20776F726C64, 12345, 54321, 123456, 654321, 1234567, 7654321, 1234567, 7654321,\n" + + " 123456789, 987654321, 123, 789, 12.34, 56.78, 90.12, 'This is a long text field', 'This is a medium text field',\n" + + " 'This is a text field', 'This is a tiny text field', 'This is a varchar field', '2022-04-27', '2022-04-27 14:30:00',\n" + + " '2023-04-27 11:08:40', 1, b'0101010101010101010101010101010101010101010101010101010101010101', 'C', 'enum2',\n" + + " 0x1B000000789C0BC9C82C5600A24485DCD494CCD25C85A49CFC2485B4CCD49C140083FF099A, 'This is a long varchar field',\n" + + " 12.345, '14:30:00', -128, 255, '{ \"key\": \"value\" }', 2022 ),\n" + + " ( 2, 0x61626374000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000,\n" + + " 0x68656C6C6F, 0x18000000789C0BC9C82C5600A244859CFCBC7485B2C4A2A4CCBCC4A24A00697308D4, NULL, 0x74696E79626C6F62,\n" + + " 0x48656C6C6F20776F726C64, 12345, 54321, 123456, 654321, 1234567, 7654321, 1234567, 7654321, 123456789, 987654321,\n" + + " 123, 789, 12.34, 56.78, 90.12, 'This is a long text field', 'This is a medium text field', 'This is a text field',\n" + + " 'This is a tiny text field', 'This is a varchar field', '2022-04-27', '2022-04-27 14:30:00', '2023-04-27 11:08:40',\n" + + " 1, b'0101010101010101010101010101010101010101010101010101010101010101', 'C', 'enum2',\n" + + " 0x1B000000789C0BC9C82C5600A24485DCD494CCD25C85A49CFC2485B4CCD49C140083FF099A, 'This is a long varchar field',\n" + + " 112.345, '14:30:00', -128, 22, '{ \"key\": \"value\" }', 2013 ),\n" + + " ( 3, 0x61626374000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000,\n" + + " 0x68656C6C6F, 0x18000000789C0BC9C82C5600A244859CFCBC7485B2C4A2A4CCBCC4A24A00697308D4, NULL, 0x74696E79626C6F62,\n" + + " 0x48656C6C6F20776F726C64, 12345, 54321, 123456, 654321, 1234567, 7654321, 1234567, 7654321, 123456789, 987654321, 123,\n" + + " 789, 12.34, 56.78, 90.12, 'This is a long text field', 'This is a medium text field', 'This is a text field',\n" + + " 'This is a tiny text field', 'This is a varchar field', '2022-04-27', '2022-04-27 14:30:00', '2023-04-27 11:08:40',\n" + + " 1, b'0101010101010101010101010101010101010101010101010101010101010101', 'C', 'enum2',\n" + + " 0x1B000000789C0BC9C82C5600A24485DCD494CCD25C85A49CFC2485B4CCD49C140083FF099A, 'This is a long varchar field', 112.345,\n" + + " '14:30:00', -128, 22, '{ \"key\": \"value\" }', 2021 )"); + } + + private void upsertDeleteSourceTable(String database, String tableName) { + executeSql( + "INSERT INTO " + + database + + "." + + tableName + + " ( id, f_binary, f_blob, f_long_varbinary, f_longblob, f_tinyblob, f_varbinary, f_smallint,\n" + + " f_smallint_unsigned, f_mediumint, f_mediumint_unsigned, f_int, f_int_unsigned, f_integer,\n" + + " f_integer_unsigned, f_bigint, f_bigint_unsigned, f_numeric, f_decimal, f_float, f_double,\n" + + " f_double_precision, f_longtext, f_mediumtext, f_text, f_tinytext, f_varchar, f_date, f_datetime,\n" + + " f_timestamp, f_bit1, f_bit64, f_char, f_enum, f_mediumblob, f_long_varchar, f_real, f_time,\n" + + " f_tinyint, f_tinyint_unsigned, f_json, f_year )\n" + + "VALUES ( 5, 0x61626374000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000,\n" + + " 0x68656C6C6F, 0x18000000789C0BC9C82C5600A244859CFCBC7485B2C4A2A4CCBCC4A24A00697308D4, NULL,\n" + + " 0x74696E79626C6F62, 0x48656C6C6F20776F726C64, 12345, 54321, 123456, 654321, 1234567, 7654321, 1234567, 7654321,\n" + + " 123456789, 987654321, 123, 789, 12.34, 56.78, 90.12, 'This is a long text field', 'This is a medium text field',\n" + + " 'This is a text field', 'This is a tiny text field', 'This is a varchar field', '2022-04-27', '2022-04-27 14:30:00',\n" + + " '2023-04-27 11:08:40', 1, b'0101010101010101010101010101010101010101010101010101010101010101', 'C', 'enum2',\n" + + " 0x1B000000789C0BC9C82C5600A24485DCD494CCD25C85A49CFC2485B4CCD49C140083FF099A, 'This is a long varchar field',\n" + + " 12.345, '14:30:00', -128, 255, '{ \"key\": \"value\" }', 1992 )"); + executeSql( + "INSERT INTO " + + database + + "." + + tableName + + " ( id, f_binary, f_blob, f_long_varbinary, f_longblob, f_tinyblob, f_varbinary, f_smallint,\n" + + " f_smallint_unsigned, f_mediumint, f_mediumint_unsigned, f_int, f_int_unsigned, f_integer,\n" + + " f_integer_unsigned, f_bigint, f_bigint_unsigned, f_numeric, f_decimal, f_float, f_double,\n" + + " f_double_precision, f_longtext, f_mediumtext, f_text, f_tinytext, f_varchar, f_date, f_datetime,\n" + + " f_timestamp, f_bit1, f_bit64, f_char, f_enum, f_mediumblob, f_long_varchar, f_real, f_time,\n" + + " f_tinyint, f_tinyint_unsigned, f_json, f_year )\n" + + "VALUES ( 6, 0x61626374000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000,\n" + + " 0x68656C6C6F, 0x18000000789C0BC9C82C5600A244859CFCBC7485B2C4A2A4CCBCC4A24A00697308D4, NULL,\n" + + " 0x74696E79626C6F62, 0x48656C6C6F20776F726C64, 12345, 54321, 123456, 654321, 1234567, 7654321, 1234567, 7654321,\n" + + " 123456789, 987654321, 123, 789, 12.34, 56.78, 90.12, 'This is a long text field', 'This is a medium text field',\n" + + " 'This is a text field', 'This is a tiny text field', 'This is a varchar field', '2022-04-27', '2022-04-27 14:30:00',\n" + + " '2023-04-27 11:08:40', 1, b'0101010101010101010101010101010101010101010101010101010101010101', 'C', 'enum2',\n" + + " 0x1B000000789C0BC9C82C5600A24485DCD494CCD25C85A49CFC2485B4CCD49C140083FF099A, 'This is a long varchar field',\n" + + " 12.345, '14:30:00', -128, 255, '{ \"key\": \"value\" }', 1999 )"); + executeSql("DELETE FROM " + database + "." + tableName + " where id = 2"); + + executeSql("UPDATE " + database + "." + tableName + " SET f_bigint = 10000 where id = 3"); + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/java/org/apache/seatunnel/e2e/connector/iceberg/IcebergSinkIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/java/org/apache/seatunnel/e2e/connector/iceberg/IcebergSinkIT.java new file mode 100644 index 000000000000..20c1b02914e5 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/java/org/apache/seatunnel/e2e/connector/iceberg/IcebergSinkIT.java @@ -0,0 +1,169 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.e2e.connector.iceberg; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.common.utils.FileUtils; +import org.apache.seatunnel.connectors.seatunnel.iceberg.IcebergTableLoader; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.CommonConfig; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.SourceConfig; +import org.apache.seatunnel.e2e.common.TestSuiteBase; +import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; +import org.apache.seatunnel.e2e.common.container.TestContainer; +import org.apache.seatunnel.e2e.common.container.TestContainerId; +import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; +import org.apache.seatunnel.e2e.common.junit.TestContainerExtension; + +import org.apache.iceberg.Table; +import org.apache.iceberg.data.IcebergGenerics; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.io.CloseableIterable; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.condition.DisabledOnOs; +import org.junit.jupiter.api.condition.OS; +import org.testcontainers.containers.Container; +import org.testcontainers.containers.GenericContainer; + +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import static org.apache.seatunnel.connectors.seatunnel.iceberg.config.IcebergCatalogType.HADOOP; +import static org.awaitility.Awaitility.given; + +@Slf4j +@DisabledOnContainer( + value = {TestContainerId.SPARK_2_4}, + type = {}, + disabledReason = "") +@DisabledOnOs(OS.WINDOWS) +public class IcebergSinkIT extends TestSuiteBase { + + private static final String CATALOG_DIR = "/tmp/seatunnel/iceberg/hadoop-sink/"; + + private static final String NAMESPACE = "seatunnel_namespace"; + + private String zstdUrl() { + return "https://repo1.maven.org/maven2/com/github/luben/zstd-jni/1.5.5-5/zstd-jni-1.5.5-5.jar"; + } + + @TestContainerExtension + protected final ContainerExtendedFactory extendedFactory = + container -> { + container.execInContainer("sh", "-c", "mkdir -p " + CATALOG_DIR); + container.execInContainer("sh", "-c", "chmod -R 777 " + CATALOG_DIR); + container.execInContainer( + "sh", + "-c", + "mkdir -p /tmp/seatunnel/plugins/Iceberg/lib && cd /tmp/seatunnel/plugins/Iceberg/lib && wget " + + zstdUrl()); + }; + + private final String NAMESPACE_TAR = NAMESPACE + ".tar.gz"; + protected final ContainerExtendedFactory containerExtendedFactory = + new ContainerExtendedFactory() { + @Override + public void extend(GenericContainer container) + throws IOException, InterruptedException { + FileUtils.createNewDir(CATALOG_DIR); + container.execInContainer( + "sh", + "-c", + "cd " + + CATALOG_DIR + + " && tar -czvf " + + NAMESPACE_TAR + + " " + + NAMESPACE); + container.copyFileFromContainer( + CATALOG_DIR + NAMESPACE_TAR, CATALOG_DIR + NAMESPACE_TAR); + extractFiles(); + } + + private void extractFiles() { + ProcessBuilder processBuilder = new ProcessBuilder(); + processBuilder.command( + "sh", "-c", "cd " + CATALOG_DIR + " && tar -zxvf " + NAMESPACE_TAR); + try { + Process process = processBuilder.start(); + // 等待命令执行完成 + int exitCode = process.waitFor(); + if (exitCode == 0) { + log.info("Extract files successful."); + } else { + log.error("Extract files failed with exit code " + exitCode); + } + } catch (IOException | InterruptedException e) { + e.printStackTrace(); + } + } + }; + + @TestTemplate + public void testInsertAndCheckDataE2e(TestContainer container) + throws IOException, InterruptedException { + Container.ExecResult textWriteResult = + container.executeJob("/iceberg/fake_to_iceberg.conf"); + Assertions.assertEquals(0, textWriteResult.getExitCode()); + // stream stage + given().ignoreExceptions() + .await() + .atMost(60000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + // copy iceberg to local + container.executeExtraCommands(containerExtendedFactory); + Assertions.assertEquals(100, loadIcebergTable().size()); + }); + } + + private List loadIcebergTable() { + List results = new ArrayList<>(); + Map configs = new HashMap<>(); + Map catalogProps = new HashMap<>(); + catalogProps.put("type", HADOOP.getType()); + catalogProps.put("warehouse", "file://" + CATALOG_DIR); + configs.put(CommonConfig.KEY_CATALOG_NAME.key(), "seatunnel_test"); + configs.put(CommonConfig.KEY_NAMESPACE.key(), "seatunnel_namespace"); + configs.put(CommonConfig.KEY_TABLE.key(), "iceberg_sink_table"); + configs.put(CommonConfig.CATALOG_PROPS.key(), catalogProps); + IcebergTableLoader tableLoader = + IcebergTableLoader.create(new SourceConfig(ReadonlyConfig.fromMap(configs))); + tableLoader.open(); + try { + Table table = tableLoader.loadTable(); + try (CloseableIterable records = IcebergGenerics.read(table).build()) { + for (Record record : records) { + results.add(record); + } + } catch (IOException e) { + e.printStackTrace(); + } + } catch (Exception ex) { + ex.printStackTrace(); + } + return results; + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/java/org/apache/seatunnel/e2e/connector/iceberg/IcebergSourceIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/java/org/apache/seatunnel/e2e/connector/iceberg/IcebergSourceIT.java index 434a1a866847..87eec5834b99 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/java/org/apache/seatunnel/e2e/connector/iceberg/IcebergSourceIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/java/org/apache/seatunnel/e2e/connector/iceberg/IcebergSourceIT.java @@ -17,8 +17,11 @@ package org.apache.seatunnel.e2e.connector.iceberg; -import org.apache.seatunnel.connectors.seatunnel.iceberg.IcebergCatalogFactory; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.connectors.seatunnel.iceberg.IcebergCatalogLoader; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.CommonConfig; import org.apache.seatunnel.connectors.seatunnel.iceberg.config.IcebergCatalogType; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.SourceConfig; import org.apache.seatunnel.e2e.common.TestResource; import org.apache.seatunnel.e2e.common.TestSuiteBase; import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; @@ -63,7 +66,9 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import static org.apache.seatunnel.connectors.seatunnel.iceberg.config.IcebergCatalogType.HADOOP; @@ -133,7 +138,19 @@ public void testIcebergSource(TestContainer container) } private void initializeIcebergTable() { - CATALOG = new IcebergCatalogFactory(CATALOG_NAME, CATALOG_TYPE, WAREHOUSE, null).create(); + + Map configs = new HashMap<>(); + // build catalog props + Map catalogProps = new HashMap<>(); + catalogProps.put("type", CATALOG_TYPE.getType()); + catalogProps.put("warehouse", WAREHOUSE); + + configs.put(CommonConfig.KEY_CATALOG_NAME.key(), CATALOG_NAME); + configs.put(CommonConfig.CATALOG_PROPS.key(), catalogProps); + + CATALOG = + new IcebergCatalogLoader(new SourceConfig(ReadonlyConfig.fromMap(configs))) + .loadCatalog(); if (!CATALOG.tableExists(TABLE)) { CATALOG.createTable(TABLE, SCHEMA); } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/ddl/inventory.sql b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/ddl/inventory.sql new file mode 100644 index 000000000000..9e9fff3f8f4c --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/ddl/inventory.sql @@ -0,0 +1,95 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one or more +-- contributor license agreements. See the NOTICE file distributed with +-- this work for additional information regarding copyright ownership. +-- The ASF licenses this file to You under the Apache License, Version 2.0 +-- (the "License"); you may not use this file except in compliance with +-- the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- + +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: inventory +-- ---------------------------------------------------------------------------------------------------------------- + +-- Create and populate our products using a single insert with many rows +CREATE TABLE products ( + id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, + name VARCHAR(255) NOT NULL DEFAULT 'SeaTunnel', + description VARCHAR(512), + weight FLOAT +); +ALTER TABLE products AUTO_INCREMENT = 101; + +INSERT INTO products +VALUES (default,"scooter","Small 2-wheel scooter",3.14), + (default,"car battery","12V car battery",8.1), + (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40 to #3",0.8), + (default,"hammer","12oz carpenter's hammer",0.75), + (default,"hammer","14oz carpenter's hammer",0.875), + (default,"hammer","16oz carpenter's hammer",1.0), + (default,"rocks","box of assorted rocks",5.3), + (default,"jacket","water resistent black wind breaker",0.1), + (default,"spare tire","24 inch spare tire",22.2); + +-- Create and populate the products on hand using multiple inserts +CREATE TABLE products_on_hand ( + product_id INTEGER NOT NULL PRIMARY KEY, + quantity INTEGER NOT NULL, + FOREIGN KEY (product_id) REFERENCES products(id) +); + +INSERT INTO products_on_hand VALUES (101,3); +INSERT INTO products_on_hand VALUES (102,8); +INSERT INTO products_on_hand VALUES (103,18); +INSERT INTO products_on_hand VALUES (104,4); +INSERT INTO products_on_hand VALUES (105,5); +INSERT INTO products_on_hand VALUES (106,0); +INSERT INTO products_on_hand VALUES (107,44); +INSERT INTO products_on_hand VALUES (108,2); +INSERT INTO products_on_hand VALUES (109,5); + +-- Create some customers ... +CREATE TABLE customers ( + id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, + first_name VARCHAR(255) NOT NULL, + last_name VARCHAR(255) NOT NULL, + email VARCHAR(255) NOT NULL UNIQUE KEY +) AUTO_INCREMENT=1001; + + +INSERT INTO customers +VALUES (default,"Sally","Thomas","sally.thomas@acme.com"), + (default,"George","Bailey","gbailey@foobar.com"), + (default,"Edward","Walker","ed@walker.com"), + (default,"Anne","Kretchmar","annek@noanswer.org"); + +-- Create some very simple orders +CREATE TABLE orders ( + order_number INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, + order_date DATE NOT NULL, + purchaser INTEGER NOT NULL, + quantity INTEGER NOT NULL, + product_id INTEGER NOT NULL, + FOREIGN KEY order_customer (purchaser) REFERENCES customers(id), + FOREIGN KEY ordered_product (product_id) REFERENCES products(id) +) AUTO_INCREMENT = 10001; + +INSERT INTO orders +VALUES (default, '2016-01-16', 1001, 1, 102), + (default, '2016-01-17', 1002, 2, 105), + (default, '2016-02-18', 1004, 3, 109), + (default, '2016-02-19', 1002, 2, 106), + (default, '16-02-21', 1003, 1, 107); + +CREATE TABLE category ( + id INT NOT NULL AUTO_INCREMENT PRIMARY KEY, + category_name VARCHAR(255) +); \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/ddl/mysql_cdc.sql b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/ddl/mysql_cdc.sql new file mode 100644 index 000000000000..63dbc16bb6ab --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/ddl/mysql_cdc.sql @@ -0,0 +1,186 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one or more +-- contributor license agreements. See the NOTICE file distributed with +-- this work for additional information regarding copyright ownership. +-- The ASF licenses this file to You under the Apache License, Version 2.0 +-- (the "License"); you may not use this file except in compliance with +-- the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- + +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: inventory +-- ---------------------------------------------------------------------------------------------------------------- +CREATE DATABASE IF NOT EXISTS `mysql_cdc`; + +use mysql_cdc; +-- Create a mysql data source table +CREATE TABLE mysql_cdc_e2e_source_table +( + `id` int NOT NULL AUTO_INCREMENT, + `f_binary` binary(64) DEFAULT NULL, + `f_blob` blob, + `f_long_varbinary` mediumblob, + `f_longblob` longblob, + `f_tinyblob` tinyblob, + `f_varbinary` varbinary(100) DEFAULT NULL, + `f_smallint` smallint DEFAULT NULL, + `f_smallint_unsigned` smallint unsigned DEFAULT NULL, + `f_mediumint` mediumint DEFAULT NULL, + `f_mediumint_unsigned` mediumint unsigned DEFAULT NULL, + `f_int` int DEFAULT NULL, + `f_int_unsigned` int unsigned DEFAULT NULL, + `f_integer` int DEFAULT NULL, + `f_integer_unsigned` int unsigned DEFAULT NULL, + `f_bigint` bigint DEFAULT NULL, + `f_bigint_unsigned` bigint unsigned DEFAULT NULL, + `f_numeric` decimal(10, 0) DEFAULT NULL, + `f_decimal` decimal(10, 0) DEFAULT NULL, + `f_float` float DEFAULT NULL, + `f_double` double DEFAULT NULL, + `f_double_precision` double DEFAULT NULL, + `f_longtext` longtext, + `f_mediumtext` mediumtext, + `f_text` text, + `f_tinytext` tinytext, + `f_varchar` varchar(100) DEFAULT NULL, + `f_date` date DEFAULT NULL, + `f_datetime` datetime DEFAULT NULL, + `f_timestamp` timestamp NULL DEFAULT NULL, + `f_bit1` bit(1) DEFAULT NULL, + `f_bit64` bit(64) DEFAULT NULL, + `f_char` char(1) DEFAULT NULL, + `f_enum` enum ('enum1','enum2','enum3') DEFAULT NULL, + `f_mediumblob` mediumblob, + `f_long_varchar` mediumtext, + `f_real` double DEFAULT NULL, + `f_time` time DEFAULT NULL, + `f_tinyint` tinyint DEFAULT NULL, + `f_tinyint_unsigned` tinyint unsigned DEFAULT NULL, + `f_json` json DEFAULT NULL, + `f_year` year DEFAULT NULL, + PRIMARY KEY (`id`) +) ENGINE = InnoDB + AUTO_INCREMENT = 2 + DEFAULT CHARSET = utf8mb4 + COLLATE = utf8mb4_0900_ai_ci; + +CREATE TABLE mysql_cdc_e2e_source_table_no_primary_key +( + `id` int NOT NULL, + `f_binary` binary(64) DEFAULT NULL, + `f_blob` blob, + `f_long_varbinary` mediumblob, + `f_longblob` longblob, + `f_tinyblob` tinyblob, + `f_varbinary` varbinary(100) DEFAULT NULL, + `f_smallint` smallint DEFAULT NULL, + `f_smallint_unsigned` smallint unsigned DEFAULT NULL, + `f_mediumint` mediumint DEFAULT NULL, + `f_mediumint_unsigned` mediumint unsigned DEFAULT NULL, + `f_int` int DEFAULT NULL, + `f_int_unsigned` int unsigned DEFAULT NULL, + `f_integer` int DEFAULT NULL, + `f_integer_unsigned` int unsigned DEFAULT NULL, + `f_bigint` bigint DEFAULT NULL, + `f_bigint_unsigned` bigint unsigned DEFAULT NULL, + `f_numeric` decimal(10, 0) DEFAULT NULL, + `f_decimal` decimal(10, 0) DEFAULT NULL, + `f_float` float DEFAULT NULL, + `f_double` double DEFAULT NULL, + `f_double_precision` double DEFAULT NULL, + `f_longtext` longtext, + `f_mediumtext` mediumtext, + `f_text` text, + `f_tinytext` tinytext, + `f_varchar` varchar(100) DEFAULT NULL, + `f_date` date DEFAULT NULL, + `f_datetime` datetime DEFAULT NULL, + `f_timestamp` timestamp NULL DEFAULT NULL, + `f_bit1` bit(1) DEFAULT NULL, + `f_bit64` bit(64) DEFAULT NULL, + `f_char` char(1) DEFAULT NULL, + `f_enum` enum ('enum1','enum2','enum3') DEFAULT NULL, + `f_mediumblob` mediumblob, + `f_long_varchar` mediumtext, + `f_real` double DEFAULT NULL, + `f_time` time DEFAULT NULL, + `f_tinyint` tinyint DEFAULT NULL, + `f_tinyint_unsigned` tinyint unsigned DEFAULT NULL, + `f_json` json DEFAULT NULL, + `f_year` year DEFAULT NULL +) ENGINE = InnoDB + DEFAULT CHARSET = utf8mb4 + COLLATE = utf8mb4_0900_ai_ci; + +truncate table mysql_cdc_e2e_source_table; +truncate table mysql_cdc_e2e_source_table_no_primary_key; + +INSERT INTO mysql_cdc_e2e_source_table ( id, f_binary, f_blob, f_long_varbinary, f_longblob, f_tinyblob, f_varbinary, f_smallint, + f_smallint_unsigned, f_mediumint, f_mediumint_unsigned, f_int, f_int_unsigned, f_integer, + f_integer_unsigned, f_bigint, f_bigint_unsigned, f_numeric, f_decimal, f_float, f_double, + f_double_precision, f_longtext, f_mediumtext, f_text, f_tinytext, f_varchar, f_date, f_datetime, + f_timestamp, f_bit1, f_bit64, f_char, f_enum, f_mediumblob, f_long_varchar, f_real, f_time, + f_tinyint, f_tinyint_unsigned, f_json, f_year ) +VALUES ( 1, 0x61626374000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000, + 0x68656C6C6F, 0x18000000789C0BC9C82C5600A244859CFCBC7485B2C4A2A4CCBCC4A24A00697308D4, NULL, + 0x74696E79626C6F62, 0x48656C6C6F20776F726C64, 12345, 54321, 123456, 654321, 1234567, 7654321, 1234567, 7654321, + 123456789, 987654321, 123, 789, 12.34, 56.78, 90.12, 'This is a long text field', 'This is a medium text field', + 'This is a text field', 'This is a tiny text field', 'This is a varchar field', '2022-04-27', '2022-04-27 14:30:00', + '2023-04-27 11:08:40', 1, b'0101010101010101010101010101010101010101010101010101010101010101', 'C', 'enum2', + 0x1B000000789C0BC9C82C5600A24485DCD494CCD25C85A49CFC2485B4CCD49C140083FF099A, 'This is a long varchar field', + 12.345, '14:30:00', -128, 255, '{ "key": "value" }', 2022 ), + ( 2, 0x61626374000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000, + 0x68656C6C6F, 0x18000000789C0BC9C82C5600A244859CFCBC7485B2C4A2A4CCBCC4A24A00697308D4, NULL, 0x74696E79626C6F62, + 0x48656C6C6F20776F726C64, 12345, 54321, 123456, 654321, 1234567, 7654321, 1234567, 7654321, 123456789, 987654321, + 123, 789, 12.34, 56.78, 90.12, 'This is a long text field', 'This is a medium text field', 'This is a text field', + 'This is a tiny text field', 'This is a varchar field', '2022-04-27', '2022-04-27 14:30:00', '2023-04-27 11:08:40', + 1, b'0101010101010101010101010101010101010101010101010101010101010101', 'C', 'enum2', + 0x1B000000789C0BC9C82C5600A24485DCD494CCD25C85A49CFC2485B4CCD49C140083FF099A, 'This is a long varchar field', + 112.345, '14:30:00', -128, 22, '{ "key": "value" }', 2013 ), + ( 3, 0x61626374000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000, + 0x68656C6C6F, 0x18000000789C0BC9C82C5600A244859CFCBC7485B2C4A2A4CCBCC4A24A00697308D4, NULL, 0x74696E79626C6F62, + 0x48656C6C6F20776F726C64, 12345, 54321, 123456, 654321, 1234567, 7654321, 1234567, 7654321, 123456789, 987654321, 123, + 789, 12.34, 56.78, 90.12, 'This is a long text field', 'This is a medium text field', 'This is a text field', + 'This is a tiny text field', 'This is a varchar field', '2022-04-27', '2022-04-27 14:30:00', '2023-04-27 11:08:40', + 1, b'0101010101010101010101010101010101010101010101010101010101010101', 'C', 'enum2', + 0x1B000000789C0BC9C82C5600A24485DCD494CCD25C85A49CFC2485B4CCD49C140083FF099A, 'This is a long varchar field', 112.345, + '14:30:00', -128, 22, '{ "key": "value" }', 2021 ); + +INSERT INTO mysql_cdc_e2e_source_table_no_primary_key ( id, f_binary, f_blob, f_long_varbinary, f_longblob, f_tinyblob, f_varbinary, f_smallint, + f_smallint_unsigned, f_mediumint, f_mediumint_unsigned, f_int, f_int_unsigned, f_integer, + f_integer_unsigned, f_bigint, f_bigint_unsigned, f_numeric, f_decimal, f_float, f_double, + f_double_precision, f_longtext, f_mediumtext, f_text, f_tinytext, f_varchar, f_date, f_datetime, + f_timestamp, f_bit1, f_bit64, f_char, f_enum, f_mediumblob, f_long_varchar, f_real, f_time, + f_tinyint, f_tinyint_unsigned, f_json, f_year ) +VALUES ( 1, 0x61626374000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000, + 0x68656C6C6F, 0x18000000789C0BC9C82C5600A244859CFCBC7485B2C4A2A4CCBCC4A24A00697308D4, NULL, + 0x74696E79626C6F62, 0x48656C6C6F20776F726C64, 12345, 54321, 123456, 654321, 1234567, 7654321, 1234567, 7654321, + 123456789, 987654321, 123, 789, 12.34, 56.78, 90.12, 'This is a long text field', 'This is a medium text field', + 'This is a text field', 'This is a tiny text field', 'This is a varchar field', '2022-04-27', '2022-04-27 14:30:00', + '2023-04-27 11:08:40', 1, b'0101010101010101010101010101010101010101010101010101010101010101', 'C', 'enum2', + 0x1B000000789C0BC9C82C5600A24485DCD494CCD25C85A49CFC2485B4CCD49C140083FF099A, 'This is a long varchar field', + 12.345, '14:30:00', -128, 255, '{ "key": "value" }', 2022 ), + ( 2, 0x61626374000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000, + 0x68656C6C6F, 0x18000000789C0BC9C82C5600A244859CFCBC7485B2C4A2A4CCBCC4A24A00697308D4, NULL, 0x74696E79626C6F62, + 0x48656C6C6F20776F726C64, 12345, 54321, 123456, 654321, 1234567, 7654321, 1234567, 7654321, 123456789, 987654321, + 123, 789, 12.34, 56.78, 90.12, 'This is a long text field', 'This is a medium text field', 'This is a text field', + 'This is a tiny text field', 'This is a varchar field', '2022-04-27', '2022-04-27 14:30:00', '2023-04-27 11:08:40', + 1, b'0101010101010101010101010101010101010101010101010101010101010101', 'C', 'enum2', + 0x1B000000789C0BC9C82C5600A24485DCD494CCD25C85A49CFC2485B4CCD49C140083FF099A, 'This is a long varchar field', + 112.345, '14:30:00', -128, 22, '{ "key": "value" }', 2013 ), + ( 3, 0x61626374000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000, + 0x68656C6C6F, 0x18000000789C0BC9C82C5600A244859CFCBC7485B2C4A2A4CCBCC4A24A00697308D4, NULL, 0x74696E79626C6F62, + 0x48656C6C6F20776F726C64, 12345, 54321, 123456, 654321, 1234567, 7654321, 1234567, 7654321, 123456789, 987654321, 123, + 789, 12.34, 56.78, 90.12, 'This is a long text field', 'This is a medium text field', 'This is a text field', + 'This is a tiny text field', 'This is a varchar field', '2022-04-27', '2022-04-27 14:30:00', '2023-04-27 11:08:40', + 1, b'0101010101010101010101010101010101010101010101010101010101010101', 'C', 'enum2', + 0x1B000000789C0BC9C82C5600A24485DCD494CCD25C85A49CFC2485B4CCD49C140083FF099A, 'This is a long varchar field', 112.345, + '14:30:00', -128, 22, '{ "key": "value" }', 2021 ); \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/iceberg/fake_to_iceberg.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/iceberg/fake_to_iceberg.conf new file mode 100644 index 000000000000..3ed5ba76f632 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/iceberg/fake_to_iceberg.conf @@ -0,0 +1,74 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + parallelism = 1 + job.mode = "BATCH" + + # You can set spark configuration here + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local +} + +source { + FakeSource { + row.num = 100 + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_decimal = "decimal(30, 8)" + c_bytes = bytes + c_date = date + c_timestamp = timestamp + } + } + result_table_name = "fake" + } +} + +transform { +} + +sink { + Iceberg { + catalog_name="seatunnel_test" + iceberg.catalog.config={ + "type"="hadoop" + "warehouse"="file:///tmp/seatunnel/iceberg/hadoop-sink/" + } + namespace="seatunnel_namespace" + table="iceberg_sink_table" + iceberg.table.write-props={ + write.format.default="parquet" + write.target-file-size-bytes=10 + } + iceberg.table.partition-keys="c_timestamp" + case_sensitive=true + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/iceberg/iceberg_source.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/iceberg/iceberg_source.conf index 969e85e1232b..d6e29483acb9 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/iceberg/iceberg_source.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/iceberg/iceberg_source.conf @@ -45,8 +45,10 @@ source { } } catalog_name = "seatunnel" - catalog_type = "hadoop" - warehouse = "file:///tmp/seatunnel/iceberg/hadoop/" + iceberg.catalog.config={ + "type"="hadoop" + "warehouse"="file:///tmp/seatunnel/iceberg/hadoop/" + } namespace = "database1" table = "source" result_table_name = "iceberg" diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/iceberg/mysql_cdc_to_iceberg.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/iceberg/mysql_cdc_to_iceberg.conf new file mode 100644 index 000000000000..6b0a5308a76c --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/iceberg/mysql_cdc_to_iceberg.conf @@ -0,0 +1,64 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + parallelism = 1 + job.mode = "STREAMING" + checkpoint.interval = 5000 +} + +source { + MySQL-CDC { + result_table_name="customer_result_table" + catalog { + factory = Mysql + } + database-names=["mysql_cdc"] + table-names = ["mysql_cdc.mysql_cdc_e2e_source_table"] + format=DEFAULT + username = "st_user" + password = "seatunnel" + base-url = "jdbc:mysql://mysql_cdc_e2e:3306/mysql_cdc" + } +} + +transform { +} + +sink { + Iceberg { + catalog_name="seatunnel_test" + iceberg.catalog.config={ + "type"="hadoop" + "warehouse"="file:///tmp/seatunnel/iceberg/hadoop-cdc-sink/" + } + namespace="seatunnel_namespace" + table="iceberg_sink_table" + iceberg.table.write-props={ + write.format.default="parquet" + write.target-file-size-bytes=10 + } + iceberg.table.primary-keys="id" + iceberg.table.partition-keys="f_datetime" + iceberg.table.upsert-mode-enabled=true + iceberg.table.schema-evolution-enabled=true + case_sensitive=true + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/iceberg/mysql_cdc_to_iceberg_for_schema_change.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/iceberg/mysql_cdc_to_iceberg_for_schema_change.conf new file mode 100644 index 000000000000..dd688a5ef539 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/iceberg/mysql_cdc_to_iceberg_for_schema_change.conf @@ -0,0 +1,68 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + parallelism = 1 + job.mode = "STREAMING" + checkpoint.interval = 5000 +} + +source { + MySQL-CDC { + result_table_name="customer_result_table" + catalog { + factory = Mysql + } + debezium = { + # include ddl + "include.schema.changes" = true + } + database-names=["mysql_cdc"] + table-names = ["mysql_cdc.mysql_cdc_e2e_source_table"] + format=DEFAULT + username = "st_user" + password = "seatunnel" + base-url = "jdbc:mysql://mysql_cdc_e2e:3306/mysql_cdc" + } +} + +transform { +} + +sink { + Iceberg { + catalog_name="seatunnel_test" + iceberg.catalog.config={ + "type"="hadoop" + "warehouse"="file:///tmp/seatunnel/iceberg/hadoop-cdc-sink/" + } + namespace="seatunnel_namespace" + table="iceberg_sink_table" + iceberg.table.write-props={ + write.format.default="parquet" + write.target-file-size-bytes=10 + } + iceberg.table.primary-keys="id" + iceberg.table.partition-keys="f_datetime" + iceberg.table.upsert-mode-enabled=true + iceberg.table.schema-evolution-enabled=true + case_sensitive=true + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/mysql/server-gtids/my.cnf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/mysql/server-gtids/my.cnf new file mode 100644 index 000000000000..a390897885d0 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/mysql/server-gtids/my.cnf @@ -0,0 +1,65 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# For advice on how to change settings please see +# http://dev.mysql.com/doc/refman/5.7/en/server-configuration-defaults.html + +[mysqld] +# +# Remove leading # and set to the amount of RAM for the most important data +# cache in MySQL. Start at 70% of total RAM for dedicated server, else 10%. +# innodb_buffer_pool_size = 128M +# +# Remove leading # to turn on a very important data integrity option: logging +# changes to the binary log between backups. +# log_bin +# +# Remove leading # to set options mainly useful for reporting servers. +# The server defaults are faster for transactions and fast SELECTs. +# Adjust sizes as needed, experiment to find the optimal values. +# join_buffer_size = 128M +# sort_buffer_size = 2M +# read_rnd_buffer_size = 2M +skip-host-cache +skip-name-resolve +#datadir=/var/lib/mysql +#socket=/var/lib/mysql/mysql.sock +secure-file-priv=/var/lib/mysql +user=mysql + +# Disabling symbolic-links is recommended to prevent assorted security risks +symbolic-links=0 + +#log-error=/var/log/mysqld.log +#pid-file=/var/run/mysqld/mysqld.pid + +# ---------------------------------------------- +# Enable the binlog for replication & CDC +# ---------------------------------------------- + +# Enable binary replication log and set the prefix, expiration, and log format. +# The prefix is arbitrary, expiration can be short for integration tests but would +# be longer on a production system. Row-level info is required for ingest to work. +# Server ID is required, but this will vary on production systems +server-id = 223344 +log_bin = mysql-bin +expire_logs_days = 1 +binlog_format = row + +# enable gtid mode +gtid_mode = on +enforce_gtid_consistency = on \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/mysql/setup.sql b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/mysql/setup.sql new file mode 100644 index 000000000000..aa4534e0ad54 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-e2e/src/test/resources/mysql/setup.sql @@ -0,0 +1,27 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one or more +-- contributor license agreements. See the NOTICE file distributed with +-- this work for additional information regarding copyright ownership. +-- The ASF licenses this file to You under the Apache License, Version 2.0 +-- (the "License"); you may not use this file except in compliance with +-- the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- + +-- In production you would almost certainly limit the replication user must be on the follower (slave) machine, +-- to prevent other clients accessing the log from other machines. For example, 'replicator'@'follower.acme.com'. +-- However, in this database we'll grant 2 users different privileges: +-- +-- 1) 'st_user' - all privileges required by the snapshot reader AND binlog reader (used for testing) +-- 2) 'mysqluser' - all privileges +-- +GRANT SELECT, RELOAD, SHOW DATABASES, REPLICATION SLAVE, REPLICATION CLIENT, DROP, LOCK TABLES ON *.* TO 'st_user'@'%'; +CREATE USER 'mysqluser' IDENTIFIED BY 'mysqlpw'; +GRANT ALL PRIVILEGES ON *.* TO 'mysqluser'@'%'; diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-hadoop3-e2e/src/test/java/org/apache/seatunnel/e2e/connector/iceberg/hadoop3/IcebergSourceIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-hadoop3-e2e/src/test/java/org/apache/seatunnel/e2e/connector/iceberg/hadoop3/IcebergSourceIT.java index f63510e3f3e5..27eb102866c3 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-hadoop3-e2e/src/test/java/org/apache/seatunnel/e2e/connector/iceberg/hadoop3/IcebergSourceIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-hadoop3-e2e/src/test/java/org/apache/seatunnel/e2e/connector/iceberg/hadoop3/IcebergSourceIT.java @@ -17,8 +17,11 @@ package org.apache.seatunnel.e2e.connector.iceberg.hadoop3; -import org.apache.seatunnel.connectors.seatunnel.iceberg.IcebergCatalogFactory; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.connectors.seatunnel.iceberg.IcebergCatalogLoader; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.CommonConfig; import org.apache.seatunnel.connectors.seatunnel.iceberg.config.IcebergCatalogType; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.SourceConfig; import org.apache.seatunnel.e2e.common.TestResource; import org.apache.seatunnel.e2e.common.TestSuiteBase; import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; @@ -63,7 +66,9 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import static org.apache.seatunnel.connectors.seatunnel.iceberg.config.IcebergCatalogType.HADOOP; @@ -133,7 +138,19 @@ public void testIcebergSource(TestContainer container) } private void initializeIcebergTable() { - CATALOG = new IcebergCatalogFactory(CATALOG_NAME, CATALOG_TYPE, WAREHOUSE, null).create(); + Map configs = new HashMap<>(); + + // add catalog properties + Map catalogProps = new HashMap<>(); + catalogProps.put("type", CATALOG_TYPE.getType()); + catalogProps.put("warehouse", WAREHOUSE); + + configs.put(CommonConfig.KEY_CATALOG_NAME.key(), CATALOG_NAME); + + configs.put(CommonConfig.CATALOG_PROPS.key(), catalogProps); + + ReadonlyConfig readonlyConfig = ReadonlyConfig.fromMap(configs); + CATALOG = new IcebergCatalogLoader(new SourceConfig(readonlyConfig)).loadCatalog(); if (!CATALOG.tableExists(TABLE)) { CATALOG.createTable(TABLE, SCHEMA); } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-hadoop3-e2e/src/test/resources/iceberg/iceberg_source.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-hadoop3-e2e/src/test/resources/iceberg/iceberg_source.conf index 97637f9a0480..c7cd57699ee9 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-hadoop3-e2e/src/test/resources/iceberg/iceberg_source.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-iceberg-hadoop3-e2e/src/test/resources/iceberg/iceberg_source.conf @@ -49,8 +49,10 @@ source { } } catalog_name = "seatunnel" - catalog_type = "hadoop" - warehouse = "file:///tmp/seatunnel/iceberg/hadoop3/" + iceberg.catalog.config={ + "type"="hadoop" + "warehouse"="file:///tmp/seatunnel/iceberg/hadoop3/" + } namespace = "database1" table = "source" result_table_name = "iceberg" diff --git a/seatunnel-shade/seatunnel-hadoop3-3.1.4-uber/pom.xml b/seatunnel-shade/seatunnel-hadoop3-3.1.4-uber/pom.xml index b80f07ea91df..00b55265c495 100644 --- a/seatunnel-shade/seatunnel-hadoop3-3.1.4-uber/pom.xml +++ b/seatunnel-shade/seatunnel-hadoop3-3.1.4-uber/pom.xml @@ -47,7 +47,6 @@ hadoop-client ${hadoop3.version} -