From afe4aec4db795f4829757d738a5bbcf1b7db8fd2 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Tue, 5 Dec 2023 17:14:09 +0100 Subject: [PATCH 01/47] Spark: Don't allow branch_ usage with VERSION AS OF (#9219) --- .../org/apache/iceberg/spark/SparkCatalog.java | 2 +- .../apache/iceberg/spark/source/SparkTable.java | 4 ++++ .../org/apache/iceberg/spark/sql/TestSelect.java | 15 +++++++++++++++ .../org/apache/iceberg/spark/SparkCatalog.java | 2 +- .../apache/iceberg/spark/source/SparkTable.java | 4 ++++ .../org/apache/iceberg/spark/sql/TestSelect.java | 15 +++++++++++++++ .../org/apache/iceberg/spark/SparkCatalog.java | 2 +- .../apache/iceberg/spark/source/SparkTable.java | 4 ++++ .../org/apache/iceberg/spark/sql/TestSelect.java | 15 +++++++++++++++ 9 files changed, 60 insertions(+), 3 deletions(-) diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java index 02bbec6824f3..48decf995fbb 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java @@ -171,7 +171,7 @@ public Table loadTable(Identifier ident, String version) throws NoSuchTableExcep SparkTable sparkTable = (SparkTable) table; Preconditions.checkArgument( - sparkTable.snapshotId() == null, + sparkTable.snapshotId() == null && sparkTable.branch() == null, "Cannot do time-travel based on both table identifier and AS OF"); try { diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java index 574d014e8335..eddcdb1819ee 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java @@ -173,6 +173,10 @@ public Long snapshotId() { return snapshotId; } + public String branch() { + return branch; + } + public SparkTable copyWithSnapshotId(long newSnapshotId) { return new SparkTable(icebergTable, newSnapshotId, refreshEagerly); } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java index e08bc4574dbf..1368c26792ee 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java @@ -426,6 +426,21 @@ public void testInvalidTimeTravelBasedOnBothAsOfAndTableIdentifier() { }); } + @Test + public void testInvalidTimeTravelAgainstBranchIdentifierWithAsOf() { + long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); + validationCatalog.loadTable(tableIdent).manageSnapshots().createBranch("b1").commit(); + + // create a second snapshot + sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); + + // using branch_b1 in the table identifier and VERSION AS OF + Assertions.assertThatThrownBy( + () -> sql("SELECT * FROM %s.branch_b1 VERSION AS OF %s", tableName, snapshotId)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); + } + @Test public void testSpecifySnapshotAndTimestamp() { // get the snapshot ID of the last write diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java index 6958ebc1034a..6b7becc77cd7 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java @@ -170,7 +170,7 @@ public Table loadTable(Identifier ident, String version) throws NoSuchTableExcep SparkTable sparkTable = (SparkTable) table; Preconditions.checkArgument( - sparkTable.snapshotId() == null, + sparkTable.snapshotId() == null && sparkTable.branch() == null, "Cannot do time-travel based on both table identifier and AS OF"); try { diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java index e200bee03e9f..bbc7434138ed 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java @@ -173,6 +173,10 @@ public Long snapshotId() { return snapshotId; } + public String branch() { + return branch; + } + public SparkTable copyWithSnapshotId(long newSnapshotId) { return new SparkTable(icebergTable, newSnapshotId, refreshEagerly); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java index dacaee7d8030..161c2e0ba637 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java @@ -421,6 +421,21 @@ public void testInvalidTimeTravelBasedOnBothAsOfAndTableIdentifier() { .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); } + @Test + public void testInvalidTimeTravelAgainstBranchIdentifierWithAsOf() { + long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); + validationCatalog.loadTable(tableIdent).manageSnapshots().createBranch("b1").commit(); + + // create a second snapshot + sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); + + // using branch_b1 in the table identifier and VERSION AS OF + Assertions.assertThatThrownBy( + () -> sql("SELECT * FROM %s.branch_b1 VERSION AS OF %s", tableName, snapshotId)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); + } + @Test public void testSpecifySnapshotAndTimestamp() { // get the snapshot ID of the last write diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java index 467b42899480..eef0f0703bc3 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java @@ -170,7 +170,7 @@ public Table loadTable(Identifier ident, String version) throws NoSuchTableExcep SparkTable sparkTable = (SparkTable) table; Preconditions.checkArgument( - sparkTable.snapshotId() == null, + sparkTable.snapshotId() == null && sparkTable.branch() == null, "Cannot do time-travel based on both table identifier and AS OF"); try { diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java index e200bee03e9f..bbc7434138ed 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java @@ -173,6 +173,10 @@ public Long snapshotId() { return snapshotId; } + public String branch() { + return branch; + } + public SparkTable copyWithSnapshotId(long newSnapshotId) { return new SparkTable(icebergTable, newSnapshotId, refreshEagerly); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java index dacaee7d8030..161c2e0ba637 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java @@ -421,6 +421,21 @@ public void testInvalidTimeTravelBasedOnBothAsOfAndTableIdentifier() { .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); } + @Test + public void testInvalidTimeTravelAgainstBranchIdentifierWithAsOf() { + long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); + validationCatalog.loadTable(tableIdent).manageSnapshots().createBranch("b1").commit(); + + // create a second snapshot + sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); + + // using branch_b1 in the table identifier and VERSION AS OF + Assertions.assertThatThrownBy( + () -> sql("SELECT * FROM %s.branch_b1 VERSION AS OF %s", tableName, snapshotId)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); + } + @Test public void testSpecifySnapshotAndTimestamp() { // get the snapshot ID of the last write From 8519224de33b26cfd7c539ffe6f123ea66165711 Mon Sep 17 00:00:00 2001 From: pvary Date: Tue, 5 Dec 2023 17:48:10 +0100 Subject: [PATCH 02/47] Flink: Document watermark generation feature (#9179) --- docs/flink-queries.md | 69 +++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 69 insertions(+) diff --git a/docs/flink-queries.md b/docs/flink-queries.md index 4cef5468cd1a..cf68fa367c21 100644 --- a/docs/flink-queries.md +++ b/docs/flink-queries.md @@ -277,6 +277,75 @@ DataStream stream = env.fromSource(source, WatermarkStrategy.noWatermarks() "Iceberg Source as Avro GenericRecord", new GenericRecordAvroTypeInfo(avroSchema)); ``` +### Emitting watermarks +Emitting watermarks from the source itself could be beneficial for several purposes, like harnessing the +[Flink Watermark Alignment](https://nightlies.apache.org/flink/flink-docs-stable/docs/dev/datastream/event-time/generating_watermarks/#watermark-alignment), +or prevent triggering [windows](https://nightlies.apache.org/flink/flink-docs-stable/docs/dev/datastream/operators/windows/) +too early when reading multiple data files concurrently. + +Enable watermark generation for an `IcebergSource` by setting the `watermarkColumn`. +The supported column types are `timestamp`, `timestamptz` and `long`. +Iceberg `timestamp` or `timestamptz` inherently contains the time precision. So there is no need +to specify the time unit. But `long` type column doesn't contain time unit information. Use +`watermarkTimeUnit` to configure the conversion for long columns. + +The watermarks are generated based on column metrics stored for data files and emitted once per split. +If multiple smaller files with different time ranges are combined into a single split, it can increase +the out-of-orderliness and extra data buffering in the Flink state. The main purpose of watermark alignment +is to reduce out-of-orderliness and excess data buffering in the Flink state. Hence it is recommended to +set `read.split.open-file-cost` to a very large value to prevent combining multiple smaller files into a +single split. The negative impact (of not combining small files into a single split) is on read throughput, +especially if there are many small files. In typical stateful processing jobs, source read throughput is not +the bottleneck. Hence this is probably a reasonable tradeoff. + +This feature requires column-level min-max stats. Make sure stats are generated for the watermark column +during write phase. By default, the column metrics are collected for the first 100 columns of the table. +If watermark column doesn't have stats enabled by default, use +[write properties](configuration.md#write-properties) starting with `write.metadata.metrics` when needed. + +The following example could be useful if watermarks are used for windowing. The source reads Iceberg data files +in order, using a timestamp column and emits watermarks: +```java +StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(); +TableLoader tableLoader = TableLoader.fromHadoopTable("hdfs://nn:8020/warehouse/path"); + +DataStream stream = + env.fromSource( + IcebergSource.forRowData() + .tableLoader(tableLoader) + // Watermark using timestamp column + .watermarkColumn("timestamp_column") + .build(), + // Watermarks are generated by the source, no need to generate it manually + WatermarkStrategy.noWatermarks() + // Extract event timestamp from records + .withTimestampAssigner((record, eventTime) -> record.getTimestamp(pos, precision).getMillisecond()), + SOURCE_NAME, + TypeInformation.of(RowData.class)); +``` + +Example for reading Iceberg table using a long event column for watermark alignment: +```java +StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(); +TableLoader tableLoader = TableLoader.fromHadoopTable("hdfs://nn:8020/warehouse/path"); + +DataStream stream = + env.fromSource( + IcebergSource source = IcebergSource.forRowData() + .tableLoader(tableLoader) + // Disable combining multiple files to a single split + .set(FlinkReadOptions.SPLIT_FILE_OPEN_COST, String.valueOf(TableProperties.SPLIT_SIZE_DEFAULT)) + // Watermark using long column + .watermarkColumn("long_column") + .watermarkTimeUnit(TimeUnit.MILLI_SCALE) + .build(), + // Watermarks are generated by the source, no need to generate it manually + WatermarkStrategy.noWatermarks() + .withWatermarkAlignment(watermarkGroup, maxAllowedWatermarkDrift), + SOURCE_NAME, + TypeInformation.of(RowData.class)); +``` + ## Options ### Read options From 68d491e6a344a0e081cc784db28dec60670d4d36 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Tue, 5 Dec 2023 18:27:27 +0100 Subject: [PATCH 03/47] Build: Bump datamodel-code-generator from 0.24.2 to 0.25.0 (#9189) Bumps [datamodel-code-generator](https://github.com/koxudaxi/datamodel-code-generator) from 0.24.2 to 0.25.0. - [Release notes](https://github.com/koxudaxi/datamodel-code-generator/releases) - [Commits](https://github.com/koxudaxi/datamodel-code-generator/compare/0.24.2...0.25.0) --- updated-dependencies: - dependency-name: datamodel-code-generator dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- open-api/requirements.txt | 2 +- open-api/rest-catalog-open-api.py | 10 +++++----- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/open-api/requirements.txt b/open-api/requirements.txt index a9be79a486fb..c20e40f4ad2a 100644 --- a/open-api/requirements.txt +++ b/open-api/requirements.txt @@ -16,4 +16,4 @@ # under the License. openapi-spec-validator==0.7.1 -datamodel-code-generator==0.24.2 +datamodel-code-generator==0.25.0 diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index 5da91a16f901..5a17b9d43856 100644 --- a/open-api/rest-catalog-open-api.py +++ b/open-api/rest-catalog-open-api.py @@ -614,7 +614,7 @@ class TransformTerm(BaseModel): term: Reference -class ReportMetricsRequest1(CommitReport): +class ReportMetricsRequest2(CommitReport): report_type: str = Field(..., alias='report-type') @@ -874,8 +874,8 @@ class LoadViewResult(BaseModel): config: Optional[Dict[str, str]] = None -class ReportMetricsRequest2(BaseModel): - __root__: Union[ReportMetricsRequest, ReportMetricsRequest1] +class ReportMetricsRequest(BaseModel): + __root__: Union[ReportMetricsRequest1, ReportMetricsRequest2] class ScanReport(BaseModel): @@ -901,7 +901,7 @@ class Schema(StructType): ) -class ReportMetricsRequest(ScanReport): +class ReportMetricsRequest1(ScanReport): report_type: str = Field(..., alias='report-type') @@ -914,4 +914,4 @@ class ReportMetricsRequest(ScanReport): AddSchemaUpdate.update_forward_refs() CreateTableRequest.update_forward_refs() CreateViewRequest.update_forward_refs() -ReportMetricsRequest2.update_forward_refs() +ReportMetricsRequest.update_forward_refs() From 7b12a4171e9e6b7829e7a66b447fc15cada151fc Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Tue, 5 Dec 2023 09:44:06 -0800 Subject: [PATCH 04/47] Flink: backport PR #9216 for disabling classloader check (#9226) --- .../source/TestIcebergSourceWithWatermarkExtractor.java | 6 +++++- .../source/TestIcebergSourceWithWatermarkExtractor.java | 6 +++++- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index 7547323871c1..ac4d07fa52d3 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -42,6 +42,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; import org.apache.flink.core.execution.JobClient; import org.apache.flink.metrics.Gauge; import org.apache.flink.runtime.metrics.MetricNames; @@ -98,7 +99,10 @@ public class TestIcebergSourceWithWatermarkExtractor implements Serializable { .setNumberTaskManagers(1) .setNumberSlotsPerTaskManager(PARALLELISM) .setRpcServiceSharing(RpcServiceSharing.DEDICATED) - .setConfiguration(reporter.addToConfiguration(new Configuration())) + .setConfiguration( + reporter.addToConfiguration( + // disable classloader check as Avro may cache class in the serializers. + new Configuration().set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false))) .withHaLeadershipControl() .build()); diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index 0bb2eb7766e9..aa4b9cd79e55 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -40,6 +40,7 @@ import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; import org.apache.flink.core.execution.JobClient; import org.apache.flink.metrics.Gauge; import org.apache.flink.runtime.metrics.MetricNames; @@ -92,7 +93,10 @@ public class TestIcebergSourceWithWatermarkExtractor implements Serializable { .setNumberTaskManagers(1) .setNumberSlotsPerTaskManager(PARALLELISM) .setRpcServiceSharing(RpcServiceSharing.DEDICATED) - .setConfiguration(reporter.addToConfiguration(new Configuration())) + .setConfiguration( + reporter.addToConfiguration( + // disable classloader check as Avro may cache class in the serializers. + new Configuration().set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false))) .withHaLeadershipControl() .build()); From 8b7a280a9fd0b51eb43538fbb9f6879d1c92de18 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 5 Dec 2023 18:45:02 +0100 Subject: [PATCH 05/47] Build: Bump actions/setup-java from 3 to 4 (#9200) Bumps [actions/setup-java](https://github.com/actions/setup-java) from 3 to 4. - [Release notes](https://github.com/actions/setup-java/releases) - [Commits](https://github.com/actions/setup-java/compare/v3...v4) --- updated-dependencies: - dependency-name: actions/setup-java dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/api-binary-compatibility.yml | 2 +- .github/workflows/delta-conversion-ci.yml | 4 ++-- .github/workflows/flink-ci.yml | 2 +- .github/workflows/hive-ci.yml | 4 ++-- .github/workflows/java-ci.yml | 6 +++--- .github/workflows/jmh-benchmarks.yml | 2 +- .github/workflows/publish-snapshot.yml | 2 +- .github/workflows/recurring-jmh-benchmarks.yml | 2 +- .github/workflows/spark-ci.yml | 6 +++--- 9 files changed, 15 insertions(+), 15 deletions(-) diff --git a/.github/workflows/api-binary-compatibility.yml b/.github/workflows/api-binary-compatibility.yml index fd7f6f14c2ef..245da4ecd4d1 100644 --- a/.github/workflows/api-binary-compatibility.yml +++ b/.github/workflows/api-binary-compatibility.yml @@ -46,7 +46,7 @@ jobs: # # See https://github.com/actions/checkout/issues/124 fetch-depth: 0 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: 11 diff --git a/.github/workflows/delta-conversion-ci.yml b/.github/workflows/delta-conversion-ci.yml index 67f5acdb32b2..5261b8217624 100644 --- a/.github/workflows/delta-conversion-ci.yml +++ b/.github/workflows/delta-conversion-ci.yml @@ -62,7 +62,7 @@ jobs: SPARK_LOCAL_IP: localhost steps: - uses: actions/checkout@v4 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: ${{ matrix.jvm }} @@ -91,7 +91,7 @@ jobs: SPARK_LOCAL_IP: localhost steps: - uses: actions/checkout@v4 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: ${{ matrix.jvm }} diff --git a/.github/workflows/flink-ci.yml b/.github/workflows/flink-ci.yml index 4ea17af6c0cb..30fe6c7723bc 100644 --- a/.github/workflows/flink-ci.yml +++ b/.github/workflows/flink-ci.yml @@ -65,7 +65,7 @@ jobs: SPARK_LOCAL_IP: localhost steps: - uses: actions/checkout@v4 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: ${{ matrix.jvm }} diff --git a/.github/workflows/hive-ci.yml b/.github/workflows/hive-ci.yml index d7474a185436..f582e516fcd1 100644 --- a/.github/workflows/hive-ci.yml +++ b/.github/workflows/hive-ci.yml @@ -60,7 +60,7 @@ jobs: SPARK_LOCAL_IP: localhost steps: - uses: actions/checkout@v4 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: ${{ matrix.jvm }} @@ -86,7 +86,7 @@ jobs: SPARK_LOCAL_IP: localhost steps: - uses: actions/checkout@v4 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: 8 diff --git a/.github/workflows/java-ci.yml b/.github/workflows/java-ci.yml index 4c8e3a5b8d92..4936e2b6514b 100644 --- a/.github/workflows/java-ci.yml +++ b/.github/workflows/java-ci.yml @@ -57,7 +57,7 @@ jobs: SPARK_LOCAL_IP: localhost steps: - uses: actions/checkout@v4 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: ${{ matrix.jvm }} @@ -81,7 +81,7 @@ jobs: runs-on: ubuntu-22.04 steps: - uses: actions/checkout@v4 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: 8 @@ -91,7 +91,7 @@ jobs: runs-on: ubuntu-22.04 steps: - uses: actions/checkout@v4 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: 8 diff --git a/.github/workflows/jmh-benchmarks.yml b/.github/workflows/jmh-benchmarks.yml index f2d6c659f61e..e76590543852 100644 --- a/.github/workflows/jmh-benchmarks.yml +++ b/.github/workflows/jmh-benchmarks.yml @@ -79,7 +79,7 @@ jobs: with: repository: ${{ github.event.inputs.repo }} ref: ${{ github.event.inputs.ref }} - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: 11 diff --git a/.github/workflows/publish-snapshot.yml b/.github/workflows/publish-snapshot.yml index 92c6b2b09223..c2809cfdcbac 100644 --- a/.github/workflows/publish-snapshot.yml +++ b/.github/workflows/publish-snapshot.yml @@ -34,7 +34,7 @@ jobs: with: # we need to fetch all tags so that getProjectVersion() in build.gradle correctly determines the next SNAPSHOT version from the newest tag fetch-depth: 0 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: 8 diff --git a/.github/workflows/recurring-jmh-benchmarks.yml b/.github/workflows/recurring-jmh-benchmarks.yml index e1f750041890..434b79755069 100644 --- a/.github/workflows/recurring-jmh-benchmarks.yml +++ b/.github/workflows/recurring-jmh-benchmarks.yml @@ -49,7 +49,7 @@ jobs: with: repository: ${{ github.event.inputs.repo }} ref: ${{ github.event.inputs.ref }} - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: 11 diff --git a/.github/workflows/spark-ci.yml b/.github/workflows/spark-ci.yml index a05c9aa19e29..3dacdf77c1a8 100644 --- a/.github/workflows/spark-ci.yml +++ b/.github/workflows/spark-ci.yml @@ -63,7 +63,7 @@ jobs: SPARK_LOCAL_IP: localhost steps: - uses: actions/checkout@v4 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: ${{ matrix.jvm }} @@ -93,7 +93,7 @@ jobs: SPARK_LOCAL_IP: localhost steps: - uses: actions/checkout@v4 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: ${{ matrix.jvm }} @@ -123,7 +123,7 @@ jobs: SPARK_LOCAL_IP: localhost steps: - uses: actions/checkout@v4 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: 17 From d80d7da3d0a956e1b6aeadbe26e0cb3f3ef2fb25 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Tue, 5 Dec 2023 19:04:47 +0100 Subject: [PATCH 06/47] Core: Handle IAE in default error handler (#9225) --- core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java | 3 +++ .../test/java/org/apache/iceberg/view/ViewCatalogTests.java | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java b/core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java index 846820a99d9f..d4083420efa6 100644 --- a/core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java +++ b/core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java @@ -202,6 +202,9 @@ public ErrorResponse parseResponse(int code, String json) { public void accept(ErrorResponse error) { switch (error.code()) { case 400: + if (IllegalArgumentException.class.getSimpleName().equals(error.type())) { + throw new IllegalArgumentException(error.message()); + } throw new BadRequestException("Malformed request: %s", error.message()); case 401: throw new NotAuthorizedException("Not authorized: %s", error.message()); diff --git a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java index 8cb77a776220..8280a200d605 100644 --- a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java +++ b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java @@ -252,7 +252,7 @@ public void createViewErrorCases() { .withQuery(trino.dialect(), trino.sql()) .withQuery(trino.dialect(), trino.sql()) .create()) - .isInstanceOf(Exception.class) + .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining( "Invalid view version: Cannot add multiple queries for dialect trino"); } From faa8b5075cb70d1cebea54700e39f038c623f08e Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Tue, 5 Dec 2023 13:13:33 -0800 Subject: [PATCH 07/47] Core: Fix logic for determining set of committed files in BaseTransaction when there are no new snapshots (#9221) --- .../org/apache/iceberg/BaseTransaction.java | 29 ++++++++++++------- 1 file changed, 18 insertions(+), 11 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/BaseTransaction.java b/core/src/main/java/org/apache/iceberg/BaseTransaction.java index 018f70eb16fa..30103fd87fe2 100644 --- a/core/src/main/java/org/apache/iceberg/BaseTransaction.java +++ b/core/src/main/java/org/apache/iceberg/BaseTransaction.java @@ -45,6 +45,7 @@ import org.apache.iceberg.metrics.MetricsReporter; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.util.PropertyUtil; @@ -446,16 +447,20 @@ private void commitSimpleTransaction() { } Set committedFiles = committedFiles(ops, newSnapshots); - // delete all of the files that were deleted in the most recent set of operation commits - Tasks.foreach(deletedFiles) - .suppressFailureWhenFinished() - .onFailure((file, exc) -> LOG.warn("Failed to delete uncommitted file: {}", file, exc)) - .run( - path -> { - if (committedFiles == null || !committedFiles.contains(path)) { - ops.io().deleteFile(path); - } - }); + if (committedFiles != null) { + // delete all of the files that were deleted in the most recent set of operation commits + Tasks.foreach(deletedFiles) + .suppressFailureWhenFinished() + .onFailure((file, exc) -> LOG.warn("Failed to delete uncommitted file: {}", file, exc)) + .run( + path -> { + if (!committedFiles.contains(path)) { + ops.io().deleteFile(path); + } + }); + } else { + LOG.warn("Failed to load metadata for a committed snapshot, skipping clean-up"); + } } catch (RuntimeException e) { LOG.warn("Failed to load committed metadata, skipping clean-up", e); @@ -502,9 +507,11 @@ private void applyUpdates(TableOperations underlyingOps) { } } + // committedFiles returns null whenever the set of committed files + // cannot be determined from the provided snapshots private static Set committedFiles(TableOperations ops, Set snapshotIds) { if (snapshotIds.isEmpty()) { - return null; + return ImmutableSet.of(); } Set committedFiles = Sets.newHashSet(); From 8e1900dc9ab917cd97a41e7145166a8c8a4bdfc5 Mon Sep 17 00:00:00 2001 From: Junhao Liu Date: Tue, 5 Dec 2023 15:48:30 -0600 Subject: [PATCH 08/47] Style: Replace Arrays.asList with Collections.singletonList (#9213) --- .../aws/s3/signer/TestS3SignRequestParser.java | 13 +++++++------ .../aws/s3/signer/TestS3SignResponseParser.java | 5 +++-- .../org/apache/iceberg/BaseRewriteManifests.java | 4 ++-- .../test/java/org/apache/iceberg/ScanTestBase.java | 12 ++++++------ .../org/apache/iceberg/TestContentFileParser.java | 3 ++- .../enumerator/TestContinuousIcebergEnumerator.java | 11 ++++++----- .../source/reader/TestIcebergSourceReader.java | 2 +- .../source/reader/TestIcebergSourceReader.java | 2 +- .../source/reader/TestIcebergSourceReader.java | 2 +- .../org/apache/iceberg/hive/TestHiveSchemaUtil.java | 7 +++++-- 10 files changed, 34 insertions(+), 27 deletions(-) diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignRequestParser.java b/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignRequestParser.java index a6928183f705..2c5f74ad8064 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignRequestParser.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignRequestParser.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.JsonNode; import java.net.URI; import java.util.Arrays; +import java.util.Collections; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; @@ -122,9 +123,9 @@ public void roundTripSerde() { "amz-sdk-request", Arrays.asList("attempt=1", "max=4"), "Content-Length", - Arrays.asList("191"), + Collections.singletonList("191"), "Content-Type", - Arrays.asList("application/json"), + Collections.singletonList("application/json"), "User-Agent", Arrays.asList("aws-sdk-java/2.20.18", "Linux/5.4.0-126"))) .build(); @@ -158,9 +159,9 @@ public void roundTripSerdeWithProperties() { "amz-sdk-request", Arrays.asList("attempt=1", "max=4"), "Content-Length", - Arrays.asList("191"), + Collections.singletonList("191"), "Content-Type", - Arrays.asList("application/json"), + Collections.singletonList("application/json"), "User-Agent", Arrays.asList("aws-sdk-java/2.20.18", "Linux/5.4.0-126"))) .properties(ImmutableMap.of("k1", "v1")) @@ -198,9 +199,9 @@ public void roundTripWithBody() { "amz-sdk-request", Arrays.asList("attempt=1", "max=4"), "Content-Length", - Arrays.asList("191"), + Collections.singletonList("191"), "Content-Type", - Arrays.asList("application/json"), + Collections.singletonList("application/json"), "User-Agent", Arrays.asList("aws-sdk-java/2.20.18", "Linux/5.4.0-126"))) .properties(ImmutableMap.of("k1", "v1")) diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignResponseParser.java b/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignResponseParser.java index d7337b1b1777..d2cf132ba598 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignResponseParser.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignResponseParser.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.JsonNode; import java.net.URI; import java.util.Arrays; +import java.util.Collections; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; @@ -70,9 +71,9 @@ public void roundTripSerde() { "amz-sdk-request", Arrays.asList("attempt=1", "max=4"), "Content-Length", - Arrays.asList("191"), + Collections.singletonList("191"), "Content-Type", - Arrays.asList("application/json"), + Collections.singletonList("application/json"), "User-Agent", Arrays.asList("aws-sdk-java/2.20.18", "Linux/5.4.0-126"))) .build(); diff --git a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java index 87768e34894a..c70dda2bd6d0 100644 --- a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java +++ b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java @@ -22,8 +22,8 @@ import static org.apache.iceberg.TableProperties.MANIFEST_TARGET_SIZE_BYTES_DEFAULT; import java.io.IOException; -import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; @@ -247,7 +247,7 @@ private void performRewrite(List currentManifests) { rewrittenManifests.add(manifest); try (ManifestReader reader = ManifestFiles.read(manifest, ops.io(), ops.current().specsById()) - .select(Arrays.asList("*"))) { + .select(Collections.singletonList("*"))) { reader .liveEntries() .forEach( diff --git a/core/src/test/java/org/apache/iceberg/ScanTestBase.java b/core/src/test/java/org/apache/iceberg/ScanTestBase.java index 5e7793939925..48a8ccbaa941 100644 --- a/core/src/test/java/org/apache/iceberg/ScanTestBase.java +++ b/core/src/test/java/org/apache/iceberg/ScanTestBase.java @@ -23,7 +23,7 @@ import java.io.File; import java.io.IOException; -import java.util.Arrays; +import java.util.Collections; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicInteger; import org.apache.iceberg.expressions.Expression; @@ -56,7 +56,7 @@ public ScanTestBase(int formatVersion) { @Test public void testTableScanHonorsSelect() { - ScanT scan = newScan().select(Arrays.asList("id")); + ScanT scan = newScan().select(Collections.singletonList("id")); Schema expectedSchema = new Schema(required(1, "id", Types.IntegerType.get())); @@ -69,20 +69,20 @@ public void testTableScanHonorsSelect() { @Test public void testTableBothProjectAndSelect() { Assertions.assertThatThrownBy( - () -> newScan().select(Arrays.asList("id")).project(SCHEMA.select("data"))) + () -> newScan().select(Collections.singletonList("id")).project(SCHEMA.select("data"))) .isInstanceOf(IllegalStateException.class) .hasMessage("Cannot set projection schema when columns are selected"); Assertions.assertThatThrownBy( - () -> newScan().project(SCHEMA.select("data")).select(Arrays.asList("id"))) + () -> newScan().project(SCHEMA.select("data")).select(Collections.singletonList("id"))) .isInstanceOf(IllegalStateException.class) .hasMessage("Cannot select columns when projection schema is set"); } @Test public void testTableScanHonorsSelectWithoutCaseSensitivity() { - ScanT scan1 = newScan().caseSensitive(false).select(Arrays.asList("ID")); + ScanT scan1 = newScan().caseSensitive(false).select(Collections.singletonList("ID")); // order of refinements shouldn't matter - ScanT scan2 = newScan().select(Arrays.asList("ID")).caseSensitive(false); + ScanT scan2 = newScan().select(Collections.singletonList("ID")).caseSensitive(false); Schema expectedSchema = new Schema(required(1, "id", Types.IntegerType.get())); diff --git a/core/src/test/java/org/apache/iceberg/TestContentFileParser.java b/core/src/test/java/org/apache/iceberg/TestContentFileParser.java index f3b5e7b60c31..4fda388159ec 100644 --- a/core/src/test/java/org/apache/iceberg/TestContentFileParser.java +++ b/core/src/test/java/org/apache/iceberg/TestContentFileParser.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.JsonNode; import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.Collections; import java.util.stream.Stream; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Comparators; @@ -270,7 +271,7 @@ private static DeleteFile deleteFileWithAllOptional(PartitionSpec spec) { metrics, new int[] {3}, 1, - Arrays.asList(128L), + Collections.singletonList(128L), ByteBuffer.wrap(new byte[16])); } diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java index 349eb11cf549..5b0ed39745c5 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg.flink.source.enumerator; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.List; @@ -173,7 +172,7 @@ public void testThrottlingDiscovery() throws Exception { enumerator.handleSourceEvent(2, new SplitRequestEvent()); // add splits[0] to the planner for next discovery - splitPlanner.addSplits(Arrays.asList(splits.get(0))); + splitPlanner.addSplits(Collections.singletonList(splits.get(0))); enumeratorContext.triggerAllActions(); // because discovered split was assigned to reader, pending splits should be empty @@ -185,7 +184,7 @@ public void testThrottlingDiscovery() throws Exception { // add the remaining 9 splits (one for every snapshot) // run discovery cycles while reader-2 still processing the splits[0] for (int i = 1; i < 10; ++i) { - splitPlanner.addSplits(Arrays.asList(splits.get(i))); + splitPlanner.addSplits(Collections.singletonList(splits.get(i))); enumeratorContext.triggerAllActions(); } @@ -196,7 +195,8 @@ public void testThrottlingDiscovery() throws Exception { splits.subList(0, 1), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); // now reader-2 finished splits[0] - enumerator.handleSourceEvent(2, new SplitRequestEvent(Arrays.asList(splits.get(0).splitId()))); + enumerator.handleSourceEvent( + 2, new SplitRequestEvent(Collections.singletonList(splits.get(0).splitId()))); enumeratorContext.triggerAllActions(); // still have 3 pending splits. After assigned splits[1] to reader-2, one more split was // discovered and added. @@ -217,7 +217,8 @@ public void testThrottlingDiscovery() throws Exception { splits.subList(0, 2), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); // now reader-2 finished splits[1] - enumerator.handleSourceEvent(2, new SplitRequestEvent(Arrays.asList(splits.get(1).splitId()))); + enumerator.handleSourceEvent( + 2, new SplitRequestEvent(Collections.singletonList(splits.get(1).splitId()))); enumeratorContext.triggerAllActions(); // still have 3 pending splits. After assigned new splits[2] to reader-2, one more split was // discovered and added. diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java index 88234c61123f..f19d57083b89 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java @@ -132,7 +132,7 @@ private void testOneSplitFetcher( ReaderUtil.createCombinedScanTask( recordBatchList, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(task); - reader.addSplits(Arrays.asList(split)); + reader.addSplits(Collections.singletonList(split)); while (readerOutput.getEmittedRecords().size() < expectedCount) { reader.pollNext(readerOutput); diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java index 88234c61123f..f19d57083b89 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java @@ -132,7 +132,7 @@ private void testOneSplitFetcher( ReaderUtil.createCombinedScanTask( recordBatchList, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(task); - reader.addSplits(Arrays.asList(split)); + reader.addSplits(Collections.singletonList(split)); while (readerOutput.getEmittedRecords().size() < expectedCount) { reader.pollNext(readerOutput); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java index 88234c61123f..f19d57083b89 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java @@ -132,7 +132,7 @@ private void testOneSplitFetcher( ReaderUtil.createCombinedScanTask( recordBatchList, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(task); - reader.addSplits(Arrays.asList(split)); + reader.addSplits(Collections.singletonList(split)); while (readerOutput.getEmittedRecords().size() < expectedCount) { reader.pollNext(readerOutput); diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveSchemaUtil.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveSchemaUtil.java index 84d11d03a741..1592a3461b40 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveSchemaUtil.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveSchemaUtil.java @@ -23,6 +23,7 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.stream.Collectors; import org.apache.hadoop.hive.metastore.api.FieldSchema; @@ -147,7 +148,9 @@ public void testSchemaConvertToIcebergSchemaForEveryPrimitiveType() { public void testNotSupportedTypes() { for (FieldSchema notSupportedField : getNotSupportedFieldSchemas()) { assertThatThrownBy( - () -> HiveSchemaUtil.convert(Lists.newArrayList(Arrays.asList(notSupportedField)))) + () -> + HiveSchemaUtil.convert( + Lists.newArrayList(Collections.singletonList(notSupportedField)))) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Unsupported Hive type"); } @@ -197,7 +200,7 @@ public void testConversionWithoutLastComment() { Arrays.asList( TypeInfoUtils.getTypeInfoFromTypeString(serdeConstants.BIGINT_TYPE_NAME), TypeInfoUtils.getTypeInfoFromTypeString(serdeConstants.STRING_TYPE_NAME)), - Arrays.asList("customer comment")); + Collections.singletonList("customer comment")); assertThat(schema.asStruct()).isEqualTo(expected.asStruct()); } From 367dc8b1b94fd8cbae2b5e4da5d01225e717c49b Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Tue, 5 Dec 2023 17:27:42 -0800 Subject: [PATCH 09/47] Core: Add comment property to ViewProperties (#9181) --- .../main/java/org/apache/iceberg/view/ViewProperties.java | 1 + .../org/apache/iceberg/view/TestViewMetadataParser.java | 7 +++++-- .../org/apache/iceberg/view/ValidViewMetadata.json | 2 +- 3 files changed, 7 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/view/ViewProperties.java b/core/src/main/java/org/apache/iceberg/view/ViewProperties.java index b356db7ce040..12b63659bcb2 100644 --- a/core/src/main/java/org/apache/iceberg/view/ViewProperties.java +++ b/core/src/main/java/org/apache/iceberg/view/ViewProperties.java @@ -25,6 +25,7 @@ public class ViewProperties { public static final String METADATA_COMPRESSION = "write.metadata.compression-codec"; public static final String METADATA_COMPRESSION_DEFAULT = "gzip"; + public static final String COMMENT = "comment"; private ViewProperties() {} } diff --git a/core/src/test/java/org/apache/iceberg/view/TestViewMetadataParser.java b/core/src/test/java/org/apache/iceberg/view/TestViewMetadataParser.java index 613b8a7fddb1..d2372363723b 100644 --- a/core/src/test/java/org/apache/iceberg/view/TestViewMetadataParser.java +++ b/core/src/test/java/org/apache/iceberg/view/TestViewMetadataParser.java @@ -107,7 +107,8 @@ public void readAndWriteValidViewMetadata() throws Exception { .addVersion(version1) .addVersion(version2) .setLocation("s3://bucket/test/location") - .setProperties(ImmutableMap.of("some-key", "some-value")) + .setProperties( + ImmutableMap.of("some-key", "some-value", ViewProperties.COMMENT, "some-comment")) .setCurrentVersionId(2) .upgradeFormatVersion(1) .build(); @@ -218,7 +219,9 @@ public void viewMetadataWithMetadataLocation() throws Exception { .addVersion(version1) .addVersion(version2) .setLocation("s3://bucket/test/location") - .setProperties(ImmutableMap.of("some-key", "some-value")) + .setProperties( + ImmutableMap.of( + "some-key", "some-value", ViewProperties.COMMENT, "some-comment")) .setCurrentVersionId(2) .upgradeFormatVersion(1) .build()) diff --git a/core/src/test/resources/org/apache/iceberg/view/ValidViewMetadata.json b/core/src/test/resources/org/apache/iceberg/view/ValidViewMetadata.json index cf022c5a7098..4b600d6f6b47 100644 --- a/core/src/test/resources/org/apache/iceberg/view/ValidViewMetadata.json +++ b/core/src/test/resources/org/apache/iceberg/view/ValidViewMetadata.json @@ -2,7 +2,7 @@ "view-uuid": "fa6506c3-7681-40c8-86dc-e36561f83385", "format-version": 1, "location": "s3://bucket/test/location", - "properties": {"some-key": "some-value"}, + "properties": {"some-key": "some-value", "comment": "some-comment"}, "current-schema-id": 0, "schemas": [ { From a89fc4646ae5e328a2ccadb98d41b79632afcf3b Mon Sep 17 00:00:00 2001 From: emkornfield Date: Wed, 6 Dec 2023 00:07:46 -0800 Subject: [PATCH 10/47] Spec: Clarify how column IDs are required (#9162) --- format/spec.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/format/spec.md b/format/spec.md index 5d6dded5ee76..80cdd6d2987f 100644 --- a/format/spec.md +++ b/format/spec.md @@ -963,7 +963,7 @@ Note that the string map case is for maps where the key type is a string. Using **Data Type Mappings** -Values should be stored in Parquet using the types and logical type annotations in the table below. Column IDs are required. +Values should be stored in Parquet using the types and logical type annotations in the table below. Column IDs are required to be stored as [field IDs](http://github.com/apache/parquet-format/blob/40699d05bd24181de6b1457babbee2c16dce3803/src/main/thrift/parquet.thrift#L459) on the parquet schema. Lists must use the [3-level representation](https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#lists). From 70ec4e5ead8db9bd946c44731192b47862bfdeea Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Wed, 6 Dec 2023 21:06:33 +0530 Subject: [PATCH 11/47] Spark: Bump Spark minor versions for 3.3 and 3.4 (#9187) --- gradle/libs.versions.toml | 4 ++-- .../iceberg/spark/extensions/TestAddFilesProcedure.java | 2 +- .../iceberg/spark/extensions/TestSnapshotTableProcedure.java | 2 +- .../integration/java/org/apache/iceberg/spark/SmokeTest.java | 4 ++-- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index d9ca188e9384..f5c391fa95b5 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -77,8 +77,8 @@ scala-collection-compat = "2.11.0" slf4j = "1.7.36" snowflake-jdbc = "3.14.3" spark-hive32 = "3.2.2" -spark-hive33 = "3.3.2" -spark-hive34 = "3.4.1" +spark-hive33 = "3.3.3" +spark-hive34 = "3.4.2" spark-hive35 = "3.5.0" spring-boot = "2.5.4" spring-web = "5.3.30" diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java index db8ffe07b8b7..63396e0969d6 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java @@ -77,7 +77,7 @@ public void setupTempDirs() { @After public void dropTables() { - sql("DROP TABLE IF EXISTS %s", sourceTableName); + sql("DROP TABLE IF EXISTS %s PURGE", sourceTableName); sql("DROP TABLE IF EXISTS %s", tableName); } diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java index 6e2bf99c5422..367d95d8f619 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java @@ -46,7 +46,7 @@ public TestSnapshotTableProcedure( @After public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); - sql("DROP TABLE IF EXISTS %S", sourceName); + sql("DROP TABLE IF EXISTS %s PURGE", sourceName); } @Test diff --git a/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java b/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java index d6fab897d62d..25d7e7471588 100644 --- a/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java +++ b/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java @@ -52,7 +52,7 @@ public void testGettingStarted() throws IOException { Assert.assertEquals( "Should have inserted 3 rows", 3L, scalarSql("SELECT COUNT(*) FROM %s", tableName)); - sql("DROP TABLE IF EXISTS source"); + sql("DROP TABLE IF EXISTS source PURGE"); sql( "CREATE TABLE source (id bigint, data string) USING parquet LOCATION '%s'", temp.newFolder()); @@ -62,7 +62,7 @@ public void testGettingStarted() throws IOException { Assert.assertEquals( "Table should now have 4 rows", 4L, scalarSql("SELECT COUNT(*) FROM %s", tableName)); - sql("DROP TABLE IF EXISTS updates"); + sql("DROP TABLE IF EXISTS updates PURGE"); sql( "CREATE TABLE updates (id bigint, data string) USING parquet LOCATION '%s'", temp.newFolder()); From d69ba0568a2e07dfb5af233350ad5668d9aef134 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 6 Dec 2023 20:22:13 +0100 Subject: [PATCH 12/47] Core: Introduce AssertViewUUID for REST catalog views (#8831) --- .../org/apache/iceberg/UpdateRequirement.java | 20 +- .../iceberg/UpdateRequirementParser.java | 17 ++ .../apache/iceberg/UpdateRequirements.java | 11 + .../iceberg/rest/RESTViewOperations.java | 7 +- .../iceberg/TestUpdateRequirementParser.java | 32 +++ .../iceberg/TestUpdateRequirements.java | 210 ++++++++++++++++++ open-api/rest-catalog-open-api.py | 5 + open-api/rest-catalog-open-api.yaml | 30 +++ 8 files changed, 325 insertions(+), 7 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/UpdateRequirement.java b/core/src/main/java/org/apache/iceberg/UpdateRequirement.java index 80ecf84efa45..fc1f55311175 100644 --- a/core/src/main/java/org/apache/iceberg/UpdateRequirement.java +++ b/core/src/main/java/org/apache/iceberg/UpdateRequirement.java @@ -25,7 +25,10 @@ /** Represents a requirement for a {@link MetadataUpdate} */ public interface UpdateRequirement { - void validate(TableMetadata base); + default void validate(TableMetadata base) { + throw new ValidationException( + "Cannot validate %s against a table", this.getClass().getSimpleName()); + } default void validate(ViewMetadata base) { throw new ValidationException( @@ -62,12 +65,25 @@ public void validate(TableMetadata base) { "Requirement failed: UUID does not match: expected %s != %s", base.uuid(), uuid); } } + } + + class AssertViewUUID implements UpdateRequirement { + private final String uuid; + + public AssertViewUUID(String uuid) { + Preconditions.checkArgument(uuid != null, "Invalid required UUID: null"); + this.uuid = uuid; + } + + public String uuid() { + return uuid; + } @Override public void validate(ViewMetadata base) { if (!uuid.equalsIgnoreCase(base.uuid())) { throw new CommitFailedException( - "Requirement failed: UUID does not match: expected %s != %s", base.uuid(), uuid); + "Requirement failed: view UUID does not match: expected %s != %s", base.uuid(), uuid); } } } diff --git a/core/src/main/java/org/apache/iceberg/UpdateRequirementParser.java b/core/src/main/java/org/apache/iceberg/UpdateRequirementParser.java index 091d4f1fc58c..5c4dc2221290 100644 --- a/core/src/main/java/org/apache/iceberg/UpdateRequirementParser.java +++ b/core/src/main/java/org/apache/iceberg/UpdateRequirementParser.java @@ -35,6 +35,7 @@ private UpdateRequirementParser() {} // assertion types static final String ASSERT_TABLE_UUID = "assert-table-uuid"; + static final String ASSERT_VIEW_UUID = "assert-view-uuid"; static final String ASSERT_TABLE_DOES_NOT_EXIST = "assert-create"; static final String ASSERT_REF_SNAPSHOT_ID = "assert-ref-snapshot-id"; static final String ASSERT_LAST_ASSIGNED_FIELD_ID = "assert-last-assigned-field-id"; @@ -68,6 +69,7 @@ private UpdateRequirementParser() {} private static final Map, String> TYPES = ImmutableMap., String>builder() .put(UpdateRequirement.AssertTableUUID.class, ASSERT_TABLE_UUID) + .put(UpdateRequirement.AssertViewUUID.class, ASSERT_VIEW_UUID) .put(UpdateRequirement.AssertTableDoesNotExist.class, ASSERT_TABLE_DOES_NOT_EXIST) .put(UpdateRequirement.AssertRefSnapshotID.class, ASSERT_REF_SNAPSHOT_ID) .put(UpdateRequirement.AssertLastAssignedFieldId.class, ASSERT_LAST_ASSIGNED_FIELD_ID) @@ -101,6 +103,9 @@ public static void toJson(UpdateRequirement updateRequirement, JsonGenerator gen case ASSERT_TABLE_UUID: writeAssertTableUUID((UpdateRequirement.AssertTableUUID) updateRequirement, generator); break; + case ASSERT_VIEW_UUID: + writeAssertViewUUID((UpdateRequirement.AssertViewUUID) updateRequirement, generator); + break; case ASSERT_REF_SNAPSHOT_ID: writeAssertRefSnapshotId( (UpdateRequirement.AssertRefSnapshotID) updateRequirement, generator); @@ -159,6 +164,8 @@ public static UpdateRequirement fromJson(JsonNode jsonNode) { return readAssertTableDoesNotExist(jsonNode); case ASSERT_TABLE_UUID: return readAssertTableUUID(jsonNode); + case ASSERT_VIEW_UUID: + return readAssertViewUUID(jsonNode); case ASSERT_REF_SNAPSHOT_ID: return readAssertRefSnapshotId(jsonNode); case ASSERT_LAST_ASSIGNED_FIELD_ID: @@ -182,6 +189,11 @@ private static void writeAssertTableUUID( gen.writeStringField(UUID, requirement.uuid()); } + private static void writeAssertViewUUID( + UpdateRequirement.AssertViewUUID requirement, JsonGenerator gen) throws IOException { + gen.writeStringField(UUID, requirement.uuid()); + } + private static void writeAssertRefSnapshotId( UpdateRequirement.AssertRefSnapshotID requirement, JsonGenerator gen) throws IOException { gen.writeStringField(NAME, requirement.refName()); @@ -231,6 +243,11 @@ private static UpdateRequirement readAssertTableUUID(JsonNode node) { return new UpdateRequirement.AssertTableUUID(uuid); } + private static UpdateRequirement readAssertViewUUID(JsonNode node) { + String uuid = JsonUtil.getString(UUID, node); + return new UpdateRequirement.AssertViewUUID(uuid); + } + private static UpdateRequirement readAssertRefSnapshotId(JsonNode node) { String name = JsonUtil.getString(NAME, node); Long snapshotId = JsonUtil.getLongOrNull(SNAPSHOT_ID, node); diff --git a/core/src/main/java/org/apache/iceberg/UpdateRequirements.java b/core/src/main/java/org/apache/iceberg/UpdateRequirements.java index 8a7a761ff2c1..6a5d07d7813d 100644 --- a/core/src/main/java/org/apache/iceberg/UpdateRequirements.java +++ b/core/src/main/java/org/apache/iceberg/UpdateRequirements.java @@ -23,6 +23,7 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.view.ViewMetadata; public class UpdateRequirements { @@ -56,6 +57,16 @@ public static List forUpdateTable( return builder.build(); } + public static List forReplaceView( + ViewMetadata base, List metadataUpdates) { + Preconditions.checkArgument(null != base, "Invalid view metadata: null"); + Preconditions.checkArgument(null != metadataUpdates, "Invalid metadata updates: null"); + Builder builder = new Builder(null, false); + builder.require(new UpdateRequirement.AssertViewUUID(base.uuid())); + metadataUpdates.forEach(builder::update); + return builder.build(); + } + private static class Builder { private final TableMetadata base; private final ImmutableList.Builder requirements = ImmutableList.builder(); diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTViewOperations.java b/core/src/main/java/org/apache/iceberg/rest/RESTViewOperations.java index 48dc075b1305..b4dafaa9031b 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTViewOperations.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTViewOperations.java @@ -21,9 +21,8 @@ import java.util.Map; import java.util.Objects; import java.util.function.Supplier; -import org.apache.iceberg.UpdateRequirement; +import org.apache.iceberg.UpdateRequirements; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.rest.requests.UpdateTableRequest; import org.apache.iceberg.rest.responses.LoadViewResponse; import org.apache.iceberg.view.ViewMetadata; @@ -62,9 +61,7 @@ public void commit(ViewMetadata base, ViewMetadata metadata) { UpdateTableRequest request = UpdateTableRequest.create( - null, - ImmutableList.of(new UpdateRequirement.AssertTableUUID(base.uuid())), - metadata.changes()); + null, UpdateRequirements.forReplaceView(base, metadata.changes()), metadata.changes()); LoadViewResponse response = client.post( diff --git a/core/src/test/java/org/apache/iceberg/TestUpdateRequirementParser.java b/core/src/test/java/org/apache/iceberg/TestUpdateRequirementParser.java index 92401b9d9ef5..cd32b9606d55 100644 --- a/core/src/test/java/org/apache/iceberg/TestUpdateRequirementParser.java +++ b/core/src/test/java/org/apache/iceberg/TestUpdateRequirementParser.java @@ -58,6 +58,25 @@ public void testAssertUUIDToJson() { .isEqualTo(expected); } + @Test + public void testAssertViewUUIDFromJson() { + String requirementType = UpdateRequirementParser.ASSERT_VIEW_UUID; + String uuid = "2cc52516-5e73-41f2-b139-545d41a4e151"; + String json = String.format("{\"type\":\"assert-view-uuid\",\"uuid\":\"%s\"}", uuid); + UpdateRequirement expected = new UpdateRequirement.AssertViewUUID(uuid); + assertEquals(requirementType, expected, UpdateRequirementParser.fromJson(json)); + } + + @Test + public void testAssertViewUUIDToJson() { + String uuid = "2cc52516-5e73-41f2-b139-545d41a4e151"; + String expected = String.format("{\"type\":\"assert-view-uuid\",\"uuid\":\"%s\"}", uuid); + UpdateRequirement actual = new UpdateRequirement.AssertViewUUID(uuid); + Assertions.assertThat(UpdateRequirementParser.toJson(actual)) + .as("AssertViewUUID should convert to the correct JSON value") + .isEqualTo(expected); + } + @Test public void testAssertTableDoesNotExistFromJson() { String requirementType = UpdateRequirementParser.ASSERT_TABLE_DOES_NOT_EXIST; @@ -262,6 +281,10 @@ public void assertEquals( (UpdateRequirement.AssertTableUUID) expected, (UpdateRequirement.AssertTableUUID) actual); break; + case UpdateRequirementParser.ASSERT_VIEW_UUID: + compareAssertViewUUID( + (UpdateRequirement.AssertViewUUID) expected, (UpdateRequirement.AssertViewUUID) actual); + break; case UpdateRequirementParser.ASSERT_TABLE_DOES_NOT_EXIST: // Don't cast here as the function explicitly tests that the types are correct, given that // the generated JSON @@ -312,6 +335,15 @@ private static void compareAssertTableUUID( .isEqualTo(expected.uuid()); } + private static void compareAssertViewUUID( + UpdateRequirement.AssertViewUUID expected, UpdateRequirement.AssertViewUUID actual) { + Assertions.assertThat(actual.uuid()) + .as("UUID from JSON should not be null") + .isNotNull() + .as("UUID should parse correctly from JSON") + .isEqualTo(expected.uuid()); + } + // AssertTableDoesNotExist does not have any fields beyond the requirement type, so just check // that the classes // are the same and as expected. diff --git a/core/src/test/java/org/apache/iceberg/TestUpdateRequirements.java b/core/src/test/java/org/apache/iceberg/TestUpdateRequirements.java index bf3c32628ab0..ed1142441736 100644 --- a/core/src/test/java/org/apache/iceberg/TestUpdateRequirements.java +++ b/core/src/test/java/org/apache/iceberg/TestUpdateRequirements.java @@ -26,11 +26,14 @@ import java.util.List; import java.util.UUID; +import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; +import org.apache.iceberg.view.ImmutableViewVersion; +import org.apache.iceberg.view.ViewMetadata; import org.assertj.core.api.InstanceOfAssertFactories; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -38,12 +41,16 @@ public class TestUpdateRequirements { private final TableMetadata metadata = mock(TableMetadata.class); private final TableMetadata updated = mock(TableMetadata.class); + private final ViewMetadata viewMetadata = mock(ViewMetadata.class); + private final ViewMetadata updatedViewMetadata = mock(ViewMetadata.class); @BeforeEach public void before() { String uuid = UUID.randomUUID().toString(); when(metadata.uuid()).thenReturn(uuid); when(updated.uuid()).thenReturn(uuid); + when(viewMetadata.uuid()).thenReturn(uuid); + when(updatedViewMetadata.uuid()).thenReturn(uuid); } @Test @@ -67,6 +74,14 @@ public void nullCheck() { assertThatThrownBy(() -> UpdateRequirements.forReplaceTable(metadata, null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid metadata updates: null"); + + assertThatThrownBy(() -> UpdateRequirements.forReplaceView(null, null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid view metadata: null"); + + assertThatThrownBy(() -> UpdateRequirements.forReplaceView(viewMetadata, null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid metadata updates: null"); } @Test @@ -87,6 +102,13 @@ public void emptyUpdatesForUpdateAndReplaceTable() { .hasOnlyElementsOfType(UpdateRequirement.AssertTableUUID.class); } + @Test + public void emptyUpdatesForReplaceView() { + assertThat(UpdateRequirements.forReplaceView(viewMetadata, ImmutableList.of())) + .hasSize(1) + .hasOnlyElementsOfType(UpdateRequirement.AssertViewUUID.class); + } + @Test public void tableAlreadyExists() { List requirements = UpdateRequirements.forCreateTable(ImmutableList.of()); @@ -129,6 +151,39 @@ public void assignUUIDFailure() { updated.uuid(), metadata.uuid())); } + @Test + public void assignUUIDToView() { + List requirements = + UpdateRequirements.forReplaceView( + viewMetadata, + ImmutableList.of( + new MetadataUpdate.AssignUUID(viewMetadata.uuid()), + new MetadataUpdate.AssignUUID(UUID.randomUUID().toString()), + new MetadataUpdate.AssignUUID(UUID.randomUUID().toString()))); + requirements.forEach(req -> req.validate(viewMetadata)); + + assertThat(requirements) + .hasSize(1) + .hasOnlyElementsOfType(UpdateRequirement.AssertViewUUID.class); + + assertViewUUID(requirements); + } + + @Test + public void assignUUIDToViewFailure() { + List requirements = + UpdateRequirements.forReplaceView( + viewMetadata, ImmutableList.of(new MetadataUpdate.AssignUUID(viewMetadata.uuid()))); + + when(updatedViewMetadata.uuid()).thenReturn(UUID.randomUUID().toString()); + assertThatThrownBy(() -> requirements.forEach(req -> req.validate(updatedViewMetadata))) + .isInstanceOf(CommitFailedException.class) + .hasMessage( + String.format( + "Requirement failed: view UUID does not match: expected %s != %s", + updatedViewMetadata.uuid(), viewMetadata.uuid())); + } + @Test public void upgradeFormatVersion() { List requirements = @@ -143,6 +198,20 @@ public void upgradeFormatVersion() { assertTableUUID(requirements); } + @Test + public void upgradeFormatVersionForView() { + List requirements = + UpdateRequirements.forReplaceView( + viewMetadata, ImmutableList.of(new MetadataUpdate.UpgradeFormatVersion(2))); + requirements.forEach(req -> req.validate(viewMetadata)); + + assertThat(requirements) + .hasSize(1) + .hasOnlyElementsOfType(UpdateRequirement.AssertViewUUID.class); + + assertViewUUID(requirements); + } + @Test public void addSchema() { int lastColumnId = 1; @@ -190,6 +259,25 @@ public void addSchemaFailure() { .hasMessage("Requirement failed: last assigned field id changed: expected id 2 != 3"); } + @Test + public void addSchemaForView() { + int lastColumnId = 1; + List requirements = + UpdateRequirements.forReplaceView( + viewMetadata, + ImmutableList.of( + new MetadataUpdate.AddSchema(new Schema(), lastColumnId), + new MetadataUpdate.AddSchema(new Schema(), lastColumnId + 1), + new MetadataUpdate.AddSchema(new Schema(), lastColumnId + 2))); + requirements.forEach(req -> req.validate(viewMetadata)); + + assertThat(requirements) + .hasSize(1) + .hasOnlyElementsOfTypes(UpdateRequirement.AssertViewUUID.class); + + assertViewUUID(requirements); + } + @Test public void setCurrentSchema() { int schemaId = 3; @@ -553,6 +641,33 @@ public void setAndRemoveProperties() { assertTableUUID(requirements); } + @Test + public void setAndRemovePropertiesForView() { + List requirements = + UpdateRequirements.forReplaceView( + viewMetadata, + ImmutableList.of(new MetadataUpdate.SetProperties(ImmutableMap.of("test", "test")))); + requirements.forEach(req -> req.validate(viewMetadata)); + + assertThat(requirements) + .hasSize(1) + .hasOnlyElementsOfTypes(UpdateRequirement.AssertViewUUID.class); + + assertViewUUID(requirements); + + requirements = + UpdateRequirements.forReplaceView( + viewMetadata, + ImmutableList.of(new MetadataUpdate.RemoveProperties(Sets.newHashSet("test")))); + requirements.forEach(req -> req.validate(viewMetadata)); + + assertThat(requirements) + .hasSize(1) + .hasOnlyElementsOfTypes(UpdateRequirement.AssertViewUUID.class); + + assertViewUUID(requirements); + } + @Test public void setLocation() { List requirements = @@ -567,6 +682,93 @@ public void setLocation() { assertTableUUID(requirements); } + @Test + public void setLocationForView() { + List requirements = + UpdateRequirements.forReplaceView( + viewMetadata, ImmutableList.of(new MetadataUpdate.SetLocation("location"))); + requirements.forEach(req -> req.validate(viewMetadata)); + + assertThat(requirements) + .hasSize(1) + .hasOnlyElementsOfTypes(UpdateRequirement.AssertViewUUID.class); + + assertViewUUID(requirements); + } + + @Test + public void addViewVersion() { + List requirements = + UpdateRequirements.forReplaceView( + viewMetadata, + ImmutableList.of( + new MetadataUpdate.AddViewVersion( + ImmutableViewVersion.builder() + .versionId(1) + .schemaId(1) + .timestampMillis(System.currentTimeMillis()) + .defaultNamespace(Namespace.of("ns")) + .build()), + new MetadataUpdate.AddViewVersion( + ImmutableViewVersion.builder() + .versionId(2) + .schemaId(1) + .timestampMillis(System.currentTimeMillis()) + .defaultNamespace(Namespace.of("ns")) + .build()), + new MetadataUpdate.AddViewVersion( + ImmutableViewVersion.builder() + .versionId(3) + .schemaId(1) + .timestampMillis(System.currentTimeMillis()) + .defaultNamespace(Namespace.of("ns")) + .build()))); + requirements.forEach(req -> req.validate(viewMetadata)); + + assertThat(requirements) + .hasSize(1) + .hasOnlyElementsOfTypes(UpdateRequirement.AssertViewUUID.class); + + assertViewUUID(requirements); + } + + @Test + public void setCurrentViewVersion() { + List requirements = + UpdateRequirements.forReplaceView( + viewMetadata, + ImmutableList.of( + new MetadataUpdate.AddViewVersion( + ImmutableViewVersion.builder() + .versionId(3) + .schemaId(1) + .timestampMillis(System.currentTimeMillis()) + .defaultNamespace(Namespace.of("ns")) + .build()), + new MetadataUpdate.AddViewVersion( + ImmutableViewVersion.builder() + .versionId(2) + .schemaId(1) + .timestampMillis(System.currentTimeMillis()) + .defaultNamespace(Namespace.of("ns")) + .build()), + new MetadataUpdate.AddViewVersion( + ImmutableViewVersion.builder() + .versionId(1) + .schemaId(1) + .timestampMillis(System.currentTimeMillis()) + .defaultNamespace(Namespace.of("ns")) + .build()), + new MetadataUpdate.SetCurrentViewVersion(2))); + requirements.forEach(req -> req.validate(viewMetadata)); + + assertThat(requirements) + .hasSize(1) + .hasOnlyElementsOfTypes(UpdateRequirement.AssertViewUUID.class); + + assertViewUUID(requirements); + } + private void assertTableUUID(List requirements) { assertThat(requirements) .element(0) @@ -574,4 +776,12 @@ private void assertTableUUID(List requirements) { .extracting(UpdateRequirement.AssertTableUUID::uuid) .isEqualTo(metadata.uuid()); } + + private void assertViewUUID(List requirements) { + assertThat(requirements) + .element(0) + .asInstanceOf(InstanceOfAssertFactories.type(UpdateRequirement.AssertViewUUID.class)) + .extracting(UpdateRequirement.AssertViewUUID::uuid) + .isEqualTo(viewMetadata.uuid()); + } } diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index 5a17b9d43856..6cd60fe9abd0 100644 --- a/open-api/rest-catalog-open-api.py +++ b/open-api/rest-catalog-open-api.py @@ -420,6 +420,10 @@ class AssertDefaultSortOrderId(TableRequirement): default_sort_order_id: int = Field(..., alias='default-sort-order-id') +class ViewRequirement(BaseModel): + __root__: Any = Field(..., discriminator='type') + + class RegisterTableRequest(BaseModel): name: str metadata_location: str = Field(..., alias='metadata-location') @@ -822,6 +826,7 @@ class CommitViewRequest(BaseModel): identifier: Optional[TableIdentifier] = Field( None, description='View identifier to update' ) + requirements: Optional[List[ViewRequirement]] = None updates: List[ViewUpdate] diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index c965793cb229..a9d30ed02c63 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -2510,6 +2510,32 @@ components: default-sort-order-id: type: integer + ViewRequirement: + discriminator: + propertyName: type + mapping: + assert-view-uuid: '#/components/schemas/AssertViewUUID' + type: object + required: + - type + properties: + type: + type: "string" + + AssertViewUUID: + allOf: + - $ref: "#/components/schemas/ViewRequirement" + description: The view UUID must match the requirement's `uuid` + required: + - type + - uuid + properties: + type: + type: string + enum: [ "assert-view-uuid" ] + uuid: + type: string + LoadTableResult: description: | Result used when a table is successfully loaded. @@ -2576,6 +2602,10 @@ components: identifier: description: View identifier to update $ref: '#/components/schemas/TableIdentifier' + requirements: + type: array + items: + $ref: '#/components/schemas/ViewRequirement' updates: type: array items: From e276753290dad4cefcb289b925d12d7287fbec16 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Wed, 6 Dec 2023 13:20:10 -0800 Subject: [PATCH 13/47] Core: Fix equality in StructLikeMap (#9236) --- .../java/org/apache/iceberg/TestHelpers.java | 46 +++++++++++++++++++ .../apache/iceberg/util/StructLikeMap.java | 21 +++------ .../iceberg/util/TestStructLikeMap.java | 38 +++++++++++++++ 3 files changed, 91 insertions(+), 14 deletions(-) diff --git a/api/src/test/java/org/apache/iceberg/TestHelpers.java b/api/src/test/java/org/apache/iceberg/TestHelpers.java index cefd765a91b5..fcb528caba96 100644 --- a/api/src/test/java/org/apache/iceberg/TestHelpers.java +++ b/api/src/test/java/org/apache/iceberg/TestHelpers.java @@ -360,6 +360,52 @@ public int hashCode() { } } + // similar to Row but has its own hashCode() and equals() implementations + // it is useful for testing custom collections that rely on wrappers + public static class CustomRow implements StructLike { + public static CustomRow of(Object... values) { + return new CustomRow(values); + } + + private final Object[] values; + + private CustomRow(Object... values) { + this.values = values; + } + + @Override + public int size() { + return values.length; + } + + @Override + public T get(int pos, Class javaClass) { + return javaClass.cast(values[pos]); + } + + @Override + public void set(int pos, T value) { + values[pos] = value; + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } else if (other == null || getClass() != other.getClass()) { + return false; + } + + CustomRow that = (CustomRow) other; + return Arrays.equals(values, that.values); + } + + @Override + public int hashCode() { + return 17 * Arrays.hashCode(values); + } + } + public static class TestFieldSummary implements ManifestFile.PartitionFieldSummary { private final boolean containsNull; private final Boolean containsNaN; diff --git a/core/src/main/java/org/apache/iceberg/util/StructLikeMap.java b/core/src/main/java/org/apache/iceberg/util/StructLikeMap.java index 58bd03041331..2bb5fa1c9d40 100644 --- a/core/src/main/java/org/apache/iceberg/util/StructLikeMap.java +++ b/core/src/main/java/org/apache/iceberg/util/StructLikeMap.java @@ -21,7 +21,6 @@ import java.util.AbstractMap; import java.util.Collection; import java.util.Map; -import java.util.Objects; import java.util.Set; import java.util.function.Function; import org.apache.iceberg.StructLike; @@ -128,9 +127,9 @@ public Set> entrySet() { private static class StructLikeEntry implements Entry { - private Map.Entry inner; + private final Entry inner; - private StructLikeEntry(Map.Entry inner) { + private StructLikeEntry(Entry inner) { this.inner = inner; } @@ -146,25 +145,19 @@ public R getValue() { @Override public int hashCode() { - int hashCode = getKey().hashCode(); - if (getValue() != null) { - hashCode ^= getValue().hashCode(); - } - return hashCode; + return inner.hashCode(); } @Override - @SuppressWarnings("unchecked") public boolean equals(Object o) { if (this == o) { return true; - } else if (!(o instanceof StructLikeEntry)) { + } else if (o == null || getClass() != o.getClass()) { return false; - } else { - StructLikeEntry that = (StructLikeEntry) o; - return Objects.equals(getKey(), that.getKey()) - && Objects.equals(getValue(), that.getValue()); } + + StructLikeEntry that = (StructLikeEntry) o; + return inner.equals(that.inner); } @Override diff --git a/core/src/test/java/org/apache/iceberg/util/TestStructLikeMap.java b/core/src/test/java/org/apache/iceberg/util/TestStructLikeMap.java index db4176ce231e..f18c48eaa344 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestStructLikeMap.java +++ b/core/src/test/java/org/apache/iceberg/util/TestStructLikeMap.java @@ -24,6 +24,8 @@ import java.util.Map; import java.util.Set; import org.apache.iceberg.StructLike; +import org.apache.iceberg.TestHelpers.CustomRow; +import org.apache.iceberg.TestHelpers.Row; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -147,4 +149,40 @@ public void testKeysWithNulls() { assertThat(map.remove(record3)).isEqualTo("aaa"); } + + @Test + public void testEqualsAndHashCode() { + Map map1 = StructLikeMap.create(STRUCT_TYPE); + Map map2 = StructLikeMap.create(STRUCT_TYPE); + + assertThat(map1).isEqualTo(map2); + assertThat(map1.hashCode()).isEqualTo(map2.hashCode()); + + map1.put(CustomRow.of(1, null), "aaa"); + map1.put(CustomRow.of(2, null), "bbb"); + + map2.put(Row.of(1, null), "aaa"); + map2.put(Row.of(2, null), "bbb"); + + assertThat(map1).isEqualTo(map2); + assertThat(map1.hashCode()).isEqualTo(map2.hashCode()); + } + + @Test + public void testKeyAndEntrySetEquality() { + Map map1 = StructLikeMap.create(STRUCT_TYPE); + Map map2 = StructLikeMap.create(STRUCT_TYPE); + + assertThat(map1.keySet()).isEqualTo(map2.keySet()); + assertThat(map1.entrySet()).isEqualTo(map2.entrySet()); + + map1.put(CustomRow.of(1, null), "aaa"); + map1.put(CustomRow.of(2, null), "bbb"); + + map2.put(Row.of(1, null), "aaa"); + map2.put(Row.of(2, null), "bbb"); + + assertThat(map1.keySet()).isEqualTo(map2.keySet()); + assertThat(map1.entrySet()).isEqualTo(map2.entrySet()); + } } From 6a9d3c77977baff4295ee2dde0150d73c8c46af1 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Wed, 6 Dec 2023 17:31:50 -0800 Subject: [PATCH 14/47] Core: Add PartitionMap (#9194) --- .../org/apache/iceberg/util/PartitionMap.java | 257 +++++++++++++++ .../org/apache/iceberg/util/PartitionSet.java | 3 +- .../apache/iceberg/util/TestPartitionMap.java | 294 ++++++++++++++++++ 3 files changed, 553 insertions(+), 1 deletion(-) create mode 100644 core/src/main/java/org/apache/iceberg/util/PartitionMap.java create mode 100644 core/src/test/java/org/apache/iceberg/util/TestPartitionMap.java diff --git a/core/src/main/java/org/apache/iceberg/util/PartitionMap.java b/core/src/main/java/org/apache/iceberg/util/PartitionMap.java new file mode 100644 index 000000000000..f16b8aadba0e --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/util/PartitionMap.java @@ -0,0 +1,257 @@ +/* + * 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.iceberg.util; + +import java.util.AbstractMap; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.function.BiFunction; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; + +/** + * A map that uses a pair of spec ID and partition tuple as keys. + * + *

This implementation internally stores provided partition tuples in {@link StructLikeMap} for + * consistent hashing and equals behavior. This ensures that objects of different types that + * represent the same structs are treated as equal keys in the map. + * + *

Note: This map is not designed for concurrent modification by multiple threads. However, it + * supports safe concurrent reads, assuming there are no concurrent writes. + * + *

Note: This map does not support null pairs but supports null as partition tuples. + * + * @param the type of values + */ +public class PartitionMap extends AbstractMap, V> { + + private final Map specs; + private final Map> partitionMaps; + + private PartitionMap(Map specs) { + this.specs = specs; + this.partitionMaps = Maps.newHashMap(); + } + + public static PartitionMap create(Map specs) { + return new PartitionMap<>(specs); + } + + @Override + public int size() { + return partitionMaps.values().stream().mapToInt(Map::size).sum(); + } + + @Override + public boolean isEmpty() { + return partitionMaps.values().stream().allMatch(Map::isEmpty); + } + + @Override + public boolean containsKey(Object key) { + return execute(key, this::containsKey, false /* default value */); + } + + public boolean containsKey(int specId, StructLike struct) { + Map partitionMap = partitionMaps.get(specId); + return partitionMap != null && partitionMap.containsKey(struct); + } + + @Override + public boolean containsValue(Object value) { + return partitionMaps.values().stream().anyMatch(map -> map.containsValue(value)); + } + + @Override + public V get(Object key) { + return execute(key, this::get, null /* default value */); + } + + public V get(int specId, StructLike struct) { + Map partitionMap = partitionMaps.get(specId); + return partitionMap != null ? partitionMap.get(struct) : null; + } + + @Override + public V put(Pair key, V value) { + return put(key.first(), key.second(), value); + } + + public V put(int specId, StructLike struct, V value) { + Map partitionMap = partitionMaps.computeIfAbsent(specId, this::newPartitionMap); + return partitionMap.put(struct, value); + } + + @Override + public void putAll(Map, ? extends V> otherMap) { + otherMap.forEach(this::put); + } + + @Override + public V remove(Object key) { + return execute(key, this::removeKey, null /* default value */); + } + + public V removeKey(int specId, StructLike struct) { + Map partitionMap = partitionMaps.get(specId); + return partitionMap != null ? partitionMap.remove(struct) : null; + } + + @Override + public void clear() { + partitionMaps.clear(); + } + + @Override + public Set> keySet() { + PartitionSet keySet = PartitionSet.create(specs); + + for (Entry> specIdAndPartitionMap : partitionMaps.entrySet()) { + int specId = specIdAndPartitionMap.getKey(); + Map partitionMap = specIdAndPartitionMap.getValue(); + for (StructLike partition : partitionMap.keySet()) { + keySet.add(specId, partition); + } + } + + return Collections.unmodifiableSet(keySet); + } + + @Override + public Collection values() { + List values = Lists.newArrayList(); + + for (Map partitionMap : partitionMaps.values()) { + values.addAll(partitionMap.values()); + } + + return Collections.unmodifiableCollection(values); + } + + @Override + public Set, V>> entrySet() { + Set, V>> entrySet = Sets.newHashSet(); + + for (Entry> specIdAndPartitionMap : partitionMaps.entrySet()) { + int specId = specIdAndPartitionMap.getKey(); + Map partitionMap = specIdAndPartitionMap.getValue(); + for (Entry structAndValue : partitionMap.entrySet()) { + entrySet.add(new PartitionEntry<>(specId, structAndValue)); + } + } + + return Collections.unmodifiableSet(entrySet); + } + + public V computeIfAbsent(int specId, StructLike struct, Supplier valueSupplier) { + Map partitionMap = partitionMaps.computeIfAbsent(specId, this::newPartitionMap); + return partitionMap.computeIfAbsent(struct, key -> valueSupplier.get()); + } + + private Map newPartitionMap(int specId) { + PartitionSpec spec = specs.get(specId); + Preconditions.checkNotNull(spec, "Cannot find spec with ID %s: %s", specId, specs); + return StructLikeMap.create(spec.partitionType()); + } + + @Override + public String toString() { + return partitionMaps.entrySet().stream() + .flatMap(this::toStrings) + .collect(Collectors.joining(", ", "{", "}")); + } + + private Stream toStrings(Entry> entry) { + PartitionSpec spec = specs.get(entry.getKey()); + return entry.getValue().entrySet().stream().map(innerEntry -> toString(spec, innerEntry)); + } + + private String toString(PartitionSpec spec, Entry entry) { + StructLike struct = entry.getKey(); + V value = entry.getValue(); + return spec.partitionToPath(struct) + " -> " + (value == this ? "(this Map)" : value); + } + + private R execute(Object key, BiFunction action, R defaultValue) { + if (key instanceof Pair) { + Object first = ((Pair) key).first(); + Object second = ((Pair) key).second(); + if (first instanceof Integer && (second == null || second instanceof StructLike)) { + return action.apply((Integer) first, (StructLike) second); + } + } else if (key == null) { + throw new NullPointerException(getClass().getName() + " does not support null keys"); + } + + return defaultValue; + } + + private static class PartitionEntry implements Entry, V> { + private final int specId; + private final Entry structAndValue; + + private PartitionEntry(int specId, Entry structAndValue) { + this.specId = specId; + this.structAndValue = structAndValue; + } + + @Override + public Pair getKey() { + return Pair.of(specId, structAndValue.getKey()); + } + + @Override + public V getValue() { + return structAndValue.getValue(); + } + + @Override + public int hashCode() { + return Objects.hash(specId, structAndValue); + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } else if (other == null || getClass() != other.getClass()) { + return false; + } + + PartitionEntry that = (PartitionEntry) other; + return specId == that.specId && Objects.equals(structAndValue, that.structAndValue); + } + + @Override + public V setValue(V newValue) { + throw new UnsupportedOperationException("Cannot set value"); + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/util/PartitionSet.java b/core/src/main/java/org/apache/iceberg/util/PartitionSet.java index fdd81ac3d005..893c3207071d 100644 --- a/core/src/main/java/org/apache/iceberg/util/PartitionSet.java +++ b/core/src/main/java/org/apache/iceberg/util/PartitionSet.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.util; +import java.util.AbstractSet; import java.util.Collection; import java.util.Iterator; import java.util.Map; @@ -32,7 +33,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; -public class PartitionSet implements Set> { +public class PartitionSet extends AbstractSet> { public static PartitionSet create(Map specsById) { return new PartitionSet(specsById); } diff --git a/core/src/test/java/org/apache/iceberg/util/TestPartitionMap.java b/core/src/test/java/org/apache/iceberg/util/TestPartitionMap.java new file mode 100644 index 000000000000..e528a1e70ee6 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/util/TestPartitionMap.java @@ -0,0 +1,294 @@ +/* + * 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.iceberg.util; + +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.TestHelpers.CustomRow; +import org.apache.iceberg.TestHelpers.Row; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestPartitionMap { + + private static final Schema SCHEMA = + new Schema( + required(1, "id", Types.IntegerType.get()), + required(2, "data", Types.StringType.get()), + required(3, "category", Types.StringType.get())); + private static final PartitionSpec UNPARTITIONED_SPEC = PartitionSpec.unpartitioned(); + private static final PartitionSpec BY_DATA_SPEC = + PartitionSpec.builderFor(SCHEMA).identity("data").withSpecId(1).build(); + private static final PartitionSpec BY_DATA_CATEGORY_BUCKET_SPEC = + PartitionSpec.builderFor(SCHEMA).identity("data").bucket("category", 8).withSpecId(3).build(); + private static final Map SPECS = + ImmutableMap.of( + UNPARTITIONED_SPEC.specId(), + UNPARTITIONED_SPEC, + BY_DATA_SPEC.specId(), + BY_DATA_SPEC, + BY_DATA_CATEGORY_BUCKET_SPEC.specId(), + BY_DATA_CATEGORY_BUCKET_SPEC); + + @Test + public void testEmptyMap() { + PartitionMap map = PartitionMap.create(SPECS); + assertThat(map).isEmpty(); + assertThat(map).hasSize(0); + assertThat(map).doesNotContainKey(Pair.of(1, Row.of(1))).doesNotContainValue("value"); + assertThat(map.values()).isEmpty(); + assertThat(map.keySet()).isEmpty(); + assertThat(map.entrySet()).isEmpty(); + } + + @Test + public void testSize() { + PartitionMap map = PartitionMap.create(SPECS); + map.put(UNPARTITIONED_SPEC.specId(), null, "v1"); + map.put(BY_DATA_SPEC.specId(), Row.of("aaa"), "v2"); + map.put(BY_DATA_SPEC.specId(), Row.of("bbb"), "v3"); + map.put(BY_DATA_CATEGORY_BUCKET_SPEC.specId(), Row.of("ccc", 2), "v4"); + assertThat(map).isNotEmpty(); + assertThat(map).hasSize(4); + } + + @Test + public void testDifferentStructLikeImplementations() { + PartitionMap map = PartitionMap.create(SPECS); + map.put(BY_DATA_SPEC.specId(), CustomRow.of("aaa"), "value"); + map.put(UNPARTITIONED_SPEC.specId(), null, "value"); + assertThat(map) + .containsEntry(Pair.of(BY_DATA_SPEC.specId(), CustomRow.of("aaa")), "value") + .containsEntry(Pair.of(BY_DATA_SPEC.specId(), Row.of("aaa")), "value") + .containsEntry(Pair.of(UNPARTITIONED_SPEC.specId(), null), "value"); + } + + @Test + public void testPutAndGet() { + PartitionMap map = PartitionMap.create(SPECS); + map.put(UNPARTITIONED_SPEC.specId(), null, "v1"); + map.put(BY_DATA_CATEGORY_BUCKET_SPEC.specId(), Row.of("aaa", 1), "v2"); + assertThat(map.get(UNPARTITIONED_SPEC.specId(), null)).isEqualTo("v1"); + assertThat(map.get(BY_DATA_CATEGORY_BUCKET_SPEC.specId(), Row.of("aaa", 1))).isEqualTo("v2"); + } + + @Test + public void testRemove() { + PartitionMap map = PartitionMap.create(SPECS); + + map.put(BY_DATA_SPEC.specId(), Row.of("aaa"), "v1"); + map.put(BY_DATA_SPEC.specId(), Row.of("bbb"), "v2"); + + map.removeKey(BY_DATA_SPEC.specId(), Row.of("aaa")); + + assertThat(map).doesNotContainKey(Pair.of(BY_DATA_SPEC.specId(), Row.of("aaa"))); + assertThat(map.get(BY_DATA_SPEC.specId(), Row.of("aaa"))).isNull(); + assertThat(map).containsKey(Pair.of(BY_DATA_SPEC.specId(), Row.of("bbb"))); + assertThat(map.get(BY_DATA_SPEC.specId(), Row.of("bbb"))).isEqualTo("v2"); + } + + @Test + public void putAll() { + PartitionMap map = PartitionMap.create(SPECS); + + Map, String> otherMap = Maps.newHashMap(); + otherMap.put(Pair.of(BY_DATA_SPEC.specId(), Row.of("aaa")), "v1"); + otherMap.put(Pair.of(BY_DATA_SPEC.specId(), Row.of("bbb")), "v2"); + map.putAll(otherMap); + + assertThat(map) + .containsEntry(Pair.of(BY_DATA_SPEC.specId(), Row.of("aaa")), "v1") + .containsEntry(Pair.of(BY_DATA_SPEC.specId(), Row.of("bbb")), "v2"); + } + + @Test + public void testClear() { + PartitionMap map = PartitionMap.create(SPECS); + map.put(UNPARTITIONED_SPEC.specId(), null, "v1"); + map.put(BY_DATA_SPEC.specId(), Row.of("aaa"), "v2"); + assertThat(map).hasSize(2); + map.clear(); + assertThat(map).isEmpty(); + } + + @Test + public void testValues() { + PartitionMap map = PartitionMap.create(SPECS); + map.put(BY_DATA_SPEC.specId(), Row.of("aaa"), 1); + map.put(BY_DATA_CATEGORY_BUCKET_SPEC.specId(), Row.of("aaa", 2), 2); + map.put(BY_DATA_SPEC.specId(), Row.of("bbb"), 3); + assertThat(map.values()).containsAll(ImmutableList.of(1, 2, 3)); + } + + @Test + public void testEntrySet() { + PartitionMap map = PartitionMap.create(SPECS); + map.put(BY_DATA_SPEC.specId(), Row.of("aaa"), "v1"); + map.put(BY_DATA_CATEGORY_BUCKET_SPEC.specId(), Row.of("bbb", 2), "v2"); + map.put(BY_DATA_SPEC.specId(), CustomRow.of("ccc"), "v3"); + assertThat(map.entrySet()).hasSize(3); + } + + @Test + public void testKeySet() { + PartitionMap map = PartitionMap.create(SPECS); + map.put(BY_DATA_SPEC.specId(), Row.of("aaa"), "v1"); + map.put(BY_DATA_SPEC.specId(), CustomRow.of("ccc"), "v2"); + assertThat(map.get(BY_DATA_SPEC.specId(), CustomRow.of("aaa"))).isEqualTo("v1"); + assertThat(map.get(BY_DATA_SPEC.specId(), Row.of("ccc"))).isEqualTo("v2"); + } + + @Test + public void testEqualsAndHashCode() { + PartitionMap map1 = PartitionMap.create(SPECS); + PartitionMap map2 = PartitionMap.create(SPECS); + + assertThat(map1).isEqualTo(map2); + assertThat(map1.hashCode()).isEqualTo(map2.hashCode()); + + map1.put(BY_DATA_SPEC.specId(), Row.of("aaa"), "v1"); + map1.put(BY_DATA_SPEC.specId(), Row.of("bbb"), "v2"); + + map2.put(BY_DATA_SPEC.specId(), CustomRow.of("aaa"), "v1"); + map2.put(BY_DATA_SPEC.specId(), CustomRow.of("bbb"), "v2"); + + assertThat(map1).isEqualTo(map2); + assertThat(map1.hashCode()).isEqualTo(map2.hashCode()); + } + + @Test + public void testToString() { + PartitionMap map = PartitionMap.create(SPECS); + + // empty map + assertThat(map.toString()).isEqualTo("{}"); + + // single entry + map.put(BY_DATA_SPEC.specId(), Row.of("aaa"), "v1"); + assertThat(map.toString()).isEqualTo("{data=aaa -> v1}"); + + // multiple entries + map.put(BY_DATA_SPEC.specId(), CustomRow.of("bbb"), "v2"); + map.put(BY_DATA_CATEGORY_BUCKET_SPEC.specId(), Row.of("ccc", 2), "v3"); + assertThat(map.toString()) + .contains("data=aaa -> v1") + .contains("data=bbb -> v2") + .contains("data=ccc/category_bucket=2 -> v3"); + } + + @Test + public void testConcurrentReadAccess() throws InterruptedException { + PartitionMap map = PartitionMap.create(SPECS); + map.put(BY_DATA_SPEC.specId(), Row.of("aaa"), "v1"); + map.put(BY_DATA_SPEC.specId(), Row.of("bbb"), "v2"); + map.put(UNPARTITIONED_SPEC.specId(), null, "v3"); + map.put(BY_DATA_SPEC.specId(), CustomRow.of("ccc"), "v4"); + + int numThreads = 10; + ExecutorService executorService = Executors.newFixedThreadPool(numThreads); + + // read the map from multiple threads to ensure thread-local wrappers are used + for (int i = 0; i < numThreads; i++) { + executorService.submit( + () -> { + assertThat(map.get(BY_DATA_SPEC.specId(), Row.of("aaa"))).isEqualTo("v1"); + assertThat(map.get(BY_DATA_SPEC.specId(), Row.of("bbb"))).isEqualTo("v2"); + assertThat(map.get(UNPARTITIONED_SPEC.specId(), null)).isEqualTo("v3"); + assertThat(map.get(BY_DATA_SPEC.specId(), Row.of("ccc"))).isEqualTo("v4"); + }); + } + + executorService.shutdown(); + assertThat(executorService.awaitTermination(1, TimeUnit.MINUTES)).isTrue(); + } + + @Test + public void testNullKey() { + PartitionMap map = PartitionMap.create(SPECS); + assertThatThrownBy(() -> map.put(null, "value")).isInstanceOf(NullPointerException.class); + assertThatThrownBy(() -> map.get(null)).isInstanceOf(NullPointerException.class); + assertThatThrownBy(() -> map.remove(null)).isInstanceOf(NullPointerException.class); + } + + @Test + public void testUnknownSpecId() { + PartitionMap map = PartitionMap.create(SPECS); + assertThatThrownBy(() -> map.put(Integer.MAX_VALUE, null, "value")) + .isInstanceOf(NullPointerException.class) + .hasMessageContaining("Cannot find spec with ID"); + } + + @Test + public void testUnmodifiableViews() { + PartitionMap map = PartitionMap.create(SPECS); + map.put(BY_DATA_SPEC.specId(), Row.of("aaa"), "v1"); + map.put(BY_DATA_SPEC.specId(), Row.of("bbb"), "v2"); + + assertThatThrownBy(() -> map.keySet().add(Pair.of(1, null))) + .isInstanceOf(UnsupportedOperationException.class); + assertThatThrownBy(() -> map.values().add("other")) + .isInstanceOf(UnsupportedOperationException.class); + assertThatThrownBy(() -> map.entrySet().add(null)) + .isInstanceOf(UnsupportedOperationException.class); + assertThatThrownBy(() -> map.entrySet().iterator().next().setValue("other")) + .isInstanceOf(UnsupportedOperationException.class); + assertThatThrownBy(() -> map.entrySet().iterator().remove()) + .isInstanceOf(UnsupportedOperationException.class); + } + + @Test + public void testKeyAndEntrySetEquality() { + PartitionMap map1 = PartitionMap.create(SPECS); + PartitionMap map2 = PartitionMap.create(SPECS); + + assertThat(map1.keySet()).isEqualTo(map2.keySet()); + assertThat(map1.entrySet()).isEqualTo(map2.entrySet()); + + map1.put(BY_DATA_SPEC.specId(), Row.of("aaa"), "v1"); + map1.put(BY_DATA_SPEC.specId(), Row.of("bbb"), "v2"); + + map2.put(BY_DATA_SPEC.specId(), CustomRow.of("aaa"), "v1"); + map2.put(BY_DATA_SPEC.specId(), CustomRow.of("bbb"), "v2"); + + assertThat(map1.keySet()).isEqualTo(map2.keySet()); + assertThat(map1.entrySet()).isEqualTo(map2.entrySet()); + } + + @Test + public void testLookupArbitraryKeyTypes() { + PartitionMap map = PartitionMap.create(SPECS); + map.put(BY_DATA_SPEC.specId(), Row.of("aaa"), "v1"); + map.put(UNPARTITIONED_SPEC.specId(), null, "v2"); + assertThat(map.containsKey("some-string")).isFalse(); + assertThat(map.get("some-string")).isNull(); + assertThat(map.remove("some-string")).isNull(); + } +} From af9522ac7e8e25dc044622c566b66301b6df9581 Mon Sep 17 00:00:00 2001 From: Wing Yew Poon Date: Thu, 7 Dec 2023 03:11:30 -0800 Subject: [PATCH 15/47] Docs: Document reading in Spark using branch and tag identifiers (#9238) --- docs/spark-queries.md | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/docs/spark-queries.md b/docs/spark-queries.md index 54b1ee6bd120..38d989f003f6 100644 --- a/docs/spark-queries.md +++ b/docs/spark-queries.md @@ -120,6 +120,17 @@ SELECT * FROM prod.db.table TIMESTAMP AS OF 499162860; SELECT * FROM prod.db.table FOR SYSTEM_TIME AS OF 499162860; ``` +The branch or tag may also be specified using a similar syntax to metadata tables, with `branch_` or `tag_`: + +```sql +SELECT * FROM prod.db.table.`branch_audit-branch`; +SELECT * FROM prod.db.table.`tag_historical-snapshot`; +``` + +(Identifiers with "-" are not valid, and so must be escaped using back quotes.) + +Note that the identifier with branch or tag may not be used in combination with `VERSION AS OF`. + #### DataFrame To select a specific table snapshot or the snapshot at some time in the DataFrame API, Iceberg supports four Spark read options: From ea7665e788cb33507e6eb19eddd4f0c24eb92983 Mon Sep 17 00:00:00 2001 From: Alexandre Dutra Date: Thu, 7 Dec 2023 12:34:21 +0100 Subject: [PATCH 16/47] Nessie: Reimplement namespace operations (#8857) This change enhances the process of creating new namespaces by retaining commit authorship information when committing the new namespace. --- .../iceberg/nessie/NessieIcebergClient.java | 371 +++++++++----- .../iceberg/nessie/NessieTableOperations.java | 60 ++- .../org/apache/iceberg/nessie/NessieUtil.java | 25 + .../iceberg/nessie/TestMultipleClients.java | 80 ++- .../apache/iceberg/nessie/TestNamespace.java | 2 +- .../nessie/TestNessieIcebergClient.java | 458 +++++++++++++++++- 6 files changed, 819 insertions(+), 177 deletions(-) diff --git a/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java b/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java index 26b3701816f5..4cbbe4a562c1 100644 --- a/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java +++ b/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java @@ -19,12 +19,17 @@ package org.apache.iceberg.nessie; import java.util.Arrays; +import java.util.Collections; +import java.util.EnumSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; +import java.util.function.Consumer; import java.util.function.Predicate; import java.util.function.Supplier; import java.util.stream.Collectors; +import javax.annotation.Nullable; import org.apache.iceberg.Snapshot; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.catalog.Namespace; @@ -36,24 +41,25 @@ import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.relocated.com.google.common.base.Suppliers; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.util.Tasks; import org.projectnessie.client.NessieConfigConstants; import org.projectnessie.client.api.CommitMultipleOperationsBuilder; +import org.projectnessie.client.api.GetContentBuilder; import org.projectnessie.client.api.NessieApiV1; import org.projectnessie.client.api.OnReferenceBuilder; import org.projectnessie.client.http.HttpClientException; import org.projectnessie.error.BaseNessieClientServerException; import org.projectnessie.error.NessieConflictException; -import org.projectnessie.error.NessieNamespaceAlreadyExistsException; -import org.projectnessie.error.NessieNamespaceNotEmptyException; -import org.projectnessie.error.NessieNamespaceNotFoundException; +import org.projectnessie.error.NessieContentNotFoundException; import org.projectnessie.error.NessieNotFoundException; +import org.projectnessie.error.NessieReferenceConflictException; import org.projectnessie.error.NessieReferenceNotFoundException; import org.projectnessie.model.Branch; +import org.projectnessie.model.Conflict; import org.projectnessie.model.Content; import org.projectnessie.model.ContentKey; import org.projectnessie.model.EntriesResponse; -import org.projectnessie.model.GetNamespacesResponse; import org.projectnessie.model.IcebergTable; import org.projectnessie.model.ImmutableCommitMeta; import org.projectnessie.model.ImmutableIcebergTable; @@ -181,133 +187,214 @@ public IcebergTable table(TableIdentifier tableIdentifier) { } public void createNamespace(Namespace namespace, Map metadata) { + getRef().checkMutable(); + if (namespace.isEmpty()) { + throw new IllegalArgumentException("Creating empty namespaces is not supported"); + } + ContentKey key = ContentKey.of(namespace.levels()); + org.projectnessie.model.Namespace content = + org.projectnessie.model.Namespace.of(key.getElements(), metadata); try { - getRef().checkMutable(); - withReference( - getApi() - .createNamespace() - .namespace(org.projectnessie.model.Namespace.of(namespace.levels())) - .properties(metadata)) - .create(); - refresh(); - } catch (NessieNamespaceAlreadyExistsException e) { - throw new AlreadyExistsException(e, "Namespace already exists: %s", namespace); + Content existing = api.getContent().reference(getReference()).key(key).get().get(key); + if (existing != null) { + throw namespaceAlreadyExists(key, existing, null); + } + try { + commitRetry("create namespace " + key, Operation.Put.of(key, content)); + } catch (NessieReferenceConflictException e) { + Optional conflict = + NessieUtil.extractSingleConflict( + e, + EnumSet.of( + Conflict.ConflictType.KEY_EXISTS, Conflict.ConflictType.NAMESPACE_ABSENT)); + if (conflict.isPresent()) { + switch (conflict.get().conflictType()) { + case KEY_EXISTS: + Content conflicting = withReference(api.getContent()).key(key).get().get(key); + throw namespaceAlreadyExists(key, conflicting, e); + case NAMESPACE_ABSENT: + throw new NoSuchNamespaceException( + e, + "Cannot create namespace '%s': parent namespace '%s' does not exist", + namespace, + conflict.get().key()); + } + } + throw new RuntimeException( + String.format("Cannot create namespace '%s': %s", namespace, e.getMessage())); + } } catch (NessieNotFoundException e) { throw new RuntimeException( String.format( - "Cannot create Namespace '%s': " + "ref '%s' is no longer valid.", + "Cannot create namespace '%s': ref '%s' is no longer valid.", namespace, getRef().getName()), e); + } catch (BaseNessieClientServerException e) { + throw new RuntimeException( + String.format("Cannot create namespace '%s': %s", namespace, e.getMessage()), e); } } public List listNamespaces(Namespace namespace) throws NoSuchNamespaceException { try { - GetNamespacesResponse response = - withReference( - getApi() - .getMultipleNamespaces() - .namespace(org.projectnessie.model.Namespace.of(namespace.levels()))) - .get(); - return response.getNamespaces().stream() - .map(ns -> Namespace.of(ns.getElements().toArray(new String[0]))) - .filter(ns -> ns.length() == namespace.length() + 1) + String filter = "entry.contentType == 'NAMESPACE' && "; + if (namespace.isEmpty()) { + filter += "size(entry.keyElements) == 1"; + } else { + org.projectnessie.model.Namespace root = + org.projectnessie.model.Namespace.of(namespace.levels()); + filter += + String.format( + "size(entry.keyElements) == %d && entry.encodedKey.startsWith('%s.')", + root.getElementCount() + 1, root.name()); + } + List entries = + withReference(api.getEntries()).filter(filter).stream() + .map(EntriesResponse.Entry::getName) + .collect(Collectors.toList()); + if (entries.isEmpty()) { + return Collections.emptyList(); + } + GetContentBuilder getContent = withReference(api.getContent()); + entries.forEach(getContent::key); + return getContent.get().values().stream() + .map(v -> v.unwrap(org.projectnessie.model.Namespace.class)) + .filter(Optional::isPresent) + .map(Optional::get) + .map(v -> Namespace.of(v.getElements().toArray(new String[0]))) .collect(Collectors.toList()); - } catch (NessieReferenceNotFoundException e) { - throw new RuntimeException( - String.format( - "Cannot list Namespaces starting from '%s': " + "ref '%s' is no longer valid.", - namespace, getRef().getName()), - e); + } catch (NessieNotFoundException e) { + if (namespace.isEmpty()) { + throw new NoSuchNamespaceException( + e, + "Cannot list top-level namespaces: ref '%s' is no longer valid.", + getRef().getName()); + } + throw new NoSuchNamespaceException( + e, + "Cannot list child namespaces from '%s': ref '%s' is no longer valid.", + namespace, + getRef().getName()); } } public boolean dropNamespace(Namespace namespace) throws NamespaceNotEmptyException { + getRef().checkMutable(); + ContentKey key = ContentKey.of(namespace.levels()); try { - getRef().checkMutable(); - withReference( - getApi() - .deleteNamespace() - .namespace(org.projectnessie.model.Namespace.of(namespace.levels()))) - .delete(); - refresh(); - return true; - } catch (NessieNamespaceNotFoundException e) { - return false; + Map contentMap = + api.getContent().reference(getReference()).key(key).get(); + Content existing = contentMap.get(key); + if (existing != null && !existing.getType().equals(Content.Type.NAMESPACE)) { + throw new NoSuchNamespaceException( + "Content object with name '%s' is not a namespace.", namespace); + } + try { + commitRetry("drop namespace " + key, Operation.Delete.of(key)); + return true; + } catch (NessieReferenceConflictException e) { + Optional conflict = + NessieUtil.extractSingleConflict( + e, + EnumSet.of( + Conflict.ConflictType.KEY_DOES_NOT_EXIST, + Conflict.ConflictType.NAMESPACE_NOT_EMPTY)); + if (conflict.isPresent()) { + Conflict.ConflictType conflictType = conflict.get().conflictType(); + switch (conflictType) { + case KEY_DOES_NOT_EXIST: + return false; + case NAMESPACE_NOT_EMPTY: + throw new NamespaceNotEmptyException(e, "Namespace '%s' is not empty.", namespace); + } + } + throw new RuntimeException( + String.format("Cannot drop namespace '%s': %s", namespace, e.getMessage())); + } } catch (NessieNotFoundException e) { LOG.error( - "Cannot drop Namespace '{}': ref '{}' is no longer valid.", + "Cannot drop namespace '{}': ref '{}' is no longer valid.", namespace, getRef().getName(), e); - return false; - } catch (NessieNamespaceNotEmptyException e) { - throw new NamespaceNotEmptyException( - e, "Namespace '%s' is not empty. One or more tables exist.", namespace); + } catch (BaseNessieClientServerException e) { + throw new RuntimeException( + String.format("Cannot drop namespace '%s': %s", namespace, e.getMessage()), e); } + return false; } public Map loadNamespaceMetadata(Namespace namespace) throws NoSuchNamespaceException { + ContentKey key = ContentKey.of(namespace.levels()); try { - return withReference( - getApi() - .getNamespace() - .namespace(org.projectnessie.model.Namespace.of(namespace.levels()))) - .get() + Map contentMap = withReference(api.getContent()).key(key).get(); + return unwrapNamespace(contentMap.get(key)) + .orElseThrow( + () -> new NoSuchNamespaceException("Namespace does not exist: %s", namespace)) .getProperties(); - } catch (NessieNamespaceNotFoundException e) { - throw new NoSuchNamespaceException(e, "Namespace does not exist: %s", namespace); - } catch (NessieReferenceNotFoundException e) { + } catch (NessieNotFoundException e) { throw new RuntimeException( String.format( - "Cannot load Namespace '%s': " + "ref '%s' is no longer valid.", + "Cannot load namespace '%s': ref '%s' is no longer valid.", namespace, getRef().getName()), e); } } public boolean setProperties(Namespace namespace, Map properties) { - try { - withReference( - getApi() - .updateProperties() - .namespace(org.projectnessie.model.Namespace.of(namespace.levels())) - .updateProperties(properties)) - .update(); - refresh(); - // always successful, otherwise an exception is thrown - return true; - } catch (NessieNamespaceNotFoundException e) { - throw new NoSuchNamespaceException(e, "Namespace does not exist: %s", namespace); - } catch (NessieNotFoundException e) { - throw new RuntimeException( - String.format( - "Cannot update properties on Namespace '%s': ref '%s' is no longer valid.", - namespace, getRef().getName()), - e); - } + return updateProperties(namespace, props -> props.putAll(properties)); } public boolean removeProperties(Namespace namespace, Set properties) { + return updateProperties(namespace, props -> props.keySet().removeAll(properties)); + } + + private boolean updateProperties(Namespace namespace, Consumer> action) { + getRef().checkMutable(); + ContentKey key = ContentKey.of(namespace.levels()); try { - withReference( - getApi() - .updateProperties() - .namespace(org.projectnessie.model.Namespace.of(namespace.levels())) - .removeProperties(properties)) - .update(); - refresh(); + commitRetry( + "update namespace " + key, + true, + commitBuilder -> { + org.projectnessie.model.Namespace oldNamespace = + unwrapNamespace(api.getContent().reference(getReference()).key(key).get().get(key)) + .orElseThrow( + () -> new NessieContentNotFoundException(key, getReference().getName())); + Map newProperties = Maps.newHashMap(oldNamespace.getProperties()); + action.accept(newProperties); + org.projectnessie.model.Namespace updatedNamespace = + org.projectnessie.model.Namespace.builder() + .from(oldNamespace) + .properties(newProperties) + .build(); + commitBuilder.operation(Operation.Put.of(key, updatedNamespace)); + return commitBuilder; + }); // always successful, otherwise an exception is thrown return true; - } catch (NessieNamespaceNotFoundException e) { - throw new NoSuchNamespaceException(e, "Namespace does not exist: %s", namespace); - } catch (NessieNotFoundException e) { + } catch (NessieReferenceConflictException e) { + Optional conflict = + NessieUtil.extractSingleConflict(e, EnumSet.of(Conflict.ConflictType.KEY_DOES_NOT_EXIST)); + if (conflict.isPresent() + && conflict.get().conflictType() == Conflict.ConflictType.KEY_DOES_NOT_EXIST) { + throw new NoSuchNamespaceException(e, "Namespace does not exist: %s", namespace); + } + throw new RuntimeException( + String.format( + "Cannot update properties on namespace '%s': %s", namespace, e.getMessage())); + } catch (NessieContentNotFoundException e) { + throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace); + } catch (NessieReferenceNotFoundException e) { throw new RuntimeException( String.format( - "Cannot remove properties from Namespace '%s': ref '%s' is no longer valid.", + "Cannot update properties on namespace '%s': ref '%s' is no longer valid.", namespace, getRef().getName()), e); + } catch (BaseNessieClientServerException e) { + throw new RuntimeException( + String.format("Cannot update namespace '%s': %s", namespace, e.getMessage()), e); } } @@ -323,28 +410,11 @@ public void renameTable(TableIdentifier from, TableIdentifier to) { throw new AlreadyExistsException("Table already exists: %s", to.name()); } - CommitMultipleOperationsBuilder operations = - getApi() - .commitMultipleOperations() - .commitMeta( - NessieUtil.buildCommitMetadata( - String.format("Iceberg rename table from '%s' to '%s'", from, to), - catalogOptions)) - .operation(Operation.Delete.of(NessieUtil.toKey(from))) - .operation(Operation.Put.of(NessieUtil.toKey(to), existingFromTable)); - try { - Tasks.foreach(operations) - .retry(5) - .stopRetryOn(NessieNotFoundException.class) - .throwFailureWhenFinished() - .onFailure((o, exception) -> refresh()) - .run( - ops -> { - Branch branch = ops.branch((Branch) getRef().getReference()).commit(); - getRef().updateReference(branch); - }, - BaseNessieClientServerException.class); + commitRetry( + String.format("Iceberg rename table from '%s' to '%s'", from, to), + Operation.Delete.of(NessieUtil.toKey(from)), + Operation.Put.of(NessieUtil.toKey(to), existingFromTable)); } catch (NessieNotFoundException e) { // important note: the NotFoundException refers to the ref only. If a table was not found it // would imply that the @@ -355,13 +425,13 @@ public void renameTable(TableIdentifier from, TableIdentifier to) { // and removed by another. throw new RuntimeException( String.format( - "Cannot rename table '%s' to '%s': " + "ref '%s' no longer exists.", + "Cannot rename table '%s' to '%s': ref '%s' no longer exists.", from.name(), to.name(), getRef().getName()), e); } catch (BaseNessieClientServerException e) { throw new CommitFailedException( e, - "Cannot rename table '%s' to '%s': " + "the current reference is not up to date.", + "Cannot rename table '%s' to '%s': the current reference is not up to date.", from.name(), to.name()); } catch (HttpClientException ex) { @@ -390,29 +460,12 @@ public boolean dropTable(TableIdentifier identifier, boolean purge) { LOG.info("Purging data for table {} was set to true but is ignored", identifier.toString()); } - CommitMultipleOperationsBuilder commitBuilderBase = - getApi() - .commitMultipleOperations() - .commitMeta( - NessieUtil.buildCommitMetadata( - String.format("Iceberg delete table %s", identifier), catalogOptions)) - .operation(Operation.Delete.of(NessieUtil.toKey(identifier))); - // We try to drop the table. Simple retry after ref update. - boolean threw = true; try { - Tasks.foreach(commitBuilderBase) - .retry(5) - .stopRetryOn(NessieNotFoundException.class) - .throwFailureWhenFinished() - .onFailure((o, exception) -> refresh()) - .run( - commitBuilder -> { - Branch branch = commitBuilder.branch((Branch) getRef().getReference()).commit(); - getRef().updateReference(branch); - }, - BaseNessieClientServerException.class); - threw = false; + commitRetry( + String.format("Iceberg delete table %s", identifier), + Operation.Delete.of(NessieUtil.toKey(identifier))); + return true; } catch (NessieConflictException e) { LOG.error( "Cannot drop table: failed after retry (update ref '{}' and retry)", @@ -423,7 +476,7 @@ public boolean dropTable(TableIdentifier identifier, boolean purge) { } catch (BaseNessieClientServerException e) { LOG.error("Cannot drop table: unknown error", e); } - return !threw; + return false; } /** @deprecated will be removed after 1.5.0 */ @@ -540,4 +593,64 @@ public void close() { api.close(); } } + + private void commitRetry(String message, Operation... ops) + throws BaseNessieClientServerException { + commitRetry(message, false, builder -> builder.operations(Arrays.asList(ops))); + } + + private void commitRetry(String message, boolean retryConflicts, CommitEnhancer commitEnhancer) + throws BaseNessieClientServerException { + // Retry all errors except for NessieNotFoundException and also NessieConflictException, unless + // retryConflicts is set to true. + Predicate shouldRetry = + e -> + !(e instanceof NessieNotFoundException) + && (!(e instanceof NessieConflictException) || retryConflicts); + Tasks.range(1) + .retry(5) + .shouldRetryTest(shouldRetry) + .throwFailureWhenFinished() + .onFailure((o, exception) -> refresh()) + .run( + i -> { + try { + Branch branch = + commitEnhancer + .enhance(api.commitMultipleOperations()) + .commitMeta(NessieUtil.buildCommitMetadata(message, catalogOptions)) + .branch((Branch) getReference()) + .commit(); + getRef().updateReference(branch); + } catch (NessieConflictException e) { + if (retryConflicts) { + refresh(); // otherwise retrying a conflict doesn't make sense + } + throw e; + } + }, + BaseNessieClientServerException.class); + } + + private static AlreadyExistsException namespaceAlreadyExists( + ContentKey key, @Nullable Content existing, @Nullable Exception ex) { + if (existing instanceof org.projectnessie.model.Namespace) { + return new AlreadyExistsException(ex, "Namespace already exists: %s", key); + } else { + return new AlreadyExistsException( + ex, "Another content object with name '%s' already exists", key); + } + } + + private static Optional unwrapNamespace(Content content) { + return content == null + ? Optional.empty() + : content.unwrap(org.projectnessie.model.Namespace.class); + } + + private interface CommitEnhancer { + + CommitMultipleOperationsBuilder enhance(CommitMultipleOperationsBuilder builder) + throws BaseNessieClientServerException; + } } diff --git a/nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java b/nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java index 0b4c293f8627..a5d7e7b21428 100644 --- a/nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java +++ b/nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java @@ -18,7 +18,7 @@ */ package org.apache.iceberg.nessie; -import java.util.List; +import java.util.EnumSet; import java.util.Map; import org.apache.iceberg.BaseMetastoreTableOperations; import org.apache.iceberg.TableMetadata; @@ -34,9 +34,7 @@ import org.projectnessie.error.NessieConflictException; import org.projectnessie.error.NessieNotFoundException; import org.projectnessie.error.NessieReferenceConflictException; -import org.projectnessie.error.ReferenceConflicts; import org.projectnessie.model.Conflict; -import org.projectnessie.model.Conflict.ConflictType; import org.projectnessie.model.Content; import org.projectnessie.model.ContentKey; import org.projectnessie.model.IcebergTable; @@ -169,35 +167,33 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) { } private static void maybeThrowSpecializedException(NessieReferenceConflictException ex) { - // Check if the server returned 'ReferenceConflicts' information - ReferenceConflicts referenceConflicts = ex.getErrorDetails(); - if (referenceConflicts == null) { - return; - } - - // Can only narrow down to a single exception, if there is only one conflict. - List conflicts = referenceConflicts.conflicts(); - if (conflicts.size() != 1) { - return; - } - - Conflict conflict = conflicts.get(0); - ConflictType conflictType = conflict.conflictType(); - if (conflictType != null) { - switch (conflictType) { - case NAMESPACE_ABSENT: - throw new NoSuchNamespaceException(ex, "Namespace does not exist: %s", conflict.key()); - case NAMESPACE_NOT_EMPTY: - throw new NamespaceNotEmptyException(ex, "Namespace not empty: %s", conflict.key()); - case KEY_DOES_NOT_EXIST: - throw new NoSuchTableException(ex, "Table or view does not exist: %s", conflict.key()); - case KEY_EXISTS: - throw new AlreadyExistsException(ex, "Table or view already exists: %s", conflict.key()); - default: - // Explicit fall-through - break; - } - } + NessieUtil.extractSingleConflict( + ex, + EnumSet.of( + Conflict.ConflictType.NAMESPACE_ABSENT, + Conflict.ConflictType.NAMESPACE_NOT_EMPTY, + Conflict.ConflictType.KEY_DOES_NOT_EXIST, + Conflict.ConflictType.KEY_EXISTS)) + .ifPresent( + conflict -> { + switch (conflict.conflictType()) { + case NAMESPACE_ABSENT: + throw new NoSuchNamespaceException( + ex, "Namespace does not exist: %s", conflict.key()); + case NAMESPACE_NOT_EMPTY: + throw new NamespaceNotEmptyException( + ex, "Namespace not empty: %s", conflict.key()); + case KEY_DOES_NOT_EXIST: + throw new NoSuchTableException( + ex, "Table or view does not exist: %s", conflict.key()); + case KEY_EXISTS: + throw new AlreadyExistsException( + ex, "Table or view already exists: %s", conflict.key()); + default: + // Explicit fall-through + break; + } + }); } @Override diff --git a/nessie/src/main/java/org/apache/iceberg/nessie/NessieUtil.java b/nessie/src/main/java/org/apache/iceberg/nessie/NessieUtil.java index 8c69f844dbfa..3c3b0afd64d3 100644 --- a/nessie/src/main/java/org/apache/iceberg/nessie/NessieUtil.java +++ b/nessie/src/main/java/org/apache/iceberg/nessie/NessieUtil.java @@ -19,9 +19,11 @@ package org.apache.iceberg.nessie; import java.util.Arrays; +import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.stream.Collectors; import javax.annotation.Nullable; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.SnapshotRef; @@ -32,7 +34,10 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.projectnessie.error.NessieReferenceConflictException; +import org.projectnessie.error.ReferenceConflicts; import org.projectnessie.model.CommitMeta; +import org.projectnessie.model.Conflict; import org.projectnessie.model.ContentKey; import org.projectnessie.model.IcebergTable; import org.projectnessie.model.ImmutableCommitMeta; @@ -165,4 +170,24 @@ public static TableMetadata updateTableMetadataWithNessieSpecificProperties( return builder.discardChanges().build(); } + + public static Optional extractSingleConflict( + NessieReferenceConflictException ex, Collection handledConflictTypes) { + // Check if the server returned 'ReferenceConflicts' information + ReferenceConflicts referenceConflicts = ex.getErrorDetails(); + if (referenceConflicts == null) { + return Optional.empty(); + } + + List conflicts = + referenceConflicts.conflicts().stream() + .filter(c -> handledConflictTypes.contains(c.conflictType())) + .collect(Collectors.toList()); + if (conflicts.size() != 1) { + return Optional.empty(); + } + + Conflict conflict = conflicts.get(0); + return Optional.of(conflict); + } } diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestMultipleClients.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestMultipleClients.java index fb7f4ea3095c..b6ae90650e89 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestMultipleClients.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestMultipleClients.java @@ -22,8 +22,8 @@ import java.io.IOException; import java.net.URI; -import java.util.AbstractMap; import java.util.Collections; +import java.util.Map; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; @@ -31,6 +31,7 @@ import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Types; import org.assertj.core.api.Assertions; @@ -39,6 +40,9 @@ import org.junit.jupiter.api.Test; import org.projectnessie.client.ext.NessieClientFactory; import org.projectnessie.client.ext.NessieClientUri; +import org.projectnessie.error.NessieConflictException; +import org.projectnessie.error.NessieNotFoundException; +import org.projectnessie.model.Branch; public class TestMultipleClients extends BaseTestIceberg { @@ -67,33 +71,87 @@ public void afterEach() throws Exception { } @Test - public void testListNamespaces() { + public void testListNamespaces() throws NessieConflictException, NessieNotFoundException { + Assertions.assertThat(catalog.listNamespaces()).isEmpty(); + Assertions.assertThat(anotherCatalog.listNamespaces()).isEmpty(); + + // listing a non-existent namespace should return empty + Assertions.assertThat(catalog.listNamespaces(Namespace.of("db1"))).isEmpty(); + Assertions.assertThat(anotherCatalog.listNamespaces(Namespace.of("db1"))).isEmpty(); + catalog.createNamespace(Namespace.of("db1"), Collections.emptyMap()); + Assertions.assertThat(catalog.listNamespaces()).containsExactlyInAnyOrder(Namespace.of("db1")); + Assertions.assertThat(anotherCatalog.listNamespaces()) + .containsExactlyInAnyOrder(Namespace.of("db1")); // another client creates a namespace with the same nessie server anotherCatalog.createNamespace(Namespace.of("db2"), Collections.emptyMap()); - Assertions.assertThat(anotherCatalog.listNamespaces()) - .containsExactlyInAnyOrder(Namespace.of("db1"), Namespace.of("db2")); Assertions.assertThat(catalog.listNamespaces()) .containsExactlyInAnyOrder(Namespace.of("db1"), Namespace.of("db2")); + Assertions.assertThat(anotherCatalog.listNamespaces()) + .containsExactlyInAnyOrder(Namespace.of("db1"), Namespace.of("db2")); + + api.deleteBranch().branch((Branch) api.getReference().refName(branch).get()).delete(); + + Assertions.assertThatThrownBy(() -> catalog.listNamespaces()) + .isInstanceOf(NoSuchNamespaceException.class) + .hasMessageContaining( + "Cannot list top-level namespaces: ref '%s' is no longer valid", branch); + Assertions.assertThatThrownBy(() -> anotherCatalog.listNamespaces(Namespace.of("db1"))) + .isInstanceOf(NoSuchNamespaceException.class) + .hasMessageContaining( + "Cannot list child namespaces from 'db1': ref '%s' is no longer valid", branch); } @Test - public void testLoadNamespaceMetadata() { + public void testLoadNamespaceMetadata() throws NessieConflictException, NessieNotFoundException { + Assertions.assertThatThrownBy(() -> catalog.loadNamespaceMetadata(Namespace.of("namespace1"))) + .isInstanceOf(NoSuchNamespaceException.class) + .hasMessageContaining("Namespace does not exist: namespace1"); + Assertions.assertThatThrownBy( + () -> anotherCatalog.loadNamespaceMetadata(Namespace.of("namespace1"))) + .isInstanceOf(NoSuchNamespaceException.class) + .hasMessageContaining("Namespace does not exist: namespace1"); + catalog.createNamespace(Namespace.of("namespace1"), Collections.emptyMap()); + + // both clients should see the namespace because we read the HEAD of the ref Assertions.assertThat(catalog.listNamespaces()) .containsExactlyInAnyOrder(Namespace.of("namespace1")); + Assertions.assertThat(anotherCatalog.listNamespaces()) + .containsExactlyInAnyOrder(Namespace.of("namespace1")); - // another client adds a metadata to the same namespace - anotherCatalog.setProperties(Namespace.of("namespace1"), Collections.singletonMap("k1", "v1")); - AbstractMap.SimpleEntry entry = new AbstractMap.SimpleEntry<>("k1", "v1"); + // the other client should not be able to update the namespace + // because it is still on the old ref hash + Assertions.assertThatThrownBy( + () -> + anotherCatalog.setProperties( + Namespace.of("namespace1"), Collections.singletonMap("k1", "v1"))) + .isInstanceOf(NoSuchNamespaceException.class) + .hasMessageContaining("Namespace does not exist: namespace1"); + // the same client adds a metadata to the namespace: expect success + catalog.setProperties(Namespace.of("namespace1"), Collections.singletonMap("k1", "v1")); + + // load metadata from the same client and another client both should work fine + // because we read the HEAD of the ref Assertions.assertThat(anotherCatalog.loadNamespaceMetadata(Namespace.of("namespace1"))) - .containsExactly(entry); - + .containsExactly(Map.entry("k1", "v1")); Assertions.assertThat(catalog.loadNamespaceMetadata(Namespace.of("namespace1"))) - .containsExactly(entry); + .containsExactly(Map.entry("k1", "v1")); + + api.deleteBranch().branch((Branch) api.getReference().refName(branch).get()).delete(); + + Assertions.assertThatThrownBy(() -> catalog.loadNamespaceMetadata(Namespace.of("namespace1"))) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining( + "Cannot load namespace 'namespace1': ref '%s' is no longer valid", branch); + Assertions.assertThatThrownBy( + () -> anotherCatalog.loadNamespaceMetadata(Namespace.of("namespace1"))) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining( + "Cannot load namespace 'namespace1': ref '%s' is no longer valid", branch); } @Test diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestNamespace.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestNamespace.java index 8267329678de..0b1af9763dae 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestNamespace.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestNamespace.java @@ -106,7 +106,7 @@ public void testCreatingAndDroppingNamespaceWithContent() throws NessieNotFoundE Assertions.assertThatThrownBy(() -> catalog.dropNamespace(namespace)) .isInstanceOf(NamespaceNotEmptyException.class) - .hasMessage("Namespace 'test' is not empty. One or more tables exist."); + .hasMessageContaining("Namespace 'test' is not empty"); catalog.dropTable(identifier, true); catalog.dropNamespace(namespace); diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieIcebergClient.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieIcebergClient.java index f1d6159d93f1..d4e079ad269c 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieIcebergClient.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieIcebergClient.java @@ -18,14 +18,36 @@ */ package org.apache.iceberg.nessie; +import static org.apache.iceberg.types.Types.NestedField.required; + import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.TableMetadata; import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.NamespaceNotEmptyException; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; import org.assertj.core.api.Assertions; +import org.assertj.core.data.Index; import org.junit.jupiter.api.Test; import org.projectnessie.error.NessieConflictException; import org.projectnessie.error.NessieNotFoundException; import org.projectnessie.model.Branch; +import org.projectnessie.model.CommitMeta; +import org.projectnessie.model.Content; +import org.projectnessie.model.ContentKey; +import org.projectnessie.model.IcebergTable; +import org.projectnessie.model.LogResponse; +import org.projectnessie.model.Operation; import org.projectnessie.model.Reference; public class TestNessieIcebergClient extends BaseTestIceberg { @@ -80,7 +102,7 @@ public void testWithReferenceAfterRecreatingBranch() client .getApi() .deleteBranch() - .branch((Branch) client.getApi().getReference().refName(branch).get()) + .branch((Branch) api.getReference().refName(branch).get()) .delete(); createBranch(branch); @@ -91,15 +113,443 @@ public void testWithReferenceAfterRecreatingBranch() Assertions.assertThat(client.withReference(branch, null)).isNotEqualTo(client); } + @Test + public void testCreateNamespace() throws NessieConflictException, NessieNotFoundException { + String branch = "createNamespaceBranch"; + createBranch(branch); + Map catalogOptions = + Map.of( + CatalogProperties.USER, "iceberg-user", + CatalogProperties.APP_ID, "iceberg-nessie"); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, catalogOptions); + + client.createNamespace(Namespace.of("a"), Map.of()); + Assertions.assertThat(client.listNamespaces(Namespace.of("a"))).isNotNull(); + + List entries = api.getCommitLog().refName(branch).get().getLogEntries(); + Assertions.assertThat(entries) + .isNotEmpty() + .first() + .extracting(LogResponse.LogEntry::getCommitMeta) + .extracting(CommitMeta::getMessage, CommitMeta::getAuthor, CommitMeta::getProperties) + .containsExactly( + "create namespace a", + "iceberg-user", + ImmutableMap.of( + "application-type", "iceberg", + "app-id", "iceberg-nessie")); + } + + @Test + public void testCreateNamespaceInvalid() throws NessieConflictException, NessieNotFoundException { + String branch = "createNamespaceInvalidBranch"; + createBranch(branch); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, Map.of()); + + Assertions.assertThatIllegalArgumentException() + .isThrownBy(() -> client.createNamespace(Namespace.empty(), Map.of())) + .withMessageContaining("Creating empty namespaces is not supported"); + + Assertions.assertThatThrownBy(() -> client.createNamespace(Namespace.of("a", "b"), Map.of())) + .isInstanceOf(NoSuchNamespaceException.class) + .hasMessageContaining("Cannot create namespace 'a.b': parent namespace 'a' does not exist"); + } + + @Test + public void testCreateNamespaceConflict() + throws NessieConflictException, NessieNotFoundException { + String branch = "createNamespaceConflictBranch"; + createBranch(branch); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, Map.of()); + + client.createNamespace(Namespace.of("a"), Map.of()); + + Assertions.assertThatThrownBy(() -> client.createNamespace(Namespace.of("a"), Map.of())) + .isInstanceOf(AlreadyExistsException.class) + .hasMessageContaining("Namespace already exists: a"); + + client.commitTable( + null, newTableMetadata(), "file:///tmp/iceberg", (String) null, ContentKey.of("a", "tbl")); + + Assertions.assertThatThrownBy(() -> client.createNamespace(Namespace.of("a", "tbl"), Map.of())) + .isInstanceOf(AlreadyExistsException.class) + .hasMessageContaining("Another content object with name 'a.tbl' already exists"); + } + + @Test + public void testCreateNamespaceExternalConflict() + throws NessieConflictException, NessieNotFoundException { + String branch = "createNamespaceExternalConflictBranch"; + createBranch(branch); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, Map.of()); + + org.projectnessie.model.Namespace nessieNs = + org.projectnessie.model.Namespace.of(ContentKey.of("a")); + commit(branch, "create namespace a", Operation.Put.of(ContentKey.of("a"), nessieNs)); + + Assertions.assertThatThrownBy(() -> client.createNamespace(Namespace.of("a"), Map.of())) + .isInstanceOf(AlreadyExistsException.class) + .hasMessageContaining("Namespace already exists: a"); + + IcebergTable table = IcebergTable.of("file:///tmp/iceberg", 1, 1, 1, 1); + commit(branch, "create table a.tbl2", Operation.Put.of(ContentKey.of("a", "tbl"), table)); + + Assertions.assertThatThrownBy(() -> client.createNamespace(Namespace.of("a", "tbl"), Map.of())) + .isInstanceOf(AlreadyExistsException.class) + .hasMessageContaining("Another content object with name 'a.tbl' already exists"); + } + + @Test + public void testCreateNamespaceNonExistingRef() + throws NessieConflictException, NessieNotFoundException { + String branch = "createNamespaceNonExistingRefBranch"; + createBranch(branch); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, Map.of()); + + client.createNamespace(Namespace.of("a"), Map.of()); + + api.deleteBranch().branch((Branch) api.getReference().refName(branch).get()).delete(); + + Assertions.assertThatThrownBy(() -> client.createNamespace(Namespace.of("b"), Map.of())) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining( + "Cannot create namespace 'b': ref 'createNamespaceNonExistingRefBranch' is no longer valid"); + } + + @Test + public void testDropNamespace() throws NessieConflictException, NessieNotFoundException { + String branch = "dropNamespaceBranch"; + createBranch(branch); + Map catalogOptions = + Map.of( + CatalogProperties.USER, "iceberg-user", + CatalogProperties.APP_ID, "iceberg-nessie"); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, catalogOptions); + + Namespace parent = Namespace.of("a"); + Namespace child = Namespace.of("a", "b"); + + Assertions.assertThat(client.dropNamespace(parent)).isFalse(); + Assertions.assertThat(client.dropNamespace(child)).isFalse(); + + client.createNamespace(parent, Map.of()); + client.createNamespace(child, Map.of()); + + Assertions.assertThat(client.dropNamespace(child)).isTrue(); + Assertions.assertThat(client.dropNamespace(parent)).isTrue(); + + List entries = api.getCommitLog().refName(branch).get().getLogEntries(); + Assertions.assertThat(entries) + .isNotEmpty() + .extracting(LogResponse.LogEntry::getCommitMeta) + .satisfies( + meta -> { + Assertions.assertThat(meta.getMessage()).contains("drop namespace a"); + Assertions.assertThat(meta.getAuthor()).isEqualTo("iceberg-user"); + Assertions.assertThat(meta.getProperties()) + .containsEntry(NessieUtil.APPLICATION_TYPE, "iceberg") + .containsEntry(CatalogProperties.APP_ID, "iceberg-nessie"); + }, + Index.atIndex(0)) + .satisfies( + meta -> { + Assertions.assertThat(meta.getMessage()).contains("drop namespace a.b"); + Assertions.assertThat(meta.getAuthor()).isEqualTo("iceberg-user"); + Assertions.assertThat(meta.getProperties()) + .containsEntry(NessieUtil.APPLICATION_TYPE, "iceberg") + .containsEntry(CatalogProperties.APP_ID, "iceberg-nessie"); + }, + Index.atIndex(1)); + } + + @Test + public void testDropNamespaceNotEmpty() throws NessieConflictException, NessieNotFoundException { + String branch = "dropNamespaceInvalidBranch"; + createBranch(branch); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, Map.of()); + + client.createNamespace(Namespace.of("a"), Map.of()); + client.createNamespace(Namespace.of("a", "b"), Map.of()); + + Assertions.assertThatThrownBy(() -> client.dropNamespace(Namespace.of("a"))) + .isInstanceOf(NamespaceNotEmptyException.class) + .hasMessageContaining("Namespace 'a' is not empty."); + } + + @Test + public void testDropNamespaceConflict() throws NessieConflictException, NessieNotFoundException { + String branch = "dropNamespaceConflictBranch"; + createBranch(branch); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, Map.of()); + + client.createNamespace(Namespace.of("a"), Map.of()); + + client.commitTable( + null, newTableMetadata(), "file:///tmp/iceberg", (String) null, ContentKey.of("a", "tbl")); + + Assertions.assertThatThrownBy(() -> client.dropNamespace(Namespace.of("a", "tbl"))) + .isInstanceOf(NoSuchNamespaceException.class) + .hasMessageContaining("Content object with name 'a.tbl' is not a namespace."); + } + + @Test + public void testDropNamespaceExternalConflict() + throws NessieConflictException, NessieNotFoundException { + String branch = "dropNamespaceExternalConflictBranch"; + createBranch(branch); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, Map.of()); + + client.createNamespace(Namespace.of("a"), Map.of()); + + org.projectnessie.model.Namespace original = fetchNamespace(ContentKey.of("a"), branch); + org.projectnessie.model.Namespace updated = + org.projectnessie.model.Namespace.builder() + .from(original) + .properties(Map.of("k1", "v1")) + .build(); + commit(branch, "update namespace a", Operation.Put.of(ContentKey.of("a"), updated)); + + Assertions.assertThatThrownBy(() -> client.dropNamespace(Namespace.of("a"))) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining( + "Cannot drop namespace 'a': Values of existing and expected content for key 'a' are different."); + } + + @Test + public void testDropNamespaceNonExistingRef() + throws NessieConflictException, NessieNotFoundException { + String branch = "dropNamespaceNonExistingRefBranch"; + createBranch(branch); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, Map.of()); + + client.createNamespace(Namespace.of("a"), Map.of()); + + api.deleteBranch().branch((Branch) api.getReference().refName(branch).get()).delete(); + + Assertions.assertThat(client.dropNamespace(Namespace.of("a"))).isFalse(); + } + + @Test + public void testSetProperties() throws NessieConflictException, NessieNotFoundException { + String branch = "setPropertiesBranch"; + createBranch(branch); + Map catalogOptions = + Map.of( + CatalogProperties.USER, "iceberg-user", + CatalogProperties.APP_ID, "iceberg-nessie"); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, catalogOptions); + + Namespace ns = Namespace.of("a"); + client.createNamespace(ns, Map.of("k1", "v1a")); + + Assertions.assertThat(client.setProperties(ns, Map.of("k1", "v1b", "k2", "v2"))).isTrue(); + + Assertions.assertThat(client.loadNamespaceMetadata(ns)) + .hasSize(2) + .containsEntry("k1", "v1b") + .containsEntry("k2", "v2"); + + List entries = api.getCommitLog().refName(branch).get().getLogEntries(); + Assertions.assertThat(entries) + .isNotEmpty() + .first() + .extracting(LogResponse.LogEntry::getCommitMeta) + .satisfies( + meta -> { + Assertions.assertThat(meta.getMessage()).contains("update namespace a"); + Assertions.assertThat(meta.getAuthor()).isEqualTo("iceberg-user"); + Assertions.assertThat(meta.getProperties()) + .containsEntry(NessieUtil.APPLICATION_TYPE, "iceberg") + .containsEntry(CatalogProperties.APP_ID, "iceberg-nessie"); + }); + } + + @Test + public void testSetPropertiesExternalConflict() + throws NessieConflictException, NessieNotFoundException { + String branch = "setPropertiesExternalConflictBranch"; + createBranch(branch); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, Map.of()); + + Namespace ns = Namespace.of("a"); + client.createNamespace(ns, Map.of("k1", "v1a")); + + ContentKey key = ContentKey.of("a"); + org.projectnessie.model.Namespace original = fetchNamespace(key, branch); + org.projectnessie.model.Namespace updated = + org.projectnessie.model.Namespace.builder() + .from(original) + .properties(Map.of("k1", "v1b", "k2", "v2")) + .build(); + commit(branch, "update namespace a", Operation.Put.of(key, updated)); + + // will generate a conflict and a retry + Assertions.assertThat(client.setProperties(ns, Map.of("k1", "v1c", "k3", "v3"))).isTrue(); + + Assertions.assertThat(client.loadNamespaceMetadata(ns)) + .hasSize(3) + .containsEntry("k1", "v1c") + .containsEntry("k2", "v2") + .containsEntry("k3", "v3"); + } + + @Test + public void testSetPropertiesNonExistingNs() + throws NessieConflictException, NessieNotFoundException { + String branch = "setPropertiesNonExistingNsBranch"; + createBranch(branch); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, Map.of()); + + client.createNamespace(Namespace.of("a"), Map.of()); + + commit(branch, "delete namespace a", Operation.Delete.of(ContentKey.of("a"))); + + Assertions.assertThatThrownBy( + () -> client.setProperties(Namespace.of("a"), Map.of("k1", "v1a"))) + .isInstanceOf(NoSuchNamespaceException.class) + .hasMessageContaining("Namespace does not exist: a"); + } + + @Test + public void testSetPropertiesNonExistingRef() + throws NessieConflictException, NessieNotFoundException { + String branch = "setPropertiesNonExistingRefBranch"; + createBranch(branch); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, Map.of()); + + client.createNamespace(Namespace.of("a"), Map.of()); + + api.deleteBranch().branch((Branch) api.getReference().refName(branch).get()).delete(); + + Assertions.assertThatThrownBy(() -> client.setProperties(Namespace.of("a"), Map.of("k1", "v1"))) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining( + "Cannot update properties on namespace 'a': ref 'setPropertiesNonExistingRefBranch' is no longer valid"); + } + + @Test + public void testRemoveProperties() throws NessieConflictException, NessieNotFoundException { + String branch = "removePropertiesBranch"; + createBranch(branch); + Map catalogOptions = + Map.of( + CatalogProperties.USER, "iceberg-user", + CatalogProperties.APP_ID, "iceberg-nessie"); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, catalogOptions); + + Namespace ns = Namespace.of("a"); + + client.createNamespace(ns, Map.of("k1", "v1", "k2", "v2")); + + Assertions.assertThat(client.removeProperties(ns, Set.of("k1"))).isTrue(); + + Assertions.assertThat(client.loadNamespaceMetadata(ns)).hasSize(1).containsOnlyKeys("k2"); + + List entries = api.getCommitLog().refName(branch).get().getLogEntries(); + Assertions.assertThat(entries) + .isNotEmpty() + .first() + .extracting(LogResponse.LogEntry::getCommitMeta) + .satisfies( + meta -> { + Assertions.assertThat(meta.getMessage()).contains("update namespace a"); + Assertions.assertThat(meta.getAuthor()).isEqualTo("iceberg-user"); + Assertions.assertThat(meta.getProperties()) + .containsEntry(NessieUtil.APPLICATION_TYPE, "iceberg") + .containsEntry(CatalogProperties.APP_ID, "iceberg-nessie"); + }); + } + + @Test + public void testRemovePropertiesExternalConflict() + throws NessieConflictException, NessieNotFoundException { + String branch = "removePropertiesExternalConflictBranch"; + createBranch(branch); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, Map.of()); + + Namespace ns = Namespace.of("a"); + client.createNamespace(ns, Map.of("k1", "v1")); + + ContentKey key = ContentKey.of("a"); + org.projectnessie.model.Namespace original = fetchNamespace(key, branch); + org.projectnessie.model.Namespace updated = + org.projectnessie.model.Namespace.builder() + .from(original) + .properties(Map.of("k2", "v2", "k3", "v3")) + .build(); + commit(branch, "update namespace a", Operation.Put.of(key, updated)); + + // will generate a conflict and a retry + Assertions.assertThat(client.removeProperties(ns, Set.of("k2"))).isTrue(); + + Assertions.assertThat(client.loadNamespaceMetadata(ns)).hasSize(1).containsOnlyKeys("k3"); + } + + @Test + public void testRemovePropertiesNonExistingNs() + throws NessieConflictException, NessieNotFoundException { + String branch = "removePropertiesNonExistingNsBranch"; + createBranch(branch); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, Map.of()); + + client.createNamespace(Namespace.of("a"), Map.of("k1", "v1")); + + commit(branch, "delete namespace a", Operation.Delete.of(ContentKey.of("a"))); + + Assertions.assertThatThrownBy(() -> client.removeProperties(Namespace.of("a"), Set.of("k1"))) + .isInstanceOf(NoSuchNamespaceException.class) + .hasMessageContaining("Namespace does not exist: a"); + } + + @Test + public void testRemovePropertiesNonExistingRef() + throws NessieConflictException, NessieNotFoundException { + String branch = "removePropertiesNonExistingRefBranch"; + createBranch(branch); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, Map.of()); + + client.createNamespace(Namespace.of("a"), Map.of("k1", "v1")); + + api.deleteBranch().branch((Branch) api.getReference().refName(branch).get()).delete(); + + Assertions.assertThatThrownBy(() -> client.removeProperties(Namespace.of("a"), Set.of("k1"))) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining( + "Cannot update properties on namespace 'a': ref 'removePropertiesNonExistingRefBranch' is no longer valid"); + } + @Test public void testInvalidClientApiVersion() throws IOException { try (NessieCatalog newCatalog = new NessieCatalog()) { newCatalog.setConf(hadoopConfig); ImmutableMap.Builder options = ImmutableMap.builder().put("client-api-version", "3"); - Assertions.assertThatThrownBy(() -> newCatalog.initialize("nessie", options.buildOrThrow())) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Unsupported client-api-version: 3. Can only be 1 or 2"); + Assertions.assertThatIllegalArgumentException() + .isThrownBy(() -> newCatalog.initialize("nessie", options.buildOrThrow())) + .withMessage("Unsupported client-api-version: 3. Can only be 1 or 2"); } } + + private void commit(String branch, String message, Operation... operations) + throws NessieNotFoundException, NessieConflictException { + Branch ref = (Branch) api.getReference().refName(branch).get(); + api.commitMultipleOperations() + .branch(ref) + .commitMeta(NessieUtil.buildCommitMetadata(message, Map.of())) + .operations(Arrays.asList(operations)) + .commit(); + } + + private org.projectnessie.model.Namespace fetchNamespace(ContentKey key, String branch) + throws NessieNotFoundException { + Reference reference = api.getReference().refName(branch).get(); + Content content = api.getContent().key(key).reference(reference).get().get(key); + return content.unwrap(org.projectnessie.model.Namespace.class).orElseThrow(); + } + + private static TableMetadata newTableMetadata() { + Schema schema = new Schema(required(1, "id", Types.LongType.get())); + return TableMetadata.newTableMetadata( + schema, PartitionSpec.unpartitioned(), SortOrder.unsorted(), null, Map.of()); + } } From d9295903abbabbbed77e12476aef62ae5260663c Mon Sep 17 00:00:00 2001 From: Yujiang Zhong <42907416+zhongyujiang@users.noreply.github.com> Date: Thu, 7 Dec 2023 21:21:35 +0800 Subject: [PATCH 17/47] Docs: Update default format version to 2. (#9239) --- docs/configuration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/configuration.md b/docs/configuration.md index ead5c763ab2b..7931394f3c0b 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -118,7 +118,7 @@ The value of these properties are not persisted as a part of the table metadata. | Property | Default | Description | | -------------- | -------- | ------------------------------------------------------------- | -| format-version | 1 | Table's format version (can be 1 or 2) as defined in the [Spec](../../../spec/#format-versioning). | +| format-version | 2 | Table's format version (can be 1 or 2) as defined in the [Spec](../../../spec/#format-versioning). Defaults to 2 since version 1.4.0. | ### Compatibility flags From 820fc3ceda386149f42db8b54e6db9171d1a3a6d Mon Sep 17 00:00:00 2001 From: Rodrigo Meneses Date: Mon, 4 Dec 2023 09:51:02 -0800 Subject: [PATCH 18/47] Flink: Move flink/v1.17 to flink/v1.18 --- flink/{v1.17 => v1.18}/build.gradle | 0 flink/{v1.17 => v1.18}/flink-runtime/LICENSE | 0 flink/{v1.17 => v1.18}/flink-runtime/NOTICE | 0 .../java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java | 0 .../src/main/java/org/apache/iceberg/flink/CatalogLoader.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkCatalog.java | 0 .../main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkConfParser.java | 0 .../main/java/org/apache/iceberg/flink/FlinkConfigOptions.java | 0 .../java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java | 0 .../java/org/apache/iceberg/flink/FlinkEnvironmentContext.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkFilters.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkReadConf.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java | 0 .../src/main/java/org/apache/iceberg/flink/IcebergTableSink.java | 0 .../src/main/java/org/apache/iceberg/flink/RowDataWrapper.java | 0 .../flink/src/main/java/org/apache/iceberg/flink/TableLoader.java | 0 .../src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java | 0 .../src/main/java/org/apache/iceberg/flink/actions/Actions.java | 0 .../org/apache/iceberg/flink/actions/RewriteDataFilesAction.java | 0 .../org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java | 0 .../main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java | 0 .../main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java | 0 .../main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java | 0 .../main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java | 0 .../main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java | 0 .../main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java | 0 .../java/org/apache/iceberg/flink/data/FlinkParquetReaders.java | 0 .../java/org/apache/iceberg/flink/data/FlinkParquetWriters.java | 0 .../java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java | 0 .../java/org/apache/iceberg/flink/data/FlinkValueReaders.java | 0 .../java/org/apache/iceberg/flink/data/FlinkValueWriters.java | 0 .../apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java | 0 .../java/org/apache/iceberg/flink/data/RowDataProjection.java | 0 .../src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java | 0 .../main/java/org/apache/iceberg/flink/data/StructRowData.java | 0 .../iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java | 0 .../java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java | 0 .../org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java | 0 .../java/org/apache/iceberg/flink/sink/BucketPartitioner.java | 0 .../java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java | 0 .../java/org/apache/iceberg/flink/sink/CachingTableSupplier.java | 0 .../main/java/org/apache/iceberg/flink/sink/CommitSummary.java | 0 .../main/java/org/apache/iceberg/flink/sink/DeltaManifests.java | 0 .../org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java | 0 .../org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java | 0 .../java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java | 0 .../org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java | 0 .../java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java | 0 .../src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java | 0 .../java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java | 0 .../apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java | 0 .../java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java | 0 .../org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java | 0 .../org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java | 0 .../java/org/apache/iceberg/flink/sink/PartitionKeySelector.java | 0 .../org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java | 0 .../org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java | 0 .../java/org/apache/iceberg/flink/sink/TaskWriterFactory.java | 0 .../org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java | 0 .../apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java | 0 .../iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java | 0 .../org/apache/iceberg/flink/sink/shuffle/DataStatistics.java | 0 .../iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java | 0 .../flink/sink/shuffle/DataStatisticsCoordinatorProvider.java | 0 .../apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java | 0 .../apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java | 0 .../apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java | 0 .../flink/sink/shuffle/DataStatisticsOrRecordSerializer.java | 0 .../org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java | 0 .../org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java | 0 .../iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java | 0 .../iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java | 0 .../main/java/org/apache/iceberg/flink/source/DataIterator.java | 0 .../main/java/org/apache/iceberg/flink/source/DataTaskReader.java | 0 .../java/org/apache/iceberg/flink/source/FileScanTaskReader.java | 0 .../java/org/apache/iceberg/flink/source/FlinkInputFormat.java | 0 .../java/org/apache/iceberg/flink/source/FlinkInputSplit.java | 0 .../main/java/org/apache/iceberg/flink/source/FlinkSource.java | 0 .../java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java | 0 .../main/java/org/apache/iceberg/flink/source/IcebergSource.java | 0 .../java/org/apache/iceberg/flink/source/IcebergTableSource.java | 0 .../apache/iceberg/flink/source/RowDataFileScanTaskReader.java | 0 .../java/org/apache/iceberg/flink/source/RowDataRewriter.java | 0 .../iceberg/flink/source/RowDataToAvroGenericRecordConverter.java | 0 .../main/java/org/apache/iceberg/flink/source/ScanContext.java | 0 .../src/main/java/org/apache/iceberg/flink/source/SourceUtil.java | 0 .../org/apache/iceberg/flink/source/StreamingMonitorFunction.java | 0 .../org/apache/iceberg/flink/source/StreamingReaderOperator.java | 0 .../apache/iceberg/flink/source/StreamingStartingStrategy.java | 0 .../iceberg/flink/source/assigner/DefaultSplitAssigner.java | 0 .../org/apache/iceberg/flink/source/assigner/GetSplitResult.java | 0 .../flink/source/assigner/OrderedSplitAssignerFactory.java | 0 .../iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java | 0 .../org/apache/iceberg/flink/source/assigner/SplitAssigner.java | 0 .../iceberg/flink/source/assigner/SplitAssignerFactory.java | 0 .../apache/iceberg/flink/source/assigner/SplitAssignerType.java | 0 .../flink/source/enumerator/AbstractIcebergEnumerator.java | 0 .../flink/source/enumerator/ContinuousEnumerationResult.java | 0 .../flink/source/enumerator/ContinuousIcebergEnumerator.java | 0 .../iceberg/flink/source/enumerator/ContinuousSplitPlanner.java | 0 .../flink/source/enumerator/ContinuousSplitPlannerImpl.java | 0 .../iceberg/flink/source/enumerator/EnumerationHistory.java | 0 .../flink/source/enumerator/IcebergEnumeratorPosition.java | 0 .../source/enumerator/IcebergEnumeratorPositionSerializer.java | 0 .../iceberg/flink/source/enumerator/IcebergEnumeratorState.java | 0 .../flink/source/enumerator/IcebergEnumeratorStateSerializer.java | 0 .../iceberg/flink/source/enumerator/StaticIcebergEnumerator.java | 0 .../org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java | 0 .../iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java | 0 .../flink/source/reader/AvroGenericRecordReaderFunction.java | 0 .../flink/source/reader/ColumnStatsWatermarkExtractor.java | 0 .../apache/iceberg/flink/source/reader/DataIteratorBatcher.java | 0 .../iceberg/flink/source/reader/DataIteratorReaderFunction.java | 0 .../apache/iceberg/flink/source/reader/IcebergSourceReader.java | 0 .../iceberg/flink/source/reader/IcebergSourceReaderMetrics.java | 0 .../iceberg/flink/source/reader/IcebergSourceSplitReader.java | 0 .../org/apache/iceberg/flink/source/reader/ListBatchRecords.java | 0 .../iceberg/flink/source/reader/ListDataIteratorBatcher.java | 0 .../iceberg/flink/source/reader/MetaDataReaderFunction.java | 0 .../org/apache/iceberg/flink/source/reader/ReaderFunction.java | 0 .../org/apache/iceberg/flink/source/reader/RecordAndPosition.java | 0 .../org/apache/iceberg/flink/source/reader/RecordFactory.java | 0 .../apache/iceberg/flink/source/reader/RowDataReaderFunction.java | 0 .../apache/iceberg/flink/source/reader/RowDataRecordFactory.java | 0 .../iceberg/flink/source/reader/SerializableRecordEmitter.java | 0 .../iceberg/flink/source/reader/SplitWatermarkExtractor.java | 0 .../flink/source/reader/WatermarkExtractorRecordEmitter.java | 0 .../org/apache/iceberg/flink/source/split/IcebergSourceSplit.java | 0 .../iceberg/flink/source/split/IcebergSourceSplitSerializer.java | 0 .../iceberg/flink/source/split/IcebergSourceSplitState.java | 0 .../iceberg/flink/source/split/IcebergSourceSplitStatus.java | 0 .../apache/iceberg/flink/source/split/SerializableComparator.java | 0 .../org/apache/iceberg/flink/source/split/SplitComparators.java | 0 .../org/apache/iceberg/flink/source/split/SplitRequestEvent.java | 0 .../java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java | 0 .../org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java | 0 .../src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java | 0 .../META-INF/services/org.apache.flink.table.factories.Factory | 0 .../services/org.apache.flink.table.factories.TableFactory | 0 .../org/apache/iceberg/flink/AvroGenericRecordConverterBase.java | 0 .../src/test/java/org/apache/iceberg/flink/DataGenerator.java | 0 .../src/test/java/org/apache/iceberg/flink/DataGenerators.java | 0 .../test/java/org/apache/iceberg/flink/FlinkCatalogTestBase.java | 0 .../src/test/java/org/apache/iceberg/flink/FlinkTestBase.java | 0 .../java/org/apache/iceberg/flink/HadoopCatalogExtension.java | 0 .../test/java/org/apache/iceberg/flink/HadoopCatalogResource.java | 0 .../test/java/org/apache/iceberg/flink/HadoopTableResource.java | 0 .../test/java/org/apache/iceberg/flink/MiniClusterResource.java | 0 .../java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java | 0 .../src/test/java/org/apache/iceberg/flink/RowDataConverter.java | 0 .../src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java | 0 .../flink/src/test/java/org/apache/iceberg/flink/TestBase.java | 0 .../src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java | 0 .../java/org/apache/iceberg/flink/TestCatalogTableLoader.java | 0 .../test/java/org/apache/iceberg/flink/TestChangeLogTable.java | 0 .../java/org/apache/iceberg/flink/TestDataFileSerialization.java | 0 .../src/test/java/org/apache/iceberg/flink/TestFixtures.java | 0 .../java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java | 0 .../java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java | 0 .../java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java | 0 .../test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java | 0 .../org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java | 0 .../test/java/org/apache/iceberg/flink/TestFlinkConfParser.java | 0 .../src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java | 0 .../test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java | 0 .../test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java | 0 .../test/java/org/apache/iceberg/flink/TestFlinkTableSink.java | 0 .../src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java | 0 .../flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java | 0 .../test/java/org/apache/iceberg/flink/TestIcebergConnector.java | 0 .../org/apache/iceberg/flink/TestManifestFileSerialization.java | 0 .../test/java/org/apache/iceberg/flink/TestRowDataWrapper.java | 0 .../src/test/java/org/apache/iceberg/flink/TestTableLoader.java | 0 .../java/org/apache/iceberg/flink/TestTableSerialization.java | 0 .../apache/iceberg/flink/actions/TestRewriteDataFilesAction.java | 0 .../test/java/org/apache/iceberg/flink/data/RandomRowData.java | 0 .../java/org/apache/iceberg/flink/data/RowDataToRowMapper.java | 0 .../org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java | 0 .../org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java | 0 .../org/apache/iceberg/flink/data/TestFlinkParquetReader.java | 0 .../org/apache/iceberg/flink/data/TestFlinkParquetWriter.java | 0 .../java/org/apache/iceberg/flink/data/TestRowDataProjection.java | 0 .../java/org/apache/iceberg/flink/data/TestRowProjection.java | 0 .../java/org/apache/iceberg/flink/data/TestStructRowData.java | 0 .../iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java | 0 .../apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java | 0 .../java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java | 0 .../iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java | 0 .../org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java | 0 .../org/apache/iceberg/flink/sink/TestCachingTableSupplier.java | 0 .../org/apache/iceberg/flink/sink/TestCompressionSettings.java | 0 .../java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java | 0 .../org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java | 0 .../org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java | 0 .../java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java | 0 .../org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java | 0 .../org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java | 0 .../org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java | 0 .../org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java | 0 .../apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java | 0 .../java/org/apache/iceberg/flink/sink/TestFlinkManifest.java | 0 .../apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java | 0 .../apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java | 0 .../apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java | 0 .../org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java | 0 .../org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java | 0 .../org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java | 0 .../org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java | 0 .../test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java | 0 .../iceberg/flink/sink/shuffle/TestAggregatedStatistics.java | 0 .../flink/sink/shuffle/TestAggregatedStatisticsTracker.java | 0 .../iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java | 0 .../flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java | 0 .../iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java | 0 .../java/org/apache/iceberg/flink/source/BoundedTableFactory.java | 0 .../java/org/apache/iceberg/flink/source/BoundedTestSource.java | 0 .../org/apache/iceberg/flink/source/ChangeLogTableTestBase.java | 0 .../test/java/org/apache/iceberg/flink/source/SplitHelpers.java | 0 .../src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java | 0 .../org/apache/iceberg/flink/source/TestBoundedTableFactory.java | 0 .../org/apache/iceberg/flink/source/TestFlinkInputFormat.java | 0 .../iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java | 0 .../org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java | 0 .../org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java | 0 .../apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java | 0 .../test/java/org/apache/iceberg/flink/source/TestFlinkScan.java | 0 .../java/org/apache/iceberg/flink/source/TestFlinkScanSql.java | 0 .../java/org/apache/iceberg/flink/source/TestFlinkSource.java | 0 .../org/apache/iceberg/flink/source/TestFlinkSourceConfig.java | 0 .../java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java | 0 .../org/apache/iceberg/flink/source/TestFlinkTableSource.java | 0 .../org/apache/iceberg/flink/source/TestIcebergSourceBounded.java | 0 .../flink/source/TestIcebergSourceBoundedGenericRecord.java | 0 .../apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java | 0 .../apache/iceberg/flink/source/TestIcebergSourceContinuous.java | 0 .../apache/iceberg/flink/source/TestIcebergSourceFailover.java | 0 .../source/TestIcebergSourceFailoverWithWatermarkExtractor.java | 0 .../iceberg/flink/source/TestIcebergSourceReaderDeletes.java | 0 .../org/apache/iceberg/flink/source/TestIcebergSourceSql.java | 0 .../flink/source/TestIcebergSourceWithWatermarkExtractor.java | 0 .../iceberg/flink/source/TestMetadataTableReadableMetrics.java | 0 .../org/apache/iceberg/flink/source/TestProjectMetaColumn.java | 0 .../flink/source/TestRowDataToAvroGenericRecordConverter.java | 0 .../test/java/org/apache/iceberg/flink/source/TestSourceUtil.java | 0 .../test/java/org/apache/iceberg/flink/source/TestSqlBase.java | 0 .../java/org/apache/iceberg/flink/source/TestStreamScanSql.java | 0 .../apache/iceberg/flink/source/TestStreamingMonitorFunction.java | 0 .../apache/iceberg/flink/source/TestStreamingReaderOperator.java | 0 .../iceberg/flink/source/assigner/SplitAssignerTestBase.java | 0 .../iceberg/flink/source/assigner/TestDefaultSplitAssigner.java | 0 .../source/assigner/TestFileSequenceNumberBasedSplitAssigner.java | 0 .../flink/source/assigner/TestWatermarkBasedSplitAssigner.java | 0 .../flink/source/enumerator/ManualContinuousSplitPlanner.java | 0 .../flink/source/enumerator/TestContinuousIcebergEnumerator.java | 0 .../flink/source/enumerator/TestContinuousSplitPlannerImpl.java | 0 .../enumerator/TestContinuousSplitPlannerImplStartStrategy.java | 0 .../iceberg/flink/source/enumerator/TestEnumerationHistory.java | 0 .../source/enumerator/TestIcebergEnumeratorStateSerializer.java | 0 .../iceberg/flink/source/reader/ReaderFunctionTestBase.java | 0 .../java/org/apache/iceberg/flink/source/reader/ReaderUtil.java | 0 .../apache/iceberg/flink/source/reader/TestArrayBatchRecords.java | 0 .../source/reader/TestArrayPoolDataIteratorBatcherRowData.java | 0 .../flink/source/reader/TestColumnStatsWatermarkExtractor.java | 0 .../iceberg/flink/source/reader/TestIcebergSourceReader.java | 0 .../iceberg/flink/source/reader/TestRowDataReaderFunction.java | 0 .../apache/iceberg/flink/source/reader/TestingMetricGroup.java | 0 .../flink/source/split/TestIcebergSourceSplitSerializer.java | 0 .../test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java | 0 .../META-INF/services/org.apache.flink.table.factories.Factory | 0 277 files changed, 0 insertions(+), 0 deletions(-) rename flink/{v1.17 => v1.18}/build.gradle (100%) rename flink/{v1.17 => v1.18}/flink-runtime/LICENSE (100%) rename flink/{v1.17 => v1.18}/flink-runtime/NOTICE (100%) rename flink/{v1.17 => v1.18}/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java (100%) rename flink/{v1.17 => v1.18}/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory (100%) rename flink/{v1.17 => v1.18}/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/FlinkCatalogTestBase.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/TestBase.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java (100%) rename flink/{v1.17 => v1.18}/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory (100%) diff --git a/flink/v1.17/build.gradle b/flink/v1.18/build.gradle similarity index 100% rename from flink/v1.17/build.gradle rename to flink/v1.18/build.gradle diff --git a/flink/v1.17/flink-runtime/LICENSE b/flink/v1.18/flink-runtime/LICENSE similarity index 100% rename from flink/v1.17/flink-runtime/LICENSE rename to flink/v1.18/flink-runtime/LICENSE diff --git a/flink/v1.17/flink-runtime/NOTICE b/flink/v1.18/flink-runtime/NOTICE similarity index 100% rename from flink/v1.17/flink-runtime/NOTICE rename to flink/v1.18/flink-runtime/NOTICE diff --git a/flink/v1.17/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java b/flink/v1.18/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java similarity index 100% rename from flink/v1.17/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java rename to flink/v1.18/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java diff --git a/flink/v1.17/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink/v1.18/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory similarity index 100% rename from flink/v1.17/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory rename to flink/v1.18/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory diff --git a/flink/v1.17/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink/v1.18/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory similarity index 100% rename from flink/v1.17/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory rename to flink/v1.18/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/FlinkCatalogTestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/FlinkCatalogTestBase.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/FlinkCatalogTestBase.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/FlinkCatalogTestBase.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestBase.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java diff --git a/flink/v1.17/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink/v1.18/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory similarity index 100% rename from flink/v1.17/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory rename to flink/v1.18/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory From b8ef64a2cd879cac284189455ab0579c5be80441 Mon Sep 17 00:00:00 2001 From: Rodrigo Meneses Date: Mon, 4 Dec 2023 09:51:34 -0800 Subject: [PATCH 19/47] Recover flink/1.17 files from history --- flink/v1.17/build.gradle | 263 ++++ flink/v1.17/flink-runtime/LICENSE | 502 +++++++ flink/v1.17/flink-runtime/NOTICE | 91 ++ .../flink/IcebergConnectorSmokeTest.java | 29 + .../apache/iceberg/flink/CatalogLoader.java | 215 +++ .../apache/iceberg/flink/FlinkCatalog.java | 833 ++++++++++++ .../iceberg/flink/FlinkCatalogFactory.java | 213 +++ .../apache/iceberg/flink/FlinkConfParser.java | 261 ++++ .../iceberg/flink/FlinkConfigOptions.java | 107 ++ .../flink/FlinkDynamicTableFactory.java | 209 +++ .../flink/FlinkEnvironmentContext.java | 31 + .../apache/iceberg/flink/FlinkFilters.java | 266 ++++ .../apache/iceberg/flink/FlinkFixupTypes.java | 50 + .../apache/iceberg/flink/FlinkReadConf.java | 193 +++ .../iceberg/flink/FlinkReadOptions.java | 112 ++ .../apache/iceberg/flink/FlinkSchemaUtil.java | 191 +++ .../iceberg/flink/FlinkSourceFilter.java | 49 + .../apache/iceberg/flink/FlinkTypeToType.java | 203 +++ .../iceberg/flink/FlinkTypeVisitor.java | 80 ++ .../apache/iceberg/flink/FlinkWriteConf.java | 205 +++ .../iceberg/flink/FlinkWriteOptions.java | 73 + .../iceberg/flink/IcebergTableSink.java | 121 ++ .../apache/iceberg/flink/RowDataWrapper.java | 142 ++ .../org/apache/iceberg/flink/TableLoader.java | 159 +++ .../apache/iceberg/flink/TypeToFlinkType.java | 134 ++ .../apache/iceberg/flink/actions/Actions.java | 52 + .../flink/actions/RewriteDataFilesAction.java | 72 + .../data/AvroWithFlinkSchemaVisitor.java | 75 ++ .../iceberg/flink/data/FlinkAvroReader.java | 169 +++ .../iceberg/flink/data/FlinkAvroWriter.java | 165 +++ .../iceberg/flink/data/FlinkOrcReader.java | 131 ++ .../iceberg/flink/data/FlinkOrcReaders.java | 283 ++++ .../iceberg/flink/data/FlinkOrcWriter.java | 163 +++ .../iceberg/flink/data/FlinkOrcWriters.java | 317 +++++ .../flink/data/FlinkParquetReaders.java | 832 ++++++++++++ .../flink/data/FlinkParquetWriters.java | 504 +++++++ .../flink/data/FlinkSchemaVisitor.java | 161 +++ .../iceberg/flink/data/FlinkValueReaders.java | 312 +++++ .../iceberg/flink/data/FlinkValueWriters.java | 253 ++++ .../data/ParquetWithFlinkSchemaVisitor.java | 222 ++++ .../iceberg/flink/data/RowDataProjection.java | 341 +++++ .../iceberg/flink/data/RowDataUtil.java | 100 ++ .../iceberg/flink/data/StructRowData.java | 300 +++++ .../AvroGenericRecordToRowDataMapper.java | 61 + .../flink/sink/BaseDeltaTaskWriter.java | 125 ++ .../sink/BucketPartitionKeySelector.java | 70 + .../iceberg/flink/sink/BucketPartitioner.java | 103 ++ .../flink/sink/BucketPartitionerUtil.java | 125 ++ .../flink/sink/CachingTableSupplier.java | 91 ++ .../iceberg/flink/sink/CommitSummary.java | 93 ++ .../iceberg/flink/sink/DeltaManifests.java | 71 + .../flink/sink/DeltaManifestsSerializer.java | 122 ++ .../flink/sink/EqualityFieldKeySelector.java | 86 ++ .../flink/sink/FlinkAppenderFactory.java | 274 ++++ .../flink/sink/FlinkFileWriterFactory.java | 293 +++++ .../iceberg/flink/sink/FlinkManifestUtil.java | 132 ++ .../apache/iceberg/flink/sink/FlinkSink.java | 654 +++++++++ .../flink/sink/IcebergFilesCommitter.java | 516 ++++++++ .../sink/IcebergFilesCommitterMetrics.java | 96 ++ .../flink/sink/IcebergStreamWriter.java | 120 ++ .../sink/IcebergStreamWriterMetrics.java | 89 ++ .../flink/sink/ManifestOutputFileFactory.java | 94 ++ .../flink/sink/PartitionKeySelector.java | 64 + .../flink/sink/PartitionedDeltaWriter.java | 97 ++ .../flink/sink/RowDataTaskWriterFactory.java | 244 ++++ .../iceberg/flink/sink/TaskWriterFactory.java | 45 + .../flink/sink/UnpartitionedDeltaWriter.java | 69 + .../sink/shuffle/AggregatedStatistics.java | 71 + .../shuffle/AggregatedStatisticsTracker.java | 133 ++ .../flink/sink/shuffle/DataStatistics.java | 61 + .../shuffle/DataStatisticsCoordinator.java | 396 ++++++ .../DataStatisticsCoordinatorProvider.java | 51 + .../sink/shuffle/DataStatisticsEvent.java | 57 + .../sink/shuffle/DataStatisticsOperator.java | 188 +++ .../sink/shuffle/DataStatisticsOrRecord.java | 113 ++ .../DataStatisticsOrRecordSerializer.java | 219 +++ .../sink/shuffle/DataStatisticsUtil.java | 97 ++ .../flink/sink/shuffle/MapDataStatistics.java | 65 + .../shuffle/MapDataStatisticsSerializer.java | 183 +++ .../AvroGenericRecordFileScanTaskReader.java | 42 + .../iceberg/flink/source/DataIterator.java | 156 +++ .../iceberg/flink/source/DataTaskReader.java | 47 + .../flink/source/FileScanTaskReader.java | 35 + .../flink/source/FlinkInputFormat.java | 141 ++ .../iceberg/flink/source/FlinkInputSplit.java | 48 + .../iceberg/flink/source/FlinkSource.java | 310 +++++ .../flink/source/FlinkSplitPlanner.java | 189 +++ .../iceberg/flink/source/IcebergSource.java | 558 ++++++++ .../flink/source/IcebergTableSource.java | 229 ++++ .../source/RowDataFileScanTaskReader.java | 243 ++++ .../iceberg/flink/source/RowDataRewriter.java | 172 +++ .../RowDataToAvroGenericRecordConverter.java | 70 + .../iceberg/flink/source/ScanContext.java | 561 ++++++++ .../iceberg/flink/source/SourceUtil.java | 77 ++ .../source/StreamingMonitorFunction.java | 269 ++++ .../flink/source/StreamingReaderOperator.java | 246 ++++ .../source/StreamingStartingStrategy.java | 54 + .../source/assigner/DefaultSplitAssigner.java | 112 ++ .../flink/source/assigner/GetSplitResult.java | 77 ++ .../assigner/OrderedSplitAssignerFactory.java | 46 + .../assigner/SimpleSplitAssignerFactory.java | 37 + .../flink/source/assigner/SplitAssigner.java | 118 ++ .../source/assigner/SplitAssignerFactory.java | 30 + .../source/assigner/SplitAssignerType.java | 33 + .../enumerator/AbstractIcebergEnumerator.java | 173 +++ .../ContinuousEnumerationResult.java | 57 + .../ContinuousIcebergEnumerator.java | 177 +++ .../enumerator/ContinuousSplitPlanner.java | 30 + .../ContinuousSplitPlannerImpl.java | 236 ++++ .../source/enumerator/EnumerationHistory.java | 96 ++ .../enumerator/IcebergEnumeratorPosition.java | 79 ++ .../IcebergEnumeratorPositionSerializer.java | 90 ++ .../enumerator/IcebergEnumeratorState.java | 65 + .../IcebergEnumeratorStateSerializer.java | 196 +++ .../enumerator/StaticIcebergEnumerator.java | 51 + .../source/reader/ArrayBatchRecords.java | 171 +++ .../reader/ArrayPoolDataIteratorBatcher.java | 130 ++ .../AvroGenericRecordReaderFunction.java | 102 ++ .../reader/ColumnStatsWatermarkExtractor.java | 98 ++ .../source/reader/DataIteratorBatcher.java | 36 + .../reader/DataIteratorReaderFunction.java | 43 + .../source/reader/IcebergSourceReader.java | 77 ++ .../reader/IcebergSourceReaderMetrics.java | 61 + .../reader/IcebergSourceSplitReader.java | 147 +++ .../flink/source/reader/ListBatchRecords.java | 85 ++ .../reader/ListDataIteratorBatcher.java | 94 ++ .../source/reader/MetaDataReaderFunction.java | 65 + .../flink/source/reader/ReaderFunction.java | 31 + .../source/reader/RecordAndPosition.java | 78 ++ .../flink/source/reader/RecordFactory.java | 34 + .../source/reader/RowDataReaderFunction.java | 79 ++ .../source/reader/RowDataRecordFactory.java | 61 + .../reader/SerializableRecordEmitter.java | 40 + .../reader/SplitWatermarkExtractor.java | 28 + .../WatermarkExtractorRecordEmitter.java | 67 + .../source/split/IcebergSourceSplit.java | 177 +++ .../split/IcebergSourceSplitSerializer.java | 60 + .../source/split/IcebergSourceSplitState.java | 37 + .../split/IcebergSourceSplitStatus.java | 25 + .../source/split/SerializableComparator.java | 24 + .../flink/source/split/SplitComparators.java | 76 ++ .../flink/source/split/SplitRequestEvent.java | 54 + .../flink/util/FlinkAlterTableUtil.java | 246 ++++ .../flink/util/FlinkCompatibilityUtil.java | 47 + .../iceberg/flink/util/FlinkPackage.java | 33 + .../org.apache.flink.table.factories.Factory | 16 + ....apache.flink.table.factories.TableFactory | 16 + .../flink/AvroGenericRecordConverterBase.java | 90 ++ .../apache/iceberg/flink/DataGenerator.java | 42 + .../apache/iceberg/flink/DataGenerators.java | 1172 +++++++++++++++++ .../iceberg/flink/FlinkCatalogTestBase.java | 155 +++ .../apache/iceberg/flink/FlinkTestBase.java | 129 ++ .../iceberg/flink/HadoopCatalogExtension.java | 104 ++ .../iceberg/flink/HadoopCatalogResource.java | 90 ++ .../iceberg/flink/HadoopTableResource.java | 64 + .../iceberg/flink/MiniClusterResource.java | 53 + .../flink/MiniFlinkClusterExtension.java | 53 + .../iceberg/flink/RowDataConverter.java | 135 ++ .../apache/iceberg/flink/SimpleDataUtil.java | 443 +++++++ .../org/apache/iceberg/flink/TestBase.java | 130 ++ .../iceberg/flink/TestCatalogLoader.java | 118 ++ .../iceberg/flink/TestCatalogTableLoader.java | 116 ++ .../iceberg/flink/TestChangeLogTable.java | 305 +++++ .../flink/TestDataFileSerialization.java | 207 +++ .../apache/iceberg/flink/TestFixtures.java | 59 + .../flink/TestFlinkAnonymousTable.java | 64 + .../flink/TestFlinkCatalogDatabase.java | 293 +++++ .../flink/TestFlinkCatalogFactory.java | 117 ++ .../iceberg/flink/TestFlinkCatalogTable.java | 692 ++++++++++ .../TestFlinkCatalogTablePartitions.java | 121 ++ .../iceberg/flink/TestFlinkConfParser.java | 61 + .../iceberg/flink/TestFlinkFilters.java | 467 +++++++ .../iceberg/flink/TestFlinkHiveCatalog.java | 105 ++ .../iceberg/flink/TestFlinkSchemaUtil.java | 416 ++++++ .../iceberg/flink/TestFlinkTableSink.java | 380 ++++++ .../apache/iceberg/flink/TestFlinkUpsert.java | 346 +++++ .../org/apache/iceberg/flink/TestHelpers.java | 611 +++++++++ .../iceberg/flink/TestIcebergConnector.java | 350 +++++ .../flink/TestManifestFileSerialization.java | 174 +++ .../iceberg/flink/TestRowDataWrapper.java | 93 ++ .../apache/iceberg/flink/TestTableLoader.java | 57 + .../iceberg/flink/TestTableSerialization.java | 110 ++ .../actions/TestRewriteDataFilesAction.java | 499 +++++++ .../iceberg/flink/data/RandomRowData.java | 38 + .../flink/data/RowDataToRowMapper.java | 50 + .../flink/data/TestFlinkAvroReaderWriter.java | 184 +++ .../flink/data/TestFlinkOrcReaderWriter.java | 106 ++ .../flink/data/TestFlinkParquetReader.java | 138 ++ .../flink/data/TestFlinkParquetWriter.java | 93 ++ .../flink/data/TestRowDataProjection.java | 593 +++++++++ .../iceberg/flink/data/TestRowProjection.java | 580 ++++++++ .../iceberg/flink/data/TestStructRowData.java | 100 ++ .../TestAvroGenericRecordToRowDataMapper.java | 37 + .../sink/TestBucketPartitionKeySelector.java | 65 + .../flink/sink/TestBucketPartitioner.java | 107 ++ ...TestBucketPartitionerFlinkIcebergSink.java | 227 ++++ .../flink/sink/TestBucketPartitionerUtil.java | 126 ++ .../flink/sink/TestCachingTableSupplier.java | 81 ++ .../flink/sink/TestCompressionSettings.java | 254 ++++ .../flink/sink/TestDeltaTaskWriter.java | 440 +++++++ .../flink/sink/TestFlinkAppenderFactory.java | 70 + .../sink/TestFlinkFileWriterFactory.java | 71 + .../flink/sink/TestFlinkIcebergSink.java | 397 ++++++ .../flink/sink/TestFlinkIcebergSinkBase.java | 64 + .../sink/TestFlinkIcebergSinkBranch.java | 137 ++ .../flink/sink/TestFlinkIcebergSinkV2.java | 236 ++++ .../sink/TestFlinkIcebergSinkV2Base.java | 362 +++++ .../sink/TestFlinkIcebergSinkV2Branch.java | 134 ++ .../iceberg/flink/sink/TestFlinkManifest.java | 315 +++++ .../sink/TestFlinkPartitioningWriters.java | 71 + .../sink/TestFlinkPositionDeltaWriters.java | 71 + .../sink/TestFlinkRollingFileWriters.java | 56 + .../flink/sink/TestFlinkWriterMetrics.java | 60 + .../flink/sink/TestIcebergFilesCommitter.java | 1152 ++++++++++++++++ .../flink/sink/TestIcebergStreamWriter.java | 388 ++++++ .../flink/sink/TestRowDataPartitionKey.java | 252 ++++ .../iceberg/flink/sink/TestTaskWriters.java | 243 ++++ .../shuffle/TestAggregatedStatistics.java | 61 + .../TestAggregatedStatisticsTracker.java | 177 +++ .../TestDataStatisticsCoordinator.java | 174 +++ ...TestDataStatisticsCoordinatorProvider.java | 147 +++ .../shuffle/TestDataStatisticsOperator.java | 227 ++++ .../flink/source/BoundedTableFactory.java | 170 +++ .../flink/source/BoundedTestSource.java | 108 ++ .../flink/source/ChangeLogTableTestBase.java | 87 ++ .../iceberg/flink/source/SplitHelpers.java | 132 ++ .../iceberg/flink/source/SqlHelpers.java | 60 + .../flink/source/TestBoundedTableFactory.java | 87 ++ .../flink/source/TestFlinkInputFormat.java | 193 +++ .../TestFlinkInputFormatReaderDeletes.java | 76 ++ .../flink/source/TestFlinkMergingMetrics.java | 71 + .../flink/source/TestFlinkMetaDataTable.java | 829 ++++++++++++ .../source/TestFlinkReaderDeletesBase.java | 110 ++ .../iceberg/flink/source/TestFlinkScan.java | 542 ++++++++ .../flink/source/TestFlinkScanSql.java | 74 ++ .../iceberg/flink/source/TestFlinkSource.java | 94 ++ .../flink/source/TestFlinkSourceConfig.java | 58 + .../flink/source/TestFlinkSourceSql.java | 85 ++ .../flink/source/TestFlinkTableSource.java | 614 +++++++++ .../source/TestIcebergSourceBounded.java | 134 ++ ...TestIcebergSourceBoundedGenericRecord.java | 203 +++ .../source/TestIcebergSourceBoundedSql.java | 80 ++ .../source/TestIcebergSourceContinuous.java | 421 ++++++ .../source/TestIcebergSourceFailover.java | 297 +++++ ...gSourceFailoverWithWatermarkExtractor.java | 112 ++ .../TestIcebergSourceReaderDeletes.java | 114 ++ .../flink/source/TestIcebergSourceSql.java | 42 + ...stIcebergSourceWithWatermarkExtractor.java | 451 +++++++ .../TestMetadataTableReadableMetrics.java | 301 +++++ .../flink/source/TestProjectMetaColumn.java | 186 +++ ...stRowDataToAvroGenericRecordConverter.java | 35 + .../iceberg/flink/source/TestSourceUtil.java | 60 + .../iceberg/flink/source/TestSqlBase.java | 158 +++ .../flink/source/TestStreamScanSql.java | 318 +++++ .../source/TestStreamingMonitorFunction.java | 407 ++++++ .../source/TestStreamingReaderOperator.java | 287 ++++ .../assigner/SplitAssignerTestBase.java | 130 ++ .../assigner/TestDefaultSplitAssigner.java | 43 + ...tFileSequenceNumberBasedSplitAssigner.java | 80 ++ .../TestWatermarkBasedSplitAssigner.java | 146 ++ .../ManualContinuousSplitPlanner.java | 97 ++ .../TestContinuousIcebergEnumerator.java | 352 +++++ .../TestContinuousSplitPlannerImpl.java | 692 ++++++++++ ...ntinuousSplitPlannerImplStartStrategy.java | 205 +++ .../enumerator/TestEnumerationHistory.java | 134 ++ .../TestIcebergEnumeratorStateSerializer.java | 145 ++ .../source/reader/ReaderFunctionTestBase.java | 219 +++ .../flink/source/reader/ReaderUtil.java | 125 ++ .../source/reader/TestArrayBatchRecords.java | 68 + ...stArrayPoolDataIteratorBatcherRowData.java | 358 +++++ .../TestColumnStatsWatermarkExtractor.java | 178 +++ .../reader/TestIcebergSourceReader.java | 184 +++ .../reader/TestRowDataReaderFunction.java | 74 ++ .../source/reader/TestingMetricGroup.java | 102 ++ .../TestIcebergSourceSplitSerializer.java | 162 +++ .../iceberg/flink/util/TestFlinkPackage.java | 31 + .../org.apache.flink.table.factories.Factory | 16 + 277 files changed, 49082 insertions(+) create mode 100644 flink/v1.17/build.gradle create mode 100644 flink/v1.17/flink-runtime/LICENSE create mode 100644 flink/v1.17/flink-runtime/NOTICE create mode 100644 flink/v1.17/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java create mode 100644 flink/v1.17/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory create mode 100644 flink/v1.17/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/FlinkCatalogTestBase.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestBase.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java create mode 100644 flink/v1.17/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory diff --git a/flink/v1.17/build.gradle b/flink/v1.17/build.gradle new file mode 100644 index 000000000000..2e081b760878 --- /dev/null +++ b/flink/v1.17/build.gradle @@ -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. + */ + +String flinkMajorVersion = '1.17' +String scalaVersion = System.getProperty("scalaVersion") != null ? System.getProperty("scalaVersion") : System.getProperty("defaultScalaVersion") + +project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { + + dependencies { + implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow') + api project(':iceberg-api') + implementation project(':iceberg-common') + implementation project(':iceberg-core') + api project(':iceberg-data') + implementation project(':iceberg-orc') + implementation project(':iceberg-parquet') + implementation project(':iceberg-hive-metastore') + + compileOnly libs.flink117.avro + // for dropwizard histogram metrics implementation + compileOnly libs.flink117.metrics.dropwizard + compileOnly libs.flink117.streaming.java + compileOnly "${libs.flink117.streaming.java.get().module}:${libs.flink117.streaming.java.get().getVersion()}:tests" + compileOnly libs.flink117.table.api.java.bridge + compileOnly "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink117.get()}" + compileOnly libs.flink117.connector.base + compileOnly libs.flink117.connector.files + + compileOnly libs.hadoop2.hdfs + compileOnly libs.hadoop2.common + compileOnly(libs.hadoop2.minicluster) { + exclude group: 'org.apache.avro', module: 'avro' + } + + implementation(libs.parquet.avro) { + exclude group: 'org.apache.avro', module: 'avro' + // already shaded by Parquet + exclude group: 'it.unimi.dsi' + exclude group: 'org.codehaus.jackson' + } + + compileOnly libs.avro.avro + + implementation("${libs.orc.core.get().module}:${libs.versions.orc.get()}:nohive") { + exclude group: 'org.apache.hadoop' + exclude group: 'commons-lang' + // These artifacts are shaded and included in the orc-core fat jar + exclude group: 'com.google.protobuf', module: 'protobuf-java' + exclude group: 'org.apache.hive', module: 'hive-storage-api' + } + + testImplementation libs.flink117.connector.test.utils + testImplementation libs.flink117.core + testImplementation libs.flink117.runtime + testImplementation(libs.flink117.test.utilsjunit) { + exclude group: 'junit' + } + testImplementation(libs.flink117.test.utils) { + exclude group: "org.apache.curator", module: 'curator-test' + exclude group: 'junit' + } + + testImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') + testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') + testImplementation project(path: ':iceberg-core', configuration: 'testArtifacts') + testImplementation project(path: ':iceberg-data', configuration: 'testArtifacts') + + // By default, hive-exec is a fat/uber jar and it exports a guava library + // that's really old. We use the core classifier to be able to override our guava + // version. Luckily, hive-exec seems to work okay so far with this version of guava + // See: https://github.com/apache/hive/blob/master/ql/pom.xml#L911 for more context. + testImplementation("${libs.hive2.exec.get().module}:${libs.hive2.exec.get().getVersion()}:core") { + exclude group: 'org.apache.avro', module: 'avro' + exclude group: 'org.slf4j', module: 'slf4j-log4j12' + exclude group: 'org.pentaho' // missing dependency + exclude group: 'org.apache.hive', module: 'hive-llap-tez' + exclude group: 'org.apache.logging.log4j' + exclude group: 'com.google.protobuf', module: 'protobuf-java' + exclude group: 'org.apache.calcite' + exclude group: 'org.apache.calcite.avatica' + exclude group: 'com.google.code.findbugs', module: 'jsr305' + } + + testImplementation(libs.hive2.metastore) { + exclude group: 'org.apache.avro', module: 'avro' + exclude group: 'org.slf4j', module: 'slf4j-log4j12' + exclude group: 'org.pentaho' // missing dependency + exclude group: 'org.apache.hbase' + exclude group: 'org.apache.logging.log4j' + exclude group: 'co.cask.tephra' + exclude group: 'com.google.code.findbugs', module: 'jsr305' + exclude group: 'org.eclipse.jetty.aggregate', module: 'jetty-all' + exclude group: 'org.eclipse.jetty.orbit', module: 'javax.servlet' + exclude group: 'org.apache.parquet', module: 'parquet-hadoop-bundle' + exclude group: 'com.tdunning', module: 'json' + exclude group: 'javax.transaction', module: 'transaction-api' + exclude group: 'com.zaxxer', module: 'HikariCP' + } + + testImplementation libs.awaitility + testImplementation libs.assertj.core + } + + test { + useJUnitPlatform() + } +} + +project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") { + apply plugin: 'com.github.johnrengelman.shadow' + + tasks.jar.dependsOn tasks.shadowJar + + sourceSets { + integration { + java.srcDir "$projectDir/src/integration/java" + resources.srcDir "$projectDir/src/integration/resources" + } + } + + configurations { + implementation { + // included in Flink + exclude group: 'org.slf4j' + exclude group: 'org.apache.commons' + exclude group: 'commons-pool' + exclude group: 'commons-codec' + exclude group: 'org.xerial.snappy' + exclude group: 'javax.xml.bind' + exclude group: 'javax.annotation' + } + } + + dependencies { + implementation(project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}")) { + exclude group: 'org.apache.flink' + } + implementation project(':iceberg-aws') + implementation project(':iceberg-azure') + implementation(project(':iceberg-aliyun')) { + exclude group: 'edu.umd.cs.findbugs', module: 'findbugs' + exclude group: 'org.apache.httpcomponents', module: 'httpclient' + exclude group: 'commons-logging', module: 'commons-logging' + } + implementation project(':iceberg-gcp') + implementation(project(':iceberg-nessie')) { + exclude group: 'com.google.code.findbugs', module: 'jsr305' + } + + // for dropwizard histogram metrics implementation + implementation libs.flink117.metrics.dropwizard + + // for integration testing with the flink-runtime-jar + // all of those dependencies are required because the integration test extends FlinkTestBase + integrationCompileOnly project(':iceberg-api') + integrationImplementation libs.junit.vintage.engine + integrationImplementation libs.assertj.core + integrationImplementation project(path: ":iceberg-flink:iceberg-flink-${flinkMajorVersion}", configuration: "testArtifacts") + integrationImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') + integrationImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') + integrationImplementation(libs.flink117.test.utils) { + exclude group: "org.apache.curator", module: 'curator-test' + exclude group: 'junit' + } + + integrationImplementation libs.flink117.table.api.java.bridge + integrationImplementation "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink117.get()}" + + integrationImplementation libs.hadoop2.common + integrationImplementation libs.hadoop2.hdfs + integrationImplementation(libs.hadoop2.minicluster) { + exclude group: 'org.apache.avro', module: 'avro' + } + + integrationImplementation(libs.hive2.metastore) { + exclude group: 'org.apache.avro', module: 'avro' + exclude group: 'org.slf4j', module: 'slf4j-log4j12' + exclude group: 'org.pentaho' // missing dependency + exclude group: 'org.apache.hbase' + exclude group: 'org.apache.logging.log4j' + exclude group: 'co.cask.tephra' + exclude group: 'com.google.code.findbugs', module: 'jsr305' + exclude group: 'org.eclipse.jetty.aggregate', module: 'jetty-all' + exclude group: 'org.eclipse.jetty.orbit', module: 'javax.servlet' + exclude group: 'org.apache.parquet', module: 'parquet-hadoop-bundle' + exclude group: 'com.tdunning', module: 'json' + exclude group: 'javax.transaction', module: 'transaction-api' + exclude group: 'com.zaxxer', module: 'HikariCP' + } + + integrationImplementation("${libs.hive2.exec.get().module}:${libs.hive2.exec.get().getVersion()}:core") { + exclude group: 'org.apache.avro', module: 'avro' + exclude group: 'org.slf4j', module: 'slf4j-log4j12' + exclude group: 'org.pentaho' // missing dependency + exclude group: 'org.apache.hive', module: 'hive-llap-tez' + exclude group: 'org.apache.logging.log4j' + exclude group: 'com.google.protobuf', module: 'protobuf-java' + exclude group: 'org.apache.calcite' + exclude group: 'org.apache.calcite.avatica' + exclude group: 'com.google.code.findbugs', module: 'jsr305' + } + } + + shadowJar { + configurations = [project.configurations.runtimeClasspath] + + zip64 true + + // include the LICENSE and NOTICE files for the shaded Jar + from(projectDir) { + include 'LICENSE' + include 'NOTICE' + } + + // Relocate dependencies to avoid conflicts + relocate 'org.apache.avro', 'org.apache.iceberg.shaded.org.apache.avro' + relocate 'org.apache.parquet', 'org.apache.iceberg.shaded.org.apache.parquet' + relocate 'com.google.errorprone', 'org.apache.iceberg.shaded.com.google.errorprone' + relocate 'com.google.flatbuffers', 'org.apache.iceberg.shaded.com.google.flatbuffers' + relocate 'com.fasterxml', 'org.apache.iceberg.shaded.com.fasterxml' + relocate 'com.github.benmanes', 'org.apache.iceberg.shaded.com.github.benmanes' + relocate 'org.checkerframework', 'org.apache.iceberg.shaded.org.checkerframework' + relocate 'shaded.parquet', 'org.apache.iceberg.shaded.org.apache.parquet.shaded' + relocate 'org.apache.orc', 'org.apache.iceberg.shaded.org.apache.orc' + relocate 'io.airlift', 'org.apache.iceberg.shaded.io.airlift' + relocate 'org.threeten.extra', 'org.apache.iceberg.shaded.org.threeten.extra' + relocate 'org.apache.hc.client5', 'org.apache.iceberg.shaded.org.apache.hc.client5' + relocate 'org.apache.hc.core5', 'org.apache.iceberg.shaded.org.apache.hc.core5' + + archiveClassifier.set(null) + } + + task integrationTest(type: Test) { + description = "Test Flink Runtime Jar against Flink ${flinkMajorVersion}" + group = "verification" + jvmArgs += project.property('extraJvmArgs') + testClassesDirs = sourceSets.integration.output.classesDirs + classpath = sourceSets.integration.runtimeClasspath + files(shadowJar.archiveFile.get().asFile.path) + inputs.file(shadowJar.archiveFile.get().asFile.path) + } + integrationTest.dependsOn shadowJar + check.dependsOn integrationTest + + jar { + enabled = false + } +} diff --git a/flink/v1.17/flink-runtime/LICENSE b/flink/v1.17/flink-runtime/LICENSE new file mode 100644 index 000000000000..8ab53469eb87 --- /dev/null +++ b/flink/v1.17/flink-runtime/LICENSE @@ -0,0 +1,502 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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 binary artifact contains Apache Avro. + +Copyright: 2014-2020 The Apache Software Foundation. +Home page: https://parquet.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains the Jackson JSON processor. + +Copyright: 2007-2020 Tatu Saloranta and other contributors +Home page: http://jackson.codehaus.org/ +License: http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +This binary artifact contains Apache Parquet. + +Copyright: 2014-2020 The Apache Software Foundation. +Home page: https://parquet.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains Apache Thrift. + +Copyright: 2006-2010 The Apache Software Foundation. +Home page: https://thrift.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains fastutil. + +Copyright: 2002-2014 Sebastiano Vigna +Home page: http://fastutil.di.unimi.it/ +License: http://www.apache.org/licenses/LICENSE-2.0.html + +-------------------------------------------------------------------------------- + +This binary artifact contains Apache ORC. + +Copyright: 2013-2020 The Apache Software Foundation. +Home page: https://orc.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains Apache Hive's storage API via ORC. + +Copyright: 2013-2020 The Apache Software Foundation. +Home page: https://hive.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains Airlift Aircompressor. + +Copyright: 2011-2020 Aircompressor authors. +Home page: https://github.com/airlift/aircompressor +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains Airlift Slice. + +Copyright: 2013-2020 Slice authors. +Home page: https://github.com/airlift/slice +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains JetBrains annotations. + +Copyright: 2000-2020 JetBrains s.r.o. +Home page: https://github.com/JetBrains/java-annotations +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains Google Guava. + +Copyright: 2006-2020 The Guava Authors +Home page: https://github.com/google/guava +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains Google Error Prone Annotations. + +Copyright: Copyright 2011-2019 The Error Prone Authors +Home page: https://github.com/google/error-prone +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains checkerframework checker-qual Annotations. + +Copyright: 2004-2020 the Checker Framework developers +Home page: https://github.com/typetools/checker-framework +License: https://github.com/typetools/checker-framework/blob/master/LICENSE.txt (MIT license) + +License text: +| The annotations are licensed under the MIT License. (The text of this +| license appears below.) More specifically, all the parts of the Checker +| Framework that you might want to include with your own program use the +| MIT License. This is the checker-qual.jar file and all the files that +| appear in it: every file in a qual/ directory, plus utility files such +| as NullnessUtil.java, RegexUtil.java, SignednessUtil.java, etc. +| In addition, the cleanroom implementations of third-party annotations, +| which the Checker Framework recognizes as aliases for its own +| annotations, are licensed under the MIT License. +| +| Permission is hereby granted, free of charge, to any person obtaining a copy +| of this software and associated documentation files (the "Software"), to deal +| in the Software without restriction, including without limitation the rights +| to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +| copies of the Software, and to permit persons to whom the Software is +| furnished to do so, subject to the following conditions: +| +| The above copyright notice and this permission notice shall be included in +| all copies or substantial portions of the Software. +| +| THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +| IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +| FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +| AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +| LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +| OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +| THE SOFTWARE. + +-------------------------------------------------------------------------------- + +This binary artifact contains Animal Sniffer Annotations. + +Copyright: 2009-2018 codehaus.org +Home page: https://www.mojohaus.org/animal-sniffer/animal-sniffer-annotations/ +License: https://www.mojohaus.org/animal-sniffer/animal-sniffer-annotations/license.html (MIT license) + +License text: +| The MIT License +| +| Copyright (c) 2009 codehaus.org. +| +| Permission is hereby granted, free of charge, to any person obtaining a copy +| of this software and associated documentation files (the "Software"), to deal +| in the Software without restriction, including without limitation the rights +| to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +| copies of the Software, and to permit persons to whom the Software is +| furnished to do so, subject to the following conditions: +| +| The above copyright notice and this permission notice shall be included in +| all copies or substantial portions of the Software. +| +| THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +| IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +| FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +| AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +| LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +| OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +| THE SOFTWARE. + +-------------------------------------------------------------------------------- + +This binary artifact contains Caffeine by Ben Manes. + +Copyright: 2014-2020 Ben Manes and contributors +Home page: https://github.com/ben-manes/caffeine +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains Apache Yetus audience annotations. + +Copyright: 2008-2020 The Apache Software Foundation. +Home page: https://yetus.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains Google protobuf. + +Copyright: 2008 Google Inc. +Home page: https://developers.google.com/protocol-buffers +License: https://github.com/protocolbuffers/protobuf/blob/master/LICENSE (BSD) + +License text: + +| Copyright 2008 Google Inc. All rights reserved. +| +| Redistribution and use in source and binary forms, with or without +| modification, are permitted provided that the following conditions are +| met: +| +| * Redistributions of source code must retain the above copyright +| notice, this list of conditions and the following disclaimer. +| * Redistributions in binary form must reproduce the above +| copyright notice, this list of conditions and the following disclaimer +| in the documentation and/or other materials provided with the +| distribution. +| * Neither the name of Google Inc. nor the names of its +| contributors may be used to endorse or promote products derived from +| this software without specific prior written permission. +| +| THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +| "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +| LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +| A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +| OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +| SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +| LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +| DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +| THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +| (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +| OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +| +| Code generated by the Protocol Buffer compiler is owned by the owner +| of the input file used when generating it. This code is not +| standalone and requires a support library to be linked with it. This +| support library is itself covered by the above license. + +-------------------------------------------------------------------------------- + +This binary artifact contains ThreeTen. + +Copyright: 2007-present, Stephen Colebourne & Michael Nascimento Santos. +Home page: https://www.threeten.org/threeten-extra/ +License: https://github.com/ThreeTen/threeten-extra/blob/master/LICENSE.txt (BSD 3-clause) + +License text: + +| All rights reserved. +| +| * Redistribution and use in source and binary forms, with or without +| modification, are permitted provided that the following conditions are met: +| +| * Redistributions of source code must retain the above copyright notice, +| this list of conditions and the following disclaimer. +| +| * Redistributions in binary form must reproduce the above copyright notice, +| this list of conditions and the following disclaimer in the documentation +| and/or other materials provided with the distribution. +| +| * Neither the name of JSR-310 nor the names of its contributors +| may be used to endorse or promote products derived from this software +| without specific prior written permission. +| +| THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +| "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +| LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +| A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR +| CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, +| EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, +| PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR +| PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF +| LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING +| NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +| SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +This binary artifact includes Project Nessie with the following in its NOTICE +file: + +| Dremio +| Copyright 2015-2017 Dremio Corporation +| +| This product includes software developed at +| The Apache Software Foundation (http://www.apache.org/). + +-------------------------------------------------------------------------------- + +This binary includes code from Apache Commons. + +* Core ArrayUtil. + +Copyright: 2020 The Apache Software Foundation +Home page: https://commons.apache.org/ +License: https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains Apache HttpComponents Client. + +Copyright: 1999-2022 The Apache Software Foundation. +Home page: https://hc.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This product includes code from Apache HttpComponents Client. + +* retry and error handling logic in ExponentialHttpRequestRetryStrategy.java + +Copyright: 1999-2022 The Apache Software Foundation. +Home page: https://hc.apache.org/ +License: https://www.apache.org/licenses/LICENSE-2.0 diff --git a/flink/v1.17/flink-runtime/NOTICE b/flink/v1.17/flink-runtime/NOTICE new file mode 100644 index 000000000000..81aa1660456a --- /dev/null +++ b/flink/v1.17/flink-runtime/NOTICE @@ -0,0 +1,91 @@ + +Apache Iceberg +Copyright 2017-2022 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +-------------------------------------------------------------------------------- + +This binary artifact includes Apache ORC with the following in its NOTICE file: + +| Apache ORC +| Copyright 2013-2019 The Apache Software Foundation +| +| This product includes software developed by The Apache Software +| Foundation (http://www.apache.org/). +| +| This product includes software developed by Hewlett-Packard: +| (c) Copyright [2014-2015] Hewlett-Packard Development Company, L.P + +-------------------------------------------------------------------------------- + +This binary artifact includes Airlift Aircompressor with the following in its +NOTICE file: + +| Snappy Copyright Notices +| ========================= +| +| * Copyright 2011 Dain Sundstrom +| * Copyright 2011, Google Inc. +| +| +| Snappy License +| =============== +| Copyright 2011, Google Inc. +| All rights reserved. +| +| Redistribution and use in source and binary forms, with or without +| modification, are permitted provided that the following conditions are +| met: +| +| * Redistributions of source code must retain the above copyright +| notice, this list of conditions and the following disclaimer. +| * Redistributions in binary form must reproduce the above +| copyright notice, this list of conditions and the following disclaimer +| in the documentation and/or other materials provided with the +| distribution. +| * Neither the name of Google Inc. nor the names of its +| contributors may be used to endorse or promote products derived from +| this software without specific prior written permission. +| +| THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +| "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +| LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +| A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +| OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +| SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +| LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +| DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +| THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +| (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +| OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +This binary artifact includes Apache Yetus with the following in its NOTICE +file: + +| Apache Yetus +| Copyright 2008-2020 The Apache Software Foundation +| +| This product includes software developed at +| The Apache Software Foundation (https://www.apache.org/). +| +| --- +| Additional licenses for the Apache Yetus Source/Website: +| --- +| +| +| See LICENSE for terms. + +-------------------------------------------------------------------------------- + +This binary artifact includes Project Nessie with the following in its NOTICE +file: + +| Dremio +| Copyright 2015-2017 Dremio Corporation +| +| This product includes software developed at +| The Apache Software Foundation (http://www.apache.org/). diff --git a/flink/v1.17/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java b/flink/v1.17/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java new file mode 100644 index 000000000000..65cfba1ec876 --- /dev/null +++ b/flink/v1.17/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java @@ -0,0 +1,29 @@ +/* + * 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.iceberg.flink; + +import java.util.Map; + +public class IcebergConnectorSmokeTest extends TestIcebergConnector { + + public IcebergConnectorSmokeTest( + String catalogName, Map properties, boolean isStreaming) { + super(catalogName, properties, isStreaming); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java new file mode 100644 index 000000000000..18473bf4f190 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java @@ -0,0 +1,215 @@ +/* + * 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.iceberg.flink; + +import java.io.Serializable; +import java.util.Map; +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 org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.rest.RESTCatalog; + +/** Serializable loader to load an Iceberg {@link Catalog}. */ +public interface CatalogLoader extends Serializable, Cloneable { + + /** + * Create a new catalog with the provided properties. NOTICE: for flink, we may initialize the + * {@link CatalogLoader} at flink sql client side or job manager side, and then serialize this + * catalog loader to task manager, finally deserialize it and create a new catalog at task manager + * side. + * + * @return a newly created {@link Catalog} + */ + Catalog loadCatalog(); + + /** Clone a CatalogLoader. */ + @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) + CatalogLoader clone(); + + static CatalogLoader hadoop( + String name, Configuration hadoopConf, Map properties) { + return new HadoopCatalogLoader(name, hadoopConf, properties); + } + + static CatalogLoader hive(String name, Configuration hadoopConf, Map properties) { + return new HiveCatalogLoader(name, hadoopConf, properties); + } + + static CatalogLoader rest(String name, Configuration hadoopConf, Map properties) { + return new RESTCatalogLoader(name, hadoopConf, properties); + } + + static CatalogLoader custom( + String name, Map properties, Configuration hadoopConf, String impl) { + return new CustomCatalogLoader(name, properties, hadoopConf, impl); + } + + class HadoopCatalogLoader implements CatalogLoader { + private final String catalogName; + private final SerializableConfiguration hadoopConf; + private final String warehouseLocation; + private final Map properties; + + private HadoopCatalogLoader( + String catalogName, Configuration conf, Map properties) { + this.catalogName = catalogName; + this.hadoopConf = new SerializableConfiguration(conf); + this.warehouseLocation = properties.get(CatalogProperties.WAREHOUSE_LOCATION); + this.properties = Maps.newHashMap(properties); + } + + @Override + public Catalog loadCatalog() { + return CatalogUtil.loadCatalog( + HadoopCatalog.class.getName(), catalogName, properties, hadoopConf.get()); + } + + @Override + @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) + public CatalogLoader clone() { + return new HadoopCatalogLoader(catalogName, new Configuration(hadoopConf.get()), properties); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("catalogName", catalogName) + .add("warehouseLocation", warehouseLocation) + .toString(); + } + } + + class HiveCatalogLoader implements CatalogLoader { + private final String catalogName; + private final SerializableConfiguration hadoopConf; + private final String uri; + private final String warehouse; + private final int clientPoolSize; + private final Map properties; + + private HiveCatalogLoader( + String catalogName, Configuration conf, Map properties) { + this.catalogName = catalogName; + this.hadoopConf = new SerializableConfiguration(conf); + this.uri = properties.get(CatalogProperties.URI); + this.warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION); + this.clientPoolSize = + properties.containsKey(CatalogProperties.CLIENT_POOL_SIZE) + ? Integer.parseInt(properties.get(CatalogProperties.CLIENT_POOL_SIZE)) + : CatalogProperties.CLIENT_POOL_SIZE_DEFAULT; + this.properties = Maps.newHashMap(properties); + } + + @Override + public Catalog loadCatalog() { + return CatalogUtil.loadCatalog( + HiveCatalog.class.getName(), catalogName, properties, hadoopConf.get()); + } + + @Override + @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) + public CatalogLoader clone() { + return new HiveCatalogLoader(catalogName, new Configuration(hadoopConf.get()), properties); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("catalogName", catalogName) + .add("uri", uri) + .add("warehouse", warehouse) + .add("clientPoolSize", clientPoolSize) + .toString(); + } + } + + class RESTCatalogLoader implements CatalogLoader { + private final String catalogName; + private final SerializableConfiguration hadoopConf; + private final Map properties; + + private RESTCatalogLoader( + String catalogName, Configuration conf, Map properties) { + this.catalogName = catalogName; + this.hadoopConf = new SerializableConfiguration(conf); + this.properties = Maps.newHashMap(properties); + } + + @Override + public Catalog loadCatalog() { + return CatalogUtil.loadCatalog( + RESTCatalog.class.getName(), catalogName, properties, hadoopConf.get()); + } + + @Override + @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) + public CatalogLoader clone() { + return new RESTCatalogLoader(catalogName, new Configuration(hadoopConf.get()), properties); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("catalogName", catalogName) + .add("properties", properties) + .toString(); + } + } + + class CustomCatalogLoader implements CatalogLoader { + + private final SerializableConfiguration hadoopConf; + private final Map properties; + private final String name; + private final String impl; + + private CustomCatalogLoader( + String name, Map properties, Configuration conf, String impl) { + this.hadoopConf = new SerializableConfiguration(conf); + this.properties = Maps.newHashMap(properties); // wrap into a hashmap for serialization + this.name = name; + this.impl = + Preconditions.checkNotNull( + impl, "Cannot initialize custom Catalog, impl class name is null"); + } + + @Override + public Catalog loadCatalog() { + return CatalogUtil.loadCatalog(impl, name, properties, hadoopConf.get()); + } + + @Override + @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) + public CatalogLoader clone() { + return new CustomCatalogLoader(name, properties, new Configuration(hadoopConf.get()), impl); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this).add("name", name).add("impl", impl).toString(); + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java new file mode 100644 index 000000000000..f022c8abcb00 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java @@ -0,0 +1,833 @@ +/* + * 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.iceberg.flink; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.AbstractCatalog; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogDatabase; +import org.apache.flink.table.catalog.CatalogDatabaseImpl; +import org.apache.flink.table.catalog.CatalogFunction; +import org.apache.flink.table.catalog.CatalogPartition; +import org.apache.flink.table.catalog.CatalogPartitionSpec; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.CatalogTableImpl; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.TableChange; +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; +import org.apache.flink.table.catalog.exceptions.FunctionNotExistException; +import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException; +import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; +import org.apache.flink.table.catalog.stats.CatalogTableStatistics; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.factories.Factory; +import org.apache.flink.util.StringUtils; +import org.apache.iceberg.CachingCatalog; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SupportsNamespaces; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.NamespaceNotEmptyException; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; +import org.apache.iceberg.flink.util.FlinkAlterTableUtil; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.base.Splitter; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; + +/** + * A Flink Catalog implementation that wraps an Iceberg {@link Catalog}. + * + *

The mapping between Flink database and Iceberg namespace: Supplying a base namespace for a + * given catalog, so if you have a catalog that supports a 2-level namespace, you would supply the + * first level in the catalog configuration and the second level would be exposed as Flink + * databases. + * + *

The Iceberg table manages its partitions by itself. The partition of the Iceberg table is + * independent of the partition of Flink. + */ +public class FlinkCatalog extends AbstractCatalog { + private final CatalogLoader catalogLoader; + private final Catalog icebergCatalog; + private final Namespace baseNamespace; + private final SupportsNamespaces asNamespaceCatalog; + private final Closeable closeable; + private final boolean cacheEnabled; + + public FlinkCatalog( + String catalogName, + String defaultDatabase, + Namespace baseNamespace, + CatalogLoader catalogLoader, + boolean cacheEnabled, + long cacheExpirationIntervalMs) { + super(catalogName, defaultDatabase); + this.catalogLoader = catalogLoader; + this.baseNamespace = baseNamespace; + this.cacheEnabled = cacheEnabled; + + Catalog originalCatalog = catalogLoader.loadCatalog(); + icebergCatalog = + cacheEnabled + ? CachingCatalog.wrap(originalCatalog, cacheExpirationIntervalMs) + : originalCatalog; + asNamespaceCatalog = + originalCatalog instanceof SupportsNamespaces ? (SupportsNamespaces) originalCatalog : null; + closeable = originalCatalog instanceof Closeable ? (Closeable) originalCatalog : null; + + FlinkEnvironmentContext.init(); + } + + @Override + public void open() throws CatalogException {} + + @Override + public void close() throws CatalogException { + if (closeable != null) { + try { + closeable.close(); + } catch (IOException e) { + throw new CatalogException(e); + } + } + } + + public Catalog catalog() { + return icebergCatalog; + } + + /** Append a new level to the base namespace */ + private static Namespace appendLevel(Namespace baseNamespace, String newLevel) { + String[] namespace = new String[baseNamespace.levels().length + 1]; + System.arraycopy(baseNamespace.levels(), 0, namespace, 0, baseNamespace.levels().length); + namespace[baseNamespace.levels().length] = newLevel; + return Namespace.of(namespace); + } + + TableIdentifier toIdentifier(ObjectPath path) { + String objectName = path.getObjectName(); + List tableName = Splitter.on('$').splitToList(objectName); + + if (tableName.size() == 1) { + return TableIdentifier.of( + appendLevel(baseNamespace, path.getDatabaseName()), path.getObjectName()); + } else if (tableName.size() == 2 && MetadataTableType.from(tableName.get(1)) != null) { + return TableIdentifier.of( + appendLevel(appendLevel(baseNamespace, path.getDatabaseName()), tableName.get(0)), + tableName.get(1)); + } else { + throw new IllegalArgumentException("Illegal table name:" + objectName); + } + } + + @Override + public List listDatabases() throws CatalogException { + if (asNamespaceCatalog == null) { + return Collections.singletonList(getDefaultDatabase()); + } + + return asNamespaceCatalog.listNamespaces(baseNamespace).stream() + .map(n -> n.level(n.levels().length - 1)) + .collect(Collectors.toList()); + } + + @Override + public CatalogDatabase getDatabase(String databaseName) + throws DatabaseNotExistException, CatalogException { + if (asNamespaceCatalog == null) { + if (!getDefaultDatabase().equals(databaseName)) { + throw new DatabaseNotExistException(getName(), databaseName); + } else { + return new CatalogDatabaseImpl(Maps.newHashMap(), ""); + } + } else { + try { + Map metadata = + Maps.newHashMap( + asNamespaceCatalog.loadNamespaceMetadata(appendLevel(baseNamespace, databaseName))); + String comment = metadata.remove("comment"); + return new CatalogDatabaseImpl(metadata, comment); + } catch (NoSuchNamespaceException e) { + throw new DatabaseNotExistException(getName(), databaseName, e); + } + } + } + + @Override + public boolean databaseExists(String databaseName) throws CatalogException { + try { + getDatabase(databaseName); + return true; + } catch (DatabaseNotExistException ignore) { + return false; + } + } + + @Override + public void createDatabase(String name, CatalogDatabase database, boolean ignoreIfExists) + throws DatabaseAlreadyExistException, CatalogException { + createDatabase( + name, mergeComment(database.getProperties(), database.getComment()), ignoreIfExists); + } + + private void createDatabase( + String databaseName, Map metadata, boolean ignoreIfExists) + throws DatabaseAlreadyExistException, CatalogException { + if (asNamespaceCatalog != null) { + try { + asNamespaceCatalog.createNamespace(appendLevel(baseNamespace, databaseName), metadata); + } catch (AlreadyExistsException e) { + if (!ignoreIfExists) { + throw new DatabaseAlreadyExistException(getName(), databaseName, e); + } + } + } else { + throw new UnsupportedOperationException( + "Namespaces are not supported by catalog: " + getName()); + } + } + + private Map mergeComment(Map metadata, String comment) { + Map ret = Maps.newHashMap(metadata); + if (metadata.containsKey("comment")) { + throw new CatalogException("Database properties should not contain key: 'comment'."); + } + + if (!StringUtils.isNullOrWhitespaceOnly(comment)) { + ret.put("comment", comment); + } + return ret; + } + + @Override + public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade) + throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException { + if (asNamespaceCatalog != null) { + try { + boolean success = asNamespaceCatalog.dropNamespace(appendLevel(baseNamespace, name)); + if (!success && !ignoreIfNotExists) { + throw new DatabaseNotExistException(getName(), name); + } + } catch (NoSuchNamespaceException e) { + if (!ignoreIfNotExists) { + throw new DatabaseNotExistException(getName(), name, e); + } + } catch (NamespaceNotEmptyException e) { + throw new DatabaseNotEmptyException(getName(), name, e); + } + } else { + if (!ignoreIfNotExists) { + throw new DatabaseNotExistException(getName(), name); + } + } + } + + @Override + public void alterDatabase(String name, CatalogDatabase newDatabase, boolean ignoreIfNotExists) + throws DatabaseNotExistException, CatalogException { + if (asNamespaceCatalog != null) { + Namespace namespace = appendLevel(baseNamespace, name); + Map updates = Maps.newHashMap(); + Set removals = Sets.newHashSet(); + + try { + Map oldProperties = asNamespaceCatalog.loadNamespaceMetadata(namespace); + Map newProperties = + mergeComment(newDatabase.getProperties(), newDatabase.getComment()); + + for (String key : oldProperties.keySet()) { + if (!newProperties.containsKey(key)) { + removals.add(key); + } + } + + for (Map.Entry entry : newProperties.entrySet()) { + if (!entry.getValue().equals(oldProperties.get(entry.getKey()))) { + updates.put(entry.getKey(), entry.getValue()); + } + } + + if (!updates.isEmpty()) { + asNamespaceCatalog.setProperties(namespace, updates); + } + + if (!removals.isEmpty()) { + asNamespaceCatalog.removeProperties(namespace, removals); + } + + } catch (NoSuchNamespaceException e) { + if (!ignoreIfNotExists) { + throw new DatabaseNotExistException(getName(), name, e); + } + } + } else { + if (getDefaultDatabase().equals(name)) { + throw new CatalogException( + "Can not alter the default database when the iceberg catalog doesn't support namespaces."); + } + if (!ignoreIfNotExists) { + throw new DatabaseNotExistException(getName(), name); + } + } + } + + @Override + public List listTables(String databaseName) + throws DatabaseNotExistException, CatalogException { + try { + return icebergCatalog.listTables(appendLevel(baseNamespace, databaseName)).stream() + .map(TableIdentifier::name) + .collect(Collectors.toList()); + } catch (NoSuchNamespaceException e) { + throw new DatabaseNotExistException(getName(), databaseName, e); + } + } + + @Override + public CatalogTable getTable(ObjectPath tablePath) + throws TableNotExistException, CatalogException { + Table table = loadIcebergTable(tablePath); + return toCatalogTable(table); + } + + private Table loadIcebergTable(ObjectPath tablePath) throws TableNotExistException { + try { + Table table = icebergCatalog.loadTable(toIdentifier(tablePath)); + if (cacheEnabled) { + table.refresh(); + } + + return table; + } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { + throw new TableNotExistException(getName(), tablePath, e); + } + } + + @Override + public boolean tableExists(ObjectPath tablePath) throws CatalogException { + return icebergCatalog.tableExists(toIdentifier(tablePath)); + } + + @Override + public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + try { + icebergCatalog.dropTable(toIdentifier(tablePath)); + } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { + if (!ignoreIfNotExists) { + throw new TableNotExistException(getName(), tablePath, e); + } + } + } + + @Override + public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists) + throws TableNotExistException, TableAlreadyExistException, CatalogException { + try { + icebergCatalog.renameTable( + toIdentifier(tablePath), + toIdentifier(new ObjectPath(tablePath.getDatabaseName(), newTableName))); + } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { + if (!ignoreIfNotExists) { + throw new TableNotExistException(getName(), tablePath, e); + } + } catch (AlreadyExistsException e) { + throw new TableAlreadyExistException(getName(), tablePath, e); + } + } + + @Override + public void createTable(ObjectPath tablePath, CatalogBaseTable table, boolean ignoreIfExists) + throws CatalogException, TableAlreadyExistException { + if (Objects.equals( + table.getOptions().get("connector"), FlinkDynamicTableFactory.FACTORY_IDENTIFIER)) { + throw new IllegalArgumentException( + "Cannot create the table with 'connector'='iceberg' table property in " + + "an iceberg catalog, Please create table with 'connector'='iceberg' property in a non-iceberg catalog or " + + "create table without 'connector'='iceberg' related properties in an iceberg table."); + } + + createIcebergTable(tablePath, table, ignoreIfExists); + } + + void createIcebergTable(ObjectPath tablePath, CatalogBaseTable table, boolean ignoreIfExists) + throws CatalogException, TableAlreadyExistException { + validateFlinkTable(table); + + Schema icebergSchema = FlinkSchemaUtil.convert(table.getSchema()); + PartitionSpec spec = toPartitionSpec(((CatalogTable) table).getPartitionKeys(), icebergSchema); + + ImmutableMap.Builder properties = ImmutableMap.builder(); + String location = null; + for (Map.Entry entry : table.getOptions().entrySet()) { + if ("location".equalsIgnoreCase(entry.getKey())) { + location = entry.getValue(); + } else { + properties.put(entry.getKey(), entry.getValue()); + } + } + + try { + icebergCatalog.createTable( + toIdentifier(tablePath), icebergSchema, spec, location, properties.build()); + } catch (AlreadyExistsException e) { + if (!ignoreIfExists) { + throw new TableAlreadyExistException(getName(), tablePath, e); + } + } + } + + private static void validateTableSchemaAndPartition(CatalogTable ct1, CatalogTable ct2) { + TableSchema ts1 = ct1.getSchema(); + TableSchema ts2 = ct2.getSchema(); + boolean equalsPrimary = false; + + if (ts1.getPrimaryKey().isPresent() && ts2.getPrimaryKey().isPresent()) { + equalsPrimary = + Objects.equals(ts1.getPrimaryKey().get().getType(), ts2.getPrimaryKey().get().getType()) + && Objects.equals( + ts1.getPrimaryKey().get().getColumns(), ts2.getPrimaryKey().get().getColumns()); + } else if (!ts1.getPrimaryKey().isPresent() && !ts2.getPrimaryKey().isPresent()) { + equalsPrimary = true; + } + + if (!(Objects.equals(ts1.getTableColumns(), ts2.getTableColumns()) + && Objects.equals(ts1.getWatermarkSpecs(), ts2.getWatermarkSpecs()) + && equalsPrimary)) { + throw new UnsupportedOperationException( + "Altering schema is not supported in the old alterTable API. " + + "To alter schema, use the other alterTable API and provide a list of TableChange's."); + } + + validateTablePartition(ct1, ct2); + } + + private static void validateTablePartition(CatalogTable ct1, CatalogTable ct2) { + if (!ct1.getPartitionKeys().equals(ct2.getPartitionKeys())) { + throw new UnsupportedOperationException("Altering partition keys is not supported yet."); + } + } + + /** + * This alterTable API only supports altering table properties. + * + *

Support for adding/removing/renaming columns cannot be done by comparing CatalogTable + * instances, unless the Flink schema contains Iceberg column IDs. + * + *

To alter columns, use the other alterTable API and provide a list of TableChange's. + * + * @param tablePath path of the table or view to be modified + * @param newTable the new table definition + * @param ignoreIfNotExists flag to specify behavior when the table or view does not exist: if set + * to false, throw an exception, if set to true, do nothing. + * @throws CatalogException in case of any runtime exception + * @throws TableNotExistException if the table does not exist + */ + @Override + public void alterTable(ObjectPath tablePath, CatalogBaseTable newTable, boolean ignoreIfNotExists) + throws CatalogException, TableNotExistException { + validateFlinkTable(newTable); + + Table icebergTable; + try { + icebergTable = loadIcebergTable(tablePath); + } catch (TableNotExistException e) { + if (!ignoreIfNotExists) { + throw e; + } else { + return; + } + } + + CatalogTable table = toCatalogTable(icebergTable); + validateTableSchemaAndPartition(table, (CatalogTable) newTable); + + Map oldProperties = table.getOptions(); + Map setProperties = Maps.newHashMap(); + + String setLocation = null; + String setSnapshotId = null; + String pickSnapshotId = null; + + for (Map.Entry entry : newTable.getOptions().entrySet()) { + String key = entry.getKey(); + String value = entry.getValue(); + + if (Objects.equals(value, oldProperties.get(key))) { + continue; + } + + if ("location".equalsIgnoreCase(key)) { + setLocation = value; + } else if ("current-snapshot-id".equalsIgnoreCase(key)) { + setSnapshotId = value; + } else if ("cherry-pick-snapshot-id".equalsIgnoreCase(key)) { + pickSnapshotId = value; + } else { + setProperties.put(key, value); + } + } + + oldProperties + .keySet() + .forEach( + k -> { + if (!newTable.getOptions().containsKey(k)) { + setProperties.put(k, null); + } + }); + + FlinkAlterTableUtil.commitChanges( + icebergTable, setLocation, setSnapshotId, pickSnapshotId, setProperties); + } + + @Override + public void alterTable( + ObjectPath tablePath, + CatalogBaseTable newTable, + List tableChanges, + boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + validateFlinkTable(newTable); + + Table icebergTable; + try { + icebergTable = loadIcebergTable(tablePath); + } catch (TableNotExistException e) { + if (!ignoreIfNotExists) { + throw e; + } else { + return; + } + } + + // Does not support altering partition yet. + validateTablePartition(toCatalogTable(icebergTable), (CatalogTable) newTable); + + String setLocation = null; + String setSnapshotId = null; + String cherrypickSnapshotId = null; + + List propertyChanges = Lists.newArrayList(); + List schemaChanges = Lists.newArrayList(); + for (TableChange change : tableChanges) { + if (change instanceof TableChange.SetOption) { + TableChange.SetOption set = (TableChange.SetOption) change; + + if ("location".equalsIgnoreCase(set.getKey())) { + setLocation = set.getValue(); + } else if ("current-snapshot-id".equalsIgnoreCase(set.getKey())) { + setSnapshotId = set.getValue(); + } else if ("cherry-pick-snapshot-id".equalsIgnoreCase(set.getKey())) { + cherrypickSnapshotId = set.getValue(); + } else { + propertyChanges.add(change); + } + } else if (change instanceof TableChange.ResetOption) { + propertyChanges.add(change); + } else { + schemaChanges.add(change); + } + } + + FlinkAlterTableUtil.commitChanges( + icebergTable, + setLocation, + setSnapshotId, + cherrypickSnapshotId, + schemaChanges, + propertyChanges); + } + + private static void validateFlinkTable(CatalogBaseTable table) { + Preconditions.checkArgument( + table instanceof CatalogTable, "The Table should be a CatalogTable."); + + TableSchema schema = table.getSchema(); + schema + .getTableColumns() + .forEach( + column -> { + if (!FlinkCompatibilityUtil.isPhysicalColumn(column)) { + throw new UnsupportedOperationException( + "Creating table with computed columns is not supported yet."); + } + }); + + if (!schema.getWatermarkSpecs().isEmpty()) { + throw new UnsupportedOperationException( + "Creating table with watermark specs is not supported yet."); + } + } + + private static PartitionSpec toPartitionSpec(List partitionKeys, Schema icebergSchema) { + PartitionSpec.Builder builder = PartitionSpec.builderFor(icebergSchema); + partitionKeys.forEach(builder::identity); + return builder.build(); + } + + private static List toPartitionKeys(PartitionSpec spec, Schema icebergSchema) { + ImmutableList.Builder partitionKeysBuilder = ImmutableList.builder(); + for (PartitionField field : spec.fields()) { + if (field.transform().isIdentity()) { + partitionKeysBuilder.add(icebergSchema.findColumnName(field.sourceId())); + } else { + // Not created by Flink SQL. + // For compatibility with iceberg tables, return empty. + // TODO modify this after Flink support partition transform. + return Collections.emptyList(); + } + } + return partitionKeysBuilder.build(); + } + + static CatalogTable toCatalogTable(Table table) { + TableSchema schema = FlinkSchemaUtil.toSchema(table.schema()); + List partitionKeys = toPartitionKeys(table.spec(), table.schema()); + + // NOTE: We can not create a IcebergCatalogTable extends CatalogTable, because Flink optimizer + // may use + // CatalogTableImpl to copy a new catalog table. + // Let's re-loading table from Iceberg catalog when creating source/sink operators. + // Iceberg does not have Table comment, so pass a null (Default comment value in Flink). + return new CatalogTableImpl(schema, partitionKeys, table.properties(), null); + } + + @Override + public Optional getFactory() { + return Optional.of(new FlinkDynamicTableFactory(this)); + } + + CatalogLoader getCatalogLoader() { + return catalogLoader; + } + + // ------------------------------ Unsupported methods + // --------------------------------------------- + + @Override + public List listViews(String databaseName) throws CatalogException { + return Collections.emptyList(); + } + + @Override + public CatalogPartition getPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public boolean partitionExists(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void createPartition( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogPartition partition, + boolean ignoreIfExists) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void dropPartition( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec, boolean ignoreIfNotExists) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterPartition( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogPartition newPartition, + boolean ignoreIfNotExists) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public List listFunctions(String dbName) throws CatalogException { + return Collections.emptyList(); + } + + @Override + public CatalogFunction getFunction(ObjectPath functionPath) + throws FunctionNotExistException, CatalogException { + throw new FunctionNotExistException(getName(), functionPath); + } + + @Override + public boolean functionExists(ObjectPath functionPath) throws CatalogException { + return false; + } + + @Override + public void createFunction( + ObjectPath functionPath, CatalogFunction function, boolean ignoreIfExists) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterFunction( + ObjectPath functionPath, CatalogFunction newFunction, boolean ignoreIfNotExists) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void dropFunction(ObjectPath functionPath, boolean ignoreIfNotExists) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterTableStatistics( + ObjectPath tablePath, CatalogTableStatistics tableStatistics, boolean ignoreIfNotExists) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterTableColumnStatistics( + ObjectPath tablePath, CatalogColumnStatistics columnStatistics, boolean ignoreIfNotExists) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterPartitionStatistics( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogTableStatistics partitionStatistics, + boolean ignoreIfNotExists) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterPartitionColumnStatistics( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogColumnStatistics columnStatistics, + boolean ignoreIfNotExists) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public List listPartitions(ObjectPath tablePath) + throws TableNotExistException, TableNotPartitionedException, CatalogException { + Table table = loadIcebergTable(tablePath); + + if (table.spec().isUnpartitioned()) { + throw new TableNotPartitionedException(icebergCatalog.name(), tablePath); + } + + Set set = Sets.newHashSet(); + try (CloseableIterable tasks = table.newScan().planFiles()) { + for (DataFile dataFile : CloseableIterable.transform(tasks, FileScanTask::file)) { + Map map = Maps.newHashMap(); + StructLike structLike = dataFile.partition(); + PartitionSpec spec = table.specs().get(dataFile.specId()); + for (int i = 0; i < structLike.size(); i++) { + map.put(spec.fields().get(i).name(), String.valueOf(structLike.get(i, Object.class))); + } + set.add(new CatalogPartitionSpec(map)); + } + } catch (IOException e) { + throw new CatalogException( + String.format("Failed to list partitions of table %s", tablePath), e); + } + + return Lists.newArrayList(set); + } + + @Override + public List listPartitions( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public List listPartitionsByFilter( + ObjectPath tablePath, List filters) throws CatalogException { + throw new UnsupportedOperationException(); + } + + // After partition pruning and filter push down, the statistics have become very inaccurate, so + // the statistics from + // here are of little significance. + // Flink will support something like SupportsReportStatistics in future. + + @Override + public CatalogTableStatistics getTableStatistics(ObjectPath tablePath) throws CatalogException { + return CatalogTableStatistics.UNKNOWN; + } + + @Override + public CatalogColumnStatistics getTableColumnStatistics(ObjectPath tablePath) + throws CatalogException { + return CatalogColumnStatistics.UNKNOWN; + } + + @Override + public CatalogTableStatistics getPartitionStatistics( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws CatalogException { + return CatalogTableStatistics.UNKNOWN; + } + + @Override + public CatalogColumnStatistics getPartitionColumnStatistics( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws CatalogException { + return CatalogColumnStatistics.UNKNOWN; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java new file mode 100644 index 000000000000..1453753849ec --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java @@ -0,0 +1,213 @@ +/* + * 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.iceberg.flink; + +import java.net.URL; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.runtime.util.HadoopUtils; +import org.apache.flink.table.catalog.Catalog; +import org.apache.flink.table.factories.CatalogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.base.Strings; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.PropertyUtil; + +/** + * A Flink Catalog factory implementation that creates {@link FlinkCatalog}. + * + *

This supports the following catalog configuration options: + * + *

    + *
  • type - Flink catalog factory key, should be "iceberg" + *
  • catalog-type - iceberg catalog type, "hive", "hadoop" or "rest" + *
  • uri - the Hive Metastore URI (Hive catalog only) + *
  • clients - the Hive Client Pool Size (Hive catalog only) + *
  • warehouse - the warehouse path (Hadoop catalog only) + *
  • default-database - a database name to use as the default + *
  • base-namespace - a base namespace as the prefix for all databases (Hadoop + * catalog only) + *
  • cache-enabled - whether to enable catalog cache + *
+ * + *

To use a custom catalog that is not a Hive or Hadoop catalog, extend this class and override + * {@link #createCatalogLoader(String, Map, Configuration)}. + */ +public class FlinkCatalogFactory implements CatalogFactory { + + // Can not just use "type", it conflicts with CATALOG_TYPE. + public static final String ICEBERG_CATALOG_TYPE = "catalog-type"; + public static final String ICEBERG_CATALOG_TYPE_HADOOP = "hadoop"; + public static final String ICEBERG_CATALOG_TYPE_HIVE = "hive"; + public static final String ICEBERG_CATALOG_TYPE_REST = "rest"; + + public static final String HIVE_CONF_DIR = "hive-conf-dir"; + public static final String HADOOP_CONF_DIR = "hadoop-conf-dir"; + public static final String DEFAULT_DATABASE = "default-database"; + public static final String DEFAULT_DATABASE_NAME = "default"; + public static final String BASE_NAMESPACE = "base-namespace"; + + public static final String TYPE = "type"; + public static final String PROPERTY_VERSION = "property-version"; + + /** + * Create an Iceberg {@link org.apache.iceberg.catalog.Catalog} loader to be used by this Flink + * catalog adapter. + * + * @param name Flink's catalog name + * @param properties Flink's catalog properties + * @param hadoopConf Hadoop configuration for catalog + * @return an Iceberg catalog loader + */ + static CatalogLoader createCatalogLoader( + String name, Map properties, Configuration hadoopConf) { + String catalogImpl = properties.get(CatalogProperties.CATALOG_IMPL); + if (catalogImpl != null) { + String catalogType = properties.get(ICEBERG_CATALOG_TYPE); + Preconditions.checkArgument( + catalogType == null, + "Cannot create catalog %s, both catalog-type and catalog-impl are set: catalog-type=%s, catalog-impl=%s", + name, + catalogType, + catalogImpl); + return CatalogLoader.custom(name, properties, hadoopConf, catalogImpl); + } + + String catalogType = properties.getOrDefault(ICEBERG_CATALOG_TYPE, ICEBERG_CATALOG_TYPE_HIVE); + switch (catalogType.toLowerCase(Locale.ENGLISH)) { + case ICEBERG_CATALOG_TYPE_HIVE: + // The values of properties 'uri', 'warehouse', 'hive-conf-dir' are allowed to be null, in + // that case it will + // fallback to parse those values from hadoop configuration which is loaded from classpath. + String hiveConfDir = properties.get(HIVE_CONF_DIR); + String hadoopConfDir = properties.get(HADOOP_CONF_DIR); + Configuration newHadoopConf = mergeHiveConf(hadoopConf, hiveConfDir, hadoopConfDir); + return CatalogLoader.hive(name, newHadoopConf, properties); + + case ICEBERG_CATALOG_TYPE_HADOOP: + return CatalogLoader.hadoop(name, hadoopConf, properties); + + case ICEBERG_CATALOG_TYPE_REST: + return CatalogLoader.rest(name, hadoopConf, properties); + + default: + throw new UnsupportedOperationException( + "Unknown catalog-type: " + catalogType + " (Must be 'hive', 'hadoop' or 'rest')"); + } + } + + @Override + public Map requiredContext() { + Map context = Maps.newHashMap(); + context.put(TYPE, "iceberg"); + context.put(PROPERTY_VERSION, "1"); + return context; + } + + @Override + public List supportedProperties() { + return ImmutableList.of("*"); + } + + @Override + public Catalog createCatalog(String name, Map properties) { + return createCatalog(name, properties, clusterHadoopConf()); + } + + protected Catalog createCatalog( + String name, Map properties, Configuration hadoopConf) { + CatalogLoader catalogLoader = createCatalogLoader(name, properties, hadoopConf); + String defaultDatabase = properties.getOrDefault(DEFAULT_DATABASE, DEFAULT_DATABASE_NAME); + + Namespace baseNamespace = Namespace.empty(); + if (properties.containsKey(BASE_NAMESPACE)) { + baseNamespace = Namespace.of(properties.get(BASE_NAMESPACE).split("\\.")); + } + + boolean cacheEnabled = + PropertyUtil.propertyAsBoolean( + properties, CatalogProperties.CACHE_ENABLED, CatalogProperties.CACHE_ENABLED_DEFAULT); + + long cacheExpirationIntervalMs = + PropertyUtil.propertyAsLong( + properties, + CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS, + CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS_OFF); + Preconditions.checkArgument( + cacheExpirationIntervalMs != 0, + "%s is not allowed to be 0.", + CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS); + + return new FlinkCatalog( + name, + defaultDatabase, + baseNamespace, + catalogLoader, + cacheEnabled, + cacheExpirationIntervalMs); + } + + private static Configuration mergeHiveConf( + Configuration hadoopConf, String hiveConfDir, String hadoopConfDir) { + Configuration newConf = new Configuration(hadoopConf); + if (!Strings.isNullOrEmpty(hiveConfDir)) { + Preconditions.checkState( + Files.exists(Paths.get(hiveConfDir, "hive-site.xml")), + "There should be a hive-site.xml file under the directory %s", + hiveConfDir); + newConf.addResource(new Path(hiveConfDir, "hive-site.xml")); + } else { + // If don't provide the hive-site.xml path explicitly, it will try to load resource from + // classpath. If still + // couldn't load the configuration file, then it will throw exception in HiveCatalog. + URL configFile = CatalogLoader.class.getClassLoader().getResource("hive-site.xml"); + if (configFile != null) { + newConf.addResource(configFile); + } + } + + if (!Strings.isNullOrEmpty(hadoopConfDir)) { + Preconditions.checkState( + Files.exists(Paths.get(hadoopConfDir, "hdfs-site.xml")), + "Failed to load Hadoop configuration: missing %s", + Paths.get(hadoopConfDir, "hdfs-site.xml")); + newConf.addResource(new Path(hadoopConfDir, "hdfs-site.xml")); + Preconditions.checkState( + Files.exists(Paths.get(hadoopConfDir, "core-site.xml")), + "Failed to load Hadoop configuration: missing %s", + Paths.get(hadoopConfDir, "core-site.xml")); + newConf.addResource(new Path(hadoopConfDir, "core-site.xml")); + } + + return newConf; + } + + public static Configuration clusterHadoopConf() { + return HadoopUtils.getHadoopConfiguration(GlobalConfiguration.loadConfiguration()); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java new file mode 100644 index 000000000000..7167859e600c --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java @@ -0,0 +1,261 @@ +/* + * 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.iceberg.flink; + +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.util.TimeUtils; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +class FlinkConfParser { + + private final Map tableProperties; + private final Map options; + private final ReadableConfig readableConfig; + + FlinkConfParser(Table table, Map options, ReadableConfig readableConfig) { + this.tableProperties = table.properties(); + this.options = options; + this.readableConfig = readableConfig; + } + + public BooleanConfParser booleanConf() { + return new BooleanConfParser(); + } + + public IntConfParser intConf() { + return new IntConfParser(); + } + + public LongConfParser longConf() { + return new LongConfParser(); + } + + public > EnumConfParser enumConfParser(Class enumClass) { + return new EnumConfParser<>(enumClass); + } + + public StringConfParser stringConf() { + return new StringConfParser(); + } + + public DurationConfParser durationConf() { + return new DurationConfParser(); + } + + class BooleanConfParser extends ConfParser { + private Boolean defaultValue; + + @Override + protected BooleanConfParser self() { + return this; + } + + public BooleanConfParser defaultValue(boolean value) { + this.defaultValue = value; + return self(); + } + + public BooleanConfParser defaultValue(String value) { + this.defaultValue = Boolean.parseBoolean(value); + return self(); + } + + public boolean parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + return parse(Boolean::parseBoolean, defaultValue); + } + } + + class IntConfParser extends ConfParser { + private Integer defaultValue; + + @Override + protected IntConfParser self() { + return this; + } + + public IntConfParser defaultValue(int value) { + this.defaultValue = value; + return self(); + } + + public int parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + return parse(Integer::parseInt, defaultValue); + } + + public Integer parseOptional() { + return parse(Integer::parseInt, null); + } + } + + class LongConfParser extends ConfParser { + private Long defaultValue; + + @Override + protected LongConfParser self() { + return this; + } + + public LongConfParser defaultValue(long value) { + this.defaultValue = value; + return self(); + } + + public long parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + return parse(Long::parseLong, defaultValue); + } + + public Long parseOptional() { + return parse(Long::parseLong, null); + } + } + + class StringConfParser extends ConfParser { + private String defaultValue; + + @Override + protected StringConfParser self() { + return this; + } + + public StringConfParser defaultValue(String value) { + this.defaultValue = value; + return self(); + } + + public String parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + return parse(Function.identity(), defaultValue); + } + + public String parseOptional() { + return parse(Function.identity(), null); + } + } + + class EnumConfParser> extends ConfParser, E> { + private E defaultValue; + private final Class enumClass; + + EnumConfParser(Class enumClass) { + this.enumClass = enumClass; + } + + @Override + protected EnumConfParser self() { + return this; + } + + public EnumConfParser defaultValue(E value) { + this.defaultValue = value; + return self(); + } + + public E parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + return parse(s -> Enum.valueOf(enumClass, s), defaultValue); + } + + public E parseOptional() { + return parse(s -> Enum.valueOf(enumClass, s), null); + } + } + + class DurationConfParser extends ConfParser { + private Duration defaultValue; + + @Override + protected DurationConfParser self() { + return this; + } + + public DurationConfParser defaultValue(Duration value) { + this.defaultValue = value; + return self(); + } + + public Duration parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + return parse(TimeUtils::parseDuration, defaultValue); + } + + public Duration parseOptional() { + return parse(TimeUtils::parseDuration, null); + } + } + + abstract class ConfParser { + private final List optionNames = Lists.newArrayList(); + private String tablePropertyName; + private ConfigOption configOption; + + protected abstract ThisT self(); + + public ThisT option(String name) { + this.optionNames.add(name); + return self(); + } + + public ThisT flinkConfig(ConfigOption newConfigOption) { + this.configOption = newConfigOption; + return self(); + } + + public ThisT tableProperty(String name) { + this.tablePropertyName = name; + return self(); + } + + protected T parse(Function conversion, T defaultValue) { + if (!optionNames.isEmpty()) { + for (String optionName : optionNames) { + String optionValue = options.get(optionName); + if (optionValue != null) { + return conversion.apply(optionValue); + } + } + } + + if (configOption != null) { + T propertyValue = readableConfig.get(configOption); + if (propertyValue != null) { + return propertyValue; + } + } + + if (tablePropertyName != null) { + String propertyValue = tableProperties.get(tablePropertyName); + if (propertyValue != null) { + return conversion.apply(propertyValue); + } + } + + return defaultValue; + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java new file mode 100644 index 000000000000..7c7afd24ed8e --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java @@ -0,0 +1,107 @@ +/* + * 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.iceberg.flink; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.description.Description; +import org.apache.flink.configuration.description.TextElement; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.iceberg.flink.source.assigner.SplitAssignerType; +import org.apache.iceberg.util.ThreadPools; + +/** + * When constructing Flink Iceberg source via Java API, configs can be set in {@link Configuration} + * passed to source builder. E.g. + * + *

+ *   configuration.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, true);
+ *   FlinkSource.forRowData()
+ *       .flinkConf(configuration)
+ *       ...
+ * 
+ * + *

When using Flink SQL/table API, connector options can be set in Flink's {@link + * TableEnvironment}. + * + *

+ *   TableEnvironment tEnv = createTableEnv();
+ *   tEnv.getConfig()
+ *        .getConfiguration()
+ *        .setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, true);
+ * 
+ */ +public class FlinkConfigOptions { + + private FlinkConfigOptions() {} + + public static final ConfigOption TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM = + ConfigOptions.key("table.exec.iceberg.infer-source-parallelism") + .booleanType() + .defaultValue(true) + .withDescription( + "If is false, parallelism of source are set by config.\n" + + "If is true, source parallelism is inferred according to splits number.\n"); + + public static final ConfigOption TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX = + ConfigOptions.key("table.exec.iceberg.infer-source-parallelism.max") + .intType() + .defaultValue(100) + .withDescription("Sets max infer parallelism for source operator."); + + public static final ConfigOption TABLE_EXEC_ICEBERG_EXPOSE_SPLIT_LOCALITY_INFO = + ConfigOptions.key("table.exec.iceberg.expose-split-locality-info") + .booleanType() + .noDefaultValue() + .withDescription( + "Expose split host information to use Flink's locality aware split assigner."); + + public static final ConfigOption SOURCE_READER_FETCH_BATCH_RECORD_COUNT = + ConfigOptions.key("table.exec.iceberg.fetch-batch-record-count") + .intType() + .defaultValue(2048) + .withDescription("The target number of records for Iceberg reader fetch batch."); + + public static final ConfigOption TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE = + ConfigOptions.key("table.exec.iceberg.worker-pool-size") + .intType() + .defaultValue(ThreadPools.WORKER_THREAD_POOL_SIZE) + .withDescription("The size of workers pool used to plan or scan manifests."); + + public static final ConfigOption TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE = + ConfigOptions.key("table.exec.iceberg.use-flip27-source") + .booleanType() + .defaultValue(false) + .withDescription("Use the FLIP-27 based Iceberg source implementation."); + + public static final ConfigOption TABLE_EXEC_SPLIT_ASSIGNER_TYPE = + ConfigOptions.key("table.exec.iceberg.split-assigner-type") + .enumType(SplitAssignerType.class) + .defaultValue(SplitAssignerType.SIMPLE) + .withDescription( + Description.builder() + .text("Split assigner type that determine how splits are assigned to readers.") + .linebreak() + .list( + TextElement.text( + SplitAssignerType.SIMPLE + + ": simple assigner that doesn't provide any guarantee on order or locality.")) + .build()); +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java new file mode 100644 index 000000000000..8e1f420b722d --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java @@ -0,0 +1,209 @@ +/* + * 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.iceberg.flink; + +import java.util.Map; +import java.util.Set; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogDatabaseImpl; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.factories.DynamicTableSinkFactory; +import org.apache.flink.table.factories.DynamicTableSourceFactory; +import org.apache.flink.table.utils.TableSchemaUtils; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.flink.source.IcebergTableSource; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; + +public class FlinkDynamicTableFactory + implements DynamicTableSinkFactory, DynamicTableSourceFactory { + static final String FACTORY_IDENTIFIER = "iceberg"; + + private static final ConfigOption CATALOG_NAME = + ConfigOptions.key("catalog-name") + .stringType() + .noDefaultValue() + .withDescription("Catalog name"); + + private static final ConfigOption CATALOG_TYPE = + ConfigOptions.key(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE) + .stringType() + .noDefaultValue() + .withDescription("Catalog type, the optional types are: custom, hadoop, hive."); + + private static final ConfigOption CATALOG_DATABASE = + ConfigOptions.key("catalog-database") + .stringType() + .defaultValue(FlinkCatalogFactory.DEFAULT_DATABASE_NAME) + .withDescription("Database name managed in the iceberg catalog."); + + private static final ConfigOption CATALOG_TABLE = + ConfigOptions.key("catalog-table") + .stringType() + .noDefaultValue() + .withDescription("Table name managed in the underlying iceberg catalog and database."); + + private final FlinkCatalog catalog; + + public FlinkDynamicTableFactory() { + this.catalog = null; + } + + public FlinkDynamicTableFactory(FlinkCatalog catalog) { + this.catalog = catalog; + } + + @Override + public DynamicTableSource createDynamicTableSource(Context context) { + ObjectIdentifier objectIdentifier = context.getObjectIdentifier(); + CatalogTable catalogTable = context.getCatalogTable(); + Map tableProps = catalogTable.getOptions(); + TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(catalogTable.getSchema()); + + TableLoader tableLoader; + if (catalog != null) { + tableLoader = createTableLoader(catalog, objectIdentifier.toObjectPath()); + } else { + tableLoader = + createTableLoader( + catalogTable, + tableProps, + objectIdentifier.getDatabaseName(), + objectIdentifier.getObjectName()); + } + + return new IcebergTableSource(tableLoader, tableSchema, tableProps, context.getConfiguration()); + } + + @Override + public DynamicTableSink createDynamicTableSink(Context context) { + ObjectIdentifier objectIdentifier = context.getObjectIdentifier(); + CatalogTable catalogTable = context.getCatalogTable(); + Map writeProps = catalogTable.getOptions(); + TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(catalogTable.getSchema()); + + TableLoader tableLoader; + if (catalog != null) { + tableLoader = createTableLoader(catalog, objectIdentifier.toObjectPath()); + } else { + tableLoader = + createTableLoader( + catalogTable, + writeProps, + objectIdentifier.getDatabaseName(), + objectIdentifier.getObjectName()); + } + + return new IcebergTableSink(tableLoader, tableSchema, context.getConfiguration(), writeProps); + } + + @Override + public Set> requiredOptions() { + Set> options = Sets.newHashSet(); + options.add(CATALOG_TYPE); + options.add(CATALOG_NAME); + return options; + } + + @Override + public Set> optionalOptions() { + Set> options = Sets.newHashSet(); + options.add(CATALOG_DATABASE); + options.add(CATALOG_TABLE); + return options; + } + + @Override + public String factoryIdentifier() { + return FACTORY_IDENTIFIER; + } + + private static TableLoader createTableLoader( + CatalogBaseTable catalogBaseTable, + Map tableProps, + String databaseName, + String tableName) { + Configuration flinkConf = new Configuration(); + tableProps.forEach(flinkConf::setString); + + String catalogName = flinkConf.getString(CATALOG_NAME); + Preconditions.checkNotNull( + catalogName, "Table property '%s' cannot be null", CATALOG_NAME.key()); + + String catalogDatabase = flinkConf.getString(CATALOG_DATABASE, databaseName); + Preconditions.checkNotNull(catalogDatabase, "The iceberg database name cannot be null"); + + String catalogTable = flinkConf.getString(CATALOG_TABLE, tableName); + Preconditions.checkNotNull(catalogTable, "The iceberg table name cannot be null"); + + org.apache.hadoop.conf.Configuration hadoopConf = FlinkCatalogFactory.clusterHadoopConf(); + FlinkCatalogFactory factory = new FlinkCatalogFactory(); + FlinkCatalog flinkCatalog = + (FlinkCatalog) factory.createCatalog(catalogName, tableProps, hadoopConf); + ObjectPath objectPath = new ObjectPath(catalogDatabase, catalogTable); + + // Create database if not exists in the external catalog. + if (!flinkCatalog.databaseExists(catalogDatabase)) { + try { + flinkCatalog.createDatabase( + catalogDatabase, new CatalogDatabaseImpl(Maps.newHashMap(), null), true); + } catch (DatabaseAlreadyExistException e) { + throw new AlreadyExistsException( + e, + "Database %s already exists in the iceberg catalog %s.", + catalogName, + catalogDatabase); + } + } + + // Create table if not exists in the external catalog. + if (!flinkCatalog.tableExists(objectPath)) { + try { + flinkCatalog.createIcebergTable(objectPath, catalogBaseTable, true); + } catch (TableAlreadyExistException e) { + throw new AlreadyExistsException( + e, + "Table %s already exists in the database %s and catalog %s", + catalogTable, + catalogDatabase, + catalogName); + } + } + + return TableLoader.fromCatalog( + flinkCatalog.getCatalogLoader(), TableIdentifier.of(catalogDatabase, catalogTable)); + } + + private static TableLoader createTableLoader(FlinkCatalog catalog, ObjectPath objectPath) { + Preconditions.checkNotNull(catalog, "Flink catalog cannot be null"); + return TableLoader.fromCatalog(catalog.getCatalogLoader(), catalog.toIdentifier(objectPath)); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java new file mode 100644 index 000000000000..f35bb577fbba --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java @@ -0,0 +1,31 @@ +/* + * 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.iceberg.flink; + +import org.apache.iceberg.EnvironmentContext; +import org.apache.iceberg.flink.util.FlinkPackage; + +class FlinkEnvironmentContext { + private FlinkEnvironmentContext() {} + + public static void init() { + EnvironmentContext.put(EnvironmentContext.ENGINE_NAME, "flink"); + EnvironmentContext.put(EnvironmentContext.ENGINE_VERSION, FlinkPackage.version()); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java new file mode 100644 index 000000000000..f2244d5137a1 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java @@ -0,0 +1,266 @@ +/* + * 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.iceberg.flink; + +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.BiFunction; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import org.apache.flink.table.expressions.CallExpression; +import org.apache.flink.table.expressions.FieldReferenceExpression; +import org.apache.flink.table.expressions.ResolvedExpression; +import org.apache.flink.table.expressions.ValueLiteralExpression; +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expression.Operation; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.util.DateTimeUtil; +import org.apache.iceberg.util.NaNUtil; + +public class FlinkFilters { + private FlinkFilters() {} + + private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%"); + + private static final Map FILTERS = + ImmutableMap.builder() + .put(BuiltInFunctionDefinitions.EQUALS, Operation.EQ) + .put(BuiltInFunctionDefinitions.NOT_EQUALS, Operation.NOT_EQ) + .put(BuiltInFunctionDefinitions.GREATER_THAN, Operation.GT) + .put(BuiltInFunctionDefinitions.GREATER_THAN_OR_EQUAL, Operation.GT_EQ) + .put(BuiltInFunctionDefinitions.LESS_THAN, Operation.LT) + .put(BuiltInFunctionDefinitions.LESS_THAN_OR_EQUAL, Operation.LT_EQ) + .put(BuiltInFunctionDefinitions.IS_NULL, Operation.IS_NULL) + .put(BuiltInFunctionDefinitions.IS_NOT_NULL, Operation.NOT_NULL) + .put(BuiltInFunctionDefinitions.AND, Operation.AND) + .put(BuiltInFunctionDefinitions.OR, Operation.OR) + .put(BuiltInFunctionDefinitions.NOT, Operation.NOT) + .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH) + .buildOrThrow(); + + /** + * Convert flink expression to iceberg expression. + * + *

the BETWEEN, NOT_BETWEEN, IN expression will be converted by flink automatically. the + * BETWEEN will be converted to (GT_EQ AND LT_EQ), the NOT_BETWEEN will be converted to (LT_EQ OR + * GT_EQ), the IN will be converted to OR, so we do not add the conversion here + * + * @param flinkExpression the flink expression + * @return the iceberg expression + */ + public static Optional convert( + org.apache.flink.table.expressions.Expression flinkExpression) { + if (!(flinkExpression instanceof CallExpression)) { + return Optional.empty(); + } + + CallExpression call = (CallExpression) flinkExpression; + Operation op = FILTERS.get(call.getFunctionDefinition()); + if (op != null) { + switch (op) { + case IS_NULL: + return onlyChildAs(call, FieldReferenceExpression.class) + .map(FieldReferenceExpression::getName) + .map(Expressions::isNull); + + case NOT_NULL: + return onlyChildAs(call, FieldReferenceExpression.class) + .map(FieldReferenceExpression::getName) + .map(Expressions::notNull); + + case LT: + return convertFieldAndLiteral(Expressions::lessThan, Expressions::greaterThan, call); + + case LT_EQ: + return convertFieldAndLiteral( + Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call); + + case GT: + return convertFieldAndLiteral(Expressions::greaterThan, Expressions::lessThan, call); + + case GT_EQ: + return convertFieldAndLiteral( + Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call); + + case EQ: + return convertFieldAndLiteral( + (ref, lit) -> { + if (NaNUtil.isNaN(lit)) { + return Expressions.isNaN(ref); + } else { + return Expressions.equal(ref, lit); + } + }, + call); + + case NOT_EQ: + return convertFieldAndLiteral( + (ref, lit) -> { + if (NaNUtil.isNaN(lit)) { + return Expressions.notNaN(ref); + } else { + return Expressions.notEqual(ref, lit); + } + }, + call); + + case NOT: + return onlyChildAs(call, CallExpression.class) + .flatMap(FlinkFilters::convert) + .map(Expressions::not); + + case AND: + return convertLogicExpression(Expressions::and, call); + + case OR: + return convertLogicExpression(Expressions::or, call); + + case STARTS_WITH: + return convertLike(call); + } + } + + return Optional.empty(); + } + + private static Optional onlyChildAs( + CallExpression call, Class expectedChildClass) { + List children = call.getResolvedChildren(); + if (children.size() != 1) { + return Optional.empty(); + } + + ResolvedExpression child = children.get(0); + if (!expectedChildClass.isInstance(child)) { + return Optional.empty(); + } + + return Optional.of(expectedChildClass.cast(child)); + } + + private static Optional convertLike(CallExpression call) { + List args = call.getResolvedChildren(); + if (args.size() != 2) { + return Optional.empty(); + } + + org.apache.flink.table.expressions.Expression left = args.get(0); + org.apache.flink.table.expressions.Expression right = args.get(1); + + if (left instanceof FieldReferenceExpression && right instanceof ValueLiteralExpression) { + String name = ((FieldReferenceExpression) left).getName(); + return convertLiteral((ValueLiteralExpression) right) + .flatMap( + lit -> { + if (lit instanceof String) { + String pattern = (String) lit; + Matcher matcher = STARTS_WITH_PATTERN.matcher(pattern); + // exclude special char of LIKE + // '_' is the wildcard of the SQL LIKE + if (!pattern.contains("_") && matcher.matches()) { + return Optional.of(Expressions.startsWith(name, matcher.group(1))); + } + } + + return Optional.empty(); + }); + } + + return Optional.empty(); + } + + private static Optional convertLogicExpression( + BiFunction function, CallExpression call) { + List args = call.getResolvedChildren(); + if (args == null || args.size() != 2) { + return Optional.empty(); + } + + Optional left = convert(args.get(0)); + Optional right = convert(args.get(1)); + if (left.isPresent() && right.isPresent()) { + return Optional.of(function.apply(left.get(), right.get())); + } + + return Optional.empty(); + } + + private static Optional convertLiteral(ValueLiteralExpression expression) { + Optional value = + expression.getValueAs( + expression.getOutputDataType().getLogicalType().getDefaultConversion()); + return value.map( + o -> { + if (o instanceof LocalDateTime) { + return DateTimeUtil.microsFromTimestamp((LocalDateTime) o); + } else if (o instanceof Instant) { + return DateTimeUtil.microsFromInstant((Instant) o); + } else if (o instanceof LocalTime) { + return DateTimeUtil.microsFromTime((LocalTime) o); + } else if (o instanceof LocalDate) { + return DateTimeUtil.daysFromDate((LocalDate) o); + } + + return o; + }); + } + + private static Optional convertFieldAndLiteral( + BiFunction expr, CallExpression call) { + return convertFieldAndLiteral(expr, expr, call); + } + + private static Optional convertFieldAndLiteral( + BiFunction convertLR, + BiFunction convertRL, + CallExpression call) { + List args = call.getResolvedChildren(); + if (args.size() != 2) { + return Optional.empty(); + } + + org.apache.flink.table.expressions.Expression left = args.get(0); + org.apache.flink.table.expressions.Expression right = args.get(1); + + if (left instanceof FieldReferenceExpression && right instanceof ValueLiteralExpression) { + String name = ((FieldReferenceExpression) left).getName(); + Optional lit = convertLiteral((ValueLiteralExpression) right); + if (lit.isPresent()) { + return Optional.of(convertLR.apply(name, lit.get())); + } + } else if (left instanceof ValueLiteralExpression + && right instanceof FieldReferenceExpression) { + Optional lit = convertLiteral((ValueLiteralExpression) left); + String name = ((FieldReferenceExpression) right).getName(); + if (lit.isPresent()) { + return Optional.of(convertRL.apply(name, lit.get())); + } + } + + return Optional.empty(); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java new file mode 100644 index 000000000000..767d4497ac91 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java @@ -0,0 +1,50 @@ +/* + * 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.iceberg.flink; + +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.FixupTypes; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; + +/** + * The uuid and fixed are converted to the same Flink type. Conversion back can produce only one, + * which may not be correct. + */ +class FlinkFixupTypes extends FixupTypes { + + private FlinkFixupTypes(Schema referenceSchema) { + super(referenceSchema); + } + + static Schema fixup(Schema schema, Schema referenceSchema) { + return new Schema( + TypeUtil.visit(schema, new FlinkFixupTypes(referenceSchema)).asStructType().fields()); + } + + @Override + protected boolean fixupPrimitive(Type.PrimitiveType type, Type source) { + if (type instanceof Types.FixedType) { + int length = ((Types.FixedType) type).length(); + return source.typeId() == Type.TypeID.UUID && length == 16; + } + return false; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java new file mode 100644 index 000000000000..0e04c9affb19 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java @@ -0,0 +1,193 @@ +/* + * 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.iceberg.flink; + +import java.time.Duration; +import java.util.Map; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.util.TimeUtils; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.source.StreamingStartingStrategy; + +public class FlinkReadConf { + + private final FlinkConfParser confParser; + + public FlinkReadConf( + Table table, Map readOptions, ReadableConfig readableConfig) { + this.confParser = new FlinkConfParser(table, readOptions, readableConfig); + } + + public Long snapshotId() { + return confParser.longConf().option(FlinkReadOptions.SNAPSHOT_ID.key()).parseOptional(); + } + + public String tag() { + return confParser.stringConf().option(FlinkReadOptions.TAG.key()).parseOptional(); + } + + public String startTag() { + return confParser.stringConf().option(FlinkReadOptions.START_TAG.key()).parseOptional(); + } + + public String endTag() { + return confParser.stringConf().option(FlinkReadOptions.END_TAG.key()).parseOptional(); + } + + public String branch() { + return confParser.stringConf().option(FlinkReadOptions.BRANCH.key()).parseOptional(); + } + + public boolean caseSensitive() { + return confParser + .booleanConf() + .option(FlinkReadOptions.CASE_SENSITIVE) + .flinkConfig(FlinkReadOptions.CASE_SENSITIVE_OPTION) + .defaultValue(FlinkReadOptions.CASE_SENSITIVE_OPTION.defaultValue()) + .parse(); + } + + public Long asOfTimestamp() { + return confParser.longConf().option(FlinkReadOptions.AS_OF_TIMESTAMP.key()).parseOptional(); + } + + public StreamingStartingStrategy startingStrategy() { + return confParser + .enumConfParser(StreamingStartingStrategy.class) + .option(FlinkReadOptions.STARTING_STRATEGY) + .flinkConfig(FlinkReadOptions.STARTING_STRATEGY_OPTION) + .defaultValue(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) + .parse(); + } + + public Long startSnapshotTimestamp() { + return confParser + .longConf() + .option(FlinkReadOptions.START_SNAPSHOT_TIMESTAMP.key()) + .parseOptional(); + } + + public Long startSnapshotId() { + return confParser.longConf().option(FlinkReadOptions.START_SNAPSHOT_ID.key()).parseOptional(); + } + + public Long endSnapshotId() { + return confParser.longConf().option(FlinkReadOptions.END_SNAPSHOT_ID.key()).parseOptional(); + } + + public long splitSize() { + return confParser + .longConf() + .option(FlinkReadOptions.SPLIT_SIZE) + .flinkConfig(FlinkReadOptions.SPLIT_SIZE_OPTION) + .tableProperty(TableProperties.SPLIT_SIZE) + .defaultValue(TableProperties.SPLIT_SIZE_DEFAULT) + .parse(); + } + + public int splitLookback() { + return confParser + .intConf() + .option(FlinkReadOptions.SPLIT_LOOKBACK) + .flinkConfig(FlinkReadOptions.SPLIT_LOOKBACK_OPTION) + .tableProperty(TableProperties.SPLIT_LOOKBACK) + .defaultValue(TableProperties.SPLIT_LOOKBACK_DEFAULT) + .parse(); + } + + public long splitFileOpenCost() { + return confParser + .longConf() + .option(FlinkReadOptions.SPLIT_FILE_OPEN_COST) + .flinkConfig(FlinkReadOptions.SPLIT_FILE_OPEN_COST_OPTION) + .tableProperty(TableProperties.SPLIT_OPEN_FILE_COST) + .defaultValue(TableProperties.SPLIT_OPEN_FILE_COST_DEFAULT) + .parse(); + } + + public boolean streaming() { + return confParser + .booleanConf() + .option(FlinkReadOptions.STREAMING) + .flinkConfig(FlinkReadOptions.STREAMING_OPTION) + .defaultValue(FlinkReadOptions.STREAMING_OPTION.defaultValue()) + .parse(); + } + + public Duration monitorInterval() { + String duration = + confParser + .stringConf() + .option(FlinkReadOptions.MONITOR_INTERVAL) + .flinkConfig(FlinkReadOptions.MONITOR_INTERVAL_OPTION) + .defaultValue(FlinkReadOptions.MONITOR_INTERVAL_OPTION.defaultValue()) + .parse(); + + return TimeUtils.parseDuration(duration); + } + + public boolean includeColumnStats() { + return confParser + .booleanConf() + .option(FlinkReadOptions.INCLUDE_COLUMN_STATS) + .flinkConfig(FlinkReadOptions.INCLUDE_COLUMN_STATS_OPTION) + .defaultValue(FlinkReadOptions.INCLUDE_COLUMN_STATS_OPTION.defaultValue()) + .parse(); + } + + public int maxPlanningSnapshotCount() { + return confParser + .intConf() + .option(FlinkReadOptions.MAX_PLANNING_SNAPSHOT_COUNT) + .flinkConfig(FlinkReadOptions.MAX_PLANNING_SNAPSHOT_COUNT_OPTION) + .defaultValue(FlinkReadOptions.MAX_PLANNING_SNAPSHOT_COUNT_OPTION.defaultValue()) + .parse(); + } + + public String nameMapping() { + return confParser.stringConf().option(TableProperties.DEFAULT_NAME_MAPPING).parseOptional(); + } + + public long limit() { + return confParser + .longConf() + .option(FlinkReadOptions.LIMIT) + .flinkConfig(FlinkReadOptions.LIMIT_OPTION) + .defaultValue(FlinkReadOptions.LIMIT_OPTION.defaultValue()) + .parse(); + } + + public int workerPoolSize() { + return confParser + .intConf() + .flinkConfig(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE) + .defaultValue(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.defaultValue()) + .parse(); + } + + public int maxAllowedPlanningFailures() { + return confParser + .intConf() + .option(FlinkReadOptions.MAX_ALLOWED_PLANNING_FAILURES) + .flinkConfig(FlinkReadOptions.MAX_ALLOWED_PLANNING_FAILURES_OPTION) + .defaultValue(FlinkReadOptions.MAX_ALLOWED_PLANNING_FAILURES_OPTION.defaultValue()) + .parse(); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java new file mode 100644 index 000000000000..55c5aca3b677 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.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.iceberg.flink; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.source.StreamingStartingStrategy; + +/** Flink source read options */ +public class FlinkReadOptions { + private static final String PREFIX = "connector.iceberg."; + + private FlinkReadOptions() {} + + public static final ConfigOption SNAPSHOT_ID = + ConfigOptions.key("snapshot-id").longType().defaultValue(null); + + public static final ConfigOption TAG = + ConfigOptions.key("tag").stringType().defaultValue(null); + + public static final ConfigOption BRANCH = + ConfigOptions.key("branch").stringType().defaultValue(null); + + public static final ConfigOption START_TAG = + ConfigOptions.key("start-tag").stringType().defaultValue(null); + + public static final ConfigOption END_TAG = + ConfigOptions.key("end-tag").stringType().defaultValue(null); + + public static final String CASE_SENSITIVE = "case-sensitive"; + public static final ConfigOption CASE_SENSITIVE_OPTION = + ConfigOptions.key(PREFIX + CASE_SENSITIVE).booleanType().defaultValue(false); + + public static final ConfigOption AS_OF_TIMESTAMP = + ConfigOptions.key("as-of-timestamp").longType().defaultValue(null); + + public static final String STARTING_STRATEGY = "starting-strategy"; + public static final ConfigOption STARTING_STRATEGY_OPTION = + ConfigOptions.key(PREFIX + STARTING_STRATEGY) + .enumType(StreamingStartingStrategy.class) + .defaultValue(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT); + + public static final ConfigOption START_SNAPSHOT_TIMESTAMP = + ConfigOptions.key("start-snapshot-timestamp").longType().defaultValue(null); + + public static final ConfigOption START_SNAPSHOT_ID = + ConfigOptions.key("start-snapshot-id").longType().defaultValue(null); + + public static final ConfigOption END_SNAPSHOT_ID = + ConfigOptions.key("end-snapshot-id").longType().defaultValue(null); + + public static final String SPLIT_SIZE = "split-size"; + public static final ConfigOption SPLIT_SIZE_OPTION = + ConfigOptions.key(PREFIX + SPLIT_SIZE) + .longType() + .defaultValue(TableProperties.SPLIT_SIZE_DEFAULT); + + public static final String SPLIT_LOOKBACK = "split-lookback"; + public static final ConfigOption SPLIT_LOOKBACK_OPTION = + ConfigOptions.key(PREFIX + SPLIT_LOOKBACK) + .intType() + .defaultValue(TableProperties.SPLIT_LOOKBACK_DEFAULT); + + public static final String SPLIT_FILE_OPEN_COST = "split-file-open-cost"; + public static final ConfigOption SPLIT_FILE_OPEN_COST_OPTION = + ConfigOptions.key(PREFIX + SPLIT_FILE_OPEN_COST) + .longType() + .defaultValue(TableProperties.SPLIT_OPEN_FILE_COST_DEFAULT); + + public static final String STREAMING = "streaming"; + public static final ConfigOption STREAMING_OPTION = + ConfigOptions.key(PREFIX + STREAMING).booleanType().defaultValue(false); + + public static final String MONITOR_INTERVAL = "monitor-interval"; + public static final ConfigOption MONITOR_INTERVAL_OPTION = + ConfigOptions.key(PREFIX + MONITOR_INTERVAL).stringType().defaultValue("60s"); + + public static final String INCLUDE_COLUMN_STATS = "include-column-stats"; + public static final ConfigOption INCLUDE_COLUMN_STATS_OPTION = + ConfigOptions.key(PREFIX + INCLUDE_COLUMN_STATS).booleanType().defaultValue(false); + + public static final String MAX_PLANNING_SNAPSHOT_COUNT = "max-planning-snapshot-count"; + public static final ConfigOption MAX_PLANNING_SNAPSHOT_COUNT_OPTION = + ConfigOptions.key(PREFIX + MAX_PLANNING_SNAPSHOT_COUNT) + .intType() + .defaultValue(Integer.MAX_VALUE); + + public static final String LIMIT = "limit"; + public static final ConfigOption LIMIT_OPTION = + ConfigOptions.key(PREFIX + LIMIT).longType().defaultValue(-1L); + + public static final String MAX_ALLOWED_PLANNING_FAILURES = "max-allowed-planning-failures"; + public static final ConfigOption MAX_ALLOWED_PLANNING_FAILURES_OPTION = + ConfigOptions.key(PREFIX + MAX_ALLOWED_PLANNING_FAILURES).intType().defaultValue(3); +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java new file mode 100644 index 000000000000..a6b53879ad80 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java @@ -0,0 +1,191 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink; + +import java.util.List; +import java.util.Set; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.iceberg.Schema; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; + +/** + * Converter between Flink types and Iceberg type. The conversion is not a 1:1 mapping that not + * allows back-and-forth conversion. So some information might get lost during the back-and-forth + * conversion. + * + *

This inconsistent types: + * + *

    + *
  • map Iceberg UUID type to Flink BinaryType(16) + *
  • map Flink VarCharType(_) and CharType(_) to Iceberg String type + *
  • map Flink VarBinaryType(_) to Iceberg Binary type + *
  • map Flink TimeType(_) to Iceberg Time type (microseconds) + *
  • map Flink TimestampType(_) to Iceberg Timestamp without zone type (microseconds) + *
  • map Flink LocalZonedTimestampType(_) to Iceberg Timestamp with zone type (microseconds) + *
  • map Flink MultiSetType to Iceberg Map type(element, int) + *
+ * + *

+ */ +public class FlinkSchemaUtil { + + private FlinkSchemaUtil() {} + + /** Convert the flink table schema to apache iceberg schema. */ + public static Schema convert(TableSchema schema) { + LogicalType schemaType = schema.toRowDataType().getLogicalType(); + Preconditions.checkArgument( + schemaType instanceof RowType, "Schema logical type should be RowType."); + + RowType root = (RowType) schemaType; + Type converted = root.accept(new FlinkTypeToType(root)); + + Schema iSchema = new Schema(converted.asStructType().fields()); + return freshIdentifierFieldIds(iSchema, schema); + } + + private static Schema freshIdentifierFieldIds(Schema iSchema, TableSchema schema) { + // Locate the identifier field id list. + Set identifierFieldIds = Sets.newHashSet(); + if (schema.getPrimaryKey().isPresent()) { + for (String column : schema.getPrimaryKey().get().getColumns()) { + Types.NestedField field = iSchema.findField(column); + Preconditions.checkNotNull( + field, + "Cannot find field ID for the primary key column %s in schema %s", + column, + iSchema); + identifierFieldIds.add(field.fieldId()); + } + } + + return new Schema(iSchema.schemaId(), iSchema.asStruct().fields(), identifierFieldIds); + } + + /** + * Convert a Flink {@link TableSchema} to a {@link Schema} based on the given schema. + * + *

This conversion does not assign new ids; it uses ids from the base schema. + * + *

Data types, field order, and nullability will match the Flink type. This conversion may + * return a schema that is not compatible with base schema. + * + * @param baseSchema a Schema on which conversion is based + * @param flinkSchema a Flink TableSchema + * @return the equivalent Schema + * @throws IllegalArgumentException if the type cannot be converted or there are missing ids + */ + public static Schema convert(Schema baseSchema, TableSchema flinkSchema) { + // convert to a type with fresh ids + Types.StructType struct = convert(flinkSchema).asStruct(); + // reassign ids to match the base schema + Schema schema = TypeUtil.reassignIds(new Schema(struct.fields()), baseSchema); + // reassign doc to match the base schema + schema = TypeUtil.reassignDoc(schema, baseSchema); + + // fix types that can't be represented in Flink (UUID) + Schema fixedSchema = FlinkFixupTypes.fixup(schema, baseSchema); + return freshIdentifierFieldIds(fixedSchema, flinkSchema); + } + + /** + * Convert a {@link Schema} to a {@link RowType Flink type}. + * + * @param schema a Schema + * @return the equivalent Flink type + * @throws IllegalArgumentException if the type cannot be converted to Flink + */ + public static RowType convert(Schema schema) { + return (RowType) TypeUtil.visit(schema, new TypeToFlinkType()); + } + + /** + * Convert a {@link Type} to a {@link LogicalType Flink type}. + * + * @param type a Type + * @return the equivalent Flink type + * @throws IllegalArgumentException if the type cannot be converted to Flink + */ + public static LogicalType convert(Type type) { + return TypeUtil.visit(type, new TypeToFlinkType()); + } + + /** + * Convert a {@link LogicalType Flink type} to a {@link Type}. + * + * @param flinkType a FlinkType + * @return the equivalent Iceberg type + */ + public static Type convert(LogicalType flinkType) { + return flinkType.accept(new FlinkTypeToType()); + } + + /** + * Convert a {@link RowType} to a {@link TableSchema}. + * + * @param rowType a RowType + * @return Flink TableSchema + */ + public static TableSchema toSchema(RowType rowType) { + TableSchema.Builder builder = TableSchema.builder(); + for (RowType.RowField field : rowType.getFields()) { + builder.field(field.getName(), TypeConversions.fromLogicalToDataType(field.getType())); + } + return builder.build(); + } + + /** + * Convert a {@link Schema} to a {@link TableSchema}. + * + * @param schema iceberg schema to convert. + * @return Flink TableSchema. + */ + public static TableSchema toSchema(Schema schema) { + TableSchema.Builder builder = TableSchema.builder(); + + // Add columns. + for (RowType.RowField field : convert(schema).getFields()) { + builder.field(field.getName(), TypeConversions.fromLogicalToDataType(field.getType())); + } + + // Add primary key. + Set identifierFieldIds = schema.identifierFieldIds(); + if (!identifierFieldIds.isEmpty()) { + List columns = Lists.newArrayListWithExpectedSize(identifierFieldIds.size()); + for (Integer identifierFieldId : identifierFieldIds) { + String columnName = schema.findColumnName(identifierFieldId); + Preconditions.checkNotNull( + columnName, "Cannot find field with id %s in schema %s", identifierFieldId, schema); + + columns.add(columnName); + } + builder.primaryKey(columns.toArray(new String[0])); + } + + return builder.build(); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java new file mode 100644 index 000000000000..5fbd84909d69 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java @@ -0,0 +1,49 @@ +/* + * 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.iceberg.flink; + +import org.apache.flink.api.common.functions.FilterFunction; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.Evaluator; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.types.Types; + +public class FlinkSourceFilter implements FilterFunction { + + private final RowType rowType; + private final Evaluator evaluator; + private final Types.StructType struct; + private volatile RowDataWrapper wrapper; + + public FlinkSourceFilter(Schema schema, Expression expr, boolean caseSensitive) { + this.rowType = FlinkSchemaUtil.convert(schema); + this.struct = schema.asStruct(); + this.evaluator = new Evaluator(struct, expr, caseSensitive); + } + + @Override + public boolean filter(RowData value) { + if (wrapper == null) { + this.wrapper = new RowDataWrapper(rowType, struct); + } + return evaluator.eval(wrapper.wrap(value)); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java new file mode 100644 index 000000000000..408065f06057 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java @@ -0,0 +1,203 @@ +/* + * 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.iceberg.flink; + +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.table.types.logical.BinaryType; +import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.CharType; +import org.apache.flink.table.types.logical.DateType; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.DoubleType; +import org.apache.flink.table.types.logical.FloatType; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.LocalZonedTimestampType; +import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.MultisetType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.SmallIntType; +import org.apache.flink.table.types.logical.TimeType; +import org.apache.flink.table.types.logical.TimestampType; +import org.apache.flink.table.types.logical.TinyIntType; +import org.apache.flink.table.types.logical.VarBinaryType; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +class FlinkTypeToType extends FlinkTypeVisitor { + + private final RowType root; + private int nextId; + + FlinkTypeToType() { + this.root = null; + } + + FlinkTypeToType(RowType root) { + this.root = root; + // the root struct's fields use the first ids + this.nextId = root.getFieldCount(); + } + + private int getNextId() { + int next = nextId; + nextId += 1; + return next; + } + + @Override + public Type visit(CharType charType) { + return Types.StringType.get(); + } + + @Override + public Type visit(VarCharType varCharType) { + return Types.StringType.get(); + } + + @Override + public Type visit(BooleanType booleanType) { + return Types.BooleanType.get(); + } + + @Override + public Type visit(BinaryType binaryType) { + return Types.FixedType.ofLength(binaryType.getLength()); + } + + @Override + public Type visit(VarBinaryType varBinaryType) { + return Types.BinaryType.get(); + } + + @Override + public Type visit(DecimalType decimalType) { + return Types.DecimalType.of(decimalType.getPrecision(), decimalType.getScale()); + } + + @Override + public Type visit(TinyIntType tinyIntType) { + return Types.IntegerType.get(); + } + + @Override + public Type visit(SmallIntType smallIntType) { + return Types.IntegerType.get(); + } + + @Override + public Type visit(IntType intType) { + return Types.IntegerType.get(); + } + + @Override + public Type visit(BigIntType bigIntType) { + return Types.LongType.get(); + } + + @Override + public Type visit(FloatType floatType) { + return Types.FloatType.get(); + } + + @Override + public Type visit(DoubleType doubleType) { + return Types.DoubleType.get(); + } + + @Override + public Type visit(DateType dateType) { + return Types.DateType.get(); + } + + @Override + public Type visit(TimeType timeType) { + return Types.TimeType.get(); + } + + @Override + public Type visit(TimestampType timestampType) { + return Types.TimestampType.withoutZone(); + } + + @Override + public Type visit(LocalZonedTimestampType localZonedTimestampType) { + return Types.TimestampType.withZone(); + } + + @Override + public Type visit(ArrayType arrayType) { + Type elementType = arrayType.getElementType().accept(this); + if (arrayType.getElementType().isNullable()) { + return Types.ListType.ofOptional(getNextId(), elementType); + } else { + return Types.ListType.ofRequired(getNextId(), elementType); + } + } + + @Override + public Type visit(MultisetType multisetType) { + Type elementType = multisetType.getElementType().accept(this); + return Types.MapType.ofRequired(getNextId(), getNextId(), elementType, Types.IntegerType.get()); + } + + @Override + public Type visit(MapType mapType) { + // keys in map are not allowed to be null. + Type keyType = mapType.getKeyType().accept(this); + Type valueType = mapType.getValueType().accept(this); + if (mapType.getValueType().isNullable()) { + return Types.MapType.ofOptional(getNextId(), getNextId(), keyType, valueType); + } else { + return Types.MapType.ofRequired(getNextId(), getNextId(), keyType, valueType); + } + } + + @Override + @SuppressWarnings("ReferenceEquality") + public Type visit(RowType rowType) { + List newFields = Lists.newArrayListWithExpectedSize(rowType.getFieldCount()); + boolean isRoot = root == rowType; + + List types = + rowType.getFields().stream() + .map(f -> f.getType().accept(this)) + .collect(Collectors.toList()); + + for (int i = 0; i < rowType.getFieldCount(); i++) { + int id = isRoot ? i : getNextId(); + + RowType.RowField field = rowType.getFields().get(i); + String name = field.getName(); + String comment = field.getDescription().orElse(null); + + if (field.getType().isNullable()) { + newFields.add(Types.NestedField.optional(id, name, types.get(i), comment)); + } else { + newFields.add(Types.NestedField.required(id, name, types.get(i), comment)); + } + } + + return Types.StructType.of(newFields); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java new file mode 100644 index 000000000000..f3de2416088c --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java @@ -0,0 +1,80 @@ +/* + * 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.iceberg.flink; + +import org.apache.flink.table.types.logical.DayTimeIntervalType; +import org.apache.flink.table.types.logical.DistinctType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeVisitor; +import org.apache.flink.table.types.logical.NullType; +import org.apache.flink.table.types.logical.RawType; +import org.apache.flink.table.types.logical.StructuredType; +import org.apache.flink.table.types.logical.SymbolType; +import org.apache.flink.table.types.logical.YearMonthIntervalType; +import org.apache.flink.table.types.logical.ZonedTimestampType; + +public abstract class FlinkTypeVisitor implements LogicalTypeVisitor { + + // ------------------------- Unsupported types ------------------------------ + + @Override + public T visit(ZonedTimestampType zonedTimestampType) { + throw new UnsupportedOperationException("Unsupported ZonedTimestampType."); + } + + @Override + public T visit(YearMonthIntervalType yearMonthIntervalType) { + throw new UnsupportedOperationException("Unsupported YearMonthIntervalType."); + } + + @Override + public T visit(DayTimeIntervalType dayTimeIntervalType) { + throw new UnsupportedOperationException("Unsupported DayTimeIntervalType."); + } + + @Override + public T visit(DistinctType distinctType) { + throw new UnsupportedOperationException("Unsupported DistinctType."); + } + + @Override + public T visit(StructuredType structuredType) { + throw new UnsupportedOperationException("Unsupported StructuredType."); + } + + @Override + public T visit(NullType nullType) { + throw new UnsupportedOperationException("Unsupported NullType."); + } + + @Override + public T visit(RawType rawType) { + throw new UnsupportedOperationException("Unsupported RawType."); + } + + @Override + public T visit(SymbolType symbolType) { + throw new UnsupportedOperationException("Unsupported SymbolType."); + } + + @Override + public T visit(LogicalType other) { + throw new UnsupportedOperationException("Unsupported type: " + other); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java new file mode 100644 index 000000000000..ca7b1120bc81 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java @@ -0,0 +1,205 @@ +/* + * 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.iceberg.flink; + +import java.time.Duration; +import java.util.Map; +import org.apache.flink.annotation.Experimental; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; + +/** + * A class for common Iceberg configs for Flink writes. + * + *

If a config is set at multiple levels, the following order of precedence is used (top to + * bottom): + * + *

    + *
  1. Write options + *
  2. flink ReadableConfig + *
  3. Table metadata + *
+ * + * The most specific value is set in write options and takes precedence over all other configs. If + * no write option is provided, this class checks the flink configuration for any overrides. If no + * applicable value is found in the write options, this class uses the table metadata. + * + *

Note this class is NOT meant to be serialized. + */ +public class FlinkWriteConf { + + private final FlinkConfParser confParser; + + public FlinkWriteConf( + Table table, Map writeOptions, ReadableConfig readableConfig) { + this.confParser = new FlinkConfParser(table, writeOptions, readableConfig); + } + + public boolean overwriteMode() { + return confParser + .booleanConf() + .option(FlinkWriteOptions.OVERWRITE_MODE.key()) + .flinkConfig(FlinkWriteOptions.OVERWRITE_MODE) + .defaultValue(FlinkWriteOptions.OVERWRITE_MODE.defaultValue()) + .parse(); + } + + public boolean upsertMode() { + return confParser + .booleanConf() + .option(FlinkWriteOptions.WRITE_UPSERT_ENABLED.key()) + .flinkConfig(FlinkWriteOptions.WRITE_UPSERT_ENABLED) + .tableProperty(TableProperties.UPSERT_ENABLED) + .defaultValue(TableProperties.UPSERT_ENABLED_DEFAULT) + .parse(); + } + + public FileFormat dataFileFormat() { + String valueAsString = + confParser + .stringConf() + .option(FlinkWriteOptions.WRITE_FORMAT.key()) + .flinkConfig(FlinkWriteOptions.WRITE_FORMAT) + .tableProperty(TableProperties.DEFAULT_FILE_FORMAT) + .defaultValue(TableProperties.DEFAULT_FILE_FORMAT_DEFAULT) + .parse(); + return FileFormat.fromString(valueAsString); + } + + public long targetDataFileSize() { + return confParser + .longConf() + .option(FlinkWriteOptions.TARGET_FILE_SIZE_BYTES.key()) + .flinkConfig(FlinkWriteOptions.TARGET_FILE_SIZE_BYTES) + .tableProperty(TableProperties.WRITE_TARGET_FILE_SIZE_BYTES) + .defaultValue(TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT) + .parse(); + } + + public String parquetCompressionCodec() { + return confParser + .stringConf() + .option(FlinkWriteOptions.COMPRESSION_CODEC.key()) + .flinkConfig(FlinkWriteOptions.COMPRESSION_CODEC) + .tableProperty(TableProperties.PARQUET_COMPRESSION) + .defaultValue(TableProperties.PARQUET_COMPRESSION_DEFAULT) + .parse(); + } + + public String parquetCompressionLevel() { + return confParser + .stringConf() + .option(FlinkWriteOptions.COMPRESSION_LEVEL.key()) + .flinkConfig(FlinkWriteOptions.COMPRESSION_LEVEL) + .tableProperty(TableProperties.PARQUET_COMPRESSION_LEVEL) + .defaultValue(TableProperties.PARQUET_COMPRESSION_LEVEL_DEFAULT) + .parseOptional(); + } + + public String avroCompressionCodec() { + return confParser + .stringConf() + .option(FlinkWriteOptions.COMPRESSION_CODEC.key()) + .flinkConfig(FlinkWriteOptions.COMPRESSION_CODEC) + .tableProperty(TableProperties.AVRO_COMPRESSION) + .defaultValue(TableProperties.AVRO_COMPRESSION_DEFAULT) + .parse(); + } + + public String avroCompressionLevel() { + return confParser + .stringConf() + .option(FlinkWriteOptions.COMPRESSION_LEVEL.key()) + .flinkConfig(FlinkWriteOptions.COMPRESSION_LEVEL) + .tableProperty(TableProperties.AVRO_COMPRESSION_LEVEL) + .defaultValue(TableProperties.AVRO_COMPRESSION_LEVEL_DEFAULT) + .parseOptional(); + } + + public String orcCompressionCodec() { + return confParser + .stringConf() + .option(FlinkWriteOptions.COMPRESSION_CODEC.key()) + .flinkConfig(FlinkWriteOptions.COMPRESSION_CODEC) + .tableProperty(TableProperties.ORC_COMPRESSION) + .defaultValue(TableProperties.ORC_COMPRESSION_DEFAULT) + .parse(); + } + + public String orcCompressionStrategy() { + return confParser + .stringConf() + .option(FlinkWriteOptions.COMPRESSION_STRATEGY.key()) + .flinkConfig(FlinkWriteOptions.COMPRESSION_STRATEGY) + .tableProperty(TableProperties.ORC_COMPRESSION_STRATEGY) + .defaultValue(TableProperties.ORC_COMPRESSION_STRATEGY_DEFAULT) + .parse(); + } + + public DistributionMode distributionMode() { + String modeName = + confParser + .stringConf() + .option(FlinkWriteOptions.DISTRIBUTION_MODE.key()) + .flinkConfig(FlinkWriteOptions.DISTRIBUTION_MODE) + .tableProperty(TableProperties.WRITE_DISTRIBUTION_MODE) + .defaultValue(TableProperties.WRITE_DISTRIBUTION_MODE_NONE) + .parse(); + return DistributionMode.fromName(modeName); + } + + public int workerPoolSize() { + return confParser + .intConf() + .flinkConfig(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE) + .defaultValue(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.defaultValue()) + .parse(); + } + + public String branch() { + return confParser + .stringConf() + .option(FlinkWriteOptions.BRANCH.key()) + .defaultValue(FlinkWriteOptions.BRANCH.defaultValue()) + .parse(); + } + + public Integer writeParallelism() { + return confParser.intConf().option(FlinkWriteOptions.WRITE_PARALLELISM.key()).parseOptional(); + } + + /** + * NOTE: This may be removed or changed in a future release. This value specifies the interval for + * refreshing the table instances in sink writer subtasks. If not specified then the default + * behavior is to not refresh the table. + * + * @return the interval for refreshing the table in sink writer subtasks + */ + @Experimental + public Duration tableRefreshInterval() { + return confParser + .durationConf() + .option(FlinkWriteOptions.TABLE_REFRESH_INTERVAL.key()) + .flinkConfig(FlinkWriteOptions.TABLE_REFRESH_INTERVAL) + .parseOptional(); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java new file mode 100644 index 000000000000..df73f2e09cac --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java @@ -0,0 +1,73 @@ +/* + * 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.iceberg.flink; + +import java.time.Duration; +import org.apache.flink.annotation.Experimental; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.iceberg.SnapshotRef; + +/** Flink sink write options */ +public class FlinkWriteOptions { + + private FlinkWriteOptions() {} + + // File format for write operations(default: Table write.format.default ) + public static final ConfigOption WRITE_FORMAT = + ConfigOptions.key("write-format").stringType().noDefaultValue(); + + // Overrides this table's write.target-file-size-bytes + public static final ConfigOption TARGET_FILE_SIZE_BYTES = + ConfigOptions.key("target-file-size-bytes").longType().noDefaultValue(); + + // Overrides this table's write..compression-codec + public static final ConfigOption COMPRESSION_CODEC = + ConfigOptions.key("compression-codec").stringType().noDefaultValue(); + + // Overrides this table's write..compression-level + public static final ConfigOption COMPRESSION_LEVEL = + ConfigOptions.key("compression-level").stringType().noDefaultValue(); + + // Overrides this table's write..compression-strategy + public static final ConfigOption COMPRESSION_STRATEGY = + ConfigOptions.key("compression-strategy").stringType().noDefaultValue(); + + // Overrides this table's write.upsert.enabled + public static final ConfigOption WRITE_UPSERT_ENABLED = + ConfigOptions.key("upsert-enabled").booleanType().noDefaultValue(); + + public static final ConfigOption OVERWRITE_MODE = + ConfigOptions.key("overwrite-enabled").booleanType().defaultValue(false); + + // Overrides the table's write.distribution-mode + public static final ConfigOption DISTRIBUTION_MODE = + ConfigOptions.key("distribution-mode").stringType().noDefaultValue(); + + // Branch to write to + public static final ConfigOption BRANCH = + ConfigOptions.key("branch").stringType().defaultValue(SnapshotRef.MAIN_BRANCH); + + public static final ConfigOption WRITE_PARALLELISM = + ConfigOptions.key("write-parallelism").intType().noDefaultValue(); + + @Experimental + public static final ConfigOption TABLE_REFRESH_INTERVAL = + ConfigOptions.key("table-refresh-interval").durationType().noDefaultValue(); +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java new file mode 100644 index 000000000000..1b9268569d9a --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java @@ -0,0 +1,121 @@ +/* + * 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.iceberg.flink; + +import java.util.List; +import java.util.Map; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.constraints.UniqueConstraint; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.ProviderContext; +import org.apache.flink.table.connector.sink.DataStreamSinkProvider; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.abilities.SupportsOverwrite; +import org.apache.flink.table.connector.sink.abilities.SupportsPartitioning; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.flink.sink.FlinkSink; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; + +public class IcebergTableSink implements DynamicTableSink, SupportsPartitioning, SupportsOverwrite { + private final TableLoader tableLoader; + private final TableSchema tableSchema; + private final ReadableConfig readableConfig; + private final Map writeProps; + + private boolean overwrite = false; + + private IcebergTableSink(IcebergTableSink toCopy) { + this.tableLoader = toCopy.tableLoader; + this.tableSchema = toCopy.tableSchema; + this.overwrite = toCopy.overwrite; + this.readableConfig = toCopy.readableConfig; + this.writeProps = toCopy.writeProps; + } + + public IcebergTableSink( + TableLoader tableLoader, + TableSchema tableSchema, + ReadableConfig readableConfig, + Map writeProps) { + this.tableLoader = tableLoader; + this.tableSchema = tableSchema; + this.readableConfig = readableConfig; + this.writeProps = writeProps; + } + + @Override + public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { + Preconditions.checkState( + !overwrite || context.isBounded(), + "Unbounded data stream doesn't support overwrite operation."); + + List equalityColumns = + tableSchema.getPrimaryKey().map(UniqueConstraint::getColumns).orElseGet(ImmutableList::of); + + return new DataStreamSinkProvider() { + @Override + public DataStreamSink consumeDataStream( + ProviderContext providerContext, DataStream dataStream) { + return FlinkSink.forRowData(dataStream) + .tableLoader(tableLoader) + .tableSchema(tableSchema) + .equalityFieldColumns(equalityColumns) + .overwrite(overwrite) + .setAll(writeProps) + .flinkConf(readableConfig) + .append(); + } + }; + } + + @Override + public void applyStaticPartition(Map partition) { + // The flink's PartitionFanoutWriter will handle the static partition write policy + // automatically. + } + + @Override + public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { + ChangelogMode.Builder builder = ChangelogMode.newBuilder(); + for (RowKind kind : requestedMode.getContainedKinds()) { + builder.addContainedKind(kind); + } + return builder.build(); + } + + @Override + public DynamicTableSink copy() { + return new IcebergTableSink(this); + } + + @Override + public String asSummaryString() { + return "Iceberg table sink"; + } + + @Override + public void applyOverwrite(boolean newOverwrite) { + this.overwrite = newOverwrite; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java new file mode 100644 index 000000000000..d4cec7a3e80b --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java @@ -0,0 +1,142 @@ +/* + * 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.iceberg.flink; + +import java.lang.reflect.Array; +import java.nio.ByteBuffer; +import java.time.LocalDateTime; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.LocalZonedTimestampType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.TimestampType; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.DateTimeUtil; +import org.apache.iceberg.util.UUIDUtil; + +public class RowDataWrapper implements StructLike { + + private final LogicalType[] types; + private final PositionalGetter[] getters; + private RowData rowData = null; + + public RowDataWrapper(RowType rowType, Types.StructType struct) { + int size = rowType.getFieldCount(); + + types = (LogicalType[]) Array.newInstance(LogicalType.class, size); + getters = (PositionalGetter[]) Array.newInstance(PositionalGetter.class, size); + + for (int i = 0; i < size; i++) { + types[i] = rowType.getTypeAt(i); + getters[i] = buildGetter(types[i], struct.fields().get(i).type()); + } + } + + public RowDataWrapper wrap(RowData data) { + this.rowData = data; + return this; + } + + @Override + public int size() { + return types.length; + } + + @Override + public T get(int pos, Class javaClass) { + if (rowData.isNullAt(pos)) { + return null; + } else if (getters[pos] != null) { + return javaClass.cast(getters[pos].get(rowData, pos)); + } + + Object value = RowData.createFieldGetter(types[pos], pos).getFieldOrNull(rowData); + return javaClass.cast(value); + } + + @Override + public void set(int pos, T value) { + throw new UnsupportedOperationException( + "Could not set a field in the RowDataWrapper because rowData is read-only"); + } + + private interface PositionalGetter { + T get(RowData data, int pos); + } + + private static PositionalGetter buildGetter(LogicalType logicalType, Type type) { + switch (logicalType.getTypeRoot()) { + case TINYINT: + return (row, pos) -> (int) row.getByte(pos); + case SMALLINT: + return (row, pos) -> (int) row.getShort(pos); + case CHAR: + case VARCHAR: + return (row, pos) -> row.getString(pos).toString(); + + case BINARY: + case VARBINARY: + if (Type.TypeID.UUID == type.typeId()) { + return (row, pos) -> UUIDUtil.convert(row.getBinary(pos)); + } else { + return (row, pos) -> ByteBuffer.wrap(row.getBinary(pos)); + } + + case DECIMAL: + DecimalType decimalType = (DecimalType) logicalType; + return (row, pos) -> + row.getDecimal(pos, decimalType.getPrecision(), decimalType.getScale()).toBigDecimal(); + + case TIME_WITHOUT_TIME_ZONE: + // Time in RowData is in milliseconds (Integer), while iceberg's time is microseconds + // (Long). + return (row, pos) -> ((long) row.getInt(pos)) * 1_000; + + case TIMESTAMP_WITHOUT_TIME_ZONE: + TimestampType timestampType = (TimestampType) logicalType; + return (row, pos) -> { + LocalDateTime localDateTime = + row.getTimestamp(pos, timestampType.getPrecision()).toLocalDateTime(); + return DateTimeUtil.microsFromTimestamp(localDateTime); + }; + + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + LocalZonedTimestampType lzTs = (LocalZonedTimestampType) logicalType; + return (row, pos) -> { + TimestampData timestampData = row.getTimestamp(pos, lzTs.getPrecision()); + return timestampData.getMillisecond() * 1000 + + timestampData.getNanoOfMillisecond() / 1000; + }; + + case ROW: + RowType rowType = (RowType) logicalType; + Types.StructType structType = (Types.StructType) type; + + RowDataWrapper nestedWrapper = new RowDataWrapper(rowType, structType); + return (row, pos) -> nestedWrapper.wrap(row.getRow(pos, rowType.getFieldCount())); + + default: + return null; + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java new file mode 100644 index 000000000000..da509451fee7 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java @@ -0,0 +1,159 @@ +/* + * 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.iceberg.flink; + +import java.io.Closeable; +import java.io.IOException; +import java.io.Serializable; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.hadoop.SerializableConfiguration; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +/** + * Serializable loader to load an Iceberg {@link Table}. Flink needs to get {@link Table} objects in + * the cluster (for example, to get splits), not just on the client side. So we need an Iceberg + * table loader to get the {@link Table} object. + */ +public interface TableLoader extends Closeable, Serializable, Cloneable { + + void open(); + + boolean isOpen(); + + Table loadTable(); + + /** Clone a TableLoader */ + @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) + TableLoader clone(); + + static TableLoader fromCatalog(CatalogLoader catalogLoader, TableIdentifier identifier) { + return new CatalogTableLoader(catalogLoader, identifier); + } + + static TableLoader fromHadoopTable(String location) { + return fromHadoopTable(location, FlinkCatalogFactory.clusterHadoopConf()); + } + + static TableLoader fromHadoopTable(String location, Configuration hadoopConf) { + return new HadoopTableLoader(location, hadoopConf); + } + + class HadoopTableLoader implements TableLoader { + + private static final long serialVersionUID = 1L; + + private final String location; + private final SerializableConfiguration hadoopConf; + + private transient HadoopTables tables; + + private HadoopTableLoader(String location, Configuration conf) { + this.location = location; + this.hadoopConf = new SerializableConfiguration(conf); + } + + @Override + public void open() { + tables = new HadoopTables(hadoopConf.get()); + } + + @Override + public boolean isOpen() { + return tables != null; + } + + @Override + public Table loadTable() { + FlinkEnvironmentContext.init(); + return tables.load(location); + } + + @Override + @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) + public TableLoader clone() { + return new HadoopTableLoader(location, new Configuration(hadoopConf.get())); + } + + @Override + public void close() {} + + @Override + public String toString() { + return MoreObjects.toStringHelper(this).add("location", location).toString(); + } + } + + class CatalogTableLoader implements TableLoader { + + private static final long serialVersionUID = 1L; + + private final CatalogLoader catalogLoader; + private final String identifier; + + private transient Catalog catalog; + + private CatalogTableLoader(CatalogLoader catalogLoader, TableIdentifier tableIdentifier) { + this.catalogLoader = catalogLoader; + this.identifier = tableIdentifier.toString(); + } + + @Override + public void open() { + catalog = catalogLoader.loadCatalog(); + } + + @Override + public boolean isOpen() { + return catalog != null; + } + + @Override + public Table loadTable() { + FlinkEnvironmentContext.init(); + return catalog.loadTable(TableIdentifier.parse(identifier)); + } + + @Override + public void close() throws IOException { + if (catalog instanceof Closeable) { + ((Closeable) catalog).close(); + } + + catalog = null; + } + + @Override + @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) + public TableLoader clone() { + return new CatalogTableLoader(catalogLoader.clone(), TableIdentifier.parse(identifier)); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("tableIdentifier", identifier) + .add("catalogLoader", catalogLoader) + .toString(); + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java new file mode 100644 index 000000000000..f8f1b74b1ceb --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java @@ -0,0 +1,134 @@ +/* + * 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.iceberg.flink; + +import java.util.List; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.table.types.logical.BinaryType; +import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.DateType; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.DoubleType; +import org.apache.flink.table.types.logical.FloatType; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.LocalZonedTimestampType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.TimeType; +import org.apache.flink.table.types.logical.TimestampType; +import org.apache.flink.table.types.logical.VarBinaryType; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; + +class TypeToFlinkType extends TypeUtil.SchemaVisitor { + TypeToFlinkType() {} + + @Override + public LogicalType schema(Schema schema, LogicalType structType) { + return structType; + } + + @Override + public LogicalType struct(Types.StructType struct, List fieldResults) { + List fields = struct.fields(); + + List flinkFields = Lists.newArrayListWithExpectedSize(fieldResults.size()); + for (int i = 0; i < fields.size(); i += 1) { + Types.NestedField field = fields.get(i); + LogicalType type = fieldResults.get(i); + RowType.RowField flinkField = + new RowType.RowField(field.name(), type.copy(field.isOptional()), field.doc()); + flinkFields.add(flinkField); + } + + return new RowType(flinkFields); + } + + @Override + public LogicalType field(Types.NestedField field, LogicalType fieldResult) { + return fieldResult; + } + + @Override + public LogicalType list(Types.ListType list, LogicalType elementResult) { + return new ArrayType(elementResult.copy(list.isElementOptional())); + } + + @Override + public LogicalType map(Types.MapType map, LogicalType keyResult, LogicalType valueResult) { + // keys in map are not allowed to be null. + return new MapType(keyResult.copy(false), valueResult.copy(map.isValueOptional())); + } + + @Override + public LogicalType primitive(Type.PrimitiveType primitive) { + switch (primitive.typeId()) { + case BOOLEAN: + return new BooleanType(); + case INTEGER: + return new IntType(); + case LONG: + return new BigIntType(); + case FLOAT: + return new FloatType(); + case DOUBLE: + return new DoubleType(); + case DATE: + return new DateType(); + case TIME: + // For the type: Flink only support TimeType with default precision (second) now. The + // precision of time is + // not supported in Flink, so we can think of it as a simple time type directly. + // For the data: Flink uses int that support mills to represent time data, so it supports + // mills precision. + return new TimeType(); + case TIMESTAMP: + Types.TimestampType timestamp = (Types.TimestampType) primitive; + if (timestamp.shouldAdjustToUTC()) { + // MICROS + return new LocalZonedTimestampType(6); + } else { + // MICROS + return new TimestampType(6); + } + case STRING: + return new VarCharType(VarCharType.MAX_LENGTH); + case UUID: + // UUID length is 16 + return new BinaryType(16); + case FIXED: + Types.FixedType fixedType = (Types.FixedType) primitive; + return new BinaryType(fixedType.length()); + case BINARY: + return new VarBinaryType(VarBinaryType.MAX_LENGTH); + case DECIMAL: + Types.DecimalType decimal = (Types.DecimalType) primitive; + return new DecimalType(decimal.precision(), decimal.scale()); + default: + throw new UnsupportedOperationException( + "Cannot convert unknown type to Flink: " + primitive); + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java new file mode 100644 index 000000000000..06ac54617ae6 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.actions; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.iceberg.Table; + +public class Actions { + + public static final Configuration CONFIG = + new Configuration() + // disable classloader check as Avro may cache class/object in the serializers. + .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); + + private StreamExecutionEnvironment env; + private Table table; + + private Actions(StreamExecutionEnvironment env, Table table) { + this.env = env; + this.table = table; + } + + public static Actions forTable(StreamExecutionEnvironment env, Table table) { + return new Actions(env, table); + } + + public static Actions forTable(Table table) { + return new Actions(StreamExecutionEnvironment.getExecutionEnvironment(CONFIG), table); + } + + public RewriteDataFilesAction rewriteDataFiles() { + return new RewriteDataFilesAction(env, table); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java new file mode 100644 index 000000000000..9876bb3861c4 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.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.iceberg.flink.actions; + +import java.util.List; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.BaseRewriteDataFilesAction; +import org.apache.iceberg.flink.source.RowDataRewriter; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +public class RewriteDataFilesAction extends BaseRewriteDataFilesAction { + + private StreamExecutionEnvironment env; + private int maxParallelism; + + public RewriteDataFilesAction(StreamExecutionEnvironment env, Table table) { + super(table); + this.env = env; + this.maxParallelism = env.getParallelism(); + } + + @Override + protected FileIO fileIO() { + return table().io(); + } + + @Override + protected List rewriteDataForTasks(List combinedScanTasks) { + int size = combinedScanTasks.size(); + int parallelism = Math.min(size, maxParallelism); + DataStream dataStream = env.fromCollection(combinedScanTasks); + RowDataRewriter rowDataRewriter = + new RowDataRewriter(table(), caseSensitive(), fileIO(), encryptionManager()); + try { + return rowDataRewriter.rewriteDataForTasks(dataStream, parallelism); + } catch (Exception e) { + throw new RuntimeException("Rewrite data file error.", e); + } + } + + @Override + protected RewriteDataFilesAction self() { + return this; + } + + public RewriteDataFilesAction maxParallelism(int parallelism) { + Preconditions.checkArgument(parallelism > 0, "Invalid max parallelism %s", parallelism); + this.maxParallelism = parallelism; + return this; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java new file mode 100644 index 000000000000..8103224a0b6c --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java @@ -0,0 +1,75 @@ +/* + * 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.iceberg.flink.data; + +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeFamily; +import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.NullType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.avro.AvroWithPartnerByStructureVisitor; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.util.Pair; + +public abstract class AvroWithFlinkSchemaVisitor + extends AvroWithPartnerByStructureVisitor { + + @Override + protected boolean isStringType(LogicalType logicalType) { + return logicalType.getTypeRoot().getFamilies().contains(LogicalTypeFamily.CHARACTER_STRING); + } + + @Override + protected boolean isMapType(LogicalType logicalType) { + return logicalType instanceof MapType; + } + + @Override + protected LogicalType arrayElementType(LogicalType arrayType) { + Preconditions.checkArgument( + arrayType instanceof ArrayType, "Invalid array: %s is not an array", arrayType); + return ((ArrayType) arrayType).getElementType(); + } + + @Override + protected LogicalType mapKeyType(LogicalType mapType) { + Preconditions.checkArgument(isMapType(mapType), "Invalid map: %s is not a map", mapType); + return ((MapType) mapType).getKeyType(); + } + + @Override + protected LogicalType mapValueType(LogicalType mapType) { + Preconditions.checkArgument(isMapType(mapType), "Invalid map: %s is not a map", mapType); + return ((MapType) mapType).getValueType(); + } + + @Override + protected Pair fieldNameAndType(LogicalType structType, int pos) { + Preconditions.checkArgument( + structType instanceof RowType, "Invalid struct: %s is not a struct", structType); + RowType.RowField field = ((RowType) structType).getFields().get(pos); + return Pair.of(field.getName(), field.getType()); + } + + @Override + protected LogicalType nullType() { + return new NullType(); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java new file mode 100644 index 000000000000..86404959735a --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.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.iceberg.flink.data; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.function.Supplier; +import org.apache.avro.LogicalType; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.io.DatumReader; +import org.apache.avro.io.Decoder; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.avro.AvroSchemaWithTypeVisitor; +import org.apache.iceberg.avro.SupportsRowPosition; +import org.apache.iceberg.avro.ValueReader; +import org.apache.iceberg.avro.ValueReaders; +import org.apache.iceberg.data.avro.DecoderResolver; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +public class FlinkAvroReader implements DatumReader, SupportsRowPosition { + + private final Schema readSchema; + private final ValueReader reader; + private Schema fileSchema = null; + + public FlinkAvroReader(org.apache.iceberg.Schema expectedSchema, Schema readSchema) { + this(expectedSchema, readSchema, ImmutableMap.of()); + } + + @SuppressWarnings("unchecked") + public FlinkAvroReader( + org.apache.iceberg.Schema expectedSchema, Schema readSchema, Map constants) { + this.readSchema = readSchema; + this.reader = + (ValueReader) + AvroSchemaWithTypeVisitor.visit(expectedSchema, readSchema, new ReadBuilder(constants)); + } + + @Override + public void setSchema(Schema newFileSchema) { + this.fileSchema = Schema.applyAliases(newFileSchema, readSchema); + } + + @Override + public RowData read(RowData reuse, Decoder decoder) throws IOException { + return DecoderResolver.resolveAndRead(decoder, readSchema, fileSchema, reader, reuse); + } + + @Override + public void setRowPositionSupplier(Supplier posSupplier) { + if (reader instanceof SupportsRowPosition) { + ((SupportsRowPosition) reader).setRowPositionSupplier(posSupplier); + } + } + + private static class ReadBuilder extends AvroSchemaWithTypeVisitor> { + private final Map idToConstant; + + private ReadBuilder(Map idToConstant) { + this.idToConstant = idToConstant; + } + + @Override + public ValueReader record( + Types.StructType expected, Schema record, List names, List> fields) { + return FlinkValueReaders.struct(fields, expected.asStructType(), idToConstant); + } + + @Override + public ValueReader union(Type expected, Schema union, List> options) { + return ValueReaders.union(options); + } + + @Override + public ValueReader array( + Types.ListType expected, Schema array, ValueReader elementReader) { + return FlinkValueReaders.array(elementReader); + } + + @Override + public ValueReader map( + Types.MapType expected, Schema map, ValueReader keyReader, ValueReader valueReader) { + return FlinkValueReaders.arrayMap(keyReader, valueReader); + } + + @Override + public ValueReader map(Types.MapType expected, Schema map, ValueReader valueReader) { + return FlinkValueReaders.map(FlinkValueReaders.strings(), valueReader); + } + + @Override + public ValueReader primitive(Type.PrimitiveType expected, Schema primitive) { + LogicalType logicalType = primitive.getLogicalType(); + if (logicalType != null) { + switch (logicalType.getName()) { + case "date": + return ValueReaders.ints(); + + case "time-micros": + return FlinkValueReaders.timeMicros(); + + case "timestamp-millis": + return FlinkValueReaders.timestampMills(); + + case "timestamp-micros": + return FlinkValueReaders.timestampMicros(); + + case "decimal": + LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) logicalType; + return FlinkValueReaders.decimal( + ValueReaders.decimalBytesReader(primitive), + decimal.getPrecision(), + decimal.getScale()); + + case "uuid": + return FlinkValueReaders.uuids(); + + default: + throw new IllegalArgumentException("Unknown logical type: " + logicalType); + } + } + + switch (primitive.getType()) { + case NULL: + return ValueReaders.nulls(); + case BOOLEAN: + return ValueReaders.booleans(); + case INT: + return ValueReaders.ints(); + case LONG: + return ValueReaders.longs(); + case FLOAT: + return ValueReaders.floats(); + case DOUBLE: + return ValueReaders.doubles(); + case STRING: + return FlinkValueReaders.strings(); + case FIXED: + return ValueReaders.fixed(primitive.getFixedSize()); + case BYTES: + return ValueReaders.bytes(); + case ENUM: + return FlinkValueReaders.enums(primitive.getEnumSymbols()); + default: + throw new IllegalArgumentException("Unsupported type: " + primitive); + } + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java new file mode 100644 index 000000000000..873e65783119 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java @@ -0,0 +1,165 @@ +/* + * 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.iceberg.flink.data; + +import java.io.IOException; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.io.Encoder; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.FieldMetrics; +import org.apache.iceberg.avro.MetricsAwareDatumWriter; +import org.apache.iceberg.avro.ValueWriter; +import org.apache.iceberg.avro.ValueWriters; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +public class FlinkAvroWriter implements MetricsAwareDatumWriter { + private final RowType rowType; + private ValueWriter writer = null; + + public FlinkAvroWriter(RowType rowType) { + this.rowType = rowType; + } + + @Override + @SuppressWarnings("unchecked") + public void setSchema(Schema schema) { + this.writer = + (ValueWriter) + AvroWithFlinkSchemaVisitor.visit(rowType, schema, new WriteBuilder()); + } + + @Override + public void write(RowData datum, Encoder out) throws IOException { + writer.write(datum, out); + } + + @Override + public Stream metrics() { + return writer.metrics(); + } + + private static class WriteBuilder extends AvroWithFlinkSchemaVisitor> { + @Override + public ValueWriter record( + LogicalType struct, Schema record, List names, List> fields) { + return FlinkValueWriters.row( + fields, + IntStream.range(0, names.size()) + .mapToObj(i -> fieldNameAndType(struct, i).second()) + .collect(Collectors.toList())); + } + + @Override + public ValueWriter union(LogicalType type, Schema union, List> options) { + Preconditions.checkArgument( + options.contains(ValueWriters.nulls()), + "Cannot create writer for non-option union: %s", + union); + Preconditions.checkArgument( + options.size() == 2, "Cannot create writer for non-option union: %s", union); + if (union.getTypes().get(0).getType() == Schema.Type.NULL) { + return ValueWriters.option(0, options.get(1)); + } else { + return ValueWriters.option(1, options.get(0)); + } + } + + @Override + public ValueWriter array(LogicalType sArray, Schema array, ValueWriter elementWriter) { + return FlinkValueWriters.array(elementWriter, arrayElementType(sArray)); + } + + @Override + public ValueWriter map(LogicalType sMap, Schema map, ValueWriter valueReader) { + return FlinkValueWriters.map( + FlinkValueWriters.strings(), mapKeyType(sMap), valueReader, mapValueType(sMap)); + } + + @Override + public ValueWriter map( + LogicalType sMap, Schema map, ValueWriter keyWriter, ValueWriter valueWriter) { + return FlinkValueWriters.arrayMap( + keyWriter, mapKeyType(sMap), valueWriter, mapValueType(sMap)); + } + + @Override + public ValueWriter primitive(LogicalType type, Schema primitive) { + org.apache.avro.LogicalType logicalType = primitive.getLogicalType(); + if (logicalType != null) { + switch (logicalType.getName()) { + case "date": + return ValueWriters.ints(); + + case "time-micros": + return FlinkValueWriters.timeMicros(); + + case "timestamp-micros": + return FlinkValueWriters.timestampMicros(); + + case "decimal": + LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) logicalType; + return FlinkValueWriters.decimal(decimal.getPrecision(), decimal.getScale()); + + case "uuid": + return ValueWriters.uuids(); + + default: + throw new IllegalArgumentException("Unsupported logical type: " + logicalType); + } + } + + switch (primitive.getType()) { + case NULL: + return ValueWriters.nulls(); + case BOOLEAN: + return ValueWriters.booleans(); + case INT: + switch (type.getTypeRoot()) { + case TINYINT: + return ValueWriters.tinyints(); + case SMALLINT: + return ValueWriters.shorts(); + default: + return ValueWriters.ints(); + } + case LONG: + return ValueWriters.longs(); + case FLOAT: + return ValueWriters.floats(); + case DOUBLE: + return ValueWriters.doubles(); + case STRING: + return FlinkValueWriters.strings(); + case FIXED: + return ValueWriters.fixed(primitive.getFixedSize()); + case BYTES: + return ValueWriters.bytes(); + default: + throw new IllegalArgumentException("Unsupported type: " + primitive); + } + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java new file mode 100644 index 000000000000..65b9d44ad4b8 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java @@ -0,0 +1,131 @@ +/* + * 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.iceberg.flink.data; + +import java.util.List; +import java.util.Map; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.orc.OrcRowReader; +import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor; +import org.apache.iceberg.orc.OrcValueReader; +import org.apache.iceberg.orc.OrcValueReaders; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.orc.TypeDescription; +import org.apache.orc.storage.ql.exec.vector.StructColumnVector; +import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; + +public class FlinkOrcReader implements OrcRowReader { + private final OrcValueReader reader; + + public FlinkOrcReader(Schema iSchema, TypeDescription readSchema) { + this(iSchema, readSchema, ImmutableMap.of()); + } + + public FlinkOrcReader(Schema iSchema, TypeDescription readSchema, Map idToConstant) { + this.reader = + OrcSchemaWithTypeVisitor.visit(iSchema, readSchema, new ReadBuilder(idToConstant)); + } + + @Override + public RowData read(VectorizedRowBatch batch, int row) { + return (RowData) reader.read(new StructColumnVector(batch.size, batch.cols), row); + } + + @Override + public void setBatchContext(long batchOffsetInFile) { + reader.setBatchContext(batchOffsetInFile); + } + + private static class ReadBuilder extends OrcSchemaWithTypeVisitor> { + private final Map idToConstant; + + private ReadBuilder(Map idToConstant) { + this.idToConstant = idToConstant; + } + + @Override + public OrcValueReader record( + Types.StructType iStruct, + TypeDescription record, + List names, + List> fields) { + return FlinkOrcReaders.struct(fields, iStruct, idToConstant); + } + + @Override + public OrcValueReader list( + Types.ListType iList, TypeDescription array, OrcValueReader elementReader) { + return FlinkOrcReaders.array(elementReader); + } + + @Override + public OrcValueReader map( + Types.MapType iMap, + TypeDescription map, + OrcValueReader keyReader, + OrcValueReader valueReader) { + return FlinkOrcReaders.map(keyReader, valueReader); + } + + @Override + public OrcValueReader primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) { + switch (iPrimitive.typeId()) { + case BOOLEAN: + return OrcValueReaders.booleans(); + case INTEGER: + return OrcValueReaders.ints(); + case LONG: + return OrcValueReaders.longs(); + case FLOAT: + return OrcValueReaders.floats(); + case DOUBLE: + return OrcValueReaders.doubles(); + case DATE: + return FlinkOrcReaders.dates(); + case TIME: + return FlinkOrcReaders.times(); + case TIMESTAMP: + Types.TimestampType timestampType = (Types.TimestampType) iPrimitive; + if (timestampType.shouldAdjustToUTC()) { + return FlinkOrcReaders.timestampTzs(); + } else { + return FlinkOrcReaders.timestamps(); + } + case STRING: + return FlinkOrcReaders.strings(); + case UUID: + case FIXED: + case BINARY: + return OrcValueReaders.bytes(); + case DECIMAL: + Types.DecimalType decimalType = (Types.DecimalType) iPrimitive; + return FlinkOrcReaders.decimals(decimalType.precision(), decimalType.scale()); + default: + throw new IllegalArgumentException( + String.format( + "Invalid iceberg type %s corresponding to ORC type %s", iPrimitive, primitive)); + } + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java new file mode 100644 index 000000000000..7a4a15c7e600 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java @@ -0,0 +1,283 @@ +/* + * 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.iceberg.flink.data; + +import java.math.BigDecimal; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneOffset; +import java.util.List; +import java.util.Map; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.iceberg.orc.OrcValueReader; +import org.apache.iceberg.orc.OrcValueReaders; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; +import org.apache.orc.storage.ql.exec.vector.BytesColumnVector; +import org.apache.orc.storage.ql.exec.vector.ColumnVector; +import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector; +import org.apache.orc.storage.ql.exec.vector.ListColumnVector; +import org.apache.orc.storage.ql.exec.vector.LongColumnVector; +import org.apache.orc.storage.ql.exec.vector.MapColumnVector; +import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector; +import org.apache.orc.storage.serde2.io.HiveDecimalWritable; + +class FlinkOrcReaders { + private FlinkOrcReaders() {} + + static OrcValueReader strings() { + return StringReader.INSTANCE; + } + + static OrcValueReader dates() { + return DateReader.INSTANCE; + } + + static OrcValueReader decimals(int precision, int scale) { + if (precision <= 18) { + return new Decimal18Reader(precision, scale); + } else if (precision <= 38) { + return new Decimal38Reader(precision, scale); + } else { + throw new IllegalArgumentException("Invalid precision: " + precision); + } + } + + static OrcValueReader times() { + return TimeReader.INSTANCE; + } + + static OrcValueReader timestamps() { + return TimestampReader.INSTANCE; + } + + static OrcValueReader timestampTzs() { + return TimestampTzReader.INSTANCE; + } + + static OrcValueReader array(OrcValueReader elementReader) { + return new ArrayReader<>(elementReader); + } + + public static OrcValueReader map( + OrcValueReader keyReader, OrcValueReader valueReader) { + return new MapReader<>(keyReader, valueReader); + } + + public static OrcValueReader struct( + List> readers, Types.StructType struct, Map idToConstant) { + return new StructReader(readers, struct, idToConstant); + } + + private static class StringReader implements OrcValueReader { + private static final StringReader INSTANCE = new StringReader(); + + @Override + public StringData nonNullRead(ColumnVector vector, int row) { + BytesColumnVector bytesVector = (BytesColumnVector) vector; + return StringData.fromBytes( + bytesVector.vector[row], bytesVector.start[row], bytesVector.length[row]); + } + } + + private static class DateReader implements OrcValueReader { + private static final DateReader INSTANCE = new DateReader(); + + @Override + public Integer nonNullRead(ColumnVector vector, int row) { + return (int) ((LongColumnVector) vector).vector[row]; + } + } + + private static class Decimal18Reader implements OrcValueReader { + private final int precision; + private final int scale; + + Decimal18Reader(int precision, int scale) { + this.precision = precision; + this.scale = scale; + } + + @Override + public DecimalData nonNullRead(ColumnVector vector, int row) { + HiveDecimalWritable value = ((DecimalColumnVector) vector).vector[row]; + + // The hive ORC writer may will adjust the scale of decimal data. + Preconditions.checkArgument( + value.precision() <= precision, + "Cannot read value as decimal(%s,%s), too large: %s", + precision, + scale, + value); + + return DecimalData.fromUnscaledLong(value.serialize64(scale), precision, scale); + } + } + + private static class Decimal38Reader implements OrcValueReader { + private final int precision; + private final int scale; + + Decimal38Reader(int precision, int scale) { + this.precision = precision; + this.scale = scale; + } + + @Override + public DecimalData nonNullRead(ColumnVector vector, int row) { + BigDecimal value = + ((DecimalColumnVector) vector).vector[row].getHiveDecimal().bigDecimalValue(); + + Preconditions.checkArgument( + value.precision() <= precision, + "Cannot read value as decimal(%s,%s), too large: %s", + precision, + scale, + value); + + return DecimalData.fromBigDecimal(value, precision, scale); + } + } + + private static class TimeReader implements OrcValueReader { + private static final TimeReader INSTANCE = new TimeReader(); + + @Override + public Integer nonNullRead(ColumnVector vector, int row) { + long micros = ((LongColumnVector) vector).vector[row]; + // Flink only support time mills, just erase micros. + return (int) (micros / 1000); + } + } + + private static class TimestampReader implements OrcValueReader { + private static final TimestampReader INSTANCE = new TimestampReader(); + + @Override + public TimestampData nonNullRead(ColumnVector vector, int row) { + TimestampColumnVector tcv = (TimestampColumnVector) vector; + LocalDateTime localDate = + Instant.ofEpochSecond(Math.floorDiv(tcv.time[row], 1_000), tcv.nanos[row]) + .atOffset(ZoneOffset.UTC) + .toLocalDateTime(); + return TimestampData.fromLocalDateTime(localDate); + } + } + + private static class TimestampTzReader implements OrcValueReader { + private static final TimestampTzReader INSTANCE = new TimestampTzReader(); + + @Override + public TimestampData nonNullRead(ColumnVector vector, int row) { + TimestampColumnVector tcv = (TimestampColumnVector) vector; + Instant instant = + Instant.ofEpochSecond(Math.floorDiv(tcv.time[row], 1_000), tcv.nanos[row]) + .atOffset(ZoneOffset.UTC) + .toInstant(); + return TimestampData.fromInstant(instant); + } + } + + private static class ArrayReader implements OrcValueReader { + private final OrcValueReader elementReader; + + private ArrayReader(OrcValueReader elementReader) { + this.elementReader = elementReader; + } + + @Override + public ArrayData nonNullRead(ColumnVector vector, int row) { + ListColumnVector listVector = (ListColumnVector) vector; + int offset = (int) listVector.offsets[row]; + int length = (int) listVector.lengths[row]; + List elements = Lists.newArrayListWithExpectedSize(length); + for (int c = 0; c < length; ++c) { + elements.add(elementReader.read(listVector.child, offset + c)); + } + return new GenericArrayData(elements.toArray()); + } + + @Override + public void setBatchContext(long batchOffsetInFile) { + elementReader.setBatchContext(batchOffsetInFile); + } + } + + private static class MapReader implements OrcValueReader { + private final OrcValueReader keyReader; + private final OrcValueReader valueReader; + + private MapReader(OrcValueReader keyReader, OrcValueReader valueReader) { + this.keyReader = keyReader; + this.valueReader = valueReader; + } + + @Override + public MapData nonNullRead(ColumnVector vector, int row) { + MapColumnVector mapVector = (MapColumnVector) vector; + int offset = (int) mapVector.offsets[row]; + long length = mapVector.lengths[row]; + + Map map = Maps.newHashMap(); + for (int c = 0; c < length; c++) { + K key = keyReader.read(mapVector.keys, offset + c); + V value = valueReader.read(mapVector.values, offset + c); + map.put(key, value); + } + + return new GenericMapData(map); + } + + @Override + public void setBatchContext(long batchOffsetInFile) { + keyReader.setBatchContext(batchOffsetInFile); + valueReader.setBatchContext(batchOffsetInFile); + } + } + + private static class StructReader extends OrcValueReaders.StructReader { + private final int numFields; + + StructReader( + List> readers, Types.StructType struct, Map idToConstant) { + super(readers, struct, idToConstant); + this.numFields = struct.fields().size(); + } + + @Override + protected RowData create() { + return new GenericRowData(numFields); + } + + @Override + protected void set(RowData struct, int pos, Object value) { + ((GenericRowData) struct).setField(pos, value); + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java new file mode 100644 index 000000000000..6a31accffd22 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java @@ -0,0 +1,163 @@ +/* + * 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.iceberg.flink.data; + +import java.util.Deque; +import java.util.List; +import java.util.stream.Stream; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.FieldMetrics; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.orc.GenericOrcWriters; +import org.apache.iceberg.orc.OrcRowWriter; +import org.apache.iceberg.orc.OrcValueWriter; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; + +public class FlinkOrcWriter implements OrcRowWriter { + private final FlinkOrcWriters.RowDataWriter writer; + + private FlinkOrcWriter(RowType rowType, Schema iSchema) { + this.writer = + (FlinkOrcWriters.RowDataWriter) + FlinkSchemaVisitor.visit(rowType, iSchema, new WriteBuilder()); + } + + public static OrcRowWriter buildWriter(RowType rowType, Schema iSchema) { + return new FlinkOrcWriter(rowType, iSchema); + } + + @Override + public void write(RowData row, VectorizedRowBatch output) { + Preconditions.checkArgument(row != null, "value must not be null"); + writer.writeRow(row, output); + } + + @Override + public List> writers() { + return writer.writers(); + } + + @Override + public Stream> metrics() { + return writer.metrics(); + } + + private static class WriteBuilder extends FlinkSchemaVisitor> { + private final Deque fieldIds = Lists.newLinkedList(); + + private WriteBuilder() {} + + @Override + public void beforeField(Types.NestedField field) { + fieldIds.push(field.fieldId()); + } + + @Override + public void afterField(Types.NestedField field) { + fieldIds.pop(); + } + + @Override + public OrcValueWriter record( + Types.StructType iStruct, List> results, List fieldType) { + return FlinkOrcWriters.struct(results, fieldType); + } + + @Override + public OrcValueWriter map( + Types.MapType iMap, + OrcValueWriter key, + OrcValueWriter value, + LogicalType keyType, + LogicalType valueType) { + return FlinkOrcWriters.map(key, value, keyType, valueType); + } + + @Override + public OrcValueWriter list( + Types.ListType iList, OrcValueWriter element, LogicalType elementType) { + return FlinkOrcWriters.list(element, elementType); + } + + @Override + public OrcValueWriter primitive(Type.PrimitiveType iPrimitive, LogicalType flinkPrimitive) { + switch (iPrimitive.typeId()) { + case BOOLEAN: + return GenericOrcWriters.booleans(); + case INTEGER: + switch (flinkPrimitive.getTypeRoot()) { + case TINYINT: + return GenericOrcWriters.bytes(); + case SMALLINT: + return GenericOrcWriters.shorts(); + } + return GenericOrcWriters.ints(); + case LONG: + return GenericOrcWriters.longs(); + case FLOAT: + Preconditions.checkArgument( + fieldIds.peek() != null, + String.format( + "[BUG] Cannot find field id for primitive field with type %s. This is likely because id " + + "information is not properly pushed during schema visiting.", + iPrimitive)); + return GenericOrcWriters.floats(fieldIds.peek()); + case DOUBLE: + Preconditions.checkArgument( + fieldIds.peek() != null, + String.format( + "[BUG] Cannot find field id for primitive field with type %s. This is likely because id " + + "information is not properly pushed during schema visiting.", + iPrimitive)); + return GenericOrcWriters.doubles(fieldIds.peek()); + case DATE: + return FlinkOrcWriters.dates(); + case TIME: + return FlinkOrcWriters.times(); + case TIMESTAMP: + Types.TimestampType timestampType = (Types.TimestampType) iPrimitive; + if (timestampType.shouldAdjustToUTC()) { + return FlinkOrcWriters.timestampTzs(); + } else { + return FlinkOrcWriters.timestamps(); + } + case STRING: + return FlinkOrcWriters.strings(); + case UUID: + case FIXED: + case BINARY: + return GenericOrcWriters.byteArrays(); + case DECIMAL: + Types.DecimalType decimalType = (Types.DecimalType) iPrimitive; + return FlinkOrcWriters.decimals(decimalType.precision(), decimalType.scale()); + default: + throw new IllegalArgumentException( + String.format( + "Invalid iceberg type %s corresponding to Flink logical type %s", + iPrimitive, flinkPrimitive)); + } + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java new file mode 100644 index 000000000000..da2f95cf822f --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java @@ -0,0 +1,317 @@ +/* + * 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.iceberg.flink.data; + +import java.time.Instant; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.util.List; +import java.util.stream.Stream; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.iceberg.FieldMetrics; +import org.apache.iceberg.data.orc.GenericOrcWriters; +import org.apache.iceberg.orc.OrcValueWriter; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.orc.storage.common.type.HiveDecimal; +import org.apache.orc.storage.ql.exec.vector.BytesColumnVector; +import org.apache.orc.storage.ql.exec.vector.ColumnVector; +import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector; +import org.apache.orc.storage.ql.exec.vector.ListColumnVector; +import org.apache.orc.storage.ql.exec.vector.LongColumnVector; +import org.apache.orc.storage.ql.exec.vector.MapColumnVector; +import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector; + +class FlinkOrcWriters { + + private FlinkOrcWriters() {} + + static OrcValueWriter strings() { + return StringWriter.INSTANCE; + } + + static OrcValueWriter dates() { + return DateWriter.INSTANCE; + } + + static OrcValueWriter times() { + return TimeWriter.INSTANCE; + } + + static OrcValueWriter timestamps() { + return TimestampWriter.INSTANCE; + } + + static OrcValueWriter timestampTzs() { + return TimestampTzWriter.INSTANCE; + } + + static OrcValueWriter decimals(int precision, int scale) { + if (precision <= 18) { + return new Decimal18Writer(precision, scale); + } else if (precision <= 38) { + return new Decimal38Writer(precision, scale); + } else { + throw new IllegalArgumentException("Invalid precision: " + precision); + } + } + + static OrcValueWriter list( + OrcValueWriter elementWriter, LogicalType elementType) { + return new ListWriter<>(elementWriter, elementType); + } + + static OrcValueWriter map( + OrcValueWriter keyWriter, + OrcValueWriter valueWriter, + LogicalType keyType, + LogicalType valueType) { + return new MapWriter<>(keyWriter, valueWriter, keyType, valueType); + } + + static OrcValueWriter struct(List> writers, List types) { + return new RowDataWriter(writers, types); + } + + private static class StringWriter implements OrcValueWriter { + private static final StringWriter INSTANCE = new StringWriter(); + + @Override + public void nonNullWrite(int rowId, StringData data, ColumnVector output) { + byte[] value = data.toBytes(); + ((BytesColumnVector) output).setRef(rowId, value, 0, value.length); + } + } + + private static class DateWriter implements OrcValueWriter { + private static final DateWriter INSTANCE = new DateWriter(); + + @Override + public void nonNullWrite(int rowId, Integer data, ColumnVector output) { + ((LongColumnVector) output).vector[rowId] = data; + } + } + + private static class TimeWriter implements OrcValueWriter { + private static final TimeWriter INSTANCE = new TimeWriter(); + + @Override + public void nonNullWrite(int rowId, Integer millis, ColumnVector output) { + // The time in flink is in millisecond, while the standard time in iceberg is microsecond. + // So we need to transform it to microsecond. + ((LongColumnVector) output).vector[rowId] = millis * 1000L; + } + } + + private static class TimestampWriter implements OrcValueWriter { + private static final TimestampWriter INSTANCE = new TimestampWriter(); + + @Override + public void nonNullWrite(int rowId, TimestampData data, ColumnVector output) { + TimestampColumnVector cv = (TimestampColumnVector) output; + cv.setIsUTC(true); + // millis + OffsetDateTime offsetDateTime = data.toInstant().atOffset(ZoneOffset.UTC); + cv.time[rowId] = + offsetDateTime.toEpochSecond() * 1_000 + offsetDateTime.getNano() / 1_000_000; + // truncate nanos to only keep microsecond precision. + cv.nanos[rowId] = (offsetDateTime.getNano() / 1_000) * 1_000; + } + } + + private static class TimestampTzWriter implements OrcValueWriter { + private static final TimestampTzWriter INSTANCE = new TimestampTzWriter(); + + @SuppressWarnings("JavaInstantGetSecondsGetNano") + @Override + public void nonNullWrite(int rowId, TimestampData data, ColumnVector output) { + TimestampColumnVector cv = (TimestampColumnVector) output; + // millis + Instant instant = data.toInstant(); + cv.time[rowId] = instant.toEpochMilli(); + // truncate nanos to only keep microsecond precision. + cv.nanos[rowId] = (instant.getNano() / 1_000) * 1_000; + } + } + + private static class Decimal18Writer implements OrcValueWriter { + private final int precision; + private final int scale; + + Decimal18Writer(int precision, int scale) { + this.precision = precision; + this.scale = scale; + } + + @Override + public void nonNullWrite(int rowId, DecimalData data, ColumnVector output) { + Preconditions.checkArgument( + scale == data.scale(), + "Cannot write value as decimal(%s,%s), wrong scale: %s", + precision, + scale, + data); + Preconditions.checkArgument( + data.precision() <= precision, + "Cannot write value as decimal(%s,%s), too large: %s", + precision, + scale, + data); + + ((DecimalColumnVector) output) + .vector[rowId].setFromLongAndScale(data.toUnscaledLong(), data.scale()); + } + } + + private static class Decimal38Writer implements OrcValueWriter { + private final int precision; + private final int scale; + + Decimal38Writer(int precision, int scale) { + this.precision = precision; + this.scale = scale; + } + + @Override + public void nonNullWrite(int rowId, DecimalData data, ColumnVector output) { + Preconditions.checkArgument( + scale == data.scale(), + "Cannot write value as decimal(%s,%s), wrong scale: %s", + precision, + scale, + data); + Preconditions.checkArgument( + data.precision() <= precision, + "Cannot write value as decimal(%s,%s), too large: %s", + precision, + scale, + data); + + ((DecimalColumnVector) output) + .vector[rowId].set(HiveDecimal.create(data.toBigDecimal(), false)); + } + } + + static class ListWriter implements OrcValueWriter { + private final OrcValueWriter elementWriter; + private final ArrayData.ElementGetter elementGetter; + + ListWriter(OrcValueWriter elementWriter, LogicalType elementType) { + this.elementWriter = elementWriter; + this.elementGetter = ArrayData.createElementGetter(elementType); + } + + @Override + @SuppressWarnings("unchecked") + public void nonNullWrite(int rowId, ArrayData data, ColumnVector output) { + ListColumnVector cv = (ListColumnVector) output; + cv.lengths[rowId] = data.size(); + cv.offsets[rowId] = cv.childCount; + cv.childCount = (int) (cv.childCount + cv.lengths[rowId]); + // make sure the child is big enough. + growColumnVector(cv.child, cv.childCount); + + for (int e = 0; e < cv.lengths[rowId]; ++e) { + Object value = elementGetter.getElementOrNull(data, e); + elementWriter.write((int) (e + cv.offsets[rowId]), (T) value, cv.child); + } + } + + @Override + public Stream> metrics() { + return elementWriter.metrics(); + } + } + + static class MapWriter implements OrcValueWriter { + private final OrcValueWriter keyWriter; + private final OrcValueWriter valueWriter; + private final ArrayData.ElementGetter keyGetter; + private final ArrayData.ElementGetter valueGetter; + + MapWriter( + OrcValueWriter keyWriter, + OrcValueWriter valueWriter, + LogicalType keyType, + LogicalType valueType) { + this.keyWriter = keyWriter; + this.valueWriter = valueWriter; + this.keyGetter = ArrayData.createElementGetter(keyType); + this.valueGetter = ArrayData.createElementGetter(valueType); + } + + @Override + @SuppressWarnings("unchecked") + public void nonNullWrite(int rowId, MapData data, ColumnVector output) { + MapColumnVector cv = (MapColumnVector) output; + ArrayData keyArray = data.keyArray(); + ArrayData valArray = data.valueArray(); + + // record the length and start of the list elements + cv.lengths[rowId] = data.size(); + cv.offsets[rowId] = cv.childCount; + cv.childCount = (int) (cv.childCount + cv.lengths[rowId]); + // make sure the child is big enough + growColumnVector(cv.keys, cv.childCount); + growColumnVector(cv.values, cv.childCount); + // Add each element + for (int e = 0; e < cv.lengths[rowId]; ++e) { + int pos = (int) (e + cv.offsets[rowId]); + keyWriter.write(pos, (K) keyGetter.getElementOrNull(keyArray, e), cv.keys); + valueWriter.write(pos, (V) valueGetter.getElementOrNull(valArray, e), cv.values); + } + } + + @Override + public Stream> metrics() { + return Stream.concat(keyWriter.metrics(), valueWriter.metrics()); + } + } + + static class RowDataWriter extends GenericOrcWriters.StructWriter { + private final List fieldGetters; + + RowDataWriter(List> writers, List types) { + super(writers); + + this.fieldGetters = Lists.newArrayListWithExpectedSize(types.size()); + for (int i = 0; i < types.size(); i++) { + fieldGetters.add(RowData.createFieldGetter(types.get(i), i)); + } + } + + @Override + protected Object get(RowData struct, int index) { + return fieldGetters.get(index).getFieldOrNull(struct); + } + } + + private static void growColumnVector(ColumnVector cv, int requestedSize) { + if (cv.isNull.length < requestedSize) { + // Use growth factor of 3 to avoid frequent array allocations + cv.ensureSize(requestedSize * 3, true); + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java new file mode 100644 index 000000000000..ab7b1174c9f3 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java @@ -0,0 +1,832 @@ +/* + * 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.iceberg.flink.data; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.time.Instant; +import java.time.ZoneOffset; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RawValueData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Schema; +import org.apache.iceberg.parquet.ParquetSchemaUtil; +import org.apache.iceberg.parquet.ParquetValueReader; +import org.apache.iceberg.parquet.ParquetValueReaders; +import org.apache.iceberg.parquet.TypeWithSchemaVisitor; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.ArrayUtil; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; + +public class FlinkParquetReaders { + private FlinkParquetReaders() {} + + public static ParquetValueReader buildReader( + Schema expectedSchema, MessageType fileSchema) { + return buildReader(expectedSchema, fileSchema, ImmutableMap.of()); + } + + @SuppressWarnings("unchecked") + public static ParquetValueReader buildReader( + Schema expectedSchema, MessageType fileSchema, Map idToConstant) { + return (ParquetValueReader) + TypeWithSchemaVisitor.visit( + expectedSchema.asStruct(), fileSchema, new ReadBuilder(fileSchema, idToConstant)); + } + + private static class ReadBuilder extends TypeWithSchemaVisitor> { + private final MessageType type; + private final Map idToConstant; + + ReadBuilder(MessageType type, Map idToConstant) { + this.type = type; + this.idToConstant = idToConstant; + } + + @Override + public ParquetValueReader message( + Types.StructType expected, MessageType message, List> fieldReaders) { + return struct(expected, message.asGroupType(), fieldReaders); + } + + @Override + public ParquetValueReader struct( + Types.StructType expected, GroupType struct, List> fieldReaders) { + // match the expected struct's order + Map> readersById = Maps.newHashMap(); + Map typesById = Maps.newHashMap(); + Map maxDefinitionLevelsById = Maps.newHashMap(); + List fields = struct.getFields(); + for (int i = 0; i < fields.size(); i += 1) { + Type fieldType = fields.get(i); + if (fieldReaders.get(i) != null) { + int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1; + if (fieldType.getId() != null) { + int id = fieldType.getId().intValue(); + readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i))); + typesById.put(id, fieldType); + if (idToConstant.containsKey(id)) { + maxDefinitionLevelsById.put(id, fieldD); + } + } + } + } + + List expectedFields = + expected != null ? expected.fields() : ImmutableList.of(); + List> reorderedFields = + Lists.newArrayListWithExpectedSize(expectedFields.size()); + List types = Lists.newArrayListWithExpectedSize(expectedFields.size()); + // Defaulting to parent max definition level + int defaultMaxDefinitionLevel = type.getMaxDefinitionLevel(currentPath()); + for (Types.NestedField field : expectedFields) { + int id = field.fieldId(); + if (idToConstant.containsKey(id)) { + // containsKey is used because the constant may be null + int fieldMaxDefinitionLevel = + maxDefinitionLevelsById.getOrDefault(id, defaultMaxDefinitionLevel); + reorderedFields.add( + ParquetValueReaders.constant(idToConstant.get(id), fieldMaxDefinitionLevel)); + types.add(null); + } else if (id == MetadataColumns.ROW_POSITION.fieldId()) { + reorderedFields.add(ParquetValueReaders.position()); + types.add(null); + } else if (id == MetadataColumns.IS_DELETED.fieldId()) { + reorderedFields.add(ParquetValueReaders.constant(false)); + types.add(null); + } else { + ParquetValueReader reader = readersById.get(id); + if (reader != null) { + reorderedFields.add(reader); + types.add(typesById.get(id)); + } else { + reorderedFields.add(ParquetValueReaders.nulls()); + types.add(null); + } + } + } + + return new RowDataReader(types, reorderedFields); + } + + @Override + public ParquetValueReader list( + Types.ListType expectedList, GroupType array, ParquetValueReader elementReader) { + if (expectedList == null) { + return null; + } + + String[] repeatedPath = currentPath(); + + int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1; + int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1; + + Type elementType = ParquetSchemaUtil.determineListElementType(array); + int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1; + + return new ArrayReader<>( + repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader)); + } + + @Override + public ParquetValueReader map( + Types.MapType expectedMap, + GroupType map, + ParquetValueReader keyReader, + ParquetValueReader valueReader) { + if (expectedMap == null) { + return null; + } + + GroupType repeatedKeyValue = map.getFields().get(0).asGroupType(); + String[] repeatedPath = currentPath(); + + int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1; + int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1; + + Type keyType = repeatedKeyValue.getType(0); + int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1; + Type valueType = repeatedKeyValue.getType(1); + int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1; + + return new MapReader<>( + repeatedD, + repeatedR, + ParquetValueReaders.option(keyType, keyD, keyReader), + ParquetValueReaders.option(valueType, valueD, valueReader)); + } + + @Override + @SuppressWarnings("CyclomaticComplexity") + public ParquetValueReader primitive( + org.apache.iceberg.types.Type.PrimitiveType expected, PrimitiveType primitive) { + if (expected == null) { + return null; + } + + ColumnDescriptor desc = type.getColumnDescription(currentPath()); + + if (primitive.getOriginalType() != null) { + switch (primitive.getOriginalType()) { + case ENUM: + case JSON: + case UTF8: + return new StringReader(desc); + case INT_8: + case INT_16: + case INT_32: + if (expected.typeId() == Types.LongType.get().typeId()) { + return new ParquetValueReaders.IntAsLongReader(desc); + } else { + return new ParquetValueReaders.UnboxedReader<>(desc); + } + case TIME_MICROS: + return new LossyMicrosToMillisTimeReader(desc); + case TIME_MILLIS: + return new MillisTimeReader(desc); + case DATE: + case INT_64: + return new ParquetValueReaders.UnboxedReader<>(desc); + case TIMESTAMP_MICROS: + if (((Types.TimestampType) expected).shouldAdjustToUTC()) { + return new MicrosToTimestampTzReader(desc); + } else { + return new MicrosToTimestampReader(desc); + } + case TIMESTAMP_MILLIS: + if (((Types.TimestampType) expected).shouldAdjustToUTC()) { + return new MillisToTimestampTzReader(desc); + } else { + return new MillisToTimestampReader(desc); + } + case DECIMAL: + DecimalLogicalTypeAnnotation decimal = + (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation(); + switch (primitive.getPrimitiveTypeName()) { + case BINARY: + case FIXED_LEN_BYTE_ARRAY: + return new BinaryDecimalReader(desc, decimal.getPrecision(), decimal.getScale()); + case INT64: + return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale()); + case INT32: + return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale()); + default: + throw new UnsupportedOperationException( + "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName()); + } + case BSON: + return new ParquetValueReaders.ByteArrayReader(desc); + default: + throw new UnsupportedOperationException( + "Unsupported logical type: " + primitive.getOriginalType()); + } + } + + switch (primitive.getPrimitiveTypeName()) { + case FIXED_LEN_BYTE_ARRAY: + case BINARY: + return new ParquetValueReaders.ByteArrayReader(desc); + case INT32: + if (expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) { + return new ParquetValueReaders.IntAsLongReader(desc); + } else { + return new ParquetValueReaders.UnboxedReader<>(desc); + } + case FLOAT: + if (expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) { + return new ParquetValueReaders.FloatAsDoubleReader(desc); + } else { + return new ParquetValueReaders.UnboxedReader<>(desc); + } + case BOOLEAN: + case INT64: + case DOUBLE: + return new ParquetValueReaders.UnboxedReader<>(desc); + default: + throw new UnsupportedOperationException("Unsupported type: " + primitive); + } + } + } + + private static class BinaryDecimalReader + extends ParquetValueReaders.PrimitiveReader { + private final int precision; + private final int scale; + + BinaryDecimalReader(ColumnDescriptor desc, int precision, int scale) { + super(desc); + this.precision = precision; + this.scale = scale; + } + + @Override + public DecimalData read(DecimalData ignored) { + Binary binary = column.nextBinary(); + BigDecimal bigDecimal = new BigDecimal(new BigInteger(binary.getBytes()), scale); + // TODO: need a unit test to write-read-validate decimal via FlinkParquetWrite/Reader + return DecimalData.fromBigDecimal(bigDecimal, precision, scale); + } + } + + private static class IntegerDecimalReader + extends ParquetValueReaders.PrimitiveReader { + private final int precision; + private final int scale; + + IntegerDecimalReader(ColumnDescriptor desc, int precision, int scale) { + super(desc); + this.precision = precision; + this.scale = scale; + } + + @Override + public DecimalData read(DecimalData ignored) { + return DecimalData.fromUnscaledLong(column.nextInteger(), precision, scale); + } + } + + private static class LongDecimalReader extends ParquetValueReaders.PrimitiveReader { + private final int precision; + private final int scale; + + LongDecimalReader(ColumnDescriptor desc, int precision, int scale) { + super(desc); + this.precision = precision; + this.scale = scale; + } + + @Override + public DecimalData read(DecimalData ignored) { + return DecimalData.fromUnscaledLong(column.nextLong(), precision, scale); + } + } + + private static class MicrosToTimestampTzReader + extends ParquetValueReaders.UnboxedReader { + MicrosToTimestampTzReader(ColumnDescriptor desc) { + super(desc); + } + + @Override + public TimestampData read(TimestampData ignored) { + long value = readLong(); + return TimestampData.fromLocalDateTime( + Instant.ofEpochSecond( + Math.floorDiv(value, 1000_000), Math.floorMod(value, 1000_000) * 1000) + .atOffset(ZoneOffset.UTC) + .toLocalDateTime()); + } + + @Override + public long readLong() { + return column.nextLong(); + } + } + + private static class MicrosToTimestampReader + extends ParquetValueReaders.UnboxedReader { + MicrosToTimestampReader(ColumnDescriptor desc) { + super(desc); + } + + @Override + public TimestampData read(TimestampData ignored) { + long value = readLong(); + return TimestampData.fromInstant( + Instant.ofEpochSecond( + Math.floorDiv(value, 1000_000), Math.floorMod(value, 1000_000) * 1000)); + } + + @Override + public long readLong() { + return column.nextLong(); + } + } + + private static class MillisToTimestampReader + extends ParquetValueReaders.UnboxedReader { + MillisToTimestampReader(ColumnDescriptor desc) { + super(desc); + } + + @Override + public TimestampData read(TimestampData ignored) { + long millis = readLong(); + return TimestampData.fromEpochMillis(millis); + } + + @Override + public long readLong() { + return column.nextLong(); + } + } + + private static class MillisToTimestampTzReader + extends ParquetValueReaders.UnboxedReader { + MillisToTimestampTzReader(ColumnDescriptor desc) { + super(desc); + } + + @Override + public TimestampData read(TimestampData ignored) { + long millis = readLong(); + return TimestampData.fromLocalDateTime( + Instant.ofEpochMilli(millis).atOffset(ZoneOffset.UTC).toLocalDateTime()); + } + + @Override + public long readLong() { + return column.nextLong(); + } + } + + private static class StringReader extends ParquetValueReaders.PrimitiveReader { + StringReader(ColumnDescriptor desc) { + super(desc); + } + + @Override + public StringData read(StringData ignored) { + Binary binary = column.nextBinary(); + ByteBuffer buffer = binary.toByteBuffer(); + if (buffer.hasArray()) { + return StringData.fromBytes( + buffer.array(), buffer.arrayOffset() + buffer.position(), buffer.remaining()); + } else { + return StringData.fromBytes(binary.getBytes()); + } + } + } + + private static class LossyMicrosToMillisTimeReader + extends ParquetValueReaders.PrimitiveReader { + LossyMicrosToMillisTimeReader(ColumnDescriptor desc) { + super(desc); + } + + @Override + public Integer read(Integer reuse) { + // Discard microseconds since Flink uses millisecond unit for TIME type. + return (int) Math.floorDiv(column.nextLong(), 1000); + } + } + + private static class MillisTimeReader extends ParquetValueReaders.PrimitiveReader { + MillisTimeReader(ColumnDescriptor desc) { + super(desc); + } + + @Override + public Integer read(Integer reuse) { + return (int) column.nextLong(); + } + } + + private static class ArrayReader + extends ParquetValueReaders.RepeatedReader { + private int readPos = 0; + private int writePos = 0; + + ArrayReader(int definitionLevel, int repetitionLevel, ParquetValueReader reader) { + super(definitionLevel, repetitionLevel, reader); + } + + @Override + protected ReusableArrayData newListData(ArrayData reuse) { + this.readPos = 0; + this.writePos = 0; + + if (reuse instanceof ReusableArrayData) { + return (ReusableArrayData) reuse; + } else { + return new ReusableArrayData(); + } + } + + @Override + @SuppressWarnings("unchecked") + protected E getElement(ReusableArrayData list) { + E value = null; + if (readPos < list.capacity()) { + value = (E) list.values[readPos]; + } + + readPos += 1; + + return value; + } + + @Override + protected void addElement(ReusableArrayData reused, E element) { + if (writePos >= reused.capacity()) { + reused.grow(); + } + + reused.values[writePos] = element; + + writePos += 1; + } + + @Override + protected ArrayData buildList(ReusableArrayData list) { + // Since ReusableArrayData is not accepted by Flink, use GenericArrayData temporarily to walk + // around it. + // Revert this to use ReusableArrayData once it is fixed in Flink. + // For your reference, https://issues.apache.org/jira/browse/FLINK-25238. + return new GenericArrayData(Arrays.copyOf(list.values, writePos)); + } + } + + private static class MapReader + extends ParquetValueReaders.RepeatedKeyValueReader { + private int readPos = 0; + private int writePos = 0; + + private final ParquetValueReaders.ReusableEntry entry = + new ParquetValueReaders.ReusableEntry<>(); + private final ParquetValueReaders.ReusableEntry nullEntry = + new ParquetValueReaders.ReusableEntry<>(); + + MapReader( + int definitionLevel, + int repetitionLevel, + ParquetValueReader keyReader, + ParquetValueReader valueReader) { + super(definitionLevel, repetitionLevel, keyReader, valueReader); + } + + @Override + protected ReusableMapData newMapData(MapData reuse) { + this.readPos = 0; + this.writePos = 0; + + if (reuse instanceof ReusableMapData) { + return (ReusableMapData) reuse; + } else { + return new ReusableMapData(); + } + } + + @Override + @SuppressWarnings("unchecked") + protected Map.Entry getPair(ReusableMapData map) { + Map.Entry kv = nullEntry; + if (readPos < map.capacity()) { + entry.set((K) map.keys.values[readPos], (V) map.values.values[readPos]); + kv = entry; + } + + readPos += 1; + + return kv; + } + + @Override + protected void addPair(ReusableMapData map, K key, V value) { + if (writePos >= map.capacity()) { + map.grow(); + } + + map.keys.values[writePos] = key; + map.values.values[writePos] = value; + + writePos += 1; + } + + @Override + protected MapData buildMap(ReusableMapData map) { + map.setNumElements(writePos); + return map; + } + } + + private static class RowDataReader + extends ParquetValueReaders.StructReader { + private final int numFields; + + RowDataReader(List types, List> readers) { + super(types, readers); + this.numFields = readers.size(); + } + + @Override + protected GenericRowData newStructData(RowData reuse) { + if (reuse instanceof GenericRowData) { + return (GenericRowData) reuse; + } else { + return new GenericRowData(numFields); + } + } + + @Override + protected Object getField(GenericRowData intermediate, int pos) { + return intermediate.getField(pos); + } + + @Override + protected RowData buildStruct(GenericRowData struct) { + return struct; + } + + @Override + protected void set(GenericRowData row, int pos, Object value) { + row.setField(pos, value); + } + + @Override + protected void setNull(GenericRowData row, int pos) { + row.setField(pos, null); + } + + @Override + protected void setBoolean(GenericRowData row, int pos, boolean value) { + row.setField(pos, value); + } + + @Override + protected void setInteger(GenericRowData row, int pos, int value) { + row.setField(pos, value); + } + + @Override + protected void setLong(GenericRowData row, int pos, long value) { + row.setField(pos, value); + } + + @Override + protected void setFloat(GenericRowData row, int pos, float value) { + row.setField(pos, value); + } + + @Override + protected void setDouble(GenericRowData row, int pos, double value) { + row.setField(pos, value); + } + } + + private static class ReusableMapData implements MapData { + private final ReusableArrayData keys; + private final ReusableArrayData values; + + private int numElements; + + private ReusableMapData() { + this.keys = new ReusableArrayData(); + this.values = new ReusableArrayData(); + } + + private void grow() { + keys.grow(); + values.grow(); + } + + private int capacity() { + return keys.capacity(); + } + + public void setNumElements(int numElements) { + this.numElements = numElements; + keys.setNumElements(numElements); + values.setNumElements(numElements); + } + + @Override + public int size() { + return numElements; + } + + @Override + public ReusableArrayData keyArray() { + return keys; + } + + @Override + public ReusableArrayData valueArray() { + return values; + } + } + + private static class ReusableArrayData implements ArrayData { + private static final Object[] EMPTY = new Object[0]; + + private Object[] values = EMPTY; + private int numElements = 0; + + private void grow() { + if (values.length == 0) { + this.values = new Object[20]; + } else { + Object[] old = values; + this.values = new Object[old.length << 1]; + // copy the old array in case it has values that can be reused + System.arraycopy(old, 0, values, 0, old.length); + } + } + + private int capacity() { + return values.length; + } + + public void setNumElements(int numElements) { + this.numElements = numElements; + } + + @Override + public int size() { + return numElements; + } + + @Override + public boolean isNullAt(int ordinal) { + return null == values[ordinal]; + } + + @Override + public boolean getBoolean(int ordinal) { + return (boolean) values[ordinal]; + } + + @Override + public byte getByte(int ordinal) { + return (byte) values[ordinal]; + } + + @Override + public short getShort(int ordinal) { + return (short) values[ordinal]; + } + + @Override + public int getInt(int ordinal) { + return (int) values[ordinal]; + } + + @Override + public long getLong(int ordinal) { + return (long) values[ordinal]; + } + + @Override + public float getFloat(int ordinal) { + return (float) values[ordinal]; + } + + @Override + public double getDouble(int ordinal) { + return (double) values[ordinal]; + } + + @Override + public StringData getString(int pos) { + return (StringData) values[pos]; + } + + @Override + public DecimalData getDecimal(int pos, int precision, int scale) { + return (DecimalData) values[pos]; + } + + @Override + public TimestampData getTimestamp(int pos, int precision) { + return (TimestampData) values[pos]; + } + + @SuppressWarnings("unchecked") + @Override + public RawValueData getRawValue(int pos) { + return (RawValueData) values[pos]; + } + + @Override + public byte[] getBinary(int ordinal) { + return (byte[]) values[ordinal]; + } + + @Override + public ArrayData getArray(int ordinal) { + return (ArrayData) values[ordinal]; + } + + @Override + public MapData getMap(int ordinal) { + return (MapData) values[ordinal]; + } + + @Override + public RowData getRow(int pos, int numFields) { + return (RowData) values[pos]; + } + + @Override + public boolean[] toBooleanArray() { + return ArrayUtil.toPrimitive((Boolean[]) values); + } + + @Override + public byte[] toByteArray() { + return ArrayUtil.toPrimitive((Byte[]) values); + } + + @Override + public short[] toShortArray() { + return ArrayUtil.toPrimitive((Short[]) values); + } + + @Override + public int[] toIntArray() { + return ArrayUtil.toPrimitive((Integer[]) values); + } + + @Override + public long[] toLongArray() { + return ArrayUtil.toPrimitive((Long[]) values); + } + + @Override + public float[] toFloatArray() { + return ArrayUtil.toPrimitive((Float[]) values); + } + + @Override + public double[] toDoubleArray() { + return ArrayUtil.toPrimitive((Double[]) values); + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java new file mode 100644 index 000000000000..db4f1730a134 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java @@ -0,0 +1,504 @@ +/* + * 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.iceberg.flink.data; + +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.RowType.RowField; +import org.apache.flink.table.types.logical.SmallIntType; +import org.apache.flink.table.types.logical.TinyIntType; +import org.apache.iceberg.parquet.ParquetValueReaders; +import org.apache.iceberg.parquet.ParquetValueWriter; +import org.apache.iceberg.parquet.ParquetValueWriters; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.DecimalUtil; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; + +public class FlinkParquetWriters { + private FlinkParquetWriters() {} + + @SuppressWarnings("unchecked") + public static ParquetValueWriter buildWriter(LogicalType schema, MessageType type) { + return (ParquetValueWriter) + ParquetWithFlinkSchemaVisitor.visit(schema, type, new WriteBuilder(type)); + } + + private static class WriteBuilder extends ParquetWithFlinkSchemaVisitor> { + private final MessageType type; + + WriteBuilder(MessageType type) { + this.type = type; + } + + @Override + public ParquetValueWriter message( + RowType sStruct, MessageType message, List> fields) { + return struct(sStruct, message.asGroupType(), fields); + } + + @Override + public ParquetValueWriter struct( + RowType sStruct, GroupType struct, List> fieldWriters) { + List fields = struct.getFields(); + List flinkFields = sStruct.getFields(); + List> writers = Lists.newArrayListWithExpectedSize(fieldWriters.size()); + List flinkTypes = Lists.newArrayList(); + for (int i = 0; i < fields.size(); i += 1) { + writers.add(newOption(struct.getType(i), fieldWriters.get(i))); + flinkTypes.add(flinkFields.get(i).getType()); + } + + return new RowDataWriter(writers, flinkTypes); + } + + @Override + public ParquetValueWriter list( + ArrayType sArray, GroupType array, ParquetValueWriter elementWriter) { + GroupType repeated = array.getFields().get(0).asGroupType(); + String[] repeatedPath = currentPath(); + + int repeatedD = type.getMaxDefinitionLevel(repeatedPath); + int repeatedR = type.getMaxRepetitionLevel(repeatedPath); + + return new ArrayDataWriter<>( + repeatedD, + repeatedR, + newOption(repeated.getType(0), elementWriter), + sArray.getElementType()); + } + + @Override + public ParquetValueWriter map( + MapType sMap, + GroupType map, + ParquetValueWriter keyWriter, + ParquetValueWriter valueWriter) { + GroupType repeatedKeyValue = map.getFields().get(0).asGroupType(); + String[] repeatedPath = currentPath(); + + int repeatedD = type.getMaxDefinitionLevel(repeatedPath); + int repeatedR = type.getMaxRepetitionLevel(repeatedPath); + + return new MapDataWriter<>( + repeatedD, + repeatedR, + newOption(repeatedKeyValue.getType(0), keyWriter), + newOption(repeatedKeyValue.getType(1), valueWriter), + sMap.getKeyType(), + sMap.getValueType()); + } + + private ParquetValueWriter newOption(Type fieldType, ParquetValueWriter writer) { + int maxD = type.getMaxDefinitionLevel(path(fieldType.getName())); + return ParquetValueWriters.option(fieldType, maxD, writer); + } + + @Override + public ParquetValueWriter primitive(LogicalType fType, PrimitiveType primitive) { + ColumnDescriptor desc = type.getColumnDescription(currentPath()); + + if (primitive.getOriginalType() != null) { + switch (primitive.getOriginalType()) { + case ENUM: + case JSON: + case UTF8: + return strings(desc); + case DATE: + case INT_8: + case INT_16: + case INT_32: + return ints(fType, desc); + case INT_64: + return ParquetValueWriters.longs(desc); + case TIME_MICROS: + return timeMicros(desc); + case TIMESTAMP_MICROS: + return timestamps(desc); + case DECIMAL: + DecimalLogicalTypeAnnotation decimal = + (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation(); + switch (primitive.getPrimitiveTypeName()) { + case INT32: + return decimalAsInteger(desc, decimal.getPrecision(), decimal.getScale()); + case INT64: + return decimalAsLong(desc, decimal.getPrecision(), decimal.getScale()); + case BINARY: + case FIXED_LEN_BYTE_ARRAY: + return decimalAsFixed(desc, decimal.getPrecision(), decimal.getScale()); + default: + throw new UnsupportedOperationException( + "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName()); + } + case BSON: + return byteArrays(desc); + default: + throw new UnsupportedOperationException( + "Unsupported logical type: " + primitive.getOriginalType()); + } + } + + switch (primitive.getPrimitiveTypeName()) { + case FIXED_LEN_BYTE_ARRAY: + case BINARY: + return byteArrays(desc); + case BOOLEAN: + return ParquetValueWriters.booleans(desc); + case INT32: + return ints(fType, desc); + case INT64: + return ParquetValueWriters.longs(desc); + case FLOAT: + return ParquetValueWriters.floats(desc); + case DOUBLE: + return ParquetValueWriters.doubles(desc); + default: + throw new UnsupportedOperationException("Unsupported type: " + primitive); + } + } + } + + private static ParquetValueWriters.PrimitiveWriter ints( + LogicalType type, ColumnDescriptor desc) { + if (type instanceof TinyIntType) { + return ParquetValueWriters.tinyints(desc); + } else if (type instanceof SmallIntType) { + return ParquetValueWriters.shorts(desc); + } + return ParquetValueWriters.ints(desc); + } + + private static ParquetValueWriters.PrimitiveWriter strings(ColumnDescriptor desc) { + return new StringDataWriter(desc); + } + + private static ParquetValueWriters.PrimitiveWriter timeMicros(ColumnDescriptor desc) { + return new TimeMicrosWriter(desc); + } + + private static ParquetValueWriters.PrimitiveWriter decimalAsInteger( + ColumnDescriptor desc, int precision, int scale) { + Preconditions.checkArgument( + precision <= 9, + "Cannot write decimal value as integer with precision larger than 9," + + " wrong precision %s", + precision); + return new IntegerDecimalWriter(desc, precision, scale); + } + + private static ParquetValueWriters.PrimitiveWriter decimalAsLong( + ColumnDescriptor desc, int precision, int scale) { + Preconditions.checkArgument( + precision <= 18, + "Cannot write decimal value as long with precision larger than 18, " + + " wrong precision %s", + precision); + return new LongDecimalWriter(desc, precision, scale); + } + + private static ParquetValueWriters.PrimitiveWriter decimalAsFixed( + ColumnDescriptor desc, int precision, int scale) { + return new FixedDecimalWriter(desc, precision, scale); + } + + private static ParquetValueWriters.PrimitiveWriter timestamps( + ColumnDescriptor desc) { + return new TimestampDataWriter(desc); + } + + private static ParquetValueWriters.PrimitiveWriter byteArrays(ColumnDescriptor desc) { + return new ByteArrayWriter(desc); + } + + private static class StringDataWriter extends ParquetValueWriters.PrimitiveWriter { + private StringDataWriter(ColumnDescriptor desc) { + super(desc); + } + + @Override + public void write(int repetitionLevel, StringData value) { + column.writeBinary(repetitionLevel, Binary.fromReusedByteArray(value.toBytes())); + } + } + + private static class TimeMicrosWriter extends ParquetValueWriters.PrimitiveWriter { + private TimeMicrosWriter(ColumnDescriptor desc) { + super(desc); + } + + @Override + public void write(int repetitionLevel, Integer value) { + long micros = value.longValue() * 1000; + column.writeLong(repetitionLevel, micros); + } + } + + private static class IntegerDecimalWriter + extends ParquetValueWriters.PrimitiveWriter { + private final int precision; + private final int scale; + + private IntegerDecimalWriter(ColumnDescriptor desc, int precision, int scale) { + super(desc); + this.precision = precision; + this.scale = scale; + } + + @Override + public void write(int repetitionLevel, DecimalData decimal) { + Preconditions.checkArgument( + decimal.scale() == scale, + "Cannot write value as decimal(%s,%s), wrong scale: %s", + precision, + scale, + decimal); + Preconditions.checkArgument( + decimal.precision() <= precision, + "Cannot write value as decimal(%s,%s), too large: %s", + precision, + scale, + decimal); + + column.writeInteger(repetitionLevel, (int) decimal.toUnscaledLong()); + } + } + + private static class LongDecimalWriter extends ParquetValueWriters.PrimitiveWriter { + private final int precision; + private final int scale; + + private LongDecimalWriter(ColumnDescriptor desc, int precision, int scale) { + super(desc); + this.precision = precision; + this.scale = scale; + } + + @Override + public void write(int repetitionLevel, DecimalData decimal) { + Preconditions.checkArgument( + decimal.scale() == scale, + "Cannot write value as decimal(%s,%s), wrong scale: %s", + precision, + scale, + decimal); + Preconditions.checkArgument( + decimal.precision() <= precision, + "Cannot write value as decimal(%s,%s), too large: %s", + precision, + scale, + decimal); + + column.writeLong(repetitionLevel, decimal.toUnscaledLong()); + } + } + + private static class FixedDecimalWriter extends ParquetValueWriters.PrimitiveWriter { + private final int precision; + private final int scale; + private final ThreadLocal bytes; + + private FixedDecimalWriter(ColumnDescriptor desc, int precision, int scale) { + super(desc); + this.precision = precision; + this.scale = scale; + this.bytes = + ThreadLocal.withInitial(() -> new byte[TypeUtil.decimalRequiredBytes(precision)]); + } + + @Override + public void write(int repetitionLevel, DecimalData decimal) { + byte[] binary = + DecimalUtil.toReusedFixLengthBytes(precision, scale, decimal.toBigDecimal(), bytes.get()); + column.writeBinary(repetitionLevel, Binary.fromReusedByteArray(binary)); + } + } + + private static class TimestampDataWriter + extends ParquetValueWriters.PrimitiveWriter { + private TimestampDataWriter(ColumnDescriptor desc) { + super(desc); + } + + @Override + public void write(int repetitionLevel, TimestampData value) { + column.writeLong( + repetitionLevel, value.getMillisecond() * 1000 + value.getNanoOfMillisecond() / 1000); + } + } + + private static class ByteArrayWriter extends ParquetValueWriters.PrimitiveWriter { + private ByteArrayWriter(ColumnDescriptor desc) { + super(desc); + } + + @Override + public void write(int repetitionLevel, byte[] bytes) { + column.writeBinary(repetitionLevel, Binary.fromReusedByteArray(bytes)); + } + } + + private static class ArrayDataWriter extends ParquetValueWriters.RepeatedWriter { + private final LogicalType elementType; + + private ArrayDataWriter( + int definitionLevel, + int repetitionLevel, + ParquetValueWriter writer, + LogicalType elementType) { + super(definitionLevel, repetitionLevel, writer); + this.elementType = elementType; + } + + @Override + protected Iterator elements(ArrayData list) { + return new ElementIterator<>(list); + } + + private class ElementIterator implements Iterator { + private final int size; + private final ArrayData list; + private final ArrayData.ElementGetter getter; + private int index; + + private ElementIterator(ArrayData list) { + this.list = list; + size = list.size(); + getter = ArrayData.createElementGetter(elementType); + index = 0; + } + + @Override + public boolean hasNext() { + return index != size; + } + + @Override + @SuppressWarnings("unchecked") + public E next() { + if (index >= size) { + throw new NoSuchElementException(); + } + + E element = (E) getter.getElementOrNull(list, index); + index += 1; + + return element; + } + } + } + + private static class MapDataWriter + extends ParquetValueWriters.RepeatedKeyValueWriter { + private final LogicalType keyType; + private final LogicalType valueType; + + private MapDataWriter( + int definitionLevel, + int repetitionLevel, + ParquetValueWriter keyWriter, + ParquetValueWriter valueWriter, + LogicalType keyType, + LogicalType valueType) { + super(definitionLevel, repetitionLevel, keyWriter, valueWriter); + this.keyType = keyType; + this.valueType = valueType; + } + + @Override + protected Iterator> pairs(MapData map) { + return new EntryIterator<>(map); + } + + private class EntryIterator implements Iterator> { + private final int size; + private final ArrayData keys; + private final ArrayData values; + private final ParquetValueReaders.ReusableEntry entry; + private final ArrayData.ElementGetter keyGetter; + private final ArrayData.ElementGetter valueGetter; + private int index; + + private EntryIterator(MapData map) { + size = map.size(); + keys = map.keyArray(); + values = map.valueArray(); + entry = new ParquetValueReaders.ReusableEntry<>(); + keyGetter = ArrayData.createElementGetter(keyType); + valueGetter = ArrayData.createElementGetter(valueType); + index = 0; + } + + @Override + public boolean hasNext() { + return index != size; + } + + @Override + @SuppressWarnings("unchecked") + public Map.Entry next() { + if (index >= size) { + throw new NoSuchElementException(); + } + + entry.set( + (K) keyGetter.getElementOrNull(keys, index), + (V) valueGetter.getElementOrNull(values, index)); + index += 1; + + return entry; + } + } + } + + private static class RowDataWriter extends ParquetValueWriters.StructWriter { + private final RowData.FieldGetter[] fieldGetter; + + RowDataWriter(List> writers, List types) { + super(writers); + fieldGetter = new RowData.FieldGetter[types.size()]; + for (int i = 0; i < types.size(); i += 1) { + fieldGetter[i] = RowData.createFieldGetter(types.get(i), i); + } + } + + @Override + protected Object get(RowData struct, int index) { + return fieldGetter[index].getFieldOrNull(struct); + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java new file mode 100644 index 000000000000..ba4e1a7a7aec --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java @@ -0,0 +1,161 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.data; + +import java.util.List; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +abstract class FlinkSchemaVisitor { + + static T visit(RowType flinkType, Schema schema, FlinkSchemaVisitor visitor) { + return visit(flinkType, schema.asStruct(), visitor); + } + + private static T visit(LogicalType flinkType, Type iType, FlinkSchemaVisitor visitor) { + switch (iType.typeId()) { + case STRUCT: + return visitRecord(flinkType, iType.asStructType(), visitor); + + case MAP: + MapType mapType = (MapType) flinkType; + Types.MapType iMapType = iType.asMapType(); + T key; + T value; + + Types.NestedField keyField = iMapType.field(iMapType.keyId()); + visitor.beforeMapKey(keyField); + try { + key = visit(mapType.getKeyType(), iMapType.keyType(), visitor); + } finally { + visitor.afterMapKey(keyField); + } + + Types.NestedField valueField = iMapType.field(iMapType.valueId()); + visitor.beforeMapValue(valueField); + try { + value = visit(mapType.getValueType(), iMapType.valueType(), visitor); + } finally { + visitor.afterMapValue(valueField); + } + + return visitor.map(iMapType, key, value, mapType.getKeyType(), mapType.getValueType()); + + case LIST: + ArrayType listType = (ArrayType) flinkType; + Types.ListType iListType = iType.asListType(); + T element; + + Types.NestedField elementField = iListType.field(iListType.elementId()); + visitor.beforeListElement(elementField); + try { + element = visit(listType.getElementType(), iListType.elementType(), visitor); + } finally { + visitor.afterListElement(elementField); + } + + return visitor.list(iListType, element, listType.getElementType()); + + default: + return visitor.primitive(iType.asPrimitiveType(), flinkType); + } + } + + private static T visitRecord( + LogicalType flinkType, Types.StructType struct, FlinkSchemaVisitor visitor) { + Preconditions.checkArgument(flinkType instanceof RowType, "%s is not a RowType.", flinkType); + RowType rowType = (RowType) flinkType; + + int fieldSize = struct.fields().size(); + List results = Lists.newArrayListWithExpectedSize(fieldSize); + List fieldTypes = Lists.newArrayListWithExpectedSize(fieldSize); + List nestedFields = struct.fields(); + + for (int i = 0; i < fieldSize; i++) { + Types.NestedField iField = nestedFields.get(i); + int fieldIndex = rowType.getFieldIndex(iField.name()); + Preconditions.checkArgument( + fieldIndex >= 0, "NestedField: %s is not found in flink RowType: %s", iField, rowType); + + LogicalType fieldFlinkType = rowType.getTypeAt(fieldIndex); + + fieldTypes.add(fieldFlinkType); + + visitor.beforeField(iField); + try { + results.add(visit(fieldFlinkType, iField.type(), visitor)); + } finally { + visitor.afterField(iField); + } + } + + return visitor.record(struct, results, fieldTypes); + } + + public T record(Types.StructType iStruct, List results, List fieldTypes) { + return null; + } + + public T list(Types.ListType iList, T element, LogicalType elementType) { + return null; + } + + public T map(Types.MapType iMap, T key, T value, LogicalType keyType, LogicalType valueType) { + return null; + } + + public T primitive(Type.PrimitiveType iPrimitive, LogicalType flinkPrimitive) { + return null; + } + + public void beforeField(Types.NestedField field) {} + + public void afterField(Types.NestedField field) {} + + public void beforeListElement(Types.NestedField elementField) { + beforeField(elementField); + } + + public void afterListElement(Types.NestedField elementField) { + afterField(elementField); + } + + public void beforeMapKey(Types.NestedField keyField) { + beforeField(keyField); + } + + public void afterMapKey(Types.NestedField keyField) { + afterField(keyField); + } + + public void beforeMapValue(Types.NestedField valueField) { + beforeField(valueField); + } + + public void afterMapValue(Types.NestedField valueField) { + afterField(valueField); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java new file mode 100644 index 000000000000..32f6c3a2ccfd --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java @@ -0,0 +1,312 @@ +/* + * 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.iceberg.flink.data; + +import java.io.IOException; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Map; +import org.apache.avro.io.Decoder; +import org.apache.avro.util.Utf8; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.iceberg.avro.ValueReader; +import org.apache.iceberg.avro.ValueReaders; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; + +public class FlinkValueReaders { + + private FlinkValueReaders() {} + + static ValueReader strings() { + return StringReader.INSTANCE; + } + + static ValueReader enums(List symbols) { + return new EnumReader(symbols); + } + + static ValueReader uuids() { + return ValueReaders.fixed(16); + } + + static ValueReader timeMicros() { + return TimeMicrosReader.INSTANCE; + } + + static ValueReader timestampMills() { + return TimestampMillsReader.INSTANCE; + } + + static ValueReader timestampMicros() { + return TimestampMicrosReader.INSTANCE; + } + + static ValueReader decimal( + ValueReader unscaledReader, int precision, int scale) { + return new DecimalReader(unscaledReader, precision, scale); + } + + static ValueReader array(ValueReader elementReader) { + return new ArrayReader(elementReader); + } + + static ValueReader arrayMap(ValueReader keyReader, ValueReader valueReader) { + return new ArrayMapReader(keyReader, valueReader); + } + + static ValueReader map(ValueReader keyReader, ValueReader valueReader) { + return new MapReader(keyReader, valueReader); + } + + static ValueReader struct( + List> readers, Types.StructType struct, Map idToConstant) { + return new StructReader(readers, struct, idToConstant); + } + + private static class StringReader implements ValueReader { + private static final StringReader INSTANCE = new StringReader(); + + private StringReader() {} + + @Override + public StringData read(Decoder decoder, Object reuse) throws IOException { + // use the decoder's readString(Utf8) method because it may be a resolving decoder + Utf8 utf8 = null; + if (reuse instanceof StringData) { + utf8 = new Utf8(((StringData) reuse).toBytes()); + } + + Utf8 string = decoder.readString(utf8); + return StringData.fromBytes(string.getBytes(), 0, string.getByteLength()); + } + } + + private static class EnumReader implements ValueReader { + private final StringData[] symbols; + + private EnumReader(List symbols) { + this.symbols = new StringData[symbols.size()]; + for (int i = 0; i < this.symbols.length; i += 1) { + this.symbols[i] = StringData.fromBytes(symbols.get(i).getBytes(StandardCharsets.UTF_8)); + } + } + + @Override + public StringData read(Decoder decoder, Object ignore) throws IOException { + int index = decoder.readEnum(); + return symbols[index]; + } + } + + private static class DecimalReader implements ValueReader { + private final ValueReader bytesReader; + private final int precision; + private final int scale; + + private DecimalReader(ValueReader bytesReader, int precision, int scale) { + this.bytesReader = bytesReader; + this.precision = precision; + this.scale = scale; + } + + @Override + public DecimalData read(Decoder decoder, Object reuse) throws IOException { + byte[] bytes = bytesReader.read(decoder, null); + return DecimalData.fromBigDecimal( + new BigDecimal(new BigInteger(bytes), scale), precision, scale); + } + } + + private static class TimeMicrosReader implements ValueReader { + private static final TimeMicrosReader INSTANCE = new TimeMicrosReader(); + + @Override + public Integer read(Decoder decoder, Object reuse) throws IOException { + long micros = decoder.readLong(); + // Flink only support time mills, just erase micros. + return (int) (micros / 1000); + } + } + + private static class TimestampMillsReader implements ValueReader { + private static final TimestampMillsReader INSTANCE = new TimestampMillsReader(); + + @Override + public TimestampData read(Decoder decoder, Object reuse) throws IOException { + return TimestampData.fromEpochMillis(decoder.readLong()); + } + } + + private static class TimestampMicrosReader implements ValueReader { + private static final TimestampMicrosReader INSTANCE = new TimestampMicrosReader(); + + @Override + public TimestampData read(Decoder decoder, Object reuse) throws IOException { + long micros = decoder.readLong(); + long mills = micros / 1000; + int nanos = ((int) (micros % 1000)) * 1000; + if (nanos < 0) { + nanos += 1_000_000; + mills -= 1; + } + return TimestampData.fromEpochMillis(mills, nanos); + } + } + + private static class ArrayReader implements ValueReader { + private final ValueReader elementReader; + private final List reusedList = Lists.newArrayList(); + + private ArrayReader(ValueReader elementReader) { + this.elementReader = elementReader; + } + + @Override + public GenericArrayData read(Decoder decoder, Object reuse) throws IOException { + reusedList.clear(); + long chunkLength = decoder.readArrayStart(); + + while (chunkLength > 0) { + for (int i = 0; i < chunkLength; i += 1) { + reusedList.add(elementReader.read(decoder, null)); + } + + chunkLength = decoder.arrayNext(); + } + + // this will convert the list to an array so it is okay to reuse the list + return new GenericArrayData(reusedList.toArray()); + } + } + + private static MapData kvArrayToMap(List keyList, List valueList) { + Map map = Maps.newHashMap(); + Object[] keys = keyList.toArray(); + Object[] values = valueList.toArray(); + for (int i = 0; i < keys.length; i++) { + map.put(keys[i], values[i]); + } + + return new GenericMapData(map); + } + + private static class ArrayMapReader implements ValueReader { + private final ValueReader keyReader; + private final ValueReader valueReader; + + private final List reusedKeyList = Lists.newArrayList(); + private final List reusedValueList = Lists.newArrayList(); + + private ArrayMapReader(ValueReader keyReader, ValueReader valueReader) { + this.keyReader = keyReader; + this.valueReader = valueReader; + } + + @Override + public MapData read(Decoder decoder, Object reuse) throws IOException { + reusedKeyList.clear(); + reusedValueList.clear(); + + long chunkLength = decoder.readArrayStart(); + + while (chunkLength > 0) { + for (int i = 0; i < chunkLength; i += 1) { + reusedKeyList.add(keyReader.read(decoder, null)); + reusedValueList.add(valueReader.read(decoder, null)); + } + + chunkLength = decoder.arrayNext(); + } + + return kvArrayToMap(reusedKeyList, reusedValueList); + } + } + + private static class MapReader implements ValueReader { + private final ValueReader keyReader; + private final ValueReader valueReader; + + private final List reusedKeyList = Lists.newArrayList(); + private final List reusedValueList = Lists.newArrayList(); + + private MapReader(ValueReader keyReader, ValueReader valueReader) { + this.keyReader = keyReader; + this.valueReader = valueReader; + } + + @Override + public MapData read(Decoder decoder, Object reuse) throws IOException { + reusedKeyList.clear(); + reusedValueList.clear(); + + long chunkLength = decoder.readMapStart(); + + while (chunkLength > 0) { + for (int i = 0; i < chunkLength; i += 1) { + reusedKeyList.add(keyReader.read(decoder, null)); + reusedValueList.add(valueReader.read(decoder, null)); + } + + chunkLength = decoder.mapNext(); + } + + return kvArrayToMap(reusedKeyList, reusedValueList); + } + } + + private static class StructReader extends ValueReaders.StructReader { + private final int numFields; + + private StructReader( + List> readers, Types.StructType struct, Map idToConstant) { + super(readers, struct, idToConstant); + this.numFields = readers.size(); + } + + @Override + protected RowData reuseOrCreate(Object reuse) { + if (reuse instanceof GenericRowData && ((GenericRowData) reuse).getArity() == numFields) { + return (GenericRowData) reuse; + } + return new GenericRowData(numFields); + } + + @Override + protected Object get(RowData struct, int pos) { + return null; + } + + @Override + protected void set(RowData struct, int pos, Object value) { + ((GenericRowData) struct).setField(pos, value); + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java new file mode 100644 index 000000000000..4e86ecce28b5 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java @@ -0,0 +1,253 @@ +/* + * 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.iceberg.flink.data; + +import java.io.IOException; +import java.lang.reflect.Array; +import java.util.List; +import org.apache.avro.io.Encoder; +import org.apache.avro.util.Utf8; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.iceberg.avro.ValueWriter; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.DecimalUtil; + +public class FlinkValueWriters { + + private FlinkValueWriters() {} + + static ValueWriter strings() { + return StringWriter.INSTANCE; + } + + static ValueWriter timeMicros() { + return TimeMicrosWriter.INSTANCE; + } + + static ValueWriter timestampMicros() { + return TimestampMicrosWriter.INSTANCE; + } + + static ValueWriter decimal(int precision, int scale) { + return new DecimalWriter(precision, scale); + } + + static ValueWriter array(ValueWriter elementWriter, LogicalType elementType) { + return new ArrayWriter<>(elementWriter, elementType); + } + + static ValueWriter arrayMap( + ValueWriter keyWriter, + LogicalType keyType, + ValueWriter valueWriter, + LogicalType valueType) { + return new ArrayMapWriter<>(keyWriter, keyType, valueWriter, valueType); + } + + static ValueWriter map( + ValueWriter keyWriter, + LogicalType keyType, + ValueWriter valueWriter, + LogicalType valueType) { + return new MapWriter<>(keyWriter, keyType, valueWriter, valueType); + } + + static ValueWriter row(List> writers, List types) { + return new RowWriter(writers, types); + } + + private static class StringWriter implements ValueWriter { + private static final StringWriter INSTANCE = new StringWriter(); + + private StringWriter() {} + + @Override + public void write(StringData s, Encoder encoder) throws IOException { + // toBytes is cheaper than Avro calling toString, which incurs encoding costs + encoder.writeString(new Utf8(s.toBytes())); + } + } + + private static class DecimalWriter implements ValueWriter { + private final int precision; + private final int scale; + private final ThreadLocal bytes; + + private DecimalWriter(int precision, int scale) { + this.precision = precision; + this.scale = scale; + this.bytes = + ThreadLocal.withInitial(() -> new byte[TypeUtil.decimalRequiredBytes(precision)]); + } + + @Override + public void write(DecimalData d, Encoder encoder) throws IOException { + encoder.writeFixed( + DecimalUtil.toReusedFixLengthBytes(precision, scale, d.toBigDecimal(), bytes.get())); + } + } + + private static class TimeMicrosWriter implements ValueWriter { + private static final TimeMicrosWriter INSTANCE = new TimeMicrosWriter(); + + @Override + public void write(Integer timeMills, Encoder encoder) throws IOException { + encoder.writeLong(timeMills * 1000L); + } + } + + private static class TimestampMicrosWriter implements ValueWriter { + private static final TimestampMicrosWriter INSTANCE = new TimestampMicrosWriter(); + + @Override + public void write(TimestampData timestampData, Encoder encoder) throws IOException { + long micros = + timestampData.getMillisecond() * 1000 + timestampData.getNanoOfMillisecond() / 1000; + encoder.writeLong(micros); + } + } + + private static class ArrayWriter implements ValueWriter { + private final ValueWriter elementWriter; + private final ArrayData.ElementGetter elementGetter; + + private ArrayWriter(ValueWriter elementWriter, LogicalType elementType) { + this.elementWriter = elementWriter; + this.elementGetter = ArrayData.createElementGetter(elementType); + } + + @Override + @SuppressWarnings("unchecked") + public void write(ArrayData array, Encoder encoder) throws IOException { + encoder.writeArrayStart(); + int numElements = array.size(); + encoder.setItemCount(numElements); + for (int i = 0; i < numElements; i += 1) { + encoder.startItem(); + elementWriter.write((T) elementGetter.getElementOrNull(array, i), encoder); + } + encoder.writeArrayEnd(); + } + } + + private static class ArrayMapWriter implements ValueWriter { + private final ValueWriter keyWriter; + private final ValueWriter valueWriter; + private final ArrayData.ElementGetter keyGetter; + private final ArrayData.ElementGetter valueGetter; + + private ArrayMapWriter( + ValueWriter keyWriter, + LogicalType keyType, + ValueWriter valueWriter, + LogicalType valueType) { + this.keyWriter = keyWriter; + this.keyGetter = ArrayData.createElementGetter(keyType); + this.valueWriter = valueWriter; + this.valueGetter = ArrayData.createElementGetter(valueType); + } + + @Override + @SuppressWarnings("unchecked") + public void write(MapData map, Encoder encoder) throws IOException { + encoder.writeArrayStart(); + int numElements = map.size(); + encoder.setItemCount(numElements); + ArrayData keyArray = map.keyArray(); + ArrayData valueArray = map.valueArray(); + for (int i = 0; i < numElements; i += 1) { + encoder.startItem(); + keyWriter.write((K) keyGetter.getElementOrNull(keyArray, i), encoder); + valueWriter.write((V) valueGetter.getElementOrNull(valueArray, i), encoder); + } + encoder.writeArrayEnd(); + } + } + + private static class MapWriter implements ValueWriter { + private final ValueWriter keyWriter; + private final ValueWriter valueWriter; + private final ArrayData.ElementGetter keyGetter; + private final ArrayData.ElementGetter valueGetter; + + private MapWriter( + ValueWriter keyWriter, + LogicalType keyType, + ValueWriter valueWriter, + LogicalType valueType) { + this.keyWriter = keyWriter; + this.keyGetter = ArrayData.createElementGetter(keyType); + this.valueWriter = valueWriter; + this.valueGetter = ArrayData.createElementGetter(valueType); + } + + @Override + @SuppressWarnings("unchecked") + public void write(MapData map, Encoder encoder) throws IOException { + encoder.writeMapStart(); + int numElements = map.size(); + encoder.setItemCount(numElements); + ArrayData keyArray = map.keyArray(); + ArrayData valueArray = map.valueArray(); + for (int i = 0; i < numElements; i += 1) { + encoder.startItem(); + keyWriter.write((K) keyGetter.getElementOrNull(keyArray, i), encoder); + valueWriter.write((V) valueGetter.getElementOrNull(valueArray, i), encoder); + } + encoder.writeMapEnd(); + } + } + + static class RowWriter implements ValueWriter { + private final ValueWriter[] writers; + private final RowData.FieldGetter[] getters; + + private RowWriter(List> writers, List types) { + this.writers = (ValueWriter[]) Array.newInstance(ValueWriter.class, writers.size()); + this.getters = new RowData.FieldGetter[writers.size()]; + for (int i = 0; i < writers.size(); i += 1) { + this.writers[i] = writers.get(i); + this.getters[i] = RowData.createFieldGetter(types.get(i), i); + } + } + + @Override + public void write(RowData row, Encoder encoder) throws IOException { + for (int i = 0; i < writers.length; i += 1) { + if (row.isNullAt(i)) { + writers[i].write(null, encoder); + } else { + write(row, i, writers[i], encoder); + } + } + } + + @SuppressWarnings("unchecked") + private void write(RowData row, int pos, ValueWriter writer, Encoder encoder) + throws IOException { + writer.write((T) getters[pos].getFieldOrNull(row), encoder); + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java new file mode 100644 index 000000000000..33feb2e32118 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java @@ -0,0 +1,222 @@ +/* + * 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.iceberg.flink.data; + +import java.util.Deque; +import java.util.List; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.RowType.RowField; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.OriginalType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; + +public class ParquetWithFlinkSchemaVisitor { + private final Deque fieldNames = Lists.newLinkedList(); + + public static T visit( + LogicalType sType, Type type, ParquetWithFlinkSchemaVisitor visitor) { + Preconditions.checkArgument(sType != null, "Invalid DataType: null"); + if (type instanceof MessageType) { + Preconditions.checkArgument( + sType instanceof RowType, "Invalid struct: %s is not a struct", sType); + RowType struct = (RowType) sType; + return visitor.message( + struct, (MessageType) type, visitFields(struct, type.asGroupType(), visitor)); + } else if (type.isPrimitive()) { + return visitor.primitive(sType, type.asPrimitiveType()); + } else { + // if not a primitive, the typeId must be a group + GroupType group = type.asGroupType(); + OriginalType annotation = group.getOriginalType(); + if (annotation != null) { + switch (annotation) { + case LIST: + Preconditions.checkArgument( + !group.isRepetition(Type.Repetition.REPEATED), + "Invalid list: top-level group is repeated: %s", + group); + Preconditions.checkArgument( + group.getFieldCount() == 1, + "Invalid list: does not contain single repeated field: %s", + group); + + GroupType repeatedElement = group.getFields().get(0).asGroupType(); + Preconditions.checkArgument( + repeatedElement.isRepetition(Type.Repetition.REPEATED), + "Invalid list: inner group is not repeated"); + Preconditions.checkArgument( + repeatedElement.getFieldCount() <= 1, + "Invalid list: repeated group is not a single field: %s", + group); + + Preconditions.checkArgument( + sType instanceof ArrayType, "Invalid list: %s is not an array", sType); + ArrayType array = (ArrayType) sType; + RowType.RowField element = + new RowField( + "element", array.getElementType(), "element of " + array.asSummaryString()); + + visitor.fieldNames.push(repeatedElement.getName()); + try { + T elementResult = null; + if (repeatedElement.getFieldCount() > 0) { + elementResult = visitField(element, repeatedElement.getType(0), visitor); + } + + return visitor.list(array, group, elementResult); + + } finally { + visitor.fieldNames.pop(); + } + + case MAP: + Preconditions.checkArgument( + !group.isRepetition(Type.Repetition.REPEATED), + "Invalid map: top-level group is repeated: %s", + group); + Preconditions.checkArgument( + group.getFieldCount() == 1, + "Invalid map: does not contain single repeated field: %s", + group); + + GroupType repeatedKeyValue = group.getType(0).asGroupType(); + Preconditions.checkArgument( + repeatedKeyValue.isRepetition(Type.Repetition.REPEATED), + "Invalid map: inner group is not repeated"); + Preconditions.checkArgument( + repeatedKeyValue.getFieldCount() <= 2, + "Invalid map: repeated group does not have 2 fields"); + + Preconditions.checkArgument( + sType instanceof MapType, "Invalid map: %s is not a map", sType); + MapType map = (MapType) sType; + RowField keyField = + new RowField("key", map.getKeyType(), "key of " + map.asSummaryString()); + RowField valueField = + new RowField("value", map.getValueType(), "value of " + map.asSummaryString()); + + visitor.fieldNames.push(repeatedKeyValue.getName()); + try { + T keyResult = null; + T valueResult = null; + switch (repeatedKeyValue.getFieldCount()) { + case 2: + // if there are 2 fields, both key and value are projected + keyResult = visitField(keyField, repeatedKeyValue.getType(0), visitor); + valueResult = visitField(valueField, repeatedKeyValue.getType(1), visitor); + break; + case 1: + // if there is just one, use the name to determine what it is + Type keyOrValue = repeatedKeyValue.getType(0); + if (keyOrValue.getName().equalsIgnoreCase("key")) { + keyResult = visitField(keyField, keyOrValue, visitor); + // value result remains null + } else { + valueResult = visitField(valueField, keyOrValue, visitor); + // key result remains null + } + break; + default: + // both results will remain null + } + + return visitor.map(map, group, keyResult, valueResult); + + } finally { + visitor.fieldNames.pop(); + } + + default: + } + } + Preconditions.checkArgument( + sType instanceof RowType, "Invalid struct: %s is not a struct", sType); + RowType struct = (RowType) sType; + return visitor.struct(struct, group, visitFields(struct, group, visitor)); + } + } + + private static T visitField( + RowType.RowField sField, Type field, ParquetWithFlinkSchemaVisitor visitor) { + visitor.fieldNames.push(field.getName()); + try { + return visit(sField.getType(), field, visitor); + } finally { + visitor.fieldNames.pop(); + } + } + + private static List visitFields( + RowType struct, GroupType group, ParquetWithFlinkSchemaVisitor visitor) { + List sFields = struct.getFields(); + Preconditions.checkArgument( + sFields.size() == group.getFieldCount(), "Structs do not match: %s and %s", struct, group); + List results = Lists.newArrayListWithExpectedSize(group.getFieldCount()); + for (int i = 0; i < sFields.size(); i += 1) { + Type field = group.getFields().get(i); + RowType.RowField sField = sFields.get(i); + Preconditions.checkArgument( + field.getName().equals(AvroSchemaUtil.makeCompatibleName(sField.getName())), + "Structs do not match: field %s != %s", + field.getName(), + sField.getName()); + results.add(visitField(sField, field, visitor)); + } + + return results; + } + + public T message(RowType sStruct, MessageType message, List fields) { + return null; + } + + public T struct(RowType sStruct, GroupType struct, List fields) { + return null; + } + + public T list(ArrayType sArray, GroupType array, T element) { + return null; + } + + public T map(MapType sMap, GroupType map, T key, T value) { + return null; + } + + public T primitive(LogicalType sPrimitive, PrimitiveType primitive) { + return null; + } + + protected String[] currentPath() { + return Lists.newArrayList(fieldNames.descendingIterator()).toArray(new String[0]); + } + + protected String[] path(String name) { + List list = Lists.newArrayList(fieldNames.descendingIterator()); + list.add(name); + return list.toArray(new String[0]); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java new file mode 100644 index 000000000000..33816c97ac29 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java @@ -0,0 +1,341 @@ +/* + * 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.iceberg.flink.data; + +import java.util.Arrays; +import java.util.Map; +import java.util.Objects; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RawValueData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.StringUtils; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; + +public class RowDataProjection implements RowData { + /** + * Creates a projecting wrapper for {@link RowData} rows. + * + *

This projection will not project the nested children types of repeated types like lists and + * maps. + * + * @param schema schema of rows wrapped by this projection + * @param projectedSchema result schema of the projected rows + * @return a wrapper to project rows + */ + public static RowDataProjection create(Schema schema, Schema projectedSchema) { + return RowDataProjection.create( + FlinkSchemaUtil.convert(schema), schema.asStruct(), projectedSchema.asStruct()); + } + + /** + * Creates a projecting wrapper for {@link RowData} rows. + * + *

This projection will not project the nested children types of repeated types like lists and + * maps. + * + * @param rowType flink row type of rows wrapped by this projection + * @param schema schema of rows wrapped by this projection + * @param projectedSchema result schema of the projected rows + * @return a wrapper to project rows + */ + public static RowDataProjection create( + RowType rowType, Types.StructType schema, Types.StructType projectedSchema) { + return new RowDataProjection(rowType, schema, projectedSchema); + } + + private final RowData.FieldGetter[] getters; + private RowData rowData; + + private RowDataProjection( + RowType rowType, Types.StructType rowStruct, Types.StructType projectType) { + Map fieldIdToPosition = Maps.newHashMap(); + for (int i = 0; i < rowStruct.fields().size(); i++) { + fieldIdToPosition.put(rowStruct.fields().get(i).fieldId(), i); + } + + this.getters = new RowData.FieldGetter[projectType.fields().size()]; + for (int i = 0; i < getters.length; i++) { + Types.NestedField projectField = projectType.fields().get(i); + Types.NestedField rowField = rowStruct.field(projectField.fieldId()); + + Preconditions.checkNotNull( + rowField, + "Cannot locate the project field <%s> in the iceberg struct <%s>", + projectField, + rowStruct); + + getters[i] = + createFieldGetter( + rowType, fieldIdToPosition.get(projectField.fieldId()), rowField, projectField); + } + } + + private static RowData.FieldGetter createFieldGetter( + RowType rowType, int position, Types.NestedField rowField, Types.NestedField projectField) { + Preconditions.checkArgument( + rowField.type().typeId() == projectField.type().typeId(), + "Different iceberg type between row field <%s> and project field <%s>", + rowField, + projectField); + + switch (projectField.type().typeId()) { + case STRUCT: + RowType nestedRowType = (RowType) rowType.getTypeAt(position); + return row -> { + // null nested struct value + if (row.isNullAt(position)) { + return null; + } + + RowData nestedRow = row.getRow(position, nestedRowType.getFieldCount()); + return RowDataProjection.create( + nestedRowType, rowField.type().asStructType(), projectField.type().asStructType()) + .wrap(nestedRow); + }; + + case MAP: + Types.MapType projectedMap = projectField.type().asMapType(); + Types.MapType originalMap = rowField.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 with non-primitive type. Trying to project <%s> out of <%s>", + projectField, + rowField); + + return RowData.createFieldGetter(rowType.getTypeAt(position), position); + + case LIST: + Types.ListType projectedList = projectField.type().asListType(); + Types.ListType originalList = rowField.type().asListType(); + + boolean elementProjectable = + !projectedList.elementType().isNestedType() + || projectedList.elementType().equals(originalList.elementType()); + Preconditions.checkArgument( + elementProjectable, + "Cannot project a partial list element with non-primitive type. Trying to project <%s> out of <%s>", + projectField, + rowField); + + return RowData.createFieldGetter(rowType.getTypeAt(position), position); + + default: + return RowData.createFieldGetter(rowType.getTypeAt(position), position); + } + } + + public RowData wrap(RowData row) { + // StructProjection allow wrapping null root struct object. + // See more discussions in https://github.com/apache/iceberg/pull/7517. + // RowDataProjection never allowed null root object to be wrapped. + // Hence, it is fine to enforce strict Preconditions check here. + Preconditions.checkArgument(row != null, "Invalid row data: null"); + this.rowData = row; + return this; + } + + private Object getValue(int pos) { + Preconditions.checkState(rowData != null, "Row data not wrapped"); + return getters[pos].getFieldOrNull(rowData); + } + + @Override + public int getArity() { + return getters.length; + } + + @Override + public RowKind getRowKind() { + Preconditions.checkState(rowData != null, "Row data not wrapped"); + return rowData.getRowKind(); + } + + @Override + public void setRowKind(RowKind kind) { + throw new UnsupportedOperationException("Cannot set row kind in the RowDataProjection"); + } + + @Override + public boolean isNullAt(int pos) { + return getValue(pos) == null; + } + + @Override + public boolean getBoolean(int pos) { + return (boolean) getValue(pos); + } + + @Override + public byte getByte(int pos) { + return (byte) getValue(pos); + } + + @Override + public short getShort(int pos) { + return (short) getValue(pos); + } + + @Override + public int getInt(int pos) { + return (int) getValue(pos); + } + + @Override + public long getLong(int pos) { + return (long) getValue(pos); + } + + @Override + public float getFloat(int pos) { + return (float) getValue(pos); + } + + @Override + public double getDouble(int pos) { + return (double) getValue(pos); + } + + @Override + public StringData getString(int pos) { + return (StringData) getValue(pos); + } + + @Override + public DecimalData getDecimal(int pos, int precision, int scale) { + return (DecimalData) getValue(pos); + } + + @Override + public TimestampData getTimestamp(int pos, int precision) { + return (TimestampData) getValue(pos); + } + + @Override + @SuppressWarnings("unchecked") + public RawValueData getRawValue(int pos) { + return (RawValueData) getValue(pos); + } + + @Override + public byte[] getBinary(int pos) { + return (byte[]) getValue(pos); + } + + @Override + public ArrayData getArray(int pos) { + return (ArrayData) getValue(pos); + } + + @Override + public MapData getMap(int pos) { + return (MapData) getValue(pos); + } + + @Override + public RowData getRow(int pos, int numFields) { + return (RowData) getValue(pos); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof RowDataProjection)) { + return false; + } + + RowDataProjection that = (RowDataProjection) o; + return deepEquals(that); + } + + @Override + public int hashCode() { + int result = Objects.hashCode(getRowKind()); + for (int pos = 0; pos < getArity(); pos++) { + if (!isNullAt(pos)) { + // Arrays.deepHashCode handles array object properly + result = 31 * result + Arrays.deepHashCode(new Object[] {getValue(pos)}); + } + } + + return result; + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getRowKind().shortString()).append("("); + for (int pos = 0; pos < getArity(); pos++) { + if (pos != 0) { + sb.append(","); + } + // copied the behavior from Flink GenericRowData + sb.append(StringUtils.arrayAwareToString(getValue(pos))); + } + + sb.append(")"); + return sb.toString(); + } + + private boolean deepEquals(RowDataProjection other) { + if (getRowKind() != other.getRowKind()) { + return false; + } + + if (getArity() != other.getArity()) { + return false; + } + + for (int pos = 0; pos < getArity(); ++pos) { + if (isNullAt(pos) && other.isNullAt(pos)) { + continue; + } + + if ((isNullAt(pos) && !other.isNullAt(pos)) || (!isNullAt(pos) && other.isNullAt(pos))) { + return false; + } + + // Objects.deepEquals handles array object properly + if (!Objects.deepEquals(getValue(pos), other.getValue(pos))) { + return false; + } + } + + return true; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java new file mode 100644 index 000000000000..c5cb51b7eae4 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.data; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import org.apache.avro.generic.GenericData; +import org.apache.avro.util.Utf8; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.ByteBuffers; +import org.apache.iceberg.util.DateTimeUtil; + +public class RowDataUtil { + + private RowDataUtil() {} + + public static Object convertConstant(Type type, Object value) { + if (value == null) { + return null; + } + + switch (type.typeId()) { + case DECIMAL: // DecimalData + Types.DecimalType decimal = (Types.DecimalType) type; + return DecimalData.fromBigDecimal((BigDecimal) value, decimal.precision(), decimal.scale()); + case STRING: // StringData + if (value instanceof Utf8) { + Utf8 utf8 = (Utf8) value; + return StringData.fromBytes(utf8.getBytes(), 0, utf8.getByteLength()); + } + return StringData.fromString(value.toString()); + case FIXED: // byte[] + if (value instanceof byte[]) { + return value; + } else if (value instanceof GenericData.Fixed) { + return ((GenericData.Fixed) value).bytes(); + } + return ByteBuffers.toByteArray((ByteBuffer) value); + case BINARY: // byte[] + return ByteBuffers.toByteArray((ByteBuffer) value); + case TIME: // int mills instead of long + return (int) ((Long) value / 1000); + case TIMESTAMP: // TimestampData + return TimestampData.fromLocalDateTime(DateTimeUtil.timestampFromMicros((Long) value)); + default: + } + return value; + } + + /** + * Similar to the private {@link RowDataSerializer#copyRowData(RowData, RowData)} method. This + * skips the check the arity of rowType and from, because the from RowData may contains additional + * column for position deletes. Using {@link RowDataSerializer#copy(RowData, RowData)} will fail + * the arity check. + */ + public static RowData clone( + RowData from, RowData reuse, RowType rowType, TypeSerializer[] fieldSerializers) { + GenericRowData ret; + if (reuse instanceof GenericRowData) { + ret = (GenericRowData) reuse; + } else { + ret = new GenericRowData(from.getArity()); + } + ret.setRowKind(from.getRowKind()); + for (int i = 0; i < rowType.getFieldCount(); i++) { + if (!from.isNullAt(i)) { + RowData.FieldGetter getter = RowData.createFieldGetter(rowType.getTypeAt(i), i); + ret.setField(i, fieldSerializers[i].copy(getter.getFieldOrNull(from))); + } else { + ret.setField(i, null); + } + } + return ret; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java new file mode 100644 index 000000000000..1019285018d0 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java @@ -0,0 +1,300 @@ +/* + * 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.iceberg.flink.data; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RawValueData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.types.RowKind; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.ByteBuffers; + +@Internal +public class StructRowData implements RowData { + private final Types.StructType type; + private RowKind kind; + private StructLike struct; + + public StructRowData(Types.StructType type) { + this(type, RowKind.INSERT); + } + + public StructRowData(Types.StructType type, RowKind kind) { + this(type, null, kind); + } + + private StructRowData(Types.StructType type, StructLike struct) { + this(type, struct, RowKind.INSERT); + } + + private StructRowData(Types.StructType type, StructLike struct, RowKind kind) { + this.type = type; + this.struct = struct; + this.kind = kind; + } + + public StructRowData setStruct(StructLike newStruct) { + this.struct = newStruct; + return this; + } + + @Override + public int getArity() { + return struct.size(); + } + + @Override + public RowKind getRowKind() { + return kind; + } + + @Override + public void setRowKind(RowKind newKind) { + Preconditions.checkNotNull(newKind, "kind can not be null"); + this.kind = newKind; + } + + @Override + public boolean isNullAt(int pos) { + return struct.get(pos, Object.class) == null; + } + + @Override + public boolean getBoolean(int pos) { + return struct.get(pos, Boolean.class); + } + + @Override + public byte getByte(int pos) { + return (byte) (int) struct.get(pos, Integer.class); + } + + @Override + public short getShort(int pos) { + return (short) (int) struct.get(pos, Integer.class); + } + + @Override + public int getInt(int pos) { + Object integer = struct.get(pos, Object.class); + + if (integer instanceof Integer) { + return (int) integer; + } else if (integer instanceof LocalDate) { + return (int) ((LocalDate) integer).toEpochDay(); + } else if (integer instanceof LocalTime) { + return (int) (((LocalTime) integer).toNanoOfDay() / 1000_000); + } else { + throw new IllegalStateException( + "Unknown type for int field. Type name: " + integer.getClass().getName()); + } + } + + @Override + public long getLong(int pos) { + Object longVal = struct.get(pos, Object.class); + + if (longVal instanceof Long) { + return (long) longVal; + } else if (longVal instanceof OffsetDateTime) { + return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000; + } else if (longVal instanceof LocalDate) { + return ((LocalDate) longVal).toEpochDay(); + } else if (longVal instanceof LocalTime) { + return ((LocalTime) longVal).toNanoOfDay(); + } else if (longVal instanceof LocalDateTime) { + return Duration.between(Instant.EPOCH, ((LocalDateTime) longVal).atOffset(ZoneOffset.UTC)) + .toNanos() + / 1000; + } else { + throw new IllegalStateException( + "Unknown type for long field. Type name: " + longVal.getClass().getName()); + } + } + + @Override + public float getFloat(int pos) { + return struct.get(pos, Float.class); + } + + @Override + public double getDouble(int pos) { + return struct.get(pos, Double.class); + } + + @Override + public StringData getString(int pos) { + return isNullAt(pos) ? null : getStringDataInternal(pos); + } + + private StringData getStringDataInternal(int pos) { + CharSequence seq = struct.get(pos, CharSequence.class); + return StringData.fromString(seq.toString()); + } + + @Override + public DecimalData getDecimal(int pos, int precision, int scale) { + return isNullAt(pos) + ? null + : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale); + } + + private BigDecimal getDecimalInternal(int pos) { + return struct.get(pos, BigDecimal.class); + } + + @Override + public TimestampData getTimestamp(int pos, int precision) { + long timeLong = getLong(pos); + return TimestampData.fromEpochMillis(timeLong / 1000, (int) (timeLong % 1000) * 1000); + } + + @Override + public RawValueData getRawValue(int pos) { + throw new UnsupportedOperationException("Not supported yet."); + } + + @Override + public byte[] getBinary(int pos) { + return isNullAt(pos) ? null : getBinaryInternal(pos); + } + + private byte[] getBinaryInternal(int pos) { + Object bytes = struct.get(pos, Object.class); + + // should only be either ByteBuffer or byte[] + if (bytes instanceof ByteBuffer) { + return ByteBuffers.toByteArray((ByteBuffer) bytes); + } else if (bytes instanceof byte[]) { + return (byte[]) bytes; + } else if (bytes instanceof UUID) { + UUID uuid = (UUID) bytes; + ByteBuffer bb = ByteBuffer.allocate(16); + bb.putLong(uuid.getMostSignificantBits()); + bb.putLong(uuid.getLeastSignificantBits()); + return bb.array(); + } else { + throw new IllegalStateException( + "Unknown type for binary field. Type name: " + bytes.getClass().getName()); + } + } + + @Override + public ArrayData getArray(int pos) { + return isNullAt(pos) + ? null + : (ArrayData) + convertValue(type.fields().get(pos).type().asListType(), struct.get(pos, List.class)); + } + + @Override + public MapData getMap(int pos) { + return isNullAt(pos) + ? null + : (MapData) + convertValue(type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class)); + } + + @Override + public RowData getRow(int pos, int numFields) { + return isNullAt(pos) ? null : getStructRowData(pos, numFields); + } + + private StructRowData getStructRowData(int pos, int numFields) { + return new StructRowData( + type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class)); + } + + private Object convertValue(Type elementType, Object value) { + switch (elementType.typeId()) { + case BOOLEAN: + case INTEGER: + case DATE: + case TIME: + case LONG: + case FLOAT: + case DOUBLE: + case DECIMAL: + return value; + case TIMESTAMP: + long millisecond = (long) value / 1000; + int nanoOfMillisecond = (int) ((Long) value % 1000) * 1000; + return TimestampData.fromEpochMillis(millisecond, nanoOfMillisecond); + case STRING: + return StringData.fromString(value.toString()); + case FIXED: + case BINARY: + return ByteBuffers.toByteArray((ByteBuffer) value); + case STRUCT: + return new StructRowData(elementType.asStructType(), (StructLike) value); + case LIST: + List list = (List) value; + Object[] array = new Object[list.size()]; + + int index = 0; + for (Object element : list) { + if (element == null) { + array[index] = null; + } else { + array[index] = convertValue(elementType.asListType().elementType(), element); + } + + index += 1; + } + return new GenericArrayData(array); + case MAP: + Types.MapType mapType = elementType.asMapType(); + Set> entries = ((Map) value).entrySet(); + Map result = Maps.newHashMap(); + for (Map.Entry entry : entries) { + final Object keyValue = convertValue(mapType.keyType(), entry.getKey()); + final Object valueValue = convertValue(mapType.valueType(), entry.getValue()); + result.put(keyValue, valueValue); + } + + return new GenericMapData(result); + default: + throw new UnsupportedOperationException("Unsupported element type: " + elementType); + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java new file mode 100644 index 000000000000..04e168385a36 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java @@ -0,0 +1,61 @@ +/* + * 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.iceberg.flink.sink; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.formats.avro.AvroToRowDataConverters; +import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.iceberg.avro.AvroSchemaUtil; + +/** + * This util class converts Avro GenericRecord to Flink RowData.
+ *
+ * Internally it uses Flink {@link AvroToRowDataConverters}. Because of the precision difference + * between how Iceberg schema (micro) and Flink {@link AvroToRowDataConverters} (milli) deal with + * time type, we can't directly use the Avro Schema converted from Iceberg schema via {@link + * AvroSchemaUtil#convert(org.apache.iceberg.Schema, String)}. + */ +public class AvroGenericRecordToRowDataMapper implements MapFunction { + + private final AvroToRowDataConverters.AvroToRowDataConverter converter; + + AvroGenericRecordToRowDataMapper(RowType rowType) { + this.converter = AvroToRowDataConverters.createRowConverter(rowType); + } + + @Override + public RowData map(GenericRecord genericRecord) throws Exception { + return (RowData) converter.convert(genericRecord); + } + + /** Create a mapper based on Avro schema. */ + public static AvroGenericRecordToRowDataMapper forAvroSchema(Schema avroSchema) { + DataType dataType = AvroSchemaConverter.convertToDataType(avroSchema.toString()); + LogicalType logicalType = TypeConversions.fromDataToLogicalType(dataType); + RowType rowType = RowType.of(logicalType.getChildren().stream().toArray(LogicalType[]::new)); + return new AvroGenericRecordToRowDataMapper(rowType); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java new file mode 100644 index 000000000000..40e0b5f2a34e --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.io.IOException; +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +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.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.data.RowDataProjection; +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.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.TypeUtil; + +abstract class BaseDeltaTaskWriter extends BaseTaskWriter { + + private final Schema schema; + private final Schema deleteSchema; + private final RowDataWrapper wrapper; + private final RowDataWrapper keyWrapper; + private final RowDataProjection keyProjection; + private final boolean upsert; + + BaseDeltaTaskWriter( + PartitionSpec spec, + FileFormat format, + FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, + FileIO io, + long targetFileSize, + Schema schema, + RowType flinkSchema, + List equalityFieldIds, + boolean upsert) { + super(spec, format, appenderFactory, fileFactory, io, targetFileSize); + this.schema = schema; + this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds)); + this.wrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); + this.keyWrapper = + new RowDataWrapper(FlinkSchemaUtil.convert(deleteSchema), deleteSchema.asStruct()); + this.keyProjection = + RowDataProjection.create(flinkSchema, schema.asStruct(), deleteSchema.asStruct()); + this.upsert = upsert; + } + + abstract RowDataDeltaWriter route(RowData row); + + RowDataWrapper wrapper() { + return wrapper; + } + + @Override + public void write(RowData row) throws IOException { + 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; // UPDATE_BEFORE is not necessary for UPSERT, we do nothing to prevent delete one + // row twice + } + 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()); + } + } + + protected class RowDataDeltaWriter extends BaseEqualityDeltaWriter { + RowDataDeltaWriter(PartitionKey partition) { + super(partition, schema, deleteSchema); + } + + @Override + protected StructLike asStructLike(RowData data) { + return wrapper.wrap(data); + } + + @Override + protected StructLike asStructLikeKey(RowData data) { + return keyWrapper.wrap(data); + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java new file mode 100644 index 000000000000..1cb6e013bd2c --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java @@ -0,0 +1,70 @@ +/* + * 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.iceberg.flink.sink; + +import java.util.stream.IntStream; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.RowDataWrapper; + +/** + * A {@link KeySelector} that extracts the bucketId from a data row's bucket partition as the key. + * To be used with the {@link BucketPartitioner}. + */ +class BucketPartitionKeySelector implements KeySelector { + + private final Schema schema; + private final PartitionKey partitionKey; + private final RowType flinkSchema; + private final int bucketFieldPosition; + + private transient RowDataWrapper rowDataWrapper; + + BucketPartitionKeySelector(PartitionSpec partitionSpec, Schema schema, RowType flinkSchema) { + this.schema = schema; + this.partitionKey = new PartitionKey(partitionSpec, schema); + this.flinkSchema = flinkSchema; + this.bucketFieldPosition = getBucketFieldPosition(partitionSpec); + } + + private int getBucketFieldPosition(PartitionSpec partitionSpec) { + int bucketFieldId = BucketPartitionerUtil.getBucketFieldId(partitionSpec); + return IntStream.range(0, partitionSpec.fields().size()) + .filter(i -> partitionSpec.fields().get(i).fieldId() == bucketFieldId) + .toArray()[0]; + } + + private RowDataWrapper lazyRowDataWrapper() { + if (rowDataWrapper == null) { + rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); + } + + return rowDataWrapper; + } + + @Override + public Integer getKey(RowData rowData) { + partitionKey.partition(lazyRowDataWrapper().wrap(rowData)); + return partitionKey.get(bucketFieldPosition, Integer.class); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java new file mode 100644 index 000000000000..9c9a117906e2 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import org.apache.flink.api.common.functions.Partitioner; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * This partitioner will redirect records to writers deterministically based on the Bucket partition + * spec. It'll attempt to optimize the file size written depending on whether numPartitions is + * greater, less or equal than the maxNumBuckets. Note: The current implementation only supports ONE + * bucket in the partition spec. + */ +class BucketPartitioner implements Partitioner { + + static final String BUCKET_NULL_MESSAGE = "bucketId cannot be null"; + static final String BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE = + "Invalid bucket ID %s: must be non-negative."; + static final String BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE = + "Invalid bucket ID %s: must be less than bucket limit: %s."; + + private final int maxNumBuckets; + + // To hold the OFFSET of the next writer to use for any bucket, only used when writers > the + // number of buckets + private final int[] currentBucketWriterOffset; + + BucketPartitioner(PartitionSpec partitionSpec) { + this.maxNumBuckets = BucketPartitionerUtil.getMaxNumBuckets(partitionSpec); + this.currentBucketWriterOffset = new int[maxNumBuckets]; + } + + /** + * Determine the partition id based on the following criteria: If the number of writers <= the + * number of buckets, an evenly distributed number of buckets will be assigned to each writer (one + * writer -> many buckets). Conversely, if the number of writers > the number of buckets the logic + * is handled by the {@link #getPartitionWithMoreWritersThanBuckets + * getPartitionWritersGreaterThanBuckets} method. + * + * @param bucketId the bucketId for each request + * @param numPartitions the total number of partitions + * @return the partition id (writer) to use for each request + */ + @Override + public int partition(Integer bucketId, int numPartitions) { + Preconditions.checkNotNull(bucketId, BUCKET_NULL_MESSAGE); + Preconditions.checkArgument(bucketId >= 0, BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE, bucketId); + Preconditions.checkArgument( + bucketId < maxNumBuckets, BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE, bucketId, maxNumBuckets); + + if (numPartitions <= maxNumBuckets) { + return bucketId % numPartitions; + } else { + return getPartitionWithMoreWritersThanBuckets(bucketId, numPartitions); + } + } + + /*- + * If the number of writers > the number of buckets each partitioner will keep a state of multiple + * writers per bucket as evenly as possible, and will round-robin the requests across them, in this + * case each writer will target only one bucket at all times (many writers -> one bucket). Example: + * Configuration: numPartitions (writers) = 5, maxBuckets = 2 + * Expected behavior: + * - Records for Bucket 0 will be "round robin" between Writers 0, 2 and 4 + * - Records for Bucket 1 will always use Writer 1 and 3 + * Notes: + * - maxNumWritersPerBucket determines when to reset the currentBucketWriterOffset to 0 for this bucketId + * - When numPartitions is not evenly divisible by maxBuckets, some buckets will have one more writer (extraWriter). + * In this example Bucket 0 has an "extra writer" to consider before resetting its offset to 0. + * + * @return the destination partition index (writer subtask id) + */ + private int getPartitionWithMoreWritersThanBuckets(int bucketId, int numPartitions) { + int currentOffset = currentBucketWriterOffset[bucketId]; + // Determine if this bucket requires an "extra writer" + int extraWriter = bucketId < (numPartitions % maxNumBuckets) ? 1 : 0; + // The max number of writers this bucket can have + int maxNumWritersPerBucket = (numPartitions / maxNumBuckets) + extraWriter; + + // Increment the writer offset or reset if it's reached the max for this bucket + int nextOffset = currentOffset == maxNumWritersPerBucket - 1 ? 0 : currentOffset + 1; + currentBucketWriterOffset[bucketId] = nextOffset; + + return bucketId + (maxNumBuckets * currentOffset); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java new file mode 100644 index 000000000000..c33207728d3e --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.transforms.PartitionSpecVisitor; + +final class BucketPartitionerUtil { + static final String BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE = + "Invalid number of buckets: %s (must be 1)"; + + private BucketPartitionerUtil() {} + + /** + * Determines whether the PartitionSpec has one and only one Bucket definition + * + * @param partitionSpec the partition spec in question + * @return whether the PartitionSpec has only one Bucket + */ + static boolean hasOneBucketField(PartitionSpec partitionSpec) { + List> bucketFields = getBucketFields(partitionSpec); + return bucketFields != null && bucketFields.size() == 1; + } + + /** + * Extracts the Bucket definition from a PartitionSpec. + * + * @param partitionSpec the partition spec in question + * @return the Bucket definition in the form of a tuple (fieldId, maxNumBuckets) + */ + private static Tuple2 getBucketFieldInfo(PartitionSpec partitionSpec) { + List> bucketFields = getBucketFields(partitionSpec); + Preconditions.checkArgument( + bucketFields.size() == 1, + BucketPartitionerUtil.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE, + bucketFields.size()); + return bucketFields.get(0); + } + + static int getBucketFieldId(PartitionSpec partitionSpec) { + return getBucketFieldInfo(partitionSpec).f0; + } + + static int getMaxNumBuckets(PartitionSpec partitionSpec) { + return getBucketFieldInfo(partitionSpec).f1; + } + + private static List> getBucketFields(PartitionSpec spec) { + return PartitionSpecVisitor.visit(spec, new BucketPartitionSpecVisitor()).stream() + .filter(Objects::nonNull) + .collect(Collectors.toList()); + } + + private static class BucketPartitionSpecVisitor + implements PartitionSpecVisitor> { + @Override + public Tuple2 identity(int fieldId, String sourceName, int sourceId) { + return null; + } + + @Override + public Tuple2 bucket( + int fieldId, String sourceName, int sourceId, int numBuckets) { + return new Tuple2<>(fieldId, numBuckets); + } + + @Override + public Tuple2 truncate( + int fieldId, String sourceName, int sourceId, int width) { + return null; + } + + @Override + public Tuple2 year(int fieldId, String sourceName, int sourceId) { + return null; + } + + @Override + public Tuple2 month(int fieldId, String sourceName, int sourceId) { + return null; + } + + @Override + public Tuple2 day(int fieldId, String sourceName, int sourceId) { + return null; + } + + @Override + public Tuple2 hour(int fieldId, String sourceName, int sourceId) { + return null; + } + + @Override + public Tuple2 alwaysNull(int fieldId, String sourceName, int sourceId) { + return null; + } + + @Override + public Tuple2 unknown( + int fieldId, String sourceName, int sourceId, String transform) { + return null; + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java new file mode 100644 index 000000000000..e9f9786f9190 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.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.iceberg.flink.sink; + +import java.time.Duration; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.util.DateTimeUtil; +import org.apache.iceberg.util.SerializableSupplier; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A table loader that will only reload a table after a certain interval has passed. WARNING: This + * table loader should be used carefully when used with writer tasks. It could result in heavy load + * on a catalog for jobs with many writers. + */ +class CachingTableSupplier implements SerializableSupplier { + + private static final Logger LOG = LoggerFactory.getLogger(CachingTableSupplier.class); + + private final Table initialTable; + private final TableLoader tableLoader; + private final Duration tableRefreshInterval; + private long lastLoadTimeMillis; + private transient Table table; + + CachingTableSupplier( + SerializableTable initialTable, TableLoader tableLoader, Duration tableRefreshInterval) { + Preconditions.checkArgument(initialTable != null, "initialTable cannot be null"); + Preconditions.checkArgument(tableLoader != null, "tableLoader cannot be null"); + Preconditions.checkArgument( + tableRefreshInterval != null, "tableRefreshInterval cannot be null"); + this.initialTable = initialTable; + this.table = initialTable; + this.tableLoader = tableLoader; + this.tableRefreshInterval = tableRefreshInterval; + this.lastLoadTimeMillis = System.currentTimeMillis(); + } + + @Override + public Table get() { + if (table == null) { + this.table = initialTable; + } + return table; + } + + Table initialTable() { + return initialTable; + } + + void refreshTable() { + if (System.currentTimeMillis() > lastLoadTimeMillis + tableRefreshInterval.toMillis()) { + try { + if (!tableLoader.isOpen()) { + tableLoader.open(); + } + + this.table = tableLoader.loadTable(); + this.lastLoadTimeMillis = System.currentTimeMillis(); + + LOG.info( + "Table {} reloaded, next min load time threshold is {}", + table.name(), + DateTimeUtil.formatTimestampMillis( + lastLoadTimeMillis + tableRefreshInterval.toMillis())); + } catch (Exception e) { + LOG.warn("An error occurred reloading table {}, table was not reloaded", table.name(), e); + } + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java new file mode 100644 index 000000000000..9a2f57181708 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java @@ -0,0 +1,93 @@ +/* + * 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.iceberg.flink.sink; + +import java.util.Arrays; +import java.util.NavigableMap; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +class CommitSummary { + + private final AtomicLong dataFilesCount = new AtomicLong(); + private final AtomicLong dataFilesRecordCount = new AtomicLong(); + private final AtomicLong dataFilesByteCount = new AtomicLong(); + private final AtomicLong deleteFilesCount = new AtomicLong(); + private final AtomicLong deleteFilesRecordCount = new AtomicLong(); + private final AtomicLong deleteFilesByteCount = new AtomicLong(); + + CommitSummary(NavigableMap pendingResults) { + pendingResults + .values() + .forEach( + writeResult -> { + dataFilesCount.addAndGet(writeResult.dataFiles().length); + Arrays.stream(writeResult.dataFiles()) + .forEach( + dataFile -> { + dataFilesRecordCount.addAndGet(dataFile.recordCount()); + dataFilesByteCount.addAndGet(dataFile.fileSizeInBytes()); + }); + deleteFilesCount.addAndGet(writeResult.deleteFiles().length); + Arrays.stream(writeResult.deleteFiles()) + .forEach( + deleteFile -> { + deleteFilesRecordCount.addAndGet(deleteFile.recordCount()); + deleteFilesByteCount.addAndGet(deleteFile.fileSizeInBytes()); + }); + }); + } + + long dataFilesCount() { + return dataFilesCount.get(); + } + + long dataFilesRecordCount() { + return dataFilesRecordCount.get(); + } + + long dataFilesByteCount() { + return dataFilesByteCount.get(); + } + + long deleteFilesCount() { + return deleteFilesCount.get(); + } + + long deleteFilesRecordCount() { + return deleteFilesRecordCount.get(); + } + + long deleteFilesByteCount() { + return deleteFilesByteCount.get(); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("dataFilesCount", dataFilesCount) + .add("dataFilesRecordCount", dataFilesRecordCount) + .add("dataFilesByteCount", dataFilesByteCount) + .add("deleteFilesCount", deleteFilesCount) + .add("deleteFilesRecordCount", deleteFilesRecordCount) + .add("deleteFilesByteCount", deleteFilesByteCount) + .toString(); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java new file mode 100644 index 000000000000..036970c06d5b --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.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.iceberg.flink.sink; + +import java.util.List; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +class DeltaManifests { + + private static final CharSequence[] EMPTY_REF_DATA_FILES = new CharSequence[0]; + + private final ManifestFile dataManifest; + private final ManifestFile deleteManifest; + private final CharSequence[] referencedDataFiles; + + DeltaManifests(ManifestFile dataManifest, ManifestFile deleteManifest) { + this(dataManifest, deleteManifest, EMPTY_REF_DATA_FILES); + } + + DeltaManifests( + ManifestFile dataManifest, ManifestFile deleteManifest, CharSequence[] referencedDataFiles) { + Preconditions.checkNotNull(referencedDataFiles, "Referenced data files shouldn't be null."); + + this.dataManifest = dataManifest; + this.deleteManifest = deleteManifest; + this.referencedDataFiles = referencedDataFiles; + } + + ManifestFile dataManifest() { + return dataManifest; + } + + ManifestFile deleteManifest() { + return deleteManifest; + } + + CharSequence[] referencedDataFiles() { + return referencedDataFiles; + } + + List manifests() { + List manifests = Lists.newArrayListWithCapacity(2); + if (dataManifest != null) { + manifests.add(dataManifest); + } + + if (deleteManifest != null) { + manifests.add(deleteManifest); + } + + return manifests; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java new file mode 100644 index 000000000000..92ca284b12ba --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java @@ -0,0 +1,122 @@ +/* + * 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.iceberg.flink.sink; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +class DeltaManifestsSerializer implements SimpleVersionedSerializer { + private static final int VERSION_1 = 1; + private static final int VERSION_2 = 2; + private static final byte[] EMPTY_BINARY = new byte[0]; + + static final DeltaManifestsSerializer INSTANCE = new DeltaManifestsSerializer(); + + @Override + public int getVersion() { + return VERSION_2; + } + + @Override + public byte[] serialize(DeltaManifests deltaManifests) throws IOException { + Preconditions.checkNotNull( + deltaManifests, "DeltaManifests to be serialized should not be null"); + + ByteArrayOutputStream binaryOut = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(binaryOut); + + byte[] dataManifestBinary = EMPTY_BINARY; + if (deltaManifests.dataManifest() != null) { + dataManifestBinary = ManifestFiles.encode(deltaManifests.dataManifest()); + } + + out.writeInt(dataManifestBinary.length); + out.write(dataManifestBinary); + + byte[] deleteManifestBinary = EMPTY_BINARY; + if (deltaManifests.deleteManifest() != null) { + deleteManifestBinary = ManifestFiles.encode(deltaManifests.deleteManifest()); + } + + out.writeInt(deleteManifestBinary.length); + out.write(deleteManifestBinary); + + CharSequence[] referencedDataFiles = deltaManifests.referencedDataFiles(); + out.writeInt(referencedDataFiles.length); + for (CharSequence referencedDataFile : referencedDataFiles) { + out.writeUTF(referencedDataFile.toString()); + } + + return binaryOut.toByteArray(); + } + + @Override + public DeltaManifests deserialize(int version, byte[] serialized) throws IOException { + if (version == VERSION_1) { + return deserializeV1(serialized); + } else if (version == VERSION_2) { + return deserializeV2(serialized); + } else { + throw new RuntimeException("Unknown serialize version: " + version); + } + } + + private DeltaManifests deserializeV1(byte[] serialized) throws IOException { + return new DeltaManifests(ManifestFiles.decode(serialized), null); + } + + private DeltaManifests deserializeV2(byte[] serialized) throws IOException { + ManifestFile dataManifest = null; + ManifestFile deleteManifest = null; + + ByteArrayInputStream binaryIn = new ByteArrayInputStream(serialized); + DataInputStream in = new DataInputStream(binaryIn); + + int dataManifestSize = in.readInt(); + if (dataManifestSize > 0) { + byte[] dataManifestBinary = new byte[dataManifestSize]; + Preconditions.checkState(in.read(dataManifestBinary) == dataManifestSize); + + dataManifest = ManifestFiles.decode(dataManifestBinary); + } + + int deleteManifestSize = in.readInt(); + if (deleteManifestSize > 0) { + byte[] deleteManifestBinary = new byte[deleteManifestSize]; + Preconditions.checkState(in.read(deleteManifestBinary) == deleteManifestSize); + + deleteManifest = ManifestFiles.decode(deleteManifestBinary); + } + + int referenceDataFileNum = in.readInt(); + CharSequence[] referencedDataFiles = new CharSequence[referenceDataFileNum]; + for (int i = 0; i < referenceDataFileNum; i++) { + referencedDataFiles[i] = in.readUTF(); + } + + return new DeltaManifests(dataManifest, deleteManifest, referencedDataFiles); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java new file mode 100644 index 000000000000..18b269d6c3e9 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java @@ -0,0 +1,86 @@ +/* + * 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.iceberg.flink.sink; + +import java.util.List; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.StructLikeWrapper; +import org.apache.iceberg.util.StructProjection; + +/** + * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record + * will be emitted to same writer in order. + */ +class EqualityFieldKeySelector implements KeySelector { + + private final Schema schema; + private final RowType flinkSchema; + private final Schema deleteSchema; + + private transient RowDataWrapper rowDataWrapper; + private transient StructProjection structProjection; + private transient StructLikeWrapper structLikeWrapper; + + EqualityFieldKeySelector(Schema schema, RowType flinkSchema, List equalityFieldIds) { + this.schema = schema; + this.flinkSchema = flinkSchema; + this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds)); + } + + /** + * Construct the {@link RowDataWrapper} lazily here because few members in it are not + * serializable. In this way, we don't have to serialize them with forcing. + */ + protected RowDataWrapper lazyRowDataWrapper() { + if (rowDataWrapper == null) { + rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); + } + return rowDataWrapper; + } + + /** Construct the {@link StructProjection} lazily because it is not serializable. */ + protected StructProjection lazyStructProjection() { + if (structProjection == null) { + structProjection = StructProjection.create(schema, deleteSchema); + } + return structProjection; + } + + /** Construct the {@link StructLikeWrapper} lazily because it is not serializable. */ + protected StructLikeWrapper lazyStructLikeWrapper() { + if (structLikeWrapper == null) { + structLikeWrapper = StructLikeWrapper.forType(deleteSchema.asStruct()); + } + return structLikeWrapper; + } + + @Override + public Integer getKey(RowData row) { + RowDataWrapper wrappedRowData = lazyRowDataWrapper().wrap(row); + StructProjection projectedRowData = lazyStructProjection().wrap(wrappedRowData); + StructLikeWrapper wrapper = lazyStructLikeWrapper().set(projectedRowData); + return wrapper.hashCode(); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java new file mode 100644 index 000000000000..b6f1392d1562 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java @@ -0,0 +1,274 @@ +/* + * 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.iceberg.flink.sink; + +import java.io.IOException; +import java.io.Serializable; +import java.io.UncheckedIOException; +import java.util.Map; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.MetricsConfig; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.deletes.EqualityDeleteWriter; +import org.apache.iceberg.deletes.PositionDeleteWriter; +import org.apache.iceberg.encryption.EncryptedOutputFile; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.data.FlinkAvroWriter; +import org.apache.iceberg.flink.data.FlinkOrcWriter; +import org.apache.iceberg.flink.data.FlinkParquetWriters; +import org.apache.iceberg.io.DataWriter; +import org.apache.iceberg.io.DeleteSchemaUtil; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +public class FlinkAppenderFactory implements FileAppenderFactory, Serializable { + private final Schema schema; + private final RowType flinkSchema; + private final Map props; + private final PartitionSpec spec; + private final int[] equalityFieldIds; + private final Schema eqDeleteRowSchema; + private final Schema posDeleteRowSchema; + private final Table table; + + private RowType eqDeleteFlinkSchema = null; + private RowType posDeleteFlinkSchema = null; + + public FlinkAppenderFactory( + Table table, + Schema schema, + RowType flinkSchema, + Map props, + PartitionSpec spec, + int[] equalityFieldIds, + Schema eqDeleteRowSchema, + Schema posDeleteRowSchema) { + Preconditions.checkNotNull(table, "Table shouldn't be null"); + this.table = table; + this.schema = schema; + this.flinkSchema = flinkSchema; + this.props = props; + this.spec = spec; + this.equalityFieldIds = equalityFieldIds; + this.eqDeleteRowSchema = eqDeleteRowSchema; + this.posDeleteRowSchema = posDeleteRowSchema; + } + + private RowType lazyEqDeleteFlinkSchema() { + if (eqDeleteFlinkSchema == null) { + Preconditions.checkNotNull(eqDeleteRowSchema, "Equality delete row schema shouldn't be null"); + this.eqDeleteFlinkSchema = FlinkSchemaUtil.convert(eqDeleteRowSchema); + } + return eqDeleteFlinkSchema; + } + + private RowType lazyPosDeleteFlinkSchema() { + if (posDeleteFlinkSchema == null) { + Preconditions.checkNotNull(posDeleteRowSchema, "Pos-delete row schema shouldn't be null"); + this.posDeleteFlinkSchema = FlinkSchemaUtil.convert(posDeleteRowSchema); + } + return this.posDeleteFlinkSchema; + } + + @Override + public FileAppender newAppender(OutputFile outputFile, FileFormat format) { + MetricsConfig metricsConfig = MetricsConfig.forTable(table); + try { + switch (format) { + case AVRO: + return Avro.write(outputFile) + .createWriterFunc(ignore -> new FlinkAvroWriter(flinkSchema)) + .setAll(props) + .schema(schema) + .metricsConfig(metricsConfig) + .overwrite() + .build(); + + case ORC: + return ORC.write(outputFile) + .createWriterFunc( + (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(flinkSchema, iSchema)) + .setAll(props) + .metricsConfig(metricsConfig) + .schema(schema) + .overwrite() + .build(); + + case PARQUET: + return Parquet.write(outputFile) + .createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(flinkSchema, msgType)) + .setAll(props) + .metricsConfig(metricsConfig) + .schema(schema) + .overwrite() + .build(); + + default: + throw new UnsupportedOperationException("Cannot write unknown file format: " + format); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + @Override + public DataWriter newDataWriter( + EncryptedOutputFile file, FileFormat format, StructLike partition) { + return new DataWriter<>( + newAppender(file.encryptingOutputFile(), format), + format, + file.encryptingOutputFile().location(), + spec, + partition, + file.keyMetadata()); + } + + @Override + public EqualityDeleteWriter newEqDeleteWriter( + EncryptedOutputFile outputFile, FileFormat format, StructLike partition) { + Preconditions.checkState( + equalityFieldIds != null && equalityFieldIds.length > 0, + "Equality field ids shouldn't be null or empty when creating equality-delete writer"); + Preconditions.checkNotNull( + eqDeleteRowSchema, + "Equality delete row schema shouldn't be null when creating equality-delete writer"); + + MetricsConfig metricsConfig = MetricsConfig.forTable(table); + try { + switch (format) { + case AVRO: + return Avro.writeDeletes(outputFile.encryptingOutputFile()) + .createWriterFunc(ignore -> new FlinkAvroWriter(lazyEqDeleteFlinkSchema())) + .withPartition(partition) + .overwrite() + .setAll(props) + .metricsConfig(metricsConfig) + .rowSchema(eqDeleteRowSchema) + .withSpec(spec) + .withKeyMetadata(outputFile.keyMetadata()) + .equalityFieldIds(equalityFieldIds) + .buildEqualityWriter(); + + case ORC: + return ORC.writeDeletes(outputFile.encryptingOutputFile()) + .createWriterFunc( + (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(flinkSchema, iSchema)) + .withPartition(partition) + .overwrite() + .setAll(props) + .metricsConfig(metricsConfig) + .rowSchema(eqDeleteRowSchema) + .withSpec(spec) + .withKeyMetadata(outputFile.keyMetadata()) + .equalityFieldIds(equalityFieldIds) + .buildEqualityWriter(); + + case PARQUET: + return Parquet.writeDeletes(outputFile.encryptingOutputFile()) + .createWriterFunc( + msgType -> FlinkParquetWriters.buildWriter(lazyEqDeleteFlinkSchema(), msgType)) + .withPartition(partition) + .overwrite() + .setAll(props) + .metricsConfig(metricsConfig) + .rowSchema(eqDeleteRowSchema) + .withSpec(spec) + .withKeyMetadata(outputFile.keyMetadata()) + .equalityFieldIds(equalityFieldIds) + .buildEqualityWriter(); + + default: + throw new UnsupportedOperationException( + "Cannot write equality-deletes for unsupported file format: " + format); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + @Override + public PositionDeleteWriter newPosDeleteWriter( + EncryptedOutputFile outputFile, FileFormat format, StructLike partition) { + MetricsConfig metricsConfig = MetricsConfig.forPositionDelete(table); + try { + switch (format) { + case AVRO: + return Avro.writeDeletes(outputFile.encryptingOutputFile()) + .createWriterFunc(ignore -> new FlinkAvroWriter(lazyPosDeleteFlinkSchema())) + .withPartition(partition) + .overwrite() + .setAll(props) + .metricsConfig(metricsConfig) + .rowSchema(posDeleteRowSchema) + .withSpec(spec) + .withKeyMetadata(outputFile.keyMetadata()) + .buildPositionWriter(); + + case ORC: + RowType orcPosDeleteSchema = + FlinkSchemaUtil.convert(DeleteSchemaUtil.posDeleteSchema(posDeleteRowSchema)); + return ORC.writeDeletes(outputFile.encryptingOutputFile()) + .createWriterFunc( + (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(orcPosDeleteSchema, iSchema)) + .withPartition(partition) + .overwrite() + .setAll(props) + .metricsConfig(metricsConfig) + .rowSchema(posDeleteRowSchema) + .withSpec(spec) + .withKeyMetadata(outputFile.keyMetadata()) + .transformPaths(path -> StringData.fromString(path.toString())) + .buildPositionWriter(); + + case PARQUET: + RowType flinkPosDeleteSchema = + FlinkSchemaUtil.convert(DeleteSchemaUtil.posDeleteSchema(posDeleteRowSchema)); + return Parquet.writeDeletes(outputFile.encryptingOutputFile()) + .createWriterFunc( + msgType -> FlinkParquetWriters.buildWriter(flinkPosDeleteSchema, msgType)) + .withPartition(partition) + .overwrite() + .setAll(props) + .metricsConfig(metricsConfig) + .rowSchema(posDeleteRowSchema) + .withSpec(spec) + .withKeyMetadata(outputFile.keyMetadata()) + .transformPaths(path -> StringData.fromString(path.toString())) + .buildPositionWriter(); + + default: + throw new UnsupportedOperationException( + "Cannot write pos-deletes for unsupported file format: " + format); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java new file mode 100644 index 000000000000..2183fe062af4 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java @@ -0,0 +1,293 @@ +/* + * 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.iceberg.flink.sink; + +import static org.apache.iceberg.MetadataColumns.DELETE_FILE_ROW_FIELD_NAME; +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.DELETE_DEFAULT_FILE_FORMAT; + +import java.io.Serializable; +import java.util.Map; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.Table; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.data.BaseFileWriterFactory; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.data.FlinkAvroWriter; +import org.apache.iceberg.flink.data.FlinkOrcWriter; +import org.apache.iceberg.flink.data.FlinkParquetWriters; +import org.apache.iceberg.io.DeleteSchemaUtil; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +class FlinkFileWriterFactory extends BaseFileWriterFactory implements Serializable { + private RowType dataFlinkType; + private RowType equalityDeleteFlinkType; + private RowType positionDeleteFlinkType; + + FlinkFileWriterFactory( + Table table, + FileFormat dataFileFormat, + Schema dataSchema, + RowType dataFlinkType, + SortOrder dataSortOrder, + FileFormat deleteFileFormat, + int[] equalityFieldIds, + Schema equalityDeleteRowSchema, + RowType equalityDeleteFlinkType, + SortOrder equalityDeleteSortOrder, + Schema positionDeleteRowSchema, + RowType positionDeleteFlinkType) { + + super( + table, + dataFileFormat, + dataSchema, + dataSortOrder, + deleteFileFormat, + equalityFieldIds, + equalityDeleteRowSchema, + equalityDeleteSortOrder, + positionDeleteRowSchema); + + this.dataFlinkType = dataFlinkType; + this.equalityDeleteFlinkType = equalityDeleteFlinkType; + this.positionDeleteFlinkType = positionDeleteFlinkType; + } + + static Builder builderFor(Table table) { + return new Builder(table); + } + + @Override + protected void configureDataWrite(Avro.DataWriteBuilder builder) { + builder.createWriterFunc(ignore -> new FlinkAvroWriter(dataFlinkType())); + } + + @Override + protected void configureEqualityDelete(Avro.DeleteWriteBuilder builder) { + builder.createWriterFunc(ignored -> new FlinkAvroWriter(equalityDeleteFlinkType())); + } + + @Override + protected void configurePositionDelete(Avro.DeleteWriteBuilder builder) { + int rowFieldIndex = positionDeleteFlinkType().getFieldIndex(DELETE_FILE_ROW_FIELD_NAME); + if (rowFieldIndex >= 0) { + // FlinkAvroWriter accepts just the Flink type of the row ignoring the path and pos + RowType positionDeleteRowFlinkType = + (RowType) positionDeleteFlinkType().getTypeAt(rowFieldIndex); + builder.createWriterFunc(ignored -> new FlinkAvroWriter(positionDeleteRowFlinkType)); + } + } + + @Override + protected void configureDataWrite(Parquet.DataWriteBuilder builder) { + builder.createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(dataFlinkType(), msgType)); + } + + @Override + protected void configureEqualityDelete(Parquet.DeleteWriteBuilder builder) { + builder.createWriterFunc( + msgType -> FlinkParquetWriters.buildWriter(equalityDeleteFlinkType(), msgType)); + } + + @Override + protected void configurePositionDelete(Parquet.DeleteWriteBuilder builder) { + builder.createWriterFunc( + msgType -> FlinkParquetWriters.buildWriter(positionDeleteFlinkType(), msgType)); + builder.transformPaths(path -> StringData.fromString(path.toString())); + } + + @Override + protected void configureDataWrite(ORC.DataWriteBuilder builder) { + builder.createWriterFunc( + (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(dataFlinkType(), iSchema)); + } + + @Override + protected void configureEqualityDelete(ORC.DeleteWriteBuilder builder) { + builder.createWriterFunc( + (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(equalityDeleteFlinkType(), iSchema)); + } + + @Override + protected void configurePositionDelete(ORC.DeleteWriteBuilder builder) { + builder.createWriterFunc( + (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(positionDeleteFlinkType(), iSchema)); + builder.transformPaths(path -> StringData.fromString(path.toString())); + } + + private RowType dataFlinkType() { + if (dataFlinkType == null) { + Preconditions.checkNotNull(dataSchema(), "Data schema must not be null"); + this.dataFlinkType = FlinkSchemaUtil.convert(dataSchema()); + } + + return dataFlinkType; + } + + private RowType equalityDeleteFlinkType() { + if (equalityDeleteFlinkType == null) { + Preconditions.checkNotNull( + equalityDeleteRowSchema(), "Equality delete schema must not be null"); + this.equalityDeleteFlinkType = FlinkSchemaUtil.convert(equalityDeleteRowSchema()); + } + + return equalityDeleteFlinkType; + } + + private RowType positionDeleteFlinkType() { + if (positionDeleteFlinkType == null) { + // wrap the optional row schema into the position delete schema that contains path and + // position + Schema positionDeleteSchema = DeleteSchemaUtil.posDeleteSchema(positionDeleteRowSchema()); + this.positionDeleteFlinkType = FlinkSchemaUtil.convert(positionDeleteSchema); + } + + return positionDeleteFlinkType; + } + + static class Builder { + private final Table table; + private FileFormat dataFileFormat; + private Schema dataSchema; + private RowType dataFlinkType; + private SortOrder dataSortOrder; + private FileFormat deleteFileFormat; + private int[] equalityFieldIds; + private Schema equalityDeleteRowSchema; + private RowType equalityDeleteFlinkType; + private SortOrder equalityDeleteSortOrder; + private Schema positionDeleteRowSchema; + private RowType positionDeleteFlinkType; + + Builder(Table table) { + this.table = table; + + Map properties = table.properties(); + + String dataFileFormatName = + properties.getOrDefault(DEFAULT_FILE_FORMAT, DEFAULT_FILE_FORMAT_DEFAULT); + this.dataFileFormat = FileFormat.fromString(dataFileFormatName); + + String deleteFileFormatName = + properties.getOrDefault(DELETE_DEFAULT_FILE_FORMAT, dataFileFormatName); + this.deleteFileFormat = FileFormat.fromString(deleteFileFormatName); + } + + Builder dataFileFormat(FileFormat newDataFileFormat) { + this.dataFileFormat = newDataFileFormat; + return this; + } + + Builder dataSchema(Schema newDataSchema) { + this.dataSchema = newDataSchema; + return this; + } + + /** + * Sets a Flink type for data. + * + *

If not set, the value is derived from the provided Iceberg schema. + */ + Builder dataFlinkType(RowType newDataFlinkType) { + this.dataFlinkType = newDataFlinkType; + return this; + } + + Builder dataSortOrder(SortOrder newDataSortOrder) { + this.dataSortOrder = newDataSortOrder; + return this; + } + + Builder deleteFileFormat(FileFormat newDeleteFileFormat) { + this.deleteFileFormat = newDeleteFileFormat; + return this; + } + + Builder equalityFieldIds(int[] newEqualityFieldIds) { + this.equalityFieldIds = newEqualityFieldIds; + return this; + } + + Builder equalityDeleteRowSchema(Schema newEqualityDeleteRowSchema) { + this.equalityDeleteRowSchema = newEqualityDeleteRowSchema; + return this; + } + + /** + * Sets a Flink type for equality deletes. + * + *

If not set, the value is derived from the provided Iceberg schema. + */ + Builder equalityDeleteFlinkType(RowType newEqualityDeleteFlinkType) { + this.equalityDeleteFlinkType = newEqualityDeleteFlinkType; + return this; + } + + Builder equalityDeleteSortOrder(SortOrder newEqualityDeleteSortOrder) { + this.equalityDeleteSortOrder = newEqualityDeleteSortOrder; + return this; + } + + Builder positionDeleteRowSchema(Schema newPositionDeleteRowSchema) { + this.positionDeleteRowSchema = newPositionDeleteRowSchema; + return this; + } + + /** + * Sets a Flink type for position deletes. + * + *

If not set, the value is derived from the provided Iceberg schema. + */ + Builder positionDeleteFlinkType(RowType newPositionDeleteFlinkType) { + this.positionDeleteFlinkType = newPositionDeleteFlinkType; + return this; + } + + FlinkFileWriterFactory build() { + boolean noEqualityDeleteConf = equalityFieldIds == null && equalityDeleteRowSchema == null; + boolean fullEqualityDeleteConf = equalityFieldIds != null && equalityDeleteRowSchema != null; + Preconditions.checkArgument( + noEqualityDeleteConf || fullEqualityDeleteConf, + "Equality field IDs and equality delete row schema must be set together"); + + return new FlinkFileWriterFactory( + table, + dataFileFormat, + dataSchema, + dataFlinkType, + dataSortOrder, + deleteFileFormat, + equalityFieldIds, + equalityDeleteRowSchema, + equalityDeleteFlinkType, + equalityDeleteSortOrder, + positionDeleteRowSchema, + positionDeleteFlinkType); + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java new file mode 100644 index 000000000000..c7e8a2dea7cb --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java @@ -0,0 +1,132 @@ +/* + * 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.iceberg.flink.sink; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.function.Supplier; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.ManifestWriter; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +class FlinkManifestUtil { + private static final int FORMAT_V2 = 2; + private static final Long DUMMY_SNAPSHOT_ID = 0L; + + private FlinkManifestUtil() {} + + static ManifestFile writeDataFiles( + OutputFile outputFile, PartitionSpec spec, List dataFiles) throws IOException { + ManifestWriter writer = + ManifestFiles.write(FORMAT_V2, spec, outputFile, DUMMY_SNAPSHOT_ID); + + try (ManifestWriter closeableWriter = writer) { + closeableWriter.addAll(dataFiles); + } + + return writer.toManifestFile(); + } + + static List readDataFiles( + ManifestFile manifestFile, FileIO io, Map specsById) + throws IOException { + try (CloseableIterable dataFiles = ManifestFiles.read(manifestFile, io, specsById)) { + return Lists.newArrayList(dataFiles); + } + } + + static ManifestOutputFileFactory createOutputFileFactory( + Supplier

tableSupplier, + Map tableProps, + String flinkJobId, + String operatorUniqueId, + int subTaskId, + long attemptNumber) { + return new ManifestOutputFileFactory( + tableSupplier, tableProps, flinkJobId, operatorUniqueId, subTaskId, attemptNumber); + } + + /** + * Write the {@link WriteResult} to temporary manifest files. + * + * @param result all those DataFiles/DeleteFiles in this WriteResult should be written with same + * partition spec + */ + static DeltaManifests writeCompletedFiles( + WriteResult result, Supplier outputFileSupplier, PartitionSpec spec) + throws IOException { + + ManifestFile dataManifest = null; + ManifestFile deleteManifest = null; + + // Write the completed data files into a newly created data manifest file. + if (result.dataFiles() != null && result.dataFiles().length > 0) { + dataManifest = + writeDataFiles(outputFileSupplier.get(), spec, Lists.newArrayList(result.dataFiles())); + } + + // Write the completed delete files into a newly created delete manifest file. + if (result.deleteFiles() != null && result.deleteFiles().length > 0) { + OutputFile deleteManifestFile = outputFileSupplier.get(); + + ManifestWriter deleteManifestWriter = + ManifestFiles.writeDeleteManifest(FORMAT_V2, spec, deleteManifestFile, DUMMY_SNAPSHOT_ID); + try (ManifestWriter writer = deleteManifestWriter) { + for (DeleteFile deleteFile : result.deleteFiles()) { + writer.add(deleteFile); + } + } + + deleteManifest = deleteManifestWriter.toManifestFile(); + } + + return new DeltaManifests(dataManifest, deleteManifest, result.referencedDataFiles()); + } + + static WriteResult readCompletedFiles( + DeltaManifests deltaManifests, FileIO io, Map specsById) + throws IOException { + WriteResult.Builder builder = WriteResult.builder(); + + // Read the completed data files from persisted data manifest file. + if (deltaManifests.dataManifest() != null) { + builder.addDataFiles(readDataFiles(deltaManifests.dataManifest(), io, specsById)); + } + + // Read the completed delete files from persisted delete manifests file. + if (deltaManifests.deleteManifest() != null) { + try (CloseableIterable deleteFiles = + ManifestFiles.readDeleteManifest(deltaManifests.deleteManifest(), io, specsById)) { + builder.addDeleteFiles(deleteFiles); + } + } + + return builder.addReferencedDataFiles(deltaManifests.referencedDataFiles()).build(); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java new file mode 100644 index 000000000000..769af7d77140 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java @@ -0,0 +1,654 @@ +/* + * 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.iceberg.flink.sink; + +import static org.apache.iceberg.TableProperties.AVRO_COMPRESSION; +import static org.apache.iceberg.TableProperties.AVRO_COMPRESSION_LEVEL; +import static org.apache.iceberg.TableProperties.ORC_COMPRESSION; +import static org.apache.iceberg.TableProperties.ORC_COMPRESSION_STRATEGY; +import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION; +import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION_LEVEL; +import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.functions.sink.DiscardingSink; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.util.DataFormatConverters; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.Row; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.FlinkWriteConf; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.SerializableSupplier; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class FlinkSink { + private static final Logger LOG = LoggerFactory.getLogger(FlinkSink.class); + + private static final String ICEBERG_STREAM_WRITER_NAME = + IcebergStreamWriter.class.getSimpleName(); + private static final String ICEBERG_FILES_COMMITTER_NAME = + IcebergFilesCommitter.class.getSimpleName(); + + private FlinkSink() {} + + /** + * Initialize a {@link Builder} to export the data from generic input data stream into iceberg + * table. We use {@link RowData} inside the sink connector, so users need to provide a mapper + * function and a {@link TypeInformation} to convert those generic records to a RowData + * DataStream. + * + * @param input the generic source input data stream. + * @param mapper function to convert the generic data to {@link RowData} + * @param outputType to define the {@link TypeInformation} for the input data. + * @param the data type of records. + * @return {@link Builder} to connect the iceberg table. + */ + public static Builder builderFor( + DataStream input, MapFunction mapper, TypeInformation outputType) { + return new Builder().forMapperOutputType(input, mapper, outputType); + } + + /** + * Initialize a {@link Builder} to export the data from input data stream with {@link Row}s into + * iceberg table. We use {@link RowData} inside the sink connector, so users need to provide a + * {@link TableSchema} for builder to convert those {@link Row}s to a {@link RowData} DataStream. + * + * @param input the source input data stream with {@link Row}s. + * @param tableSchema defines the {@link TypeInformation} for input data. + * @return {@link Builder} to connect the iceberg table. + */ + public static Builder forRow(DataStream input, TableSchema tableSchema) { + RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType(); + DataType[] fieldDataTypes = tableSchema.getFieldDataTypes(); + + DataFormatConverters.RowConverter rowConverter = + new DataFormatConverters.RowConverter(fieldDataTypes); + return builderFor(input, rowConverter::toInternal, FlinkCompatibilityUtil.toTypeInfo(rowType)) + .tableSchema(tableSchema); + } + + /** + * Initialize a {@link Builder} to export the data from input data stream with {@link RowData}s + * into iceberg table. + * + * @param input the source input data stream with {@link RowData}s. + * @return {@link Builder} to connect the iceberg table. + */ + public static Builder forRowData(DataStream input) { + return new Builder().forRowData(input); + } + + public static class Builder { + private Function> inputCreator = null; + private TableLoader tableLoader; + private Table table; + private TableSchema tableSchema; + private List equalityFieldColumns = null; + private String uidPrefix = null; + private final Map snapshotProperties = Maps.newHashMap(); + private ReadableConfig readableConfig = new Configuration(); + private final Map writeOptions = Maps.newHashMap(); + private FlinkWriteConf flinkWriteConf = null; + + private Builder() {} + + private Builder forRowData(DataStream newRowDataInput) { + this.inputCreator = ignored -> newRowDataInput; + return this; + } + + private Builder forMapperOutputType( + DataStream input, MapFunction mapper, TypeInformation outputType) { + this.inputCreator = + newUidPrefix -> { + // Input stream order is crucial for some situation(e.g. in cdc case). Therefore, we + // need to set the parallelism + // of map operator same as its input to keep map operator chaining its input, and avoid + // rebalanced by default. + SingleOutputStreamOperator inputStream = + input.map(mapper, outputType).setParallelism(input.getParallelism()); + if (newUidPrefix != null) { + inputStream.name(operatorName(newUidPrefix)).uid(newUidPrefix + "-mapper"); + } + return inputStream; + }; + return this; + } + + /** + * This iceberg {@link Table} instance is used for initializing {@link IcebergStreamWriter} + * which will write all the records into {@link DataFile}s and emit them to downstream operator. + * Providing a table would avoid so many table loading from each separate task. + * + * @param newTable the loaded iceberg table instance. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder table(Table newTable) { + this.table = newTable; + return this; + } + + /** + * The table loader is used for loading tables in {@link IcebergFilesCommitter} lazily, we need + * this loader because {@link Table} is not serializable and could not just use the loaded table + * from Builder#table in the remote task manager. + * + * @param newTableLoader to load iceberg table inside tasks. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder tableLoader(TableLoader newTableLoader) { + this.tableLoader = newTableLoader; + return this; + } + + /** + * Set the write properties for Flink sink. View the supported properties in {@link + * FlinkWriteOptions} + */ + public Builder set(String property, String value) { + writeOptions.put(property, value); + return this; + } + + /** + * Set the write properties for Flink sink. View the supported properties in {@link + * FlinkWriteOptions} + */ + public Builder setAll(Map properties) { + writeOptions.putAll(properties); + return this; + } + + public Builder tableSchema(TableSchema newTableSchema) { + this.tableSchema = newTableSchema; + return this; + } + + public Builder overwrite(boolean newOverwrite) { + writeOptions.put(FlinkWriteOptions.OVERWRITE_MODE.key(), Boolean.toString(newOverwrite)); + return this; + } + + public Builder flinkConf(ReadableConfig config) { + this.readableConfig = config; + return this; + } + + /** + * Configure the write {@link DistributionMode} that the flink sink will use. Currently, flink + * support {@link DistributionMode#NONE} and {@link DistributionMode#HASH}. + * + * @param mode to specify the write distribution mode. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder distributionMode(DistributionMode mode) { + Preconditions.checkArgument( + !DistributionMode.RANGE.equals(mode), + "Flink does not support 'range' write distribution mode now."); + if (mode != null) { + writeOptions.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), mode.modeName()); + } + return this; + } + + /** + * Configuring the write parallel number for iceberg stream writer. + * + * @param newWriteParallelism the number of parallel iceberg stream writer. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder writeParallelism(int newWriteParallelism) { + writeOptions.put( + FlinkWriteOptions.WRITE_PARALLELISM.key(), Integer.toString(newWriteParallelism)); + return this; + } + + /** + * All INSERT/UPDATE_AFTER events from input stream will be transformed to UPSERT events, which + * means it will DELETE the old records and then INSERT the new records. In partitioned table, + * the partition fields should be a subset of equality fields, otherwise the old row that + * located in partition-A could not be deleted by the new row that located in partition-B. + * + * @param enabled indicate whether it should transform all INSERT/UPDATE_AFTER events to UPSERT. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder upsert(boolean enabled) { + writeOptions.put(FlinkWriteOptions.WRITE_UPSERT_ENABLED.key(), Boolean.toString(enabled)); + return this; + } + + /** + * Configuring the equality field columns for iceberg table that accept CDC or UPSERT events. + * + * @param columns defines the iceberg table's key. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder equalityFieldColumns(List columns) { + this.equalityFieldColumns = columns; + return this; + } + + /** + * Set the uid prefix for FlinkSink operators. Note that FlinkSink internally consists of + * multiple operators (like writer, committer, dummy sink etc.) Actually operator uid will be + * appended with a suffix like "uidPrefix-writer".
+ *
+ * If provided, this prefix is also applied to operator names.
+ *
+ * Flink auto generates operator uid if not set explicitly. It is a recommended + * best-practice to set uid for all operators before deploying to production. Flink has an + * option to {@code pipeline.auto-generate-uid=false} to disable auto-generation and force + * explicit setting of all operator uid.
+ *
+ * Be careful with setting this for an existing job, because now we are changing the operator + * uid from an auto-generated one to this new value. When deploying the change with a + * checkpoint, Flink won't be able to restore the previous Flink sink operator state (more + * specifically the committer operator state). You need to use {@code --allowNonRestoredState} + * to ignore the previous sink state. During restore Flink sink state is used to check if last + * commit was actually successful or not. {@code --allowNonRestoredState} can lead to data loss + * if the Iceberg commit failed in the last completed checkpoint. + * + * @param newPrefix prefix for Flink sink operator uid and name + * @return {@link Builder} to connect the iceberg table. + */ + public Builder uidPrefix(String newPrefix) { + this.uidPrefix = newPrefix; + return this; + } + + public Builder setSnapshotProperties(Map properties) { + snapshotProperties.putAll(properties); + return this; + } + + public Builder setSnapshotProperty(String property, String value) { + snapshotProperties.put(property, value); + return this; + } + + public Builder toBranch(String branch) { + writeOptions.put(FlinkWriteOptions.BRANCH.key(), branch); + return this; + } + + private DataStreamSink chainIcebergOperators() { + Preconditions.checkArgument( + inputCreator != null, + "Please use forRowData() or forMapperOutputType() to initialize the input DataStream."); + Preconditions.checkNotNull(tableLoader, "Table loader shouldn't be null"); + + DataStream rowDataInput = inputCreator.apply(uidPrefix); + + if (table == null) { + if (!tableLoader.isOpen()) { + tableLoader.open(); + } + + try (TableLoader loader = tableLoader) { + this.table = loader.loadTable(); + } catch (IOException e) { + throw new UncheckedIOException( + "Failed to load iceberg table from table loader: " + tableLoader, e); + } + } + + flinkWriteConf = new FlinkWriteConf(table, writeOptions, readableConfig); + + // Find out the equality field id list based on the user-provided equality field column names. + List equalityFieldIds = checkAndGetEqualityFieldIds(); + + // Convert the requested flink table schema to flink row type. + RowType flinkRowType = toFlinkRowType(table.schema(), tableSchema); + + // Distribute the records from input data stream based on the write.distribution-mode and + // equality fields. + DataStream distributeStream = + distributeDataStream( + rowDataInput, equalityFieldIds, table.spec(), table.schema(), flinkRowType); + + // Add parallel writers that append rows to files + SingleOutputStreamOperator writerStream = + appendWriter(distributeStream, flinkRowType, equalityFieldIds); + + // Add single-parallelism committer that commits files + // after successful checkpoint or end of input + SingleOutputStreamOperator committerStream = appendCommitter(writerStream); + + // Add dummy discard sink + return appendDummySink(committerStream); + } + + /** + * Append the iceberg sink operators to write records to iceberg table. + * + * @return {@link DataStreamSink} for sink. + */ + public DataStreamSink append() { + return chainIcebergOperators(); + } + + private String operatorName(String suffix) { + return uidPrefix != null ? uidPrefix + "-" + suffix : suffix; + } + + @VisibleForTesting + List checkAndGetEqualityFieldIds() { + List equalityFieldIds = Lists.newArrayList(table.schema().identifierFieldIds()); + if (equalityFieldColumns != null && !equalityFieldColumns.isEmpty()) { + Set equalityFieldSet = + Sets.newHashSetWithExpectedSize(equalityFieldColumns.size()); + for (String column : equalityFieldColumns) { + org.apache.iceberg.types.Types.NestedField field = table.schema().findField(column); + Preconditions.checkNotNull( + field, + "Missing required equality field column '%s' in table schema %s", + column, + table.schema()); + equalityFieldSet.add(field.fieldId()); + } + + if (!equalityFieldSet.equals(table.schema().identifierFieldIds())) { + LOG.warn( + "The configured equality field column IDs {} are not matched with the schema identifier field IDs" + + " {}, use job specified equality field columns as the equality fields by default.", + equalityFieldSet, + table.schema().identifierFieldIds()); + } + equalityFieldIds = Lists.newArrayList(equalityFieldSet); + } + return equalityFieldIds; + } + + @SuppressWarnings("unchecked") + private DataStreamSink appendDummySink( + SingleOutputStreamOperator committerStream) { + DataStreamSink resultStream = + committerStream + .addSink(new DiscardingSink()) + .name(operatorName(String.format("IcebergSink %s", this.table.name()))) + .setParallelism(1); + if (uidPrefix != null) { + resultStream = resultStream.uid(uidPrefix + "-dummysink"); + } + return resultStream; + } + + private SingleOutputStreamOperator appendCommitter( + SingleOutputStreamOperator writerStream) { + IcebergFilesCommitter filesCommitter = + new IcebergFilesCommitter( + tableLoader, + flinkWriteConf.overwriteMode(), + snapshotProperties, + flinkWriteConf.workerPoolSize(), + flinkWriteConf.branch(), + table.spec()); + SingleOutputStreamOperator committerStream = + writerStream + .transform(operatorName(ICEBERG_FILES_COMMITTER_NAME), Types.VOID, filesCommitter) + .setParallelism(1) + .setMaxParallelism(1); + if (uidPrefix != null) { + committerStream = committerStream.uid(uidPrefix + "-committer"); + } + return committerStream; + } + + private SingleOutputStreamOperator appendWriter( + DataStream input, RowType flinkRowType, List equalityFieldIds) { + // Validate the equality fields and partition fields if we enable the upsert mode. + if (flinkWriteConf.upsertMode()) { + Preconditions.checkState( + !flinkWriteConf.overwriteMode(), + "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); + Preconditions.checkState( + !equalityFieldIds.isEmpty(), + "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + if (!table.spec().isUnpartitioned()) { + for (PartitionField partitionField : table.spec().fields()) { + Preconditions.checkState( + equalityFieldIds.contains(partitionField.sourceId()), + "In UPSERT mode, partition field '%s' should be included in equality fields: '%s'", + partitionField, + equalityFieldColumns); + } + } + } + + SerializableTable serializableTable = (SerializableTable) SerializableTable.copyOf(table); + Duration tableRefreshInterval = flinkWriteConf.tableRefreshInterval(); + + SerializableSupplier
tableSupplier; + if (tableRefreshInterval != null) { + tableSupplier = + new CachingTableSupplier(serializableTable, tableLoader, tableRefreshInterval); + } else { + tableSupplier = () -> serializableTable; + } + + IcebergStreamWriter streamWriter = + createStreamWriter(tableSupplier, flinkWriteConf, flinkRowType, equalityFieldIds); + + int parallelism = + flinkWriteConf.writeParallelism() == null + ? input.getParallelism() + : flinkWriteConf.writeParallelism(); + SingleOutputStreamOperator writerStream = + input + .transform( + operatorName(ICEBERG_STREAM_WRITER_NAME), + TypeInformation.of(WriteResult.class), + streamWriter) + .setParallelism(parallelism); + if (uidPrefix != null) { + writerStream = writerStream.uid(uidPrefix + "-writer"); + } + return writerStream; + } + + private DataStream distributeDataStream( + DataStream input, + List equalityFieldIds, + PartitionSpec partitionSpec, + Schema iSchema, + RowType flinkRowType) { + DistributionMode writeMode = flinkWriteConf.distributionMode(); + + LOG.info("Write distribution mode is '{}'", writeMode.modeName()); + switch (writeMode) { + case NONE: + if (equalityFieldIds.isEmpty()) { + return input; + } else { + LOG.info("Distribute rows by equality fields, because there are equality fields set"); + return input.keyBy( + new EqualityFieldKeySelector(iSchema, flinkRowType, equalityFieldIds)); + } + + case HASH: + if (equalityFieldIds.isEmpty()) { + if (partitionSpec.isUnpartitioned()) { + LOG.warn( + "Fallback to use 'none' distribution mode, because there are no equality fields set " + + "and table is unpartitioned"); + return input; + } else { + return input.keyBy(new PartitionKeySelector(partitionSpec, iSchema, flinkRowType)); + } + } else { + if (partitionSpec.isUnpartitioned()) { + LOG.info( + "Distribute rows by equality fields, because there are equality fields set " + + "and table is unpartitioned"); + return input.keyBy( + new EqualityFieldKeySelector(iSchema, flinkRowType, equalityFieldIds)); + } else { + for (PartitionField partitionField : partitionSpec.fields()) { + Preconditions.checkState( + equalityFieldIds.contains(partitionField.sourceId()), + "In 'hash' distribution mode with equality fields set, partition field '%s' " + + "should be included in equality fields: '%s'", + partitionField, + equalityFieldColumns); + } + return input.keyBy(new PartitionKeySelector(partitionSpec, iSchema, flinkRowType)); + } + } + + case RANGE: + if (equalityFieldIds.isEmpty()) { + LOG.warn( + "Fallback to use 'none' distribution mode, because there are no equality fields set " + + "and {}=range is not supported yet in flink", + WRITE_DISTRIBUTION_MODE); + return input; + } else { + LOG.info( + "Distribute rows by equality fields, because there are equality fields set " + + "and{}=range is not supported yet in flink", + WRITE_DISTRIBUTION_MODE); + return input.keyBy( + new EqualityFieldKeySelector(iSchema, flinkRowType, equalityFieldIds)); + } + + default: + throw new RuntimeException("Unrecognized " + WRITE_DISTRIBUTION_MODE + ": " + writeMode); + } + } + } + + static RowType toFlinkRowType(Schema schema, TableSchema requestedSchema) { + if (requestedSchema != null) { + // Convert the flink schema to iceberg schema firstly, then reassign ids to match the existing + // iceberg schema. + Schema writeSchema = TypeUtil.reassignIds(FlinkSchemaUtil.convert(requestedSchema), schema); + TypeUtil.validateWriteSchema(schema, writeSchema, true, true); + + // We use this flink schema to read values from RowData. The flink's TINYINT and SMALLINT will + // be promoted to + // iceberg INTEGER, that means if we use iceberg's table schema to read TINYINT (backend by 1 + // 'byte'), we will + // read 4 bytes rather than 1 byte, it will mess up the byte array in BinaryRowData. So here + // we must use flink + // schema. + return (RowType) requestedSchema.toRowDataType().getLogicalType(); + } else { + return FlinkSchemaUtil.convert(schema); + } + } + + static IcebergStreamWriter createStreamWriter( + SerializableSupplier
tableSupplier, + FlinkWriteConf flinkWriteConf, + RowType flinkRowType, + List equalityFieldIds) { + Preconditions.checkArgument(tableSupplier != null, "Iceberg table supplier shouldn't be null"); + + Table initTable = tableSupplier.get(); + FileFormat format = flinkWriteConf.dataFileFormat(); + TaskWriterFactory taskWriterFactory = + new RowDataTaskWriterFactory( + tableSupplier, + flinkRowType, + flinkWriteConf.targetDataFileSize(), + format, + writeProperties(initTable, format, flinkWriteConf), + equalityFieldIds, + flinkWriteConf.upsertMode()); + + return new IcebergStreamWriter<>(initTable.name(), taskWriterFactory); + } + + /** + * Based on the {@link FileFormat} overwrites the table level compression properties for the table + * write. + * + * @param table The table to get the table level settings + * @param format The FileFormat to use + * @param conf The write configuration + * @return The properties to use for writing + */ + private static Map writeProperties( + Table table, FileFormat format, FlinkWriteConf conf) { + Map writeProperties = Maps.newHashMap(table.properties()); + + switch (format) { + case PARQUET: + writeProperties.put(PARQUET_COMPRESSION, conf.parquetCompressionCodec()); + String parquetCompressionLevel = conf.parquetCompressionLevel(); + if (parquetCompressionLevel != null) { + writeProperties.put(PARQUET_COMPRESSION_LEVEL, parquetCompressionLevel); + } + + break; + case AVRO: + writeProperties.put(AVRO_COMPRESSION, conf.avroCompressionCodec()); + String avroCompressionLevel = conf.avroCompressionLevel(); + if (avroCompressionLevel != null) { + writeProperties.put(AVRO_COMPRESSION_LEVEL, conf.avroCompressionLevel()); + } + + break; + case ORC: + writeProperties.put(ORC_COMPRESSION, conf.orcCompressionCodec()); + writeProperties.put(ORC_COMPRESSION_STRATEGY, conf.orcCompressionStrategy()); + break; + default: + throw new IllegalArgumentException(String.format("Unknown file format %s", format)); + } + + return writeProperties; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java new file mode 100644 index 000000000000..b9bceaa9311d --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java @@ -0,0 +1,516 @@ +/* + * 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.iceberg.flink.sink; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.SortedMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; +import org.apache.flink.core.io.SimpleVersionedSerialization; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.BoundedOneInput; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.table.runtime.typeutils.SortedMapTypeInfo; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.ReplacePartitions; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotUpdate; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.base.Strings; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Comparators; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class IcebergFilesCommitter extends AbstractStreamOperator + implements OneInputStreamOperator, BoundedOneInput { + + private static final long serialVersionUID = 1L; + private static final long INITIAL_CHECKPOINT_ID = -1L; + private static final byte[] EMPTY_MANIFEST_DATA = new byte[0]; + + private static final Logger LOG = LoggerFactory.getLogger(IcebergFilesCommitter.class); + private static final String FLINK_JOB_ID = "flink.job-id"; + private static final String OPERATOR_ID = "flink.operator-id"; + + // The max checkpoint id we've committed to iceberg table. As the flink's checkpoint is always + // increasing, so we could correctly commit all the data files whose checkpoint id is greater than + // the max committed one to iceberg table, for avoiding committing the same data files twice. This + // id will be attached to iceberg's meta when committing the iceberg transaction. + private static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed-checkpoint-id"; + static final String MAX_CONTINUOUS_EMPTY_COMMITS = "flink.max-continuous-empty-commits"; + + // TableLoader to load iceberg table lazily. + private final TableLoader tableLoader; + private final boolean replacePartitions; + private final Map snapshotProperties; + + // A sorted map to maintain the completed data files for each pending checkpointId (which have not + // been committed to iceberg table). We need a sorted map here because there's possible that few + // checkpoints snapshot failed, for example: the 1st checkpoint have 2 data files <1, >, the 2st checkpoint have 1 data files <2, >. Snapshot for checkpoint#1 + // interrupted because of network/disk failure etc, while we don't expect any data loss in iceberg + // table. So we keep the finished files <1, > in memory and retry to commit iceberg + // table when the next checkpoint happen. + private final NavigableMap dataFilesPerCheckpoint = Maps.newTreeMap(); + + // The completed files cache for current checkpoint. Once the snapshot barrier received, it will + // be flushed to the 'dataFilesPerCheckpoint'. + private final List writeResultsOfCurrentCkpt = Lists.newArrayList(); + private final String branch; + + // It will have an unique identifier for one job. + private transient String flinkJobId; + private transient String operatorUniqueId; + private transient Table table; + private transient IcebergFilesCommitterMetrics committerMetrics; + private transient ManifestOutputFileFactory manifestOutputFileFactory; + private transient long maxCommittedCheckpointId; + private transient int continuousEmptyCheckpoints; + private transient int maxContinuousEmptyCommits; + // There're two cases that we restore from flink checkpoints: the first case is restoring from + // snapshot created by the same flink job; another case is restoring from snapshot created by + // another different job. For the second case, we need to maintain the old flink job's id in flink + // state backend to find the max-committed-checkpoint-id when traversing iceberg table's + // snapshots. + private static final ListStateDescriptor JOB_ID_DESCRIPTOR = + new ListStateDescriptor<>("iceberg-flink-job-id", BasicTypeInfo.STRING_TYPE_INFO); + private transient ListState jobIdState; + // All pending checkpoints states for this function. + private static final ListStateDescriptor> STATE_DESCRIPTOR = + buildStateDescriptor(); + private transient ListState> checkpointsState; + + private final Integer workerPoolSize; + private final PartitionSpec spec; + private transient ExecutorService workerPool; + + IcebergFilesCommitter( + TableLoader tableLoader, + boolean replacePartitions, + Map snapshotProperties, + Integer workerPoolSize, + String branch, + PartitionSpec spec) { + this.tableLoader = tableLoader; + this.replacePartitions = replacePartitions; + this.snapshotProperties = snapshotProperties; + this.workerPoolSize = workerPoolSize; + this.branch = branch; + this.spec = spec; + } + + @Override + public void initializeState(StateInitializationContext context) throws Exception { + super.initializeState(context); + this.flinkJobId = getContainingTask().getEnvironment().getJobID().toString(); + this.operatorUniqueId = getRuntimeContext().getOperatorUniqueID(); + + // Open the table loader and load the table. + this.tableLoader.open(); + this.table = tableLoader.loadTable(); + this.committerMetrics = new IcebergFilesCommitterMetrics(super.metrics, table.name()); + + maxContinuousEmptyCommits = + PropertyUtil.propertyAsInt(table.properties(), MAX_CONTINUOUS_EMPTY_COMMITS, 10); + Preconditions.checkArgument( + maxContinuousEmptyCommits > 0, MAX_CONTINUOUS_EMPTY_COMMITS + " must be positive"); + + int subTaskId = getRuntimeContext().getIndexOfThisSubtask(); + int attemptId = getRuntimeContext().getAttemptNumber(); + this.manifestOutputFileFactory = + FlinkManifestUtil.createOutputFileFactory( + () -> table, table.properties(), flinkJobId, operatorUniqueId, subTaskId, attemptId); + this.maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID; + + this.checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR); + this.jobIdState = context.getOperatorStateStore().getListState(JOB_ID_DESCRIPTOR); + if (context.isRestored()) { + Iterable jobIdIterable = jobIdState.get(); + if (jobIdIterable == null || !jobIdIterable.iterator().hasNext()) { + LOG.warn( + "Failed to restore committer state. This can happen when operator uid changed and Flink " + + "allowNonRestoredState is enabled. Best practice is to explicitly set the operator id " + + "via FlinkSink#Builder#uidPrefix() so that the committer operator uid is stable. " + + "Otherwise, Flink auto generate an operator uid based on job topology." + + "With that, operator uid is subjective to change upon topology change."); + return; + } + + String restoredFlinkJobId = jobIdIterable.iterator().next(); + Preconditions.checkState( + !Strings.isNullOrEmpty(restoredFlinkJobId), + "Flink job id parsed from checkpoint snapshot shouldn't be null or empty"); + + // Since flink's checkpoint id will start from the max-committed-checkpoint-id + 1 in the new + // flink job even if it's restored from a snapshot created by another different flink job, so + // it's safe to assign the max committed checkpoint id from restored flink job to the current + // flink job. + this.maxCommittedCheckpointId = + getMaxCommittedCheckpointId(table, restoredFlinkJobId, operatorUniqueId, branch); + + NavigableMap uncommittedDataFiles = + Maps.newTreeMap(checkpointsState.get().iterator().next()) + .tailMap(maxCommittedCheckpointId, false); + if (!uncommittedDataFiles.isEmpty()) { + // Committed all uncommitted data files from the old flink job to iceberg table. + long maxUncommittedCheckpointId = uncommittedDataFiles.lastKey(); + commitUpToCheckpoint( + uncommittedDataFiles, restoredFlinkJobId, operatorUniqueId, maxUncommittedCheckpointId); + } + } + } + + @Override + public void snapshotState(StateSnapshotContext context) throws Exception { + super.snapshotState(context); + long checkpointId = context.getCheckpointId(); + LOG.info( + "Start to flush snapshot state to state backend, table: {}, checkpointId: {}", + table, + checkpointId); + + // Update the checkpoint state. + long startNano = System.nanoTime(); + dataFilesPerCheckpoint.put(checkpointId, writeToManifest(checkpointId)); + // Reset the snapshot state to the latest state. + checkpointsState.clear(); + checkpointsState.add(dataFilesPerCheckpoint); + + jobIdState.clear(); + jobIdState.add(flinkJobId); + + // Clear the local buffer for current checkpoint. + writeResultsOfCurrentCkpt.clear(); + committerMetrics.checkpointDuration( + TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + super.notifyCheckpointComplete(checkpointId); + // It's possible that we have the following events: + // 1. snapshotState(ckpId); + // 2. snapshotState(ckpId+1); + // 3. notifyCheckpointComplete(ckpId+1); + // 4. notifyCheckpointComplete(ckpId); + // For step#4, we don't need to commit iceberg table again because in step#3 we've committed all + // the files, + // Besides, we need to maintain the max-committed-checkpoint-id to be increasing. + if (checkpointId > maxCommittedCheckpointId) { + LOG.info("Checkpoint {} completed. Attempting commit.", checkpointId); + commitUpToCheckpoint(dataFilesPerCheckpoint, flinkJobId, operatorUniqueId, checkpointId); + this.maxCommittedCheckpointId = checkpointId; + } else { + LOG.info( + "Skipping committing checkpoint {}. {} is already committed.", + checkpointId, + maxCommittedCheckpointId); + } + + // reload the table in case new configuration is needed + this.table = tableLoader.loadTable(); + } + + private void commitUpToCheckpoint( + NavigableMap deltaManifestsMap, + String newFlinkJobId, + String operatorId, + long checkpointId) + throws IOException { + NavigableMap pendingMap = deltaManifestsMap.headMap(checkpointId, true); + List manifests = Lists.newArrayList(); + NavigableMap pendingResults = Maps.newTreeMap(); + for (Map.Entry e : pendingMap.entrySet()) { + if (Arrays.equals(EMPTY_MANIFEST_DATA, e.getValue())) { + // Skip the empty flink manifest. + continue; + } + + DeltaManifests deltaManifests = + SimpleVersionedSerialization.readVersionAndDeSerialize( + DeltaManifestsSerializer.INSTANCE, e.getValue()); + pendingResults.put( + e.getKey(), + FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs())); + manifests.addAll(deltaManifests.manifests()); + } + + CommitSummary summary = new CommitSummary(pendingResults); + commitPendingResult(pendingResults, summary, newFlinkJobId, operatorId, checkpointId); + committerMetrics.updateCommitSummary(summary); + pendingMap.clear(); + deleteCommittedManifests(manifests, newFlinkJobId, checkpointId); + } + + private void commitPendingResult( + NavigableMap pendingResults, + CommitSummary summary, + String newFlinkJobId, + String operatorId, + long checkpointId) { + long totalFiles = summary.dataFilesCount() + summary.deleteFilesCount(); + continuousEmptyCheckpoints = totalFiles == 0 ? continuousEmptyCheckpoints + 1 : 0; + if (totalFiles != 0 || continuousEmptyCheckpoints % maxContinuousEmptyCommits == 0) { + if (replacePartitions) { + replacePartitions(pendingResults, summary, newFlinkJobId, operatorId, checkpointId); + } else { + commitDeltaTxn(pendingResults, summary, newFlinkJobId, operatorId, checkpointId); + } + continuousEmptyCheckpoints = 0; + } else { + LOG.info("Skip commit for checkpoint {} due to no data files or delete files.", checkpointId); + } + } + + private void deleteCommittedManifests( + List manifests, String newFlinkJobId, long checkpointId) { + for (ManifestFile manifest : manifests) { + try { + table.io().deleteFile(manifest.path()); + } catch (Exception e) { + // The flink manifests cleaning failure shouldn't abort the completed checkpoint. + String details = + MoreObjects.toStringHelper(this) + .add("flinkJobId", newFlinkJobId) + .add("checkpointId", checkpointId) + .add("manifestPath", manifest.path()) + .toString(); + LOG.warn( + "The iceberg transaction has been committed, but we failed to clean the temporary flink manifests: {}", + details, + e); + } + } + } + + private void replacePartitions( + NavigableMap pendingResults, + CommitSummary summary, + String newFlinkJobId, + String operatorId, + long checkpointId) { + Preconditions.checkState( + summary.deleteFilesCount() == 0, "Cannot overwrite partitions with delete files."); + // Commit the overwrite transaction. + ReplacePartitions dynamicOverwrite = table.newReplacePartitions().scanManifestsWith(workerPool); + for (WriteResult result : pendingResults.values()) { + Preconditions.checkState( + result.referencedDataFiles().length == 0, "Should have no referenced data files."); + Arrays.stream(result.dataFiles()).forEach(dynamicOverwrite::addFile); + } + + commitOperation( + dynamicOverwrite, + summary, + "dynamic partition overwrite", + newFlinkJobId, + operatorId, + checkpointId); + } + + private void commitDeltaTxn( + NavigableMap pendingResults, + CommitSummary summary, + String newFlinkJobId, + String operatorId, + long checkpointId) { + if (summary.deleteFilesCount() == 0) { + // To be compatible with iceberg format V1. + AppendFiles appendFiles = table.newAppend().scanManifestsWith(workerPool); + for (WriteResult result : pendingResults.values()) { + Preconditions.checkState( + result.referencedDataFiles().length == 0, + "Should have no referenced data files for append."); + Arrays.stream(result.dataFiles()).forEach(appendFiles::appendFile); + } + commitOperation(appendFiles, summary, "append", newFlinkJobId, operatorId, checkpointId); + } else { + // To be compatible with iceberg format V2. + for (Map.Entry e : pendingResults.entrySet()) { + // We don't commit the merged result into a single transaction because for the sequential + // transaction txn1 and txn2, the equality-delete files of txn2 are required to be applied + // to data files from txn1. Committing the merged one will lead to the incorrect delete + // semantic. + WriteResult result = e.getValue(); + + // Row delta validations are not needed for streaming changes that write equality deletes. + // Equality deletes are applied to data in all previous sequence numbers, so retries may + // push deletes further in the future, but do not affect correctness. Position deletes + // committed to the table in this path are used only to delete rows from data files that are + // being added in this commit. There is no way for data files added along with the delete + // files to be concurrently removed, so there is no need to validate the files referenced by + // the position delete files that are being committed. + RowDelta rowDelta = table.newRowDelta().scanManifestsWith(workerPool); + + Arrays.stream(result.dataFiles()).forEach(rowDelta::addRows); + Arrays.stream(result.deleteFiles()).forEach(rowDelta::addDeletes); + commitOperation(rowDelta, summary, "rowDelta", newFlinkJobId, operatorId, e.getKey()); + } + } + } + + private void commitOperation( + SnapshotUpdate operation, + CommitSummary summary, + String description, + String newFlinkJobId, + String operatorId, + long checkpointId) { + LOG.info( + "Committing {} for checkpoint {} to table {} branch {} with summary: {}", + description, + checkpointId, + table.name(), + branch, + summary); + snapshotProperties.forEach(operation::set); + // custom snapshot metadata properties will be overridden if they conflict with internal ones + // used by the sink. + operation.set(MAX_COMMITTED_CHECKPOINT_ID, Long.toString(checkpointId)); + operation.set(FLINK_JOB_ID, newFlinkJobId); + operation.set(OPERATOR_ID, operatorId); + operation.toBranch(branch); + + long startNano = System.nanoTime(); + operation.commit(); // abort is automatically called if this fails. + long durationMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano); + LOG.info( + "Committed {} to table: {}, branch: {}, checkpointId {} in {} ms", + description, + table.name(), + branch, + checkpointId, + durationMs); + committerMetrics.commitDuration(durationMs); + } + + @Override + public void processElement(StreamRecord element) { + this.writeResultsOfCurrentCkpt.add(element.getValue()); + } + + @Override + public void endInput() throws IOException { + // Flush the buffered data files into 'dataFilesPerCheckpoint' firstly. + long currentCheckpointId = Long.MAX_VALUE; + dataFilesPerCheckpoint.put(currentCheckpointId, writeToManifest(currentCheckpointId)); + writeResultsOfCurrentCkpt.clear(); + + commitUpToCheckpoint(dataFilesPerCheckpoint, flinkJobId, operatorUniqueId, currentCheckpointId); + } + + /** + * Write all the complete data files to a newly created manifest file and return the manifest's + * avro serialized bytes. + */ + private byte[] writeToManifest(long checkpointId) throws IOException { + if (writeResultsOfCurrentCkpt.isEmpty()) { + return EMPTY_MANIFEST_DATA; + } + + WriteResult result = WriteResult.builder().addAll(writeResultsOfCurrentCkpt).build(); + DeltaManifests deltaManifests = + FlinkManifestUtil.writeCompletedFiles( + result, () -> manifestOutputFileFactory.create(checkpointId), spec); + + return SimpleVersionedSerialization.writeVersionAndSerialize( + DeltaManifestsSerializer.INSTANCE, deltaManifests); + } + + @Override + public void open() throws Exception { + super.open(); + + final String operatorID = getRuntimeContext().getOperatorUniqueID(); + this.workerPool = + ThreadPools.newWorkerPool("iceberg-worker-pool-" + operatorID, workerPoolSize); + } + + @Override + public void close() throws Exception { + if (tableLoader != null) { + tableLoader.close(); + } + + if (workerPool != null) { + workerPool.shutdown(); + } + } + + @VisibleForTesting + static ListStateDescriptor> buildStateDescriptor() { + Comparator longComparator = Comparators.forType(Types.LongType.get()); + // Construct a SortedMapTypeInfo. + SortedMapTypeInfo sortedMapTypeInfo = + new SortedMapTypeInfo<>( + BasicTypeInfo.LONG_TYPE_INFO, + PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO, + longComparator); + return new ListStateDescriptor<>("iceberg-files-committer-state", sortedMapTypeInfo); + } + + static long getMaxCommittedCheckpointId( + Table table, String flinkJobId, String operatorId, String branch) { + Snapshot snapshot = table.snapshot(branch); + long lastCommittedCheckpointId = INITIAL_CHECKPOINT_ID; + + while (snapshot != null) { + Map summary = snapshot.summary(); + String snapshotFlinkJobId = summary.get(FLINK_JOB_ID); + String snapshotOperatorId = summary.get(OPERATOR_ID); + if (flinkJobId.equals(snapshotFlinkJobId) + && (snapshotOperatorId == null || snapshotOperatorId.equals(operatorId))) { + String value = summary.get(MAX_COMMITTED_CHECKPOINT_ID); + if (value != null) { + lastCommittedCheckpointId = Long.parseLong(value); + break; + } + } + Long parentSnapshotId = snapshot.parentId(); + snapshot = parentSnapshotId != null ? table.snapshot(parentSnapshotId) : null; + } + + return lastCommittedCheckpointId; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java new file mode 100644 index 000000000000..9de0d6aaa551 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.MetricGroup; + +class IcebergFilesCommitterMetrics { + private final AtomicLong lastCheckpointDurationMs = new AtomicLong(); + private final AtomicLong lastCommitDurationMs = new AtomicLong(); + private final ElapsedTimeGauge elapsedSecondsSinceLastSuccessfulCommit; + private final Counter committedDataFilesCount; + private final Counter committedDataFilesRecordCount; + private final Counter committedDataFilesByteCount; + private final Counter committedDeleteFilesCount; + private final Counter committedDeleteFilesRecordCount; + private final Counter committedDeleteFilesByteCount; + + IcebergFilesCommitterMetrics(MetricGroup metrics, String fullTableName) { + MetricGroup committerMetrics = + metrics.addGroup("IcebergFilesCommitter").addGroup("table", fullTableName); + committerMetrics.gauge("lastCheckpointDurationMs", lastCheckpointDurationMs::get); + committerMetrics.gauge("lastCommitDurationMs", lastCommitDurationMs::get); + this.elapsedSecondsSinceLastSuccessfulCommit = new ElapsedTimeGauge(TimeUnit.SECONDS); + committerMetrics.gauge( + "elapsedSecondsSinceLastSuccessfulCommit", elapsedSecondsSinceLastSuccessfulCommit); + this.committedDataFilesCount = committerMetrics.counter("committedDataFilesCount"); + this.committedDataFilesRecordCount = committerMetrics.counter("committedDataFilesRecordCount"); + this.committedDataFilesByteCount = committerMetrics.counter("committedDataFilesByteCount"); + this.committedDeleteFilesCount = committerMetrics.counter("committedDeleteFilesCount"); + this.committedDeleteFilesRecordCount = + committerMetrics.counter("committedDeleteFilesRecordCount"); + this.committedDeleteFilesByteCount = committerMetrics.counter("committedDeleteFilesByteCount"); + } + + void checkpointDuration(long checkpointDurationMs) { + lastCheckpointDurationMs.set(checkpointDurationMs); + } + + void commitDuration(long commitDurationMs) { + lastCommitDurationMs.set(commitDurationMs); + } + + /** This is called upon a successful commit. */ + void updateCommitSummary(CommitSummary stats) { + elapsedSecondsSinceLastSuccessfulCommit.refreshLastRecordedTime(); + committedDataFilesCount.inc(stats.dataFilesCount()); + committedDataFilesRecordCount.inc(stats.dataFilesRecordCount()); + committedDataFilesByteCount.inc(stats.dataFilesByteCount()); + committedDeleteFilesCount.inc(stats.deleteFilesCount()); + committedDeleteFilesRecordCount.inc(stats.deleteFilesRecordCount()); + committedDeleteFilesByteCount.inc(stats.deleteFilesByteCount()); + } + + /** + * This gauge measures the elapsed time between now and last recorded time set by {@link + * ElapsedTimeGauge#refreshLastRecordedTime()}. + */ + private static class ElapsedTimeGauge implements Gauge { + private final TimeUnit reportUnit; + private volatile long lastRecordedTimeNano; + + ElapsedTimeGauge(TimeUnit timeUnit) { + this.reportUnit = timeUnit; + this.lastRecordedTimeNano = System.nanoTime(); + } + + void refreshLastRecordedTime() { + this.lastRecordedTimeNano = System.nanoTime(); + } + + @Override + public Long getValue() { + return reportUnit.convert(System.nanoTime() - lastRecordedTimeNano, TimeUnit.NANOSECONDS); + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java new file mode 100644 index 000000000000..9ea0349fb057 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.io.IOException; +import java.util.concurrent.TimeUnit; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.BoundedOneInput; +import org.apache.flink.streaming.api.operators.ChainingStrategy; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +class IcebergStreamWriter extends AbstractStreamOperator + implements OneInputStreamOperator, BoundedOneInput { + + private static final long serialVersionUID = 1L; + + private final String fullTableName; + private final TaskWriterFactory taskWriterFactory; + + private transient TaskWriter writer; + private transient int subTaskId; + private transient int attemptId; + private transient IcebergStreamWriterMetrics writerMetrics; + + IcebergStreamWriter(String fullTableName, TaskWriterFactory taskWriterFactory) { + this.fullTableName = fullTableName; + this.taskWriterFactory = taskWriterFactory; + setChainingStrategy(ChainingStrategy.ALWAYS); + } + + @Override + public void open() { + this.subTaskId = getRuntimeContext().getIndexOfThisSubtask(); + this.attemptId = getRuntimeContext().getAttemptNumber(); + this.writerMetrics = new IcebergStreamWriterMetrics(super.metrics, fullTableName); + + // Initialize the task writer factory. + this.taskWriterFactory.initialize(subTaskId, attemptId); + + // Initialize the task writer. + this.writer = taskWriterFactory.create(); + } + + @Override + public void prepareSnapshotPreBarrier(long checkpointId) throws Exception { + flush(); + this.writer = taskWriterFactory.create(); + } + + @Override + public void processElement(StreamRecord element) throws Exception { + writer.write(element.getValue()); + } + + @Override + public void close() throws Exception { + super.close(); + if (writer != null) { + writer.close(); + writer = null; + } + } + + @Override + public void endInput() throws IOException { + // For bounded stream, it may don't enable the checkpoint mechanism so we'd better to emit the + // remaining completed files to downstream before closing the writer so that we won't miss any + // of them. + // Note that if the task is not closed after calling endInput, checkpoint may be triggered again + // causing files to be sent repeatedly, the writer is marked as null after the last file is sent + // to guard against duplicated writes. + flush(); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("table_name", fullTableName) + .add("subtask_id", subTaskId) + .add("attempt_id", attemptId) + .toString(); + } + + /** close all open files and emit files to downstream committer operator */ + private void flush() throws IOException { + if (writer == null) { + return; + } + + long startNano = System.nanoTime(); + WriteResult result = writer.complete(); + writerMetrics.updateFlushResult(result); + output.collect(new StreamRecord<>(result)); + writerMetrics.flushDuration(TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); + + // Set writer to null to prevent duplicate flushes in the corner case of + // prepareSnapshotPreBarrier happening after endInput. + writer = null; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java new file mode 100644 index 000000000000..ce2a6c583fdf --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import com.codahale.metrics.SlidingWindowReservoir; +import java.util.Arrays; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.dropwizard.metrics.DropwizardHistogramWrapper; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Histogram; +import org.apache.flink.metrics.MetricGroup; +import org.apache.iceberg.io.WriteResult; + +class IcebergStreamWriterMetrics { + // 1,024 reservoir size should cost about 8KB, which is quite small. + // It should also produce good accuracy for histogram distribution (like percentiles). + private static final int HISTOGRAM_RESERVOIR_SIZE = 1024; + + private final Counter flushedDataFiles; + private final Counter flushedDeleteFiles; + private final Counter flushedReferencedDataFiles; + private final AtomicLong lastFlushDurationMs; + private final Histogram dataFilesSizeHistogram; + private final Histogram deleteFilesSizeHistogram; + + IcebergStreamWriterMetrics(MetricGroup metrics, String fullTableName) { + MetricGroup writerMetrics = + metrics.addGroup("IcebergStreamWriter").addGroup("table", fullTableName); + this.flushedDataFiles = writerMetrics.counter("flushedDataFiles"); + this.flushedDeleteFiles = writerMetrics.counter("flushedDeleteFiles"); + this.flushedReferencedDataFiles = writerMetrics.counter("flushedReferencedDataFiles"); + this.lastFlushDurationMs = new AtomicLong(); + writerMetrics.gauge("lastFlushDurationMs", lastFlushDurationMs::get); + + com.codahale.metrics.Histogram dropwizardDataFilesSizeHistogram = + new com.codahale.metrics.Histogram(new SlidingWindowReservoir(HISTOGRAM_RESERVOIR_SIZE)); + this.dataFilesSizeHistogram = + writerMetrics.histogram( + "dataFilesSizeHistogram", + new DropwizardHistogramWrapper(dropwizardDataFilesSizeHistogram)); + com.codahale.metrics.Histogram dropwizardDeleteFilesSizeHistogram = + new com.codahale.metrics.Histogram(new SlidingWindowReservoir(HISTOGRAM_RESERVOIR_SIZE)); + this.deleteFilesSizeHistogram = + writerMetrics.histogram( + "deleteFilesSizeHistogram", + new DropwizardHistogramWrapper(dropwizardDeleteFilesSizeHistogram)); + } + + void updateFlushResult(WriteResult result) { + flushedDataFiles.inc(result.dataFiles().length); + flushedDeleteFiles.inc(result.deleteFiles().length); + flushedReferencedDataFiles.inc(result.referencedDataFiles().length); + + // For file size distribution histogram, we don't have to update them after successful commits. + // This should works equally well and we avoided the overhead of tracking the list of file sizes + // in the {@link CommitSummary}, which currently stores simple stats for counters and gauges + // metrics. + Arrays.stream(result.dataFiles()) + .forEach( + dataFile -> { + dataFilesSizeHistogram.update(dataFile.fileSizeInBytes()); + }); + Arrays.stream(result.deleteFiles()) + .forEach( + deleteFile -> { + deleteFilesSizeHistogram.update(deleteFile.fileSizeInBytes()); + }); + } + + void flushDuration(long flushDurationMs) { + lastFlushDurationMs.set(flushDurationMs); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java new file mode 100644 index 000000000000..da5e6e7627ae --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java @@ -0,0 +1,94 @@ +/* + * 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.iceberg.flink.sink; + +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Supplier; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.relocated.com.google.common.base.Strings; + +class ManifestOutputFileFactory { + // Users could define their own flink manifests directory by setting this value in table + // properties. + static final String FLINK_MANIFEST_LOCATION = "flink.manifests.location"; + + private final Supplier
tableSupplier; + private final Map props; + private final String flinkJobId; + private final String operatorUniqueId; + private final int subTaskId; + private final long attemptNumber; + private final AtomicInteger fileCount = new AtomicInteger(0); + + ManifestOutputFileFactory( + Supplier
tableSupplier, + Map props, + String flinkJobId, + String operatorUniqueId, + int subTaskId, + long attemptNumber) { + this.tableSupplier = tableSupplier; + this.props = props; + this.flinkJobId = flinkJobId; + this.operatorUniqueId = operatorUniqueId; + this.subTaskId = subTaskId; + this.attemptNumber = attemptNumber; + } + + private String generatePath(long checkpointId) { + return FileFormat.AVRO.addExtension( + String.format( + "%s-%s-%05d-%d-%d-%05d", + flinkJobId, + operatorUniqueId, + subTaskId, + attemptNumber, + checkpointId, + fileCount.incrementAndGet())); + } + + OutputFile create(long checkpointId) { + String flinkManifestDir = props.get(FLINK_MANIFEST_LOCATION); + TableOperations ops = ((HasTableOperations) tableSupplier.get()).operations(); + + String newManifestFullPath; + if (Strings.isNullOrEmpty(flinkManifestDir)) { + // User don't specify any flink manifest directory, so just use the default metadata path. + newManifestFullPath = ops.metadataFileLocation(generatePath(checkpointId)); + } else { + newManifestFullPath = + String.format("%s/%s", stripTrailingSlash(flinkManifestDir), generatePath(checkpointId)); + } + + return tableSupplier.get().io().newOutputFile(newManifestFullPath); + } + + private static String stripTrailingSlash(String path) { + String result = path; + while (result.endsWith("/")) { + result = result.substring(0, result.length() - 1); + } + return result; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java new file mode 100644 index 000000000000..df951684b446 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.RowDataWrapper; + +/** + * Create a {@link KeySelector} to shuffle by partition key, then each partition/bucket will be + * wrote by only one task. That will reduce lots of small files in partitioned fanout write policy + * for {@link FlinkSink}. + */ +class PartitionKeySelector implements KeySelector { + + private final Schema schema; + private final PartitionKey partitionKey; + private final RowType flinkSchema; + + private transient RowDataWrapper rowDataWrapper; + + PartitionKeySelector(PartitionSpec spec, Schema schema, RowType flinkSchema) { + this.schema = schema; + this.partitionKey = new PartitionKey(spec, schema); + this.flinkSchema = flinkSchema; + } + + /** + * Construct the {@link RowDataWrapper} lazily here because few members in it are not + * serializable. In this way, we don't have to serialize them with forcing. + */ + private RowDataWrapper lazyRowDataWrapper() { + if (rowDataWrapper == null) { + rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); + } + return rowDataWrapper; + } + + @Override + public String getKey(RowData row) { + partitionKey.partition(lazyRowDataWrapper().wrap(row)); + return partitionKey.toPath(); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java new file mode 100644 index 000000000000..38062dd1a2c4 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java @@ -0,0 +1,97 @@ +/* + * 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.iceberg.flink.sink; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.List; +import java.util.Map; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.Tasks; + +class PartitionedDeltaWriter extends BaseDeltaTaskWriter { + + private final PartitionKey partitionKey; + + private final Map writers = Maps.newHashMap(); + + PartitionedDeltaWriter( + PartitionSpec spec, + FileFormat format, + FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, + FileIO io, + long targetFileSize, + Schema schema, + RowType flinkSchema, + List equalityFieldIds, + boolean upsert) { + super( + spec, + format, + appenderFactory, + fileFactory, + io, + targetFileSize, + schema, + flinkSchema, + equalityFieldIds, + upsert); + this.partitionKey = new PartitionKey(spec, schema); + } + + @Override + RowDataDeltaWriter route(RowData 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/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java new file mode 100644 index 000000000000..67422a1afeb1 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java @@ -0,0 +1,244 @@ +/* + * 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.iceberg.flink.sink; + +import java.util.List; +import java.util.Map; +import java.util.function.Supplier; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.io.PartitionedFanoutWriter; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.io.UnpartitionedWriter; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.ArrayUtil; +import org.apache.iceberg.util.SerializableSupplier; + +public class RowDataTaskWriterFactory implements TaskWriterFactory { + private final Supplier
tableSupplier; + private final Schema schema; + private final RowType flinkSchema; + private final PartitionSpec spec; + private final long targetFileSizeBytes; + private final FileFormat format; + private final List equalityFieldIds; + private final boolean upsert; + private final FileAppenderFactory appenderFactory; + + private transient OutputFileFactory outputFileFactory; + + public RowDataTaskWriterFactory( + Table table, + RowType flinkSchema, + long targetFileSizeBytes, + FileFormat format, + Map writeProperties, + List equalityFieldIds, + boolean upsert) { + this( + () -> table, + flinkSchema, + targetFileSizeBytes, + format, + writeProperties, + equalityFieldIds, + upsert); + } + + public RowDataTaskWriterFactory( + SerializableSupplier
tableSupplier, + RowType flinkSchema, + long targetFileSizeBytes, + FileFormat format, + Map writeProperties, + List equalityFieldIds, + boolean upsert) { + this.tableSupplier = tableSupplier; + + Table table; + if (tableSupplier instanceof CachingTableSupplier) { + // rely on the initial table metadata for schema, etc., until schema evolution is supported + table = ((CachingTableSupplier) tableSupplier).initialTable(); + } else { + table = tableSupplier.get(); + } + + this.schema = table.schema(); + this.flinkSchema = flinkSchema; + this.spec = table.spec(); + this.targetFileSizeBytes = targetFileSizeBytes; + this.format = format; + this.equalityFieldIds = equalityFieldIds; + this.upsert = upsert; + + if (equalityFieldIds == null || equalityFieldIds.isEmpty()) { + this.appenderFactory = + new FlinkAppenderFactory( + table, schema, flinkSchema, writeProperties, spec, null, null, null); + } else if (upsert) { + // In upsert mode, only the new row is emitted using INSERT row kind. Therefore, any column of + // the inserted row + // may differ from the deleted row other than the primary key fields, and the delete file must + // contain values + // that are correct for the deleted row. Therefore, only write the equality delete fields. + this.appenderFactory = + new FlinkAppenderFactory( + table, + schema, + flinkSchema, + writeProperties, + spec, + ArrayUtil.toIntArray(equalityFieldIds), + TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds)), + null); + } else { + this.appenderFactory = + new FlinkAppenderFactory( + table, + schema, + flinkSchema, + writeProperties, + spec, + ArrayUtil.toIntArray(equalityFieldIds), + schema, + null); + } + } + + @Override + public void initialize(int taskId, int attemptId) { + Table table; + if (tableSupplier instanceof CachingTableSupplier) { + // rely on the initial table metadata for schema, etc., until schema evolution is supported + table = ((CachingTableSupplier) tableSupplier).initialTable(); + } else { + table = tableSupplier.get(); + } + + refreshTable(); + + this.outputFileFactory = + OutputFileFactory.builderFor(table, taskId, attemptId) + .format(format) + .ioSupplier(() -> tableSupplier.get().io()) + .build(); + } + + @Override + public TaskWriter create() { + Preconditions.checkNotNull( + outputFileFactory, + "The outputFileFactory shouldn't be null if we have invoked the initialize()."); + + refreshTable(); + + if (equalityFieldIds == null || equalityFieldIds.isEmpty()) { + // Initialize a task writer to write INSERT only. + if (spec.isUnpartitioned()) { + return new UnpartitionedWriter<>( + spec, + format, + appenderFactory, + outputFileFactory, + tableSupplier.get().io(), + targetFileSizeBytes); + } else { + return new RowDataPartitionedFanoutWriter( + spec, + format, + appenderFactory, + outputFileFactory, + tableSupplier.get().io(), + targetFileSizeBytes, + schema, + flinkSchema); + } + } else { + // Initialize a task writer to write both INSERT and equality DELETE. + if (spec.isUnpartitioned()) { + return new UnpartitionedDeltaWriter( + spec, + format, + appenderFactory, + outputFileFactory, + tableSupplier.get().io(), + targetFileSizeBytes, + schema, + flinkSchema, + equalityFieldIds, + upsert); + } else { + return new PartitionedDeltaWriter( + spec, + format, + appenderFactory, + outputFileFactory, + tableSupplier.get().io(), + targetFileSizeBytes, + schema, + flinkSchema, + equalityFieldIds, + upsert); + } + } + } + + void refreshTable() { + if (tableSupplier instanceof CachingTableSupplier) { + ((CachingTableSupplier) tableSupplier).refreshTable(); + } + } + + private static class RowDataPartitionedFanoutWriter extends PartitionedFanoutWriter { + + private final PartitionKey partitionKey; + private final RowDataWrapper rowDataWrapper; + + RowDataPartitionedFanoutWriter( + PartitionSpec spec, + FileFormat format, + FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, + FileIO io, + long targetFileSize, + Schema schema, + RowType flinkSchema) { + super(spec, format, appenderFactory, fileFactory, io, targetFileSize); + this.partitionKey = new PartitionKey(spec, schema); + this.rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); + } + + @Override + protected PartitionKey partition(RowData row) { + partitionKey.partition(rowDataWrapper.wrap(row)); + return partitionKey; + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java new file mode 100644 index 000000000000..e3a1245e8cbd --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java @@ -0,0 +1,45 @@ +/* + * 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.iceberg.flink.sink; + +import java.io.Serializable; +import org.apache.iceberg.io.TaskWriter; + +/** + * Factory to create {@link TaskWriter} + * + * @param data type of record. + */ +public interface TaskWriterFactory extends Serializable { + + /** + * Initialize the factory with a given taskId and attemptId. + * + * @param taskId the identifier of task. + * @param attemptId the attempt id of this task. + */ + void initialize(int taskId, int attemptId); + + /** + * Initialize a {@link TaskWriter} with given task id and attempt id. + * + * @return a newly created task writer. + */ + TaskWriter create(); +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java new file mode 100644 index 000000000000..7680fb933b20 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java @@ -0,0 +1,69 @@ +/* + * 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.iceberg.flink.sink; + +import java.io.IOException; +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFileFactory; + +class UnpartitionedDeltaWriter extends BaseDeltaTaskWriter { + private final RowDataDeltaWriter writer; + + UnpartitionedDeltaWriter( + PartitionSpec spec, + FileFormat format, + FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, + FileIO io, + long targetFileSize, + Schema schema, + RowType flinkSchema, + List equalityFieldIds, + boolean upsert) { + super( + spec, + format, + appenderFactory, + fileFactory, + io, + targetFileSize, + schema, + flinkSchema, + equalityFieldIds, + upsert); + this.writer = new RowDataDeltaWriter(null); + } + + @Override + RowDataDeltaWriter route(RowData row) { + return writer; + } + + @Override + public void close() throws IOException { + writer.close(); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java new file mode 100644 index 000000000000..157f04b8b0ed --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.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.iceberg.flink.sink.shuffle; + +import java.io.Serializable; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * AggregatedStatistics is used by {@link DataStatisticsCoordinator} to collect {@link + * DataStatistics} from {@link DataStatisticsOperator} subtasks for specific checkpoint. It stores + * the merged {@link DataStatistics} result from all reported subtasks. + */ +class AggregatedStatistics, S> implements Serializable { + + private final long checkpointId; + private final DataStatistics dataStatistics; + + AggregatedStatistics(long checkpoint, TypeSerializer> statisticsSerializer) { + this.checkpointId = checkpoint; + this.dataStatistics = statisticsSerializer.createInstance(); + } + + AggregatedStatistics(long checkpoint, DataStatistics dataStatistics) { + this.checkpointId = checkpoint; + this.dataStatistics = dataStatistics; + } + + long checkpointId() { + return checkpointId; + } + + DataStatistics dataStatistics() { + return dataStatistics; + } + + void mergeDataStatistic(String operatorName, long eventCheckpointId, D eventDataStatistics) { + Preconditions.checkArgument( + checkpointId == eventCheckpointId, + "Received unexpected event from operator %s checkpoint %s. Expected checkpoint %s", + operatorName, + eventCheckpointId, + checkpointId); + dataStatistics.merge(eventDataStatistics); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("checkpointId", checkpointId) + .add("dataStatistics", dataStatistics) + .toString(); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java new file mode 100644 index 000000000000..e8ff61dbeb27 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java @@ -0,0 +1,133 @@ +/* + * 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.iceberg.flink.sink.shuffle; + +import java.util.Set; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * AggregatedStatisticsTracker is used by {@link DataStatisticsCoordinator} to track the in progress + * {@link AggregatedStatistics} received from {@link DataStatisticsOperator} subtasks for specific + * checkpoint. + */ +class AggregatedStatisticsTracker, S> { + private static final Logger LOG = LoggerFactory.getLogger(AggregatedStatisticsTracker.class); + private static final double ACCEPT_PARTIAL_AGGR_THRESHOLD = 90; + private final String operatorName; + private final TypeSerializer> statisticsSerializer; + private final int parallelism; + private final Set inProgressSubtaskSet; + private volatile AggregatedStatistics inProgressStatistics; + + AggregatedStatisticsTracker( + String operatorName, + TypeSerializer> statisticsSerializer, + int parallelism) { + this.operatorName = operatorName; + this.statisticsSerializer = statisticsSerializer; + this.parallelism = parallelism; + this.inProgressSubtaskSet = Sets.newHashSet(); + } + + AggregatedStatistics updateAndCheckCompletion( + int subtask, DataStatisticsEvent event) { + long checkpointId = event.checkpointId(); + + if (inProgressStatistics != null && inProgressStatistics.checkpointId() > checkpointId) { + LOG.info( + "Expect data statistics for operator {} checkpoint {}, but receive event from older checkpoint {}. Ignore it.", + operatorName, + inProgressStatistics.checkpointId(), + checkpointId); + return null; + } + + AggregatedStatistics completedStatistics = null; + if (inProgressStatistics != null && inProgressStatistics.checkpointId() < checkpointId) { + if ((double) inProgressSubtaskSet.size() / parallelism * 100 + >= ACCEPT_PARTIAL_AGGR_THRESHOLD) { + completedStatistics = inProgressStatistics; + LOG.info( + "Received data statistics from {} subtasks out of total {} for operator {} at checkpoint {}. " + + "Complete data statistics aggregation at checkpoint {} as it is more than the threshold of {} percentage", + inProgressSubtaskSet.size(), + parallelism, + operatorName, + checkpointId, + inProgressStatistics.checkpointId(), + ACCEPT_PARTIAL_AGGR_THRESHOLD); + } else { + LOG.info( + "Received data statistics from {} subtasks out of total {} for operator {} at checkpoint {}. " + + "Aborting the incomplete aggregation for checkpoint {}", + inProgressSubtaskSet.size(), + parallelism, + operatorName, + checkpointId, + inProgressStatistics.checkpointId()); + } + + inProgressStatistics = null; + inProgressSubtaskSet.clear(); + } + + if (inProgressStatistics == null) { + LOG.info("Starting a new data statistics for checkpoint {}", checkpointId); + inProgressStatistics = new AggregatedStatistics<>(checkpointId, statisticsSerializer); + inProgressSubtaskSet.clear(); + } + + if (!inProgressSubtaskSet.add(subtask)) { + LOG.debug( + "Ignore duplicated data statistics from operator {} subtask {} for checkpoint {}.", + operatorName, + subtask, + checkpointId); + } else { + inProgressStatistics.mergeDataStatistic( + operatorName, + event.checkpointId(), + DataStatisticsUtil.deserializeDataStatistics( + event.statisticsBytes(), statisticsSerializer)); + } + + if (inProgressSubtaskSet.size() == parallelism) { + completedStatistics = inProgressStatistics; + LOG.info( + "Received data statistics from all {} operators {} for checkpoint {}. Return last completed aggregator {}.", + parallelism, + operatorName, + inProgressStatistics.checkpointId(), + completedStatistics.dataStatistics()); + inProgressStatistics = new AggregatedStatistics<>(checkpointId + 1, statisticsSerializer); + inProgressSubtaskSet.clear(); + } + + return completedStatistics; + } + + @VisibleForTesting + AggregatedStatistics inProgressStatistics() { + return inProgressStatistics; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java new file mode 100644 index 000000000000..28a05201c02f --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java @@ -0,0 +1,61 @@ +/* + * 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.iceberg.flink.sink.shuffle; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.data.RowData; + +/** + * DataStatistics defines the interface to collect data distribution information. + * + *

Data statistics tracks traffic volume distribution across data keys. For low-cardinality key, + * a simple map of (key, count) can be used. For high-cardinality key, probabilistic data structures + * (sketching) can be used. + */ +@Internal +interface DataStatistics { + + /** + * Check if data statistics contains any statistics information. + * + * @return true if data statistics doesn't contain any statistics information + */ + boolean isEmpty(); + + /** + * Add data key to data statistics. + * + * @param key generate from data by applying key selector + */ + void add(RowData key); + + /** + * Merge current statistics with other statistics. + * + * @param otherStatistics the statistics to be merged + */ + void merge(D otherStatistics); + + /** + * Get the underline statistics. + * + * @return the underline statistics + */ + S statistics(); +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java new file mode 100644 index 000000000000..c8ac79c61bf6 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java @@ -0,0 +1,396 @@ +/* + * 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.iceberg.flink.sink.shuffle; + +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FatalExitExceptionHandler; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.ThrowableCatchingRunnable; +import org.apache.flink.util.function.ThrowingRunnable; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * DataStatisticsCoordinator receives {@link DataStatisticsEvent} from {@link + * DataStatisticsOperator} every subtask and then merge them together. Once aggregation for all + * subtasks data statistics completes, DataStatisticsCoordinator will send the aggregated data + * statistics back to {@link DataStatisticsOperator}. In the end a custom partitioner will + * distribute traffic based on the aggregated data statistics to improve data clustering. + */ +@Internal +class DataStatisticsCoordinator, S> implements OperatorCoordinator { + private static final Logger LOG = LoggerFactory.getLogger(DataStatisticsCoordinator.class); + + private final String operatorName; + private final ExecutorService coordinatorExecutor; + private final OperatorCoordinator.Context operatorCoordinatorContext; + private final SubtaskGateways subtaskGateways; + private final CoordinatorExecutorThreadFactory coordinatorThreadFactory; + private final TypeSerializer> statisticsSerializer; + private final transient AggregatedStatisticsTracker aggregatedStatisticsTracker; + private volatile AggregatedStatistics completedStatistics; + private volatile boolean started; + + DataStatisticsCoordinator( + String operatorName, + OperatorCoordinator.Context context, + TypeSerializer> statisticsSerializer) { + this.operatorName = operatorName; + this.coordinatorThreadFactory = + new CoordinatorExecutorThreadFactory( + "DataStatisticsCoordinator-" + operatorName, context.getUserCodeClassloader()); + this.coordinatorExecutor = Executors.newSingleThreadExecutor(coordinatorThreadFactory); + this.operatorCoordinatorContext = context; + this.subtaskGateways = new SubtaskGateways(operatorName, parallelism()); + this.statisticsSerializer = statisticsSerializer; + this.aggregatedStatisticsTracker = + new AggregatedStatisticsTracker<>(operatorName, statisticsSerializer, parallelism()); + } + + @Override + public void start() throws Exception { + LOG.info("Starting data statistics coordinator: {}.", operatorName); + started = true; + } + + @Override + public void close() throws Exception { + coordinatorExecutor.shutdown(); + LOG.info("Closed data statistics coordinator: {}.", operatorName); + } + + @VisibleForTesting + void callInCoordinatorThread(Callable callable, String errorMessage) { + ensureStarted(); + // Ensure the task is done by the coordinator executor. + if (!coordinatorThreadFactory.isCurrentThreadCoordinatorThread()) { + try { + Callable guardedCallable = + () -> { + try { + return callable.call(); + } catch (Throwable t) { + LOG.error( + "Uncaught Exception in data statistics coordinator: {} executor", + operatorName, + t); + ExceptionUtils.rethrowException(t); + return null; + } + }; + + coordinatorExecutor.submit(guardedCallable).get(); + } catch (InterruptedException | ExecutionException e) { + throw new FlinkRuntimeException(errorMessage, e); + } + } else { + try { + callable.call(); + } catch (Throwable t) { + LOG.error( + "Uncaught Exception in data statistics coordinator: {} executor", operatorName, t); + throw new FlinkRuntimeException(errorMessage, t); + } + } + } + + public void runInCoordinatorThread(Runnable runnable) { + this.coordinatorExecutor.execute( + new ThrowableCatchingRunnable( + throwable -> + this.coordinatorThreadFactory.uncaughtException(Thread.currentThread(), throwable), + runnable)); + } + + private void runInCoordinatorThread(ThrowingRunnable action, String actionString) { + ensureStarted(); + runInCoordinatorThread( + () -> { + try { + action.run(); + } catch (Throwable t) { + ExceptionUtils.rethrowIfFatalErrorOrOOM(t); + LOG.error( + "Uncaught exception in the data statistics coordinator: {} while {}. Triggering job failover", + operatorName, + actionString, + t); + operatorCoordinatorContext.failJob(t); + } + }); + } + + private void ensureStarted() { + Preconditions.checkState(started, "The coordinator of %s has not started yet.", operatorName); + } + + private int parallelism() { + return operatorCoordinatorContext.currentParallelism(); + } + + private void handleDataStatisticRequest(int subtask, DataStatisticsEvent event) { + AggregatedStatistics aggregatedStatistics = + aggregatedStatisticsTracker.updateAndCheckCompletion(subtask, event); + + if (aggregatedStatistics != null) { + completedStatistics = aggregatedStatistics; + sendDataStatisticsToSubtasks( + completedStatistics.checkpointId(), completedStatistics.dataStatistics()); + } + } + + @SuppressWarnings("FutureReturnValueIgnored") + private void sendDataStatisticsToSubtasks( + long checkpointId, DataStatistics globalDataStatistics) { + callInCoordinatorThread( + () -> { + DataStatisticsEvent dataStatisticsEvent = + DataStatisticsEvent.create(checkpointId, globalDataStatistics, statisticsSerializer); + int parallelism = parallelism(); + for (int i = 0; i < parallelism; ++i) { + subtaskGateways.getSubtaskGateway(i).sendEvent(dataStatisticsEvent); + } + + return null; + }, + String.format( + "Failed to send operator %s coordinator global data statistics for checkpoint %d", + operatorName, checkpointId)); + } + + @Override + @SuppressWarnings("unchecked") + public void handleEventFromOperator(int subtask, int attemptNumber, OperatorEvent event) { + runInCoordinatorThread( + () -> { + LOG.debug( + "Handling event from subtask {} (#{}) of {}: {}", + subtask, + attemptNumber, + operatorName, + event); + Preconditions.checkArgument(event instanceof DataStatisticsEvent); + handleDataStatisticRequest(subtask, ((DataStatisticsEvent) event)); + }, + String.format( + "handling operator event %s from subtask %d (#%d)", + event.getClass(), subtask, attemptNumber)); + } + + @Override + public void checkpointCoordinator(long checkpointId, CompletableFuture resultFuture) { + runInCoordinatorThread( + () -> { + LOG.debug( + "Snapshotting data statistics coordinator {} for checkpoint {}", + operatorName, + checkpointId); + resultFuture.complete( + DataStatisticsUtil.serializeAggregatedStatistics( + completedStatistics, statisticsSerializer)); + }, + String.format("taking checkpoint %d", checkpointId)); + } + + @Override + public void notifyCheckpointComplete(long checkpointId) {} + + @Override + public void resetToCheckpoint(long checkpointId, @Nullable byte[] checkpointData) + throws Exception { + Preconditions.checkState( + !started, "The coordinator %s can only be reset if it was not yet started", operatorName); + + if (checkpointData == null) { + LOG.info( + "Data statistic coordinator {} has nothing to restore from checkpoint {}", + operatorName, + checkpointId); + return; + } + + LOG.info( + "Restoring data statistic coordinator {} from checkpoint {}", operatorName, checkpointId); + completedStatistics = + DataStatisticsUtil.deserializeAggregatedStatistics(checkpointData, statisticsSerializer); + } + + @Override + public void subtaskReset(int subtask, long checkpointId) { + runInCoordinatorThread( + () -> { + LOG.info( + "Operator {} subtask {} is reset to checkpoint {}", + operatorName, + subtask, + checkpointId); + Preconditions.checkState( + this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); + subtaskGateways.reset(subtask); + }, + String.format("handling subtask %d recovery to checkpoint %d", subtask, checkpointId)); + } + + @Override + public void executionAttemptFailed(int subtask, int attemptNumber, @Nullable Throwable reason) { + runInCoordinatorThread( + () -> { + LOG.info( + "Unregistering gateway after failure for subtask {} (#{}) of data statistic {}", + subtask, + attemptNumber, + operatorName); + Preconditions.checkState( + this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); + subtaskGateways.unregisterSubtaskGateway(subtask, attemptNumber); + }, + String.format("handling subtask %d (#%d) failure", subtask, attemptNumber)); + } + + @Override + public void executionAttemptReady(int subtask, int attemptNumber, SubtaskGateway gateway) { + Preconditions.checkArgument(subtask == gateway.getSubtask()); + Preconditions.checkArgument(attemptNumber == gateway.getExecution().getAttemptNumber()); + runInCoordinatorThread( + () -> { + Preconditions.checkState( + this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); + subtaskGateways.registerSubtaskGateway(gateway); + }, + String.format( + "making event gateway to subtask %d (#%d) available", subtask, attemptNumber)); + } + + @VisibleForTesting + AggregatedStatistics completedStatistics() { + return completedStatistics; + } + + private static class SubtaskGateways { + private final String operatorName; + private final Map[] gateways; + + private SubtaskGateways(String operatorName, int parallelism) { + this.operatorName = operatorName; + gateways = new Map[parallelism]; + + for (int i = 0; i < parallelism; ++i) { + gateways[i] = Maps.newHashMap(); + } + } + + private void registerSubtaskGateway(OperatorCoordinator.SubtaskGateway gateway) { + int subtaskIndex = gateway.getSubtask(); + int attemptNumber = gateway.getExecution().getAttemptNumber(); + Preconditions.checkState( + !gateways[subtaskIndex].containsKey(attemptNumber), + "Coordinator of %s already has a subtask gateway for %d (#%d)", + operatorName, + subtaskIndex, + attemptNumber); + LOG.debug( + "Coordinator of {} registers gateway for subtask {} attempt {}", + operatorName, + subtaskIndex, + attemptNumber); + gateways[subtaskIndex].put(attemptNumber, gateway); + } + + private void unregisterSubtaskGateway(int subtaskIndex, int attemptNumber) { + LOG.debug( + "Coordinator of {} unregisters gateway for subtask {} attempt {}", + operatorName, + subtaskIndex, + attemptNumber); + gateways[subtaskIndex].remove(attemptNumber); + } + + private OperatorCoordinator.SubtaskGateway getSubtaskGateway(int subtaskIndex) { + Preconditions.checkState( + !gateways[subtaskIndex].isEmpty(), + "Coordinator of %s subtask %d is not ready yet to receive events", + operatorName, + subtaskIndex); + return Iterables.getOnlyElement(gateways[subtaskIndex].values()); + } + + private void reset(int subtaskIndex) { + gateways[subtaskIndex].clear(); + } + } + + private static class CoordinatorExecutorThreadFactory + implements ThreadFactory, Thread.UncaughtExceptionHandler { + + private final String coordinatorThreadName; + private final ClassLoader classLoader; + private final Thread.UncaughtExceptionHandler errorHandler; + + @javax.annotation.Nullable private Thread thread; + + CoordinatorExecutorThreadFactory( + final String coordinatorThreadName, final ClassLoader contextClassLoader) { + this(coordinatorThreadName, contextClassLoader, FatalExitExceptionHandler.INSTANCE); + } + + @org.apache.flink.annotation.VisibleForTesting + CoordinatorExecutorThreadFactory( + final String coordinatorThreadName, + final ClassLoader contextClassLoader, + final Thread.UncaughtExceptionHandler errorHandler) { + this.coordinatorThreadName = coordinatorThreadName; + this.classLoader = contextClassLoader; + this.errorHandler = errorHandler; + } + + @Override + public synchronized Thread newThread(@NotNull Runnable runnable) { + thread = new Thread(runnable, coordinatorThreadName); + thread.setContextClassLoader(classLoader); + thread.setUncaughtExceptionHandler(this); + return thread; + } + + @Override + public synchronized void uncaughtException(Thread t, Throwable e) { + errorHandler.uncaughtException(t, e); + } + + boolean isCurrentThreadCoordinatorThread() { + return Thread.currentThread() == thread; + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java new file mode 100644 index 000000000000..47dbfc3cfbe1 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java @@ -0,0 +1,51 @@ +/* + * 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.iceberg.flink.sink.shuffle; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +import org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator; + +/** + * DataStatisticsCoordinatorProvider provides the method to create new {@link + * DataStatisticsCoordinator} + */ +@Internal +public class DataStatisticsCoordinatorProvider, S> + extends RecreateOnResetOperatorCoordinator.Provider { + + private final String operatorName; + private final TypeSerializer> statisticsSerializer; + + public DataStatisticsCoordinatorProvider( + String operatorName, + OperatorID operatorID, + TypeSerializer> statisticsSerializer) { + super(operatorID); + this.operatorName = operatorName; + this.statisticsSerializer = statisticsSerializer; + } + + @Override + public OperatorCoordinator getCoordinator(OperatorCoordinator.Context context) { + return new DataStatisticsCoordinator<>(operatorName, context, statisticsSerializer); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java new file mode 100644 index 000000000000..852d2157b8cb --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; + +/** + * DataStatisticsEvent is sent between data statistics coordinator and operator to transmit data + * statistics in bytes + */ +@Internal +class DataStatisticsEvent, S> implements OperatorEvent { + + private static final long serialVersionUID = 1L; + private final long checkpointId; + private final byte[] statisticsBytes; + + private DataStatisticsEvent(long checkpointId, byte[] statisticsBytes) { + this.checkpointId = checkpointId; + this.statisticsBytes = statisticsBytes; + } + + static , S> DataStatisticsEvent create( + long checkpointId, + DataStatistics dataStatistics, + TypeSerializer> statisticsSerializer) { + return new DataStatisticsEvent<>( + checkpointId, + DataStatisticsUtil.serializeDataStatistics(dataStatistics, statisticsSerializer)); + } + + long checkpointId() { + return checkpointId; + } + + byte[] statisticsBytes() { + return statisticsBytes; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java new file mode 100644 index 000000000000..d00d5d2e5aa9 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java @@ -0,0 +1,188 @@ +/* + * 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.iceberg.flink.sink.shuffle; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; +import org.apache.flink.runtime.operators.coordination.OperatorEventHandler; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * DataStatisticsOperator collects traffic distribution statistics. A custom partitioner shall be + * attached to the DataStatisticsOperator output. The custom partitioner leverages the statistics to + * shuffle record to improve data clustering while maintaining relative balanced traffic + * distribution to downstream subtasks. + */ +@Internal +class DataStatisticsOperator, S> + extends AbstractStreamOperator> + implements OneInputStreamOperator>, OperatorEventHandler { + private static final long serialVersionUID = 1L; + + private final String operatorName; + // keySelector will be used to generate key from data for collecting data statistics + private final KeySelector keySelector; + private final OperatorEventGateway operatorEventGateway; + private final TypeSerializer> statisticsSerializer; + private transient volatile DataStatistics localStatistics; + private transient volatile DataStatistics globalStatistics; + private transient ListState> globalStatisticsState; + + DataStatisticsOperator( + String operatorName, + KeySelector keySelector, + OperatorEventGateway operatorEventGateway, + TypeSerializer> statisticsSerializer) { + this.operatorName = operatorName; + this.keySelector = keySelector; + this.operatorEventGateway = operatorEventGateway; + this.statisticsSerializer = statisticsSerializer; + } + + @Override + public void initializeState(StateInitializationContext context) throws Exception { + localStatistics = statisticsSerializer.createInstance(); + globalStatisticsState = + context + .getOperatorStateStore() + .getUnionListState( + new ListStateDescriptor<>("globalStatisticsState", statisticsSerializer)); + + if (context.isRestored()) { + int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask(); + if (globalStatisticsState.get() == null + || !globalStatisticsState.get().iterator().hasNext()) { + LOG.warn( + "Operator {} subtask {} doesn't have global statistics state to restore", + operatorName, + subtaskIndex); + globalStatistics = statisticsSerializer.createInstance(); + } else { + LOG.info( + "Restoring operator {} global statistics state for subtask {}", + operatorName, + subtaskIndex); + globalStatistics = globalStatisticsState.get().iterator().next(); + } + } else { + globalStatistics = statisticsSerializer.createInstance(); + } + } + + @Override + public void open() throws Exception { + if (!globalStatistics.isEmpty()) { + output.collect( + new StreamRecord<>(DataStatisticsOrRecord.fromDataStatistics(globalStatistics))); + } + } + + @Override + @SuppressWarnings("unchecked") + public void handleOperatorEvent(OperatorEvent event) { + int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask(); + Preconditions.checkArgument( + event instanceof DataStatisticsEvent, + String.format( + "Operator %s subtask %s received unexpected operator event %s", + operatorName, subtaskIndex, event.getClass())); + DataStatisticsEvent statisticsEvent = (DataStatisticsEvent) event; + LOG.info( + "Operator {} received global data event from coordinator checkpoint {}", + operatorName, + statisticsEvent.checkpointId()); + globalStatistics = + DataStatisticsUtil.deserializeDataStatistics( + statisticsEvent.statisticsBytes(), statisticsSerializer); + output.collect(new StreamRecord<>(DataStatisticsOrRecord.fromDataStatistics(globalStatistics))); + } + + @Override + public void processElement(StreamRecord streamRecord) throws Exception { + RowData record = streamRecord.getValue(); + RowData key = keySelector.getKey(record); + localStatistics.add(key); + output.collect(new StreamRecord<>(DataStatisticsOrRecord.fromRecord(record))); + } + + @Override + public void snapshotState(StateSnapshotContext context) throws Exception { + long checkpointId = context.getCheckpointId(); + int subTaskId = getRuntimeContext().getIndexOfThisSubtask(); + LOG.info( + "Snapshotting data statistics operator {} for checkpoint {} in subtask {}", + operatorName, + checkpointId, + subTaskId); + + // Pass global statistics to partitioners so that all the operators refresh statistics + // at same checkpoint barrier + if (!globalStatistics.isEmpty()) { + output.collect( + new StreamRecord<>(DataStatisticsOrRecord.fromDataStatistics(globalStatistics))); + } + + // Only subtask 0 saves the state so that globalStatisticsState(UnionListState) stores + // an exact copy of globalStatistics + if (!globalStatistics.isEmpty() && getRuntimeContext().getIndexOfThisSubtask() == 0) { + globalStatisticsState.clear(); + LOG.info( + "Saving operator {} global statistics {} to state in subtask {}", + operatorName, + globalStatistics, + subTaskId); + globalStatisticsState.add(globalStatistics); + } + + // For now, local statistics are sent to coordinator at checkpoint + operatorEventGateway.sendEventToCoordinator( + DataStatisticsEvent.create(checkpointId, localStatistics, statisticsSerializer)); + LOG.debug( + "Subtask {} of operator {} sent local statistics to coordinator at checkpoint{}: {}", + subTaskId, + operatorName, + checkpointId, + localStatistics); + + // Recreate the local statistics + localStatistics = statisticsSerializer.createInstance(); + } + + @VisibleForTesting + DataStatistics localDataStatistics() { + return localStatistics; + } + + @VisibleForTesting + DataStatistics globalDataStatistics() { + return globalStatistics; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java new file mode 100644 index 000000000000..889e85112e16 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java @@ -0,0 +1,113 @@ +/* + * 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.iceberg.flink.sink.shuffle; + +import java.io.Serializable; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * The wrapper class for data statistics and record. It is the only way for data statistics operator + * to send global data statistics to custom partitioner to distribute data based on statistics + * + *

DataStatisticsOrRecord contains either data statistics(globally aggregated) or a record. It is + * sent from {@link DataStatisticsOperator} to partitioner. Once partitioner receives the data + * statistics, it will use that to decide the coming record should send to which writer subtask. + * After shuffling, a filter and mapper are required to filter out the data distribution weight, + * unwrap the object and extract the original record type T. + */ +class DataStatisticsOrRecord, S> implements Serializable { + + private static final long serialVersionUID = 1L; + + private DataStatistics statistics; + private RowData record; + + private DataStatisticsOrRecord(DataStatistics statistics, RowData record) { + Preconditions.checkArgument( + record != null ^ statistics != null, "DataStatistics or record, not neither or both"); + this.statistics = statistics; + this.record = record; + } + + static , S> DataStatisticsOrRecord fromRecord( + RowData record) { + return new DataStatisticsOrRecord<>(null, record); + } + + static , S> DataStatisticsOrRecord fromDataStatistics( + DataStatistics statistics) { + return new DataStatisticsOrRecord<>(statistics, null); + } + + static , S> DataStatisticsOrRecord reuseRecord( + DataStatisticsOrRecord reuse, TypeSerializer recordSerializer) { + if (reuse.hasRecord()) { + return reuse; + } else { + // not reusable + return DataStatisticsOrRecord.fromRecord(recordSerializer.createInstance()); + } + } + + static , S> DataStatisticsOrRecord reuseStatistics( + DataStatisticsOrRecord reuse, + TypeSerializer> statisticsSerializer) { + if (reuse.hasDataStatistics()) { + return reuse; + } else { + // not reusable + return DataStatisticsOrRecord.fromDataStatistics(statisticsSerializer.createInstance()); + } + } + + boolean hasDataStatistics() { + return statistics != null; + } + + boolean hasRecord() { + return record != null; + } + + DataStatistics dataStatistics() { + return statistics; + } + + void dataStatistics(DataStatistics newStatistics) { + this.statistics = newStatistics; + } + + RowData record() { + return record; + } + + void record(RowData newRecord) { + this.record = newRecord; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("statistics", statistics) + .add("record", record) + .toString(); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java new file mode 100644 index 000000000000..e9a6fa0cbfc5 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java @@ -0,0 +1,219 @@ +/* + * 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.iceberg.flink.sink.shuffle; + +import java.io.IOException; +import java.util.Objects; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.table.data.RowData; + +@Internal +class DataStatisticsOrRecordSerializer, S> + extends TypeSerializer> { + private final TypeSerializer> statisticsSerializer; + private final TypeSerializer recordSerializer; + + DataStatisticsOrRecordSerializer( + TypeSerializer> statisticsSerializer, + TypeSerializer recordSerializer) { + this.statisticsSerializer = statisticsSerializer; + this.recordSerializer = recordSerializer; + } + + @Override + public boolean isImmutableType() { + return false; + } + + @SuppressWarnings("ReferenceEquality") + @Override + public TypeSerializer> duplicate() { + TypeSerializer> duplicateStatisticsSerializer = + statisticsSerializer.duplicate(); + TypeSerializer duplicateRowDataSerializer = recordSerializer.duplicate(); + if ((statisticsSerializer != duplicateStatisticsSerializer) + || (recordSerializer != duplicateRowDataSerializer)) { + return new DataStatisticsOrRecordSerializer<>( + duplicateStatisticsSerializer, duplicateRowDataSerializer); + } else { + return this; + } + } + + @Override + public DataStatisticsOrRecord createInstance() { + // arbitrarily always create RowData value instance + return DataStatisticsOrRecord.fromRecord(recordSerializer.createInstance()); + } + + @Override + public DataStatisticsOrRecord copy(DataStatisticsOrRecord from) { + if (from.hasRecord()) { + return DataStatisticsOrRecord.fromRecord(recordSerializer.copy(from.record())); + } else { + return DataStatisticsOrRecord.fromDataStatistics( + statisticsSerializer.copy(from.dataStatistics())); + } + } + + @Override + public DataStatisticsOrRecord copy( + DataStatisticsOrRecord from, DataStatisticsOrRecord reuse) { + DataStatisticsOrRecord to; + if (from.hasRecord()) { + to = DataStatisticsOrRecord.reuseRecord(reuse, recordSerializer); + RowData record = recordSerializer.copy(from.record(), to.record()); + to.record(record); + } else { + to = DataStatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); + DataStatistics statistics = + statisticsSerializer.copy(from.dataStatistics(), to.dataStatistics()); + to.dataStatistics(statistics); + } + + return to; + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(DataStatisticsOrRecord statisticsOrRecord, DataOutputView target) + throws IOException { + if (statisticsOrRecord.hasRecord()) { + target.writeBoolean(true); + recordSerializer.serialize(statisticsOrRecord.record(), target); + } else { + target.writeBoolean(false); + statisticsSerializer.serialize(statisticsOrRecord.dataStatistics(), target); + } + } + + @Override + public DataStatisticsOrRecord deserialize(DataInputView source) throws IOException { + boolean isRecord = source.readBoolean(); + if (isRecord) { + return DataStatisticsOrRecord.fromRecord(recordSerializer.deserialize(source)); + } else { + return DataStatisticsOrRecord.fromDataStatistics(statisticsSerializer.deserialize(source)); + } + } + + @Override + public DataStatisticsOrRecord deserialize( + DataStatisticsOrRecord reuse, DataInputView source) throws IOException { + DataStatisticsOrRecord to; + boolean isRecord = source.readBoolean(); + if (isRecord) { + to = DataStatisticsOrRecord.reuseRecord(reuse, recordSerializer); + RowData record = recordSerializer.deserialize(to.record(), source); + to.record(record); + } else { + to = DataStatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); + DataStatistics statistics = + statisticsSerializer.deserialize(to.dataStatistics(), source); + to.dataStatistics(statistics); + } + + return to; + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + boolean hasRecord = source.readBoolean(); + target.writeBoolean(hasRecord); + if (hasRecord) { + recordSerializer.copy(source, target); + } else { + statisticsSerializer.copy(source, target); + } + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof DataStatisticsOrRecordSerializer)) { + return false; + } + + @SuppressWarnings("unchecked") + DataStatisticsOrRecordSerializer other = (DataStatisticsOrRecordSerializer) obj; + return Objects.equals(statisticsSerializer, other.statisticsSerializer) + && Objects.equals(recordSerializer, other.recordSerializer); + } + + @Override + public int hashCode() { + return Objects.hash(statisticsSerializer, recordSerializer); + } + + @Override + public TypeSerializerSnapshot> snapshotConfiguration() { + return new DataStatisticsOrRecordSerializerSnapshot<>(this); + } + + public static class DataStatisticsOrRecordSerializerSnapshot, S> + extends CompositeTypeSerializerSnapshot< + DataStatisticsOrRecord, DataStatisticsOrRecordSerializer> { + private static final int CURRENT_VERSION = 1; + + // constructors need to public. Otherwise, Flink state restore would complain + // "The class has no (implicit) public nullary constructor". + @SuppressWarnings("checkstyle:RedundantModifier") + public DataStatisticsOrRecordSerializerSnapshot() { + super(DataStatisticsOrRecordSerializer.class); + } + + @SuppressWarnings("checkstyle:RedundantModifier") + public DataStatisticsOrRecordSerializerSnapshot( + DataStatisticsOrRecordSerializer serializer) { + super(serializer); + } + + @SuppressWarnings("checkstyle:RedundantModifier") + @Override + protected int getCurrentOuterSnapshotVersion() { + return CURRENT_VERSION; + } + + @Override + protected TypeSerializer[] getNestedSerializers( + DataStatisticsOrRecordSerializer outerSerializer) { + return new TypeSerializer[] { + outerSerializer.statisticsSerializer, outerSerializer.recordSerializer + }; + } + + @SuppressWarnings("unchecked") + @Override + protected DataStatisticsOrRecordSerializer createOuterSerializerWithNestedSerializers( + TypeSerializer[] nestedSerializers) { + TypeSerializer> statisticsSerializer = + (TypeSerializer>) nestedSerializers[0]; + TypeSerializer recordSerializer = (TypeSerializer) nestedSerializers[1]; + return new DataStatisticsOrRecordSerializer<>(statisticsSerializer, recordSerializer); + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java new file mode 100644 index 000000000000..2737b1346f0f --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java @@ -0,0 +1,97 @@ +/* + * 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.iceberg.flink.sink.shuffle; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; + +/** + * DataStatisticsUtil is the utility to serialize and deserialize {@link DataStatistics} and {@link + * AggregatedStatistics} + */ +class DataStatisticsUtil { + + private DataStatisticsUtil() {} + + static , S> byte[] serializeDataStatistics( + DataStatistics dataStatistics, + TypeSerializer> statisticsSerializer) { + DataOutputSerializer out = new DataOutputSerializer(64); + try { + statisticsSerializer.serialize(dataStatistics, out); + return out.getCopyOfBuffer(); + } catch (IOException e) { + throw new IllegalStateException("Fail to serialize data statistics", e); + } + } + + @SuppressWarnings("unchecked") + static , S> D deserializeDataStatistics( + byte[] bytes, TypeSerializer> statisticsSerializer) { + DataInputDeserializer input = new DataInputDeserializer(bytes, 0, bytes.length); + try { + return (D) statisticsSerializer.deserialize(input); + } catch (IOException e) { + throw new IllegalStateException("Fail to deserialize data statistics", e); + } + } + + static , S> byte[] serializeAggregatedStatistics( + AggregatedStatistics aggregatedStatistics, + TypeSerializer> statisticsSerializer) + throws IOException { + ByteArrayOutputStream bytes = new ByteArrayOutputStream(); + ObjectOutputStream out = new ObjectOutputStream(bytes); + + DataOutputSerializer outSerializer = new DataOutputSerializer(64); + out.writeLong(aggregatedStatistics.checkpointId()); + statisticsSerializer.serialize(aggregatedStatistics.dataStatistics(), outSerializer); + byte[] statisticsBytes = outSerializer.getCopyOfBuffer(); + out.writeInt(statisticsBytes.length); + out.write(statisticsBytes); + out.flush(); + + return bytes.toByteArray(); + } + + @SuppressWarnings("unchecked") + static , S> + AggregatedStatistics deserializeAggregatedStatistics( + byte[] bytes, TypeSerializer> statisticsSerializer) + throws IOException { + ByteArrayInputStream bytesIn = new ByteArrayInputStream(bytes); + ObjectInputStream in = new ObjectInputStream(bytesIn); + + long completedCheckpointId = in.readLong(); + int statisticsBytesLength = in.readInt(); + byte[] statisticsBytes = new byte[statisticsBytesLength]; + in.readFully(statisticsBytes); + DataInputDeserializer input = + new DataInputDeserializer(statisticsBytes, 0, statisticsBytesLength); + DataStatistics dataStatistics = statisticsSerializer.deserialize(input); + + return new AggregatedStatistics<>(completedCheckpointId, dataStatistics); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java new file mode 100644 index 000000000000..246b56526fd6 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java @@ -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. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.Map; +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +/** MapDataStatistics uses map to count key frequency */ +@Internal +class MapDataStatistics implements DataStatistics> { + private final Map statistics; + + MapDataStatistics() { + this.statistics = Maps.newHashMap(); + } + + MapDataStatistics(Map statistics) { + this.statistics = statistics; + } + + @Override + public boolean isEmpty() { + return statistics.size() == 0; + } + + @Override + public void add(RowData key) { + // increase count of occurrence by one in the dataStatistics map + statistics.merge(key, 1L, Long::sum); + } + + @Override + public void merge(MapDataStatistics otherStatistics) { + otherStatistics.statistics().forEach((key, count) -> statistics.merge(key, count, Long::sum)); + } + + @Override + public Map statistics() { + return statistics; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this).add("statistics", statistics).toString(); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java new file mode 100644 index 000000000000..6d07637b29b3 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java @@ -0,0 +1,183 @@ +/* + * 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.iceberg.flink.sink.shuffle; + +import java.io.IOException; +import java.util.Map; +import java.util.Objects; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.common.typeutils.base.MapSerializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.table.data.RowData; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +@Internal +class MapDataStatisticsSerializer + extends TypeSerializer>> { + private final MapSerializer mapSerializer; + + static TypeSerializer>> fromKeySerializer( + TypeSerializer keySerializer) { + return new MapDataStatisticsSerializer( + new MapSerializer<>(keySerializer, LongSerializer.INSTANCE)); + } + + MapDataStatisticsSerializer(MapSerializer mapSerializer) { + this.mapSerializer = mapSerializer; + } + + @Override + public boolean isImmutableType() { + return false; + } + + @SuppressWarnings("ReferenceEquality") + @Override + public TypeSerializer>> duplicate() { + MapSerializer duplicateMapSerializer = + (MapSerializer) mapSerializer.duplicate(); + return (duplicateMapSerializer == mapSerializer) + ? this + : new MapDataStatisticsSerializer(duplicateMapSerializer); + } + + @Override + public DataStatistics> createInstance() { + return new MapDataStatistics(); + } + + @Override + public DataStatistics> copy(DataStatistics obj) { + Preconditions.checkArgument( + obj instanceof MapDataStatistics, "Invalid data statistics type: " + obj.getClass()); + MapDataStatistics from = (MapDataStatistics) obj; + TypeSerializer keySerializer = mapSerializer.getKeySerializer(); + Map newMap = Maps.newHashMapWithExpectedSize(from.statistics().size()); + for (Map.Entry entry : from.statistics().entrySet()) { + RowData newKey = keySerializer.copy(entry.getKey()); + // no need to copy value since it is just a Long + newMap.put(newKey, entry.getValue()); + } + + return new MapDataStatistics(newMap); + } + + @Override + public DataStatistics> copy( + DataStatistics from, DataStatistics reuse) { + // not much benefit to reuse + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(DataStatistics obj, DataOutputView target) throws IOException { + Preconditions.checkArgument( + obj instanceof MapDataStatistics, "Invalid data statistics type: " + obj.getClass()); + MapDataStatistics mapStatistics = (MapDataStatistics) obj; + mapSerializer.serialize(mapStatistics.statistics(), target); + } + + @Override + public DataStatistics> deserialize(DataInputView source) + throws IOException { + return new MapDataStatistics(mapSerializer.deserialize(source)); + } + + @Override + public DataStatistics> deserialize( + DataStatistics reuse, DataInputView source) throws IOException { + // not much benefit to reuse + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + mapSerializer.copy(source, target); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof MapDataStatisticsSerializer)) { + return false; + } + + MapDataStatisticsSerializer other = (MapDataStatisticsSerializer) obj; + return Objects.equals(mapSerializer, other.mapSerializer); + } + + @Override + public int hashCode() { + return mapSerializer.hashCode(); + } + + @Override + public TypeSerializerSnapshot>> + snapshotConfiguration() { + return new MapDataStatisticsSerializerSnapshot(this); + } + + public static class MapDataStatisticsSerializerSnapshot + extends CompositeTypeSerializerSnapshot< + DataStatistics>, MapDataStatisticsSerializer> { + private static final int CURRENT_VERSION = 1; + + // constructors need to public. Otherwise, Flink state restore would complain + // "The class has no (implicit) public nullary constructor". + @SuppressWarnings("checkstyle:RedundantModifier") + public MapDataStatisticsSerializerSnapshot() { + super(MapDataStatisticsSerializer.class); + } + + @SuppressWarnings("checkstyle:RedundantModifier") + public MapDataStatisticsSerializerSnapshot(MapDataStatisticsSerializer serializer) { + super(serializer); + } + + @Override + protected int getCurrentOuterSnapshotVersion() { + return CURRENT_VERSION; + } + + @Override + protected TypeSerializer[] getNestedSerializers( + MapDataStatisticsSerializer outerSerializer) { + return new TypeSerializer[] {outerSerializer.mapSerializer}; + } + + @Override + protected MapDataStatisticsSerializer createOuterSerializerWithNestedSerializers( + TypeSerializer[] nestedSerializers) { + @SuppressWarnings("unchecked") + MapSerializer mapSerializer = + (MapSerializer) nestedSerializers[0]; + return new MapDataStatisticsSerializer(mapSerializer); + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java new file mode 100644 index 000000000000..796434c45136 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java @@ -0,0 +1,42 @@ +/* + * 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.iceberg.flink.source; + +import org.apache.avro.generic.GenericRecord; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.encryption.InputFilesDecryptor; +import org.apache.iceberg.io.CloseableIterator; + +public class AvroGenericRecordFileScanTaskReader implements FileScanTaskReader { + private final RowDataFileScanTaskReader rowDataReader; + private final RowDataToAvroGenericRecordConverter converter; + + public AvroGenericRecordFileScanTaskReader( + RowDataFileScanTaskReader rowDataReader, RowDataToAvroGenericRecordConverter converter) { + this.rowDataReader = rowDataReader; + this.converter = converter; + } + + @Override + public CloseableIterator open( + FileScanTask fileScanTask, InputFilesDecryptor inputFilesDecryptor) { + return CloseableIterator.transform( + rowDataReader.open(fileScanTask, inputFilesDecryptor), converter); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java new file mode 100644 index 000000000000..91d975349b19 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java @@ -0,0 +1,156 @@ +/* + * 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.iceberg.flink.source; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Iterator; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.InputFilesDecryptor; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * Flink data iterator that reads {@link CombinedScanTask} into a {@link CloseableIterator} + * + * @param is the output data type returned by this iterator. + */ +@Internal +public class DataIterator implements CloseableIterator { + + private final FileScanTaskReader fileScanTaskReader; + + private final InputFilesDecryptor inputFilesDecryptor; + private final CombinedScanTask combinedTask; + + private Iterator tasks; + private CloseableIterator currentIterator; + private int fileOffset; + private long recordOffset; + + public DataIterator( + FileScanTaskReader fileScanTaskReader, + CombinedScanTask task, + FileIO io, + EncryptionManager encryption) { + this.fileScanTaskReader = fileScanTaskReader; + + this.inputFilesDecryptor = new InputFilesDecryptor(task, io, encryption); + this.combinedTask = task; + + this.tasks = task.files().iterator(); + this.currentIterator = CloseableIterator.empty(); + + // fileOffset starts at -1 because we started + // from an empty iterator that is not from the split files. + this.fileOffset = -1; + // record offset points to the record that next() should return when called + this.recordOffset = 0L; + } + + /** + * (startingFileOffset, startingRecordOffset) points to the next row that reader should resume + * from. E.g., if the seek position is (file=0, record=1), seek moves the iterator position to the + * 2nd row in file 0. When next() is called after seek, 2nd row from file 0 should be returned. + */ + public void seek(int startingFileOffset, long startingRecordOffset) { + Preconditions.checkState( + fileOffset == -1, "Seek should be called before any other iterator actions"); + // skip files + Preconditions.checkState( + startingFileOffset < combinedTask.files().size(), + "Invalid starting file offset %s for combined scan task with %s files: %s", + startingFileOffset, + combinedTask.files().size(), + combinedTask); + for (long i = 0L; i < startingFileOffset; ++i) { + tasks.next(); + } + + updateCurrentIterator(); + // skip records within the file + for (long i = 0; i < startingRecordOffset; ++i) { + if (currentFileHasNext() && hasNext()) { + next(); + } else { + throw new IllegalStateException( + String.format( + "Invalid starting record offset %d for file %d from CombinedScanTask: %s", + startingRecordOffset, startingFileOffset, combinedTask)); + } + } + + fileOffset = startingFileOffset; + recordOffset = startingRecordOffset; + } + + @Override + public boolean hasNext() { + updateCurrentIterator(); + return currentIterator.hasNext(); + } + + @Override + public T next() { + updateCurrentIterator(); + recordOffset += 1; + return currentIterator.next(); + } + + public boolean currentFileHasNext() { + return currentIterator.hasNext(); + } + + /** Updates the current iterator field to ensure that the current Iterator is not exhausted. */ + private void updateCurrentIterator() { + try { + while (!currentIterator.hasNext() && tasks.hasNext()) { + currentIterator.close(); + currentIterator = openTaskIterator(tasks.next()); + fileOffset += 1; + recordOffset = 0L; + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + private CloseableIterator openTaskIterator(FileScanTask scanTask) { + return fileScanTaskReader.open(scanTask, inputFilesDecryptor); + } + + @Override + public void close() throws IOException { + // close the current iterator + currentIterator.close(); + tasks = null; + } + + public int fileOffset() { + return fileOffset; + } + + public long recordOffset() { + return recordOffset; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java new file mode 100644 index 000000000000..4394dab4d4cc --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java @@ -0,0 +1,47 @@ +/* + * 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.iceberg.flink.source; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Schema; +import org.apache.iceberg.encryption.InputFilesDecryptor; +import org.apache.iceberg.flink.data.StructRowData; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.CloseableIterator; + +@Internal +public class DataTaskReader implements FileScanTaskReader { + + private final Schema readSchema; + + public DataTaskReader(Schema readSchema) { + this.readSchema = readSchema; + } + + @Override + public CloseableIterator open( + FileScanTask task, InputFilesDecryptor inputFilesDecryptor) { + StructRowData row = new StructRowData(readSchema.asStruct()); + CloseableIterable iterable = + CloseableIterable.transform(task.asDataTask().rows(), row::setStruct); + return iterable.iterator(); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java new file mode 100644 index 000000000000..927a804a4792 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java @@ -0,0 +1,35 @@ +/* + * 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.iceberg.flink.source; + +import java.io.Serializable; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.encryption.InputFilesDecryptor; +import org.apache.iceberg.io.CloseableIterator; + +/** + * Read a {@link FileScanTask} into a {@link CloseableIterator} + * + * @param is the output data type returned by this iterator. + */ +@Internal +public interface FileScanTaskReader extends Serializable { + CloseableIterator open(FileScanTask fileScanTask, InputFilesDecryptor inputFilesDecryptor); +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java new file mode 100644 index 000000000000..9a5123dc489e --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java @@ -0,0 +1,141 @@ +/* + * 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.iceberg.flink.source; + +import java.io.IOException; +import java.util.concurrent.ExecutorService; +import org.apache.flink.api.common.io.DefaultInputSplitAssigner; +import org.apache.flink.api.common.io.InputFormat; +import org.apache.flink.api.common.io.LocatableInputSplitAssigner; +import org.apache.flink.api.common.io.RichInputFormat; +import org.apache.flink.api.common.io.statistics.BaseStatistics; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.io.InputSplitAssigner; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.BaseMetadataTable; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.util.ThreadPools; + +/** Flink {@link InputFormat} for Iceberg. */ +public class FlinkInputFormat extends RichInputFormat { + + private static final long serialVersionUID = 1L; + + private final TableLoader tableLoader; + private final FileIO io; + private final EncryptionManager encryption; + private final ScanContext context; + private final FileScanTaskReader rowDataReader; + + private transient DataIterator iterator; + private transient long currentReadCount = 0L; + + FlinkInputFormat( + TableLoader tableLoader, + Schema tableSchema, + FileIO io, + EncryptionManager encryption, + ScanContext context) { + this.tableLoader = tableLoader; + this.io = io; + this.encryption = encryption; + this.context = context; + + tableLoader.open(); + Table table = tableLoader.loadTable(); + if (table instanceof BaseMetadataTable) { + this.rowDataReader = new DataTaskReader(context.project()); + } else { + this.rowDataReader = + new RowDataFileScanTaskReader( + tableSchema, + context.project(), + context.nameMapping(), + context.caseSensitive(), + context.filters()); + } + } + + @VisibleForTesting + Schema projectedSchema() { + return context.project(); + } + + @Override + public BaseStatistics getStatistics(BaseStatistics cachedStatistics) { + // Legacy method, not be used. + return null; + } + + @Override + public FlinkInputSplit[] createInputSplits(int minNumSplits) throws IOException { + // Called in Job manager, so it is OK to load table from catalog. + tableLoader.open(); + final ExecutorService workerPool = + ThreadPools.newWorkerPool("iceberg-plan-worker-pool", context.planParallelism()); + try (TableLoader loader = tableLoader) { + Table table = loader.loadTable(); + return FlinkSplitPlanner.planInputSplits(table, context, workerPool); + } finally { + workerPool.shutdown(); + } + } + + @Override + public InputSplitAssigner getInputSplitAssigner(FlinkInputSplit[] inputSplits) { + return context.exposeLocality() + ? new LocatableInputSplitAssigner(inputSplits) + : new DefaultInputSplitAssigner(inputSplits); + } + + @Override + public void configure(Configuration parameters) {} + + @Override + public void open(FlinkInputSplit split) { + this.iterator = new DataIterator<>(rowDataReader, split.getTask(), io, encryption); + } + + @Override + public boolean reachedEnd() { + if (context.limit() > 0 && currentReadCount >= context.limit()) { + return true; + } else { + return !iterator.hasNext(); + } + } + + @Override + public RowData nextRecord(RowData reuse) { + currentReadCount++; + return iterator.next(); + } + + @Override + public void close() throws IOException { + if (iterator != null) { + iterator.close(); + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java new file mode 100644 index 000000000000..16fd4f39596c --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.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.iceberg.flink.source; + +import java.util.Arrays; +import javax.annotation.Nullable; +import org.apache.flink.core.io.LocatableInputSplit; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +public class FlinkInputSplit extends LocatableInputSplit { + + private final CombinedScanTask task; + + FlinkInputSplit(int splitNumber, CombinedScanTask task, @Nullable String[] hostnames) { + super(splitNumber, hostnames); + this.task = task; + } + + CombinedScanTask getTask() { + return task; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("splitNumber", getSplitNumber()) + .add("task", task) + .add("hosts", Arrays.toString(getHostnames())) + .toString(); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java new file mode 100644 index 000000000000..fa1656c55278 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java @@ -0,0 +1,310 @@ +/* + * 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.iceberg.flink.source; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.time.Duration; +import java.util.List; +import java.util.Map; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.FlinkReadOptions; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.PropertyUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class FlinkSource { + private static final Logger LOG = LoggerFactory.getLogger(FlinkSource.class); + + private FlinkSource() {} + + /** + * Initialize a {@link Builder} to read the data from iceberg table. Equivalent to {@link + * TableScan}. See more options in {@link ScanContext}. + * + *

The Source can be read static data in bounded mode. It can also continuously check the + * arrival of new data and read records incrementally. + * + *

    + *
  • Without startSnapshotId: Bounded + *
  • With startSnapshotId and with endSnapshotId: Bounded + *
  • With startSnapshotId (-1 means unbounded preceding) and Without endSnapshotId: Unbounded + *
+ * + *

+ * + * @return {@link Builder} to connect the iceberg table. + */ + public static Builder forRowData() { + return new Builder(); + } + + /** Source builder to build {@link DataStream}. */ + public static class Builder { + private StreamExecutionEnvironment env; + private Table table; + private TableLoader tableLoader; + private TableSchema projectedSchema; + private ReadableConfig readableConfig = new Configuration(); + private final ScanContext.Builder contextBuilder = ScanContext.builder(); + private Boolean exposeLocality; + + private final Map readOptions = Maps.newHashMap(); + + public Builder tableLoader(TableLoader newLoader) { + this.tableLoader = newLoader; + return this; + } + + public Builder table(Table newTable) { + this.table = newTable; + return this; + } + + public Builder env(StreamExecutionEnvironment newEnv) { + this.env = newEnv; + return this; + } + + public Builder filters(List filters) { + contextBuilder.filters(filters); + return this; + } + + public Builder project(TableSchema schema) { + this.projectedSchema = schema; + return this; + } + + public Builder limit(Long newLimit) { + if (newLimit != null) { + readOptions.put(FlinkReadOptions.LIMIT, Long.toString(newLimit)); + } + return this; + } + + public Builder set(String property, String value) { + readOptions.put(property, value); + return this; + } + + public Builder setAll(Map properties) { + readOptions.putAll(properties); + return this; + } + + /** @deprecated Use {@link #setAll} instead. */ + @Deprecated + public Builder properties(Map properties) { + readOptions.putAll(properties); + return this; + } + + public Builder caseSensitive(boolean caseSensitive) { + readOptions.put(FlinkReadOptions.CASE_SENSITIVE, Boolean.toString(caseSensitive)); + return this; + } + + public Builder snapshotId(Long snapshotId) { + readOptions.put(FlinkReadOptions.SNAPSHOT_ID.key(), Long.toString(snapshotId)); + return this; + } + + public Builder branch(String branch) { + readOptions.put(FlinkReadOptions.BRANCH.key(), branch); + return this; + } + + public Builder tag(String tag) { + readOptions.put(FlinkReadOptions.TAG.key(), tag); + return this; + } + + public Builder startSnapshotId(Long startSnapshotId) { + readOptions.put(FlinkReadOptions.START_SNAPSHOT_ID.key(), Long.toString(startSnapshotId)); + return this; + } + + public Builder endSnapshotId(Long endSnapshotId) { + readOptions.put(FlinkReadOptions.END_SNAPSHOT_ID.key(), Long.toString(endSnapshotId)); + return this; + } + + public Builder startTag(String startTag) { + readOptions.put(FlinkReadOptions.START_TAG.key(), startTag); + return this; + } + + public Builder endTag(String endTag) { + readOptions.put(FlinkReadOptions.END_TAG.key(), endTag); + return this; + } + + public Builder asOfTimestamp(Long asOfTimestamp) { + readOptions.put(FlinkReadOptions.AS_OF_TIMESTAMP.key(), Long.toString(asOfTimestamp)); + return this; + } + + public Builder splitSize(Long splitSize) { + readOptions.put(FlinkReadOptions.SPLIT_SIZE, Long.toString(splitSize)); + return this; + } + + public Builder splitLookback(Integer splitLookback) { + readOptions.put(FlinkReadOptions.SPLIT_LOOKBACK, Integer.toString(splitLookback)); + return this; + } + + public Builder splitOpenFileCost(Long splitOpenFileCost) { + readOptions.put(FlinkReadOptions.SPLIT_FILE_OPEN_COST, Long.toString(splitOpenFileCost)); + return this; + } + + public Builder streaming(boolean streaming) { + readOptions.put(FlinkReadOptions.STREAMING, Boolean.toString(streaming)); + return this; + } + + public Builder exposeLocality(boolean newExposeLocality) { + this.exposeLocality = newExposeLocality; + return this; + } + + public Builder nameMapping(String nameMapping) { + readOptions.put(TableProperties.DEFAULT_NAME_MAPPING, nameMapping); + return this; + } + + public Builder monitorInterval(Duration interval) { + readOptions.put(FlinkReadOptions.MONITOR_INTERVAL, interval.toNanos() + " ns"); + return this; + } + + public Builder maxPlanningSnapshotCount(int newMaxPlanningSnapshotCount) { + readOptions.put( + FlinkReadOptions.MAX_PLANNING_SNAPSHOT_COUNT, + Integer.toString(newMaxPlanningSnapshotCount)); + return this; + } + + public Builder flinkConf(ReadableConfig config) { + this.readableConfig = config; + return this; + } + + public FlinkInputFormat buildFormat() { + Preconditions.checkNotNull(tableLoader, "TableLoader should not be null"); + + Schema icebergSchema; + FileIO io; + EncryptionManager encryption; + if (table == null) { + // load required fields by table loader. + tableLoader.open(); + try (TableLoader loader = tableLoader) { + table = loader.loadTable(); + icebergSchema = table.schema(); + io = table.io(); + encryption = table.encryption(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } else { + icebergSchema = table.schema(); + io = table.io(); + encryption = table.encryption(); + } + + if (projectedSchema == null) { + contextBuilder.project(icebergSchema); + } else { + contextBuilder.project(FlinkSchemaUtil.convert(icebergSchema, projectedSchema)); + } + + contextBuilder.exposeLocality( + SourceUtil.isLocalityEnabled(table, readableConfig, exposeLocality)); + contextBuilder.planParallelism( + readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE)); + + contextBuilder.resolveConfig(table, readOptions, readableConfig); + + return new FlinkInputFormat( + tableLoader, icebergSchema, io, encryption, contextBuilder.build()); + } + + public DataStream build() { + Preconditions.checkNotNull(env, "StreamExecutionEnvironment should not be null"); + FlinkInputFormat format = buildFormat(); + + ScanContext context = contextBuilder.build(); + TypeInformation typeInfo = + FlinkCompatibilityUtil.toTypeInfo(FlinkSchemaUtil.convert(context.project())); + + if (!context.isStreaming()) { + int parallelism = + SourceUtil.inferParallelism( + readableConfig, + context.limit(), + () -> { + try { + return format.createInputSplits(0).length; + } catch (IOException e) { + throw new UncheckedIOException( + "Failed to create iceberg input splits for table: " + table, e); + } + }); + if (env.getMaxParallelism() > 0) { + parallelism = Math.min(parallelism, env.getMaxParallelism()); + } + return env.createInput(format, typeInfo).setParallelism(parallelism); + } else { + StreamingMonitorFunction function = new StreamingMonitorFunction(tableLoader, context); + + String monitorFunctionName = String.format("Iceberg table (%s) monitor", table); + String readerOperatorName = String.format("Iceberg table (%s) reader", table); + + return env.addSource(function, monitorFunctionName) + .transform(readerOperatorName, typeInfo, StreamingReaderOperator.factory(format)); + } + } + } + + public static boolean isBounded(Map properties) { + return !PropertyUtil.propertyAsBoolean(properties, FlinkReadOptions.STREAMING, false); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java new file mode 100644 index 000000000000..15078809714f --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.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.iceberg.flink.source; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.List; +import java.util.concurrent.ExecutorService; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.IncrementalAppendScan; +import org.apache.iceberg.Scan; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.hadoop.Util; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.Tasks; + +@Internal +public class FlinkSplitPlanner { + private FlinkSplitPlanner() {} + + static FlinkInputSplit[] planInputSplits( + Table table, ScanContext context, ExecutorService workerPool) { + try (CloseableIterable tasksIterable = + planTasks(table, context, workerPool)) { + List tasks = Lists.newArrayList(tasksIterable); + FlinkInputSplit[] splits = new FlinkInputSplit[tasks.size()]; + boolean exposeLocality = context.exposeLocality(); + + Tasks.range(tasks.size()) + .stopOnFailure() + .executeWith(exposeLocality ? workerPool : null) + .run( + index -> { + CombinedScanTask task = tasks.get(index); + String[] hostnames = null; + if (exposeLocality) { + hostnames = Util.blockLocations(table.io(), task); + } + splits[index] = new FlinkInputSplit(index, task, hostnames); + }); + return splits; + } catch (IOException e) { + throw new UncheckedIOException("Failed to process tasks iterable", e); + } + } + + /** This returns splits for the FLIP-27 source */ + public static List planIcebergSourceSplits( + Table table, ScanContext context, ExecutorService workerPool) { + try (CloseableIterable tasksIterable = + planTasks(table, context, workerPool)) { + return Lists.newArrayList( + CloseableIterable.transform(tasksIterable, IcebergSourceSplit::fromCombinedScanTask)); + } catch (IOException e) { + throw new UncheckedIOException("Failed to process task iterable: ", e); + } + } + + static CloseableIterable planTasks( + Table table, ScanContext context, ExecutorService workerPool) { + ScanMode scanMode = checkScanMode(context); + if (scanMode == ScanMode.INCREMENTAL_APPEND_SCAN) { + IncrementalAppendScan scan = table.newIncrementalAppendScan(); + scan = refineScanWithBaseConfigs(scan, context, workerPool); + + if (context.startTag() != null) { + Preconditions.checkArgument( + table.snapshot(context.startTag()) != null, + "Cannot find snapshot with tag %s", + context.startTag()); + scan = scan.fromSnapshotExclusive(table.snapshot(context.startTag()).snapshotId()); + } + + if (context.startSnapshotId() != null) { + Preconditions.checkArgument( + context.startTag() == null, "START_SNAPSHOT_ID and START_TAG cannot both be set"); + scan = scan.fromSnapshotExclusive(context.startSnapshotId()); + } + + if (context.endTag() != null) { + Preconditions.checkArgument( + table.snapshot(context.endTag()) != null, + "Cannot find snapshot with tag %s", + context.endTag()); + scan = scan.toSnapshot(table.snapshot(context.endTag()).snapshotId()); + } + + if (context.endSnapshotId() != null) { + Preconditions.checkArgument( + context.endTag() == null, "END_SNAPSHOT_ID and END_TAG cannot both be set"); + scan = scan.toSnapshot(context.endSnapshotId()); + } + + return scan.planTasks(); + } else { + TableScan scan = table.newScan(); + scan = refineScanWithBaseConfigs(scan, context, workerPool); + + if (context.snapshotId() != null) { + scan = scan.useSnapshot(context.snapshotId()); + } else if (context.tag() != null) { + scan = scan.useRef(context.tag()); + } else if (context.branch() != null) { + scan = scan.useRef(context.branch()); + } + + if (context.asOfTimestamp() != null) { + scan = scan.asOfTime(context.asOfTimestamp()); + } + + return scan.planTasks(); + } + } + + @VisibleForTesting + enum ScanMode { + BATCH, + INCREMENTAL_APPEND_SCAN + } + + @VisibleForTesting + static ScanMode checkScanMode(ScanContext context) { + if (context.startSnapshotId() != null + || context.endSnapshotId() != null + || context.startTag() != null + || context.endTag() != null) { + return ScanMode.INCREMENTAL_APPEND_SCAN; + } else { + return ScanMode.BATCH; + } + } + + /** refine scan with common configs */ + private static > T refineScanWithBaseConfigs( + T scan, ScanContext context, ExecutorService workerPool) { + T refinedScan = + scan.caseSensitive(context.caseSensitive()).project(context.project()).planWith(workerPool); + + if (context.includeColumnStats()) { + refinedScan = refinedScan.includeColumnStats(); + } + + if (context.includeStatsForColumns() != null) { + refinedScan = refinedScan.includeColumnStats(context.includeStatsForColumns()); + } + + refinedScan = refinedScan.option(TableProperties.SPLIT_SIZE, context.splitSize().toString()); + + refinedScan = + refinedScan.option(TableProperties.SPLIT_LOOKBACK, context.splitLookback().toString()); + + refinedScan = + refinedScan.option( + TableProperties.SPLIT_OPEN_FILE_COST, context.splitOpenFileCost().toString()); + + if (context.filters() != null) { + for (Expression filter : context.filters()) { + refinedScan = refinedScan.filter(filter); + } + } + + return refinedScan; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java new file mode 100644 index 000000000000..179253cb3a18 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -0,0 +1,558 @@ +/* + * 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.iceberg.flink.source; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import javax.annotation.Nullable; +import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.BaseMetadataTable; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.FlinkReadOptions; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.source.assigner.OrderedSplitAssignerFactory; +import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; +import org.apache.iceberg.flink.source.assigner.SplitAssigner; +import org.apache.iceberg.flink.source.assigner.SplitAssignerFactory; +import org.apache.iceberg.flink.source.enumerator.ContinuousIcebergEnumerator; +import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlanner; +import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlannerImpl; +import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorState; +import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer; +import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator; +import org.apache.iceberg.flink.source.reader.ColumnStatsWatermarkExtractor; +import org.apache.iceberg.flink.source.reader.IcebergSourceReader; +import org.apache.iceberg.flink.source.reader.IcebergSourceReaderMetrics; +import org.apache.iceberg.flink.source.reader.MetaDataReaderFunction; +import org.apache.iceberg.flink.source.reader.ReaderFunction; +import org.apache.iceberg.flink.source.reader.RowDataReaderFunction; +import org.apache.iceberg.flink.source.reader.SerializableRecordEmitter; +import org.apache.iceberg.flink.source.reader.SplitWatermarkExtractor; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer; +import org.apache.iceberg.flink.source.split.SerializableComparator; +import org.apache.iceberg.flink.source.split.SplitComparators; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.util.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Experimental +public class IcebergSource implements Source { + private static final Logger LOG = LoggerFactory.getLogger(IcebergSource.class); + + private final TableLoader tableLoader; + private final ScanContext scanContext; + private final ReaderFunction readerFunction; + private final SplitAssignerFactory assignerFactory; + private final SerializableComparator splitComparator; + private final SerializableRecordEmitter emitter; + + // Can't use SerializableTable as enumerator needs a regular table + // that can discover table changes + private transient Table table; + + IcebergSource( + TableLoader tableLoader, + ScanContext scanContext, + ReaderFunction readerFunction, + SplitAssignerFactory assignerFactory, + SerializableComparator splitComparator, + Table table, + SerializableRecordEmitter emitter) { + this.tableLoader = tableLoader; + this.scanContext = scanContext; + this.readerFunction = readerFunction; + this.assignerFactory = assignerFactory; + this.splitComparator = splitComparator; + this.table = table; + this.emitter = emitter; + } + + String name() { + return "IcebergSource-" + lazyTable().name(); + } + + private String planningThreadName() { + // Ideally, operatorId should be used as the threadPoolName as Flink guarantees its uniqueness + // within a job. SplitEnumeratorContext doesn't expose the OperatorCoordinator.Context, which + // would contain the OperatorID. Need to discuss with Flink community whether it is ok to expose + // a public API like the protected method "OperatorCoordinator.Context getCoordinatorContext()" + // from SourceCoordinatorContext implementation. For now,

- is used as + // the unique thread pool name. + return lazyTable().name() + "-" + UUID.randomUUID(); + } + + private List planSplitsForBatch(String threadName) { + ExecutorService workerPool = + ThreadPools.newWorkerPool(threadName, scanContext.planParallelism()); + try { + List splits = + FlinkSplitPlanner.planIcebergSourceSplits(lazyTable(), scanContext, workerPool); + LOG.info( + "Discovered {} splits from table {} during job initialization", + splits.size(), + lazyTable().name()); + return splits; + } finally { + workerPool.shutdown(); + } + } + + private Table lazyTable() { + if (table == null) { + tableLoader.open(); + try (TableLoader loader = tableLoader) { + this.table = loader.loadTable(); + } catch (IOException e) { + throw new UncheckedIOException("Failed to close table loader", e); + } + } + + return table; + } + + @Override + public Boundedness getBoundedness() { + return scanContext.isStreaming() ? Boundedness.CONTINUOUS_UNBOUNDED : Boundedness.BOUNDED; + } + + @Override + public SourceReader createReader(SourceReaderContext readerContext) { + IcebergSourceReaderMetrics metrics = + new IcebergSourceReaderMetrics(readerContext.metricGroup(), lazyTable().name()); + return new IcebergSourceReader<>( + emitter, metrics, readerFunction, splitComparator, readerContext); + } + + @Override + public SplitEnumerator createEnumerator( + SplitEnumeratorContext enumContext) { + return createEnumerator(enumContext, null); + } + + @Override + public SplitEnumerator restoreEnumerator( + SplitEnumeratorContext enumContext, IcebergEnumeratorState enumState) { + return createEnumerator(enumContext, enumState); + } + + @Override + public SimpleVersionedSerializer getSplitSerializer() { + return new IcebergSourceSplitSerializer(scanContext.caseSensitive()); + } + + @Override + public SimpleVersionedSerializer getEnumeratorCheckpointSerializer() { + return new IcebergEnumeratorStateSerializer(scanContext.caseSensitive()); + } + + private SplitEnumerator createEnumerator( + SplitEnumeratorContext enumContext, + @Nullable IcebergEnumeratorState enumState) { + SplitAssigner assigner; + if (enumState == null) { + assigner = assignerFactory.createAssigner(); + } else { + LOG.info( + "Iceberg source restored {} splits from state for table {}", + enumState.pendingSplits().size(), + lazyTable().name()); + assigner = assignerFactory.createAssigner(enumState.pendingSplits()); + } + + if (scanContext.isStreaming()) { + ContinuousSplitPlanner splitPlanner = + new ContinuousSplitPlannerImpl(tableLoader.clone(), scanContext, planningThreadName()); + return new ContinuousIcebergEnumerator( + enumContext, assigner, scanContext, splitPlanner, enumState); + } else { + List splits = planSplitsForBatch(planningThreadName()); + assigner.onDiscoveredSplits(splits); + return new StaticIcebergEnumerator(enumContext, assigner); + } + } + + public static Builder builder() { + return new Builder<>(); + } + + public static Builder forRowData() { + return new Builder<>(); + } + + public static class Builder { + private TableLoader tableLoader; + private Table table; + private SplitAssignerFactory splitAssignerFactory; + private SerializableComparator splitComparator; + private String watermarkColumn; + private TimeUnit watermarkTimeUnit = TimeUnit.MICROSECONDS; + private ReaderFunction readerFunction; + private ReadableConfig flinkConfig = new Configuration(); + private final ScanContext.Builder contextBuilder = ScanContext.builder(); + private TableSchema projectedFlinkSchema; + private Boolean exposeLocality; + + private final Map readOptions = Maps.newHashMap(); + + Builder() {} + + public Builder tableLoader(TableLoader loader) { + this.tableLoader = loader; + return this; + } + + public Builder table(Table newTable) { + this.table = newTable; + return this; + } + + public Builder assignerFactory(SplitAssignerFactory assignerFactory) { + Preconditions.checkArgument( + watermarkColumn == null, + "Watermark column and SplitAssigner should not be set in the same source"); + this.splitAssignerFactory = assignerFactory; + return this; + } + + public Builder splitComparator( + SerializableComparator newSplitComparator) { + this.splitComparator = newSplitComparator; + return this; + } + + public Builder readerFunction(ReaderFunction newReaderFunction) { + this.readerFunction = newReaderFunction; + return this; + } + + public Builder flinkConfig(ReadableConfig config) { + this.flinkConfig = config; + return this; + } + + public Builder caseSensitive(boolean newCaseSensitive) { + readOptions.put(FlinkReadOptions.CASE_SENSITIVE, Boolean.toString(newCaseSensitive)); + return this; + } + + public Builder useSnapshotId(Long newSnapshotId) { + if (newSnapshotId != null) { + readOptions.put(FlinkReadOptions.SNAPSHOT_ID.key(), Long.toString(newSnapshotId)); + } + return this; + } + + public Builder streamingStartingStrategy(StreamingStartingStrategy newStartingStrategy) { + readOptions.put(FlinkReadOptions.STARTING_STRATEGY, newStartingStrategy.name()); + return this; + } + + public Builder startSnapshotTimestamp(Long newStartSnapshotTimestamp) { + if (newStartSnapshotTimestamp != null) { + readOptions.put( + FlinkReadOptions.START_SNAPSHOT_TIMESTAMP.key(), + Long.toString(newStartSnapshotTimestamp)); + } + return this; + } + + public Builder startSnapshotId(Long newStartSnapshotId) { + if (newStartSnapshotId != null) { + readOptions.put( + FlinkReadOptions.START_SNAPSHOT_ID.key(), Long.toString(newStartSnapshotId)); + } + return this; + } + + public Builder tag(String tag) { + readOptions.put(FlinkReadOptions.TAG.key(), tag); + return this; + } + + public Builder branch(String branch) { + readOptions.put(FlinkReadOptions.BRANCH.key(), branch); + return this; + } + + public Builder startTag(String startTag) { + readOptions.put(FlinkReadOptions.START_TAG.key(), startTag); + return this; + } + + public Builder endTag(String endTag) { + readOptions.put(FlinkReadOptions.END_TAG.key(), endTag); + return this; + } + + public Builder endSnapshotId(Long newEndSnapshotId) { + if (newEndSnapshotId != null) { + readOptions.put(FlinkReadOptions.END_SNAPSHOT_ID.key(), Long.toString(newEndSnapshotId)); + } + return this; + } + + public Builder asOfTimestamp(Long newAsOfTimestamp) { + if (newAsOfTimestamp != null) { + readOptions.put(FlinkReadOptions.AS_OF_TIMESTAMP.key(), Long.toString(newAsOfTimestamp)); + } + return this; + } + + public Builder splitSize(Long newSplitSize) { + if (newSplitSize != null) { + readOptions.put(FlinkReadOptions.SPLIT_SIZE, Long.toString(newSplitSize)); + } + return this; + } + + public Builder splitLookback(Integer newSplitLookback) { + if (newSplitLookback != null) { + readOptions.put(FlinkReadOptions.SPLIT_LOOKBACK, Integer.toString(newSplitLookback)); + } + return this; + } + + public Builder splitOpenFileCost(Long newSplitOpenFileCost) { + if (newSplitOpenFileCost != null) { + readOptions.put(FlinkReadOptions.SPLIT_FILE_OPEN_COST, Long.toString(newSplitOpenFileCost)); + } + + return this; + } + + public Builder streaming(boolean streaming) { + readOptions.put(FlinkReadOptions.STREAMING, Boolean.toString(streaming)); + return this; + } + + public Builder monitorInterval(Duration newMonitorInterval) { + if (newMonitorInterval != null) { + readOptions.put(FlinkReadOptions.MONITOR_INTERVAL, newMonitorInterval.toNanos() + " ns"); + } + return this; + } + + public Builder nameMapping(String newNameMapping) { + readOptions.put(TableProperties.DEFAULT_NAME_MAPPING, newNameMapping); + return this; + } + + public Builder project(Schema newProjectedSchema) { + this.contextBuilder.project(newProjectedSchema); + return this; + } + + public Builder project(TableSchema newProjectedFlinkSchema) { + this.projectedFlinkSchema = newProjectedFlinkSchema; + return this; + } + + public Builder filters(List newFilters) { + this.contextBuilder.filters(newFilters); + return this; + } + + public Builder limit(Long newLimit) { + if (newLimit != null) { + readOptions.put(FlinkReadOptions.LIMIT, Long.toString(newLimit)); + } + return this; + } + + public Builder includeColumnStats(boolean newIncludeColumnStats) { + readOptions.put( + FlinkReadOptions.INCLUDE_COLUMN_STATS, Boolean.toString(newIncludeColumnStats)); + return this; + } + + public Builder planParallelism(int planParallelism) { + readOptions.put( + FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.key(), + Integer.toString(planParallelism)); + return this; + } + + public Builder exposeLocality(boolean newExposeLocality) { + this.exposeLocality = newExposeLocality; + return this; + } + + public Builder maxAllowedPlanningFailures(int maxAllowedPlanningFailures) { + readOptions.put( + FlinkReadOptions.MAX_ALLOWED_PLANNING_FAILURES_OPTION.key(), + Integer.toString(maxAllowedPlanningFailures)); + return this; + } + + /** + * Set the read properties for Flink source. View the supported properties in {@link + * FlinkReadOptions} + */ + public Builder set(String property, String value) { + readOptions.put(property, value); + return this; + } + + /** + * Set the read properties for Flink source. View the supported properties in {@link + * FlinkReadOptions} + */ + public Builder setAll(Map properties) { + readOptions.putAll(properties); + return this; + } + + /** + * Emits watermarks once per split based on the min value of column statistics from files + * metadata in the given split. The generated watermarks are also used for ordering the splits + * for read. Accepted column types are timestamp/timestamptz/long. For long columns consider + * setting {@link #watermarkTimeUnit(TimeUnit)}. + * + *

Consider setting `read.split.open-file-cost` to prevent combining small files to a single + * split when the watermark is used for watermark alignment. + */ + public Builder watermarkColumn(String columnName) { + Preconditions.checkArgument( + splitAssignerFactory == null, + "Watermark column and SplitAssigner should not be set in the same source"); + this.watermarkColumn = columnName; + return this; + } + + /** + * When the type of the {@link #watermarkColumn} is {@link + * org.apache.iceberg.types.Types.LongType}, then sets the {@link TimeUnit} to convert the + * value. The default value is {@link TimeUnit#MICROSECONDS}. + */ + public Builder watermarkTimeUnit(TimeUnit timeUnit) { + this.watermarkTimeUnit = timeUnit; + return this; + } + + /** @deprecated Use {@link #setAll} instead. */ + @Deprecated + public Builder properties(Map properties) { + readOptions.putAll(properties); + return this; + } + + public IcebergSource build() { + if (table == null) { + try (TableLoader loader = tableLoader) { + loader.open(); + this.table = tableLoader.loadTable(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + contextBuilder.resolveConfig(table, readOptions, flinkConfig); + + Schema icebergSchema = table.schema(); + if (projectedFlinkSchema != null) { + contextBuilder.project(FlinkSchemaUtil.convert(icebergSchema, projectedFlinkSchema)); + } + + SerializableRecordEmitter emitter = SerializableRecordEmitter.defaultEmitter(); + if (watermarkColumn != null) { + // Column statistics is needed for watermark generation + contextBuilder.includeColumnStats(Sets.newHashSet(watermarkColumn)); + + SplitWatermarkExtractor watermarkExtractor = + new ColumnStatsWatermarkExtractor(icebergSchema, watermarkColumn, watermarkTimeUnit); + emitter = SerializableRecordEmitter.emitterWithWatermark(watermarkExtractor); + splitAssignerFactory = + new OrderedSplitAssignerFactory(SplitComparators.watermark(watermarkExtractor)); + } + + ScanContext context = contextBuilder.build(); + if (readerFunction == null) { + if (table instanceof BaseMetadataTable) { + MetaDataReaderFunction rowDataReaderFunction = + new MetaDataReaderFunction( + flinkConfig, table.schema(), context.project(), table.io(), table.encryption()); + this.readerFunction = (ReaderFunction) rowDataReaderFunction; + } else { + RowDataReaderFunction rowDataReaderFunction = + new RowDataReaderFunction( + flinkConfig, + table.schema(), + context.project(), + context.nameMapping(), + context.caseSensitive(), + table.io(), + table.encryption(), + context.filters()); + this.readerFunction = (ReaderFunction) rowDataReaderFunction; + } + } + + if (splitAssignerFactory == null) { + if (splitComparator == null) { + splitAssignerFactory = new SimpleSplitAssignerFactory(); + } else { + splitAssignerFactory = new OrderedSplitAssignerFactory(splitComparator); + } + } + + checkRequired(); + // Since builder already load the table, pass it to the source to avoid double loading + return new IcebergSource<>( + tableLoader, + context, + readerFunction, + splitAssignerFactory, + splitComparator, + table, + emitter); + } + + private void checkRequired() { + Preconditions.checkNotNull(tableLoader, "tableLoader is required."); + Preconditions.checkNotNull(splitAssignerFactory, "assignerFactory is required."); + Preconditions.checkNotNull(readerFunction, "readerFunction is required."); + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java new file mode 100644 index 000000000000..610657e8d47b --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java @@ -0,0 +1,229 @@ +/* + * 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.iceberg.flink.source; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.ProviderContext; +import org.apache.flink.table.connector.source.DataStreamScanProvider; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.connector.source.ScanTableSource; +import org.apache.flink.table.connector.source.abilities.SupportsFilterPushDown; +import org.apache.flink.table.connector.source.abilities.SupportsLimitPushDown; +import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.expressions.ResolvedExpression; +import org.apache.flink.table.types.DataType; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.FlinkFilters; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.source.assigner.SplitAssignerType; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +/** Flink Iceberg table source. */ +@Internal +public class IcebergTableSource + implements ScanTableSource, + SupportsProjectionPushDown, + SupportsFilterPushDown, + SupportsLimitPushDown { + + private int[] projectedFields; + private Long limit; + private List filters; + + private final TableLoader loader; + private final TableSchema schema; + private final Map properties; + private final boolean isLimitPushDown; + private final ReadableConfig readableConfig; + + private IcebergTableSource(IcebergTableSource toCopy) { + this.loader = toCopy.loader; + this.schema = toCopy.schema; + this.properties = toCopy.properties; + this.projectedFields = toCopy.projectedFields; + this.isLimitPushDown = toCopy.isLimitPushDown; + this.limit = toCopy.limit; + this.filters = toCopy.filters; + this.readableConfig = toCopy.readableConfig; + } + + public IcebergTableSource( + TableLoader loader, + TableSchema schema, + Map properties, + ReadableConfig readableConfig) { + this(loader, schema, properties, null, false, null, ImmutableList.of(), readableConfig); + } + + private IcebergTableSource( + TableLoader loader, + TableSchema schema, + Map properties, + int[] projectedFields, + boolean isLimitPushDown, + Long limit, + List filters, + ReadableConfig readableConfig) { + this.loader = loader; + this.schema = schema; + this.properties = properties; + this.projectedFields = projectedFields; + this.isLimitPushDown = isLimitPushDown; + this.limit = limit; + this.filters = filters; + this.readableConfig = readableConfig; + } + + @Override + public void applyProjection(int[][] projectFields) { + this.projectedFields = new int[projectFields.length]; + for (int i = 0; i < projectFields.length; i++) { + Preconditions.checkArgument( + projectFields[i].length == 1, "Don't support nested projection in iceberg source now."); + this.projectedFields[i] = projectFields[i][0]; + } + } + + private DataStream createDataStream(StreamExecutionEnvironment execEnv) { + return FlinkSource.forRowData() + .env(execEnv) + .tableLoader(loader) + .properties(properties) + .project(getProjectedSchema()) + .limit(limit) + .filters(filters) + .flinkConf(readableConfig) + .build(); + } + + private DataStreamSource createFLIP27Stream(StreamExecutionEnvironment env) { + SplitAssignerType assignerType = + readableConfig.get(FlinkConfigOptions.TABLE_EXEC_SPLIT_ASSIGNER_TYPE); + IcebergSource source = + IcebergSource.forRowData() + .tableLoader(loader) + .assignerFactory(assignerType.factory()) + .properties(properties) + .project(getProjectedSchema()) + .limit(limit) + .filters(filters) + .flinkConfig(readableConfig) + .build(); + DataStreamSource stream = + env.fromSource( + source, + WatermarkStrategy.noWatermarks(), + source.name(), + TypeInformation.of(RowData.class)); + return stream; + } + + private TableSchema getProjectedSchema() { + if (projectedFields == null) { + return schema; + } else { + String[] fullNames = schema.getFieldNames(); + DataType[] fullTypes = schema.getFieldDataTypes(); + return TableSchema.builder() + .fields( + Arrays.stream(projectedFields).mapToObj(i -> fullNames[i]).toArray(String[]::new), + Arrays.stream(projectedFields).mapToObj(i -> fullTypes[i]).toArray(DataType[]::new)) + .build(); + } + } + + @Override + public void applyLimit(long newLimit) { + this.limit = newLimit; + } + + @Override + public Result applyFilters(List flinkFilters) { + List acceptedFilters = Lists.newArrayList(); + List expressions = Lists.newArrayList(); + + for (ResolvedExpression resolvedExpression : flinkFilters) { + Optional icebergExpression = FlinkFilters.convert(resolvedExpression); + if (icebergExpression.isPresent()) { + expressions.add(icebergExpression.get()); + acceptedFilters.add(resolvedExpression); + } + } + + this.filters = expressions; + return Result.of(acceptedFilters, flinkFilters); + } + + @Override + public boolean supportsNestedProjection() { + // TODO: support nested projection + return false; + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.insertOnly(); + } + + @Override + public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) { + return new DataStreamScanProvider() { + @Override + public DataStream produceDataStream( + ProviderContext providerContext, StreamExecutionEnvironment execEnv) { + if (readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE)) { + return createFLIP27Stream(execEnv); + } else { + return createDataStream(execEnv); + } + } + + @Override + public boolean isBounded() { + return FlinkSource.isBounded(properties); + } + }; + } + + @Override + public DynamicTableSource copy() { + return new IcebergTableSource(this); + } + + @Override + public String asSummaryString() { + return "Iceberg table source"; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java new file mode 100644 index 000000000000..88364f4e87b1 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java @@ -0,0 +1,243 @@ +/* + * 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.iceberg.flink.source; + +import java.util.List; +import java.util.Map; +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.data.DeleteFilter; +import org.apache.iceberg.encryption.InputFilesDecryptor; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.FlinkSourceFilter; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.data.FlinkAvroReader; +import org.apache.iceberg.flink.data.FlinkOrcReader; +import org.apache.iceberg.flink.data.FlinkParquetReaders; +import org.apache.iceberg.flink.data.RowDataProjection; +import org.apache.iceberg.flink.data.RowDataUtil; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.mapping.NameMappingParser; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.PartitionUtil; + +@Internal +public class RowDataFileScanTaskReader implements FileScanTaskReader { + + private final Schema tableSchema; + private final Schema projectedSchema; + private final String nameMapping; + private final boolean caseSensitive; + private final FlinkSourceFilter rowFilter; + + public RowDataFileScanTaskReader( + Schema tableSchema, + Schema projectedSchema, + String nameMapping, + boolean caseSensitive, + List filters) { + this.tableSchema = tableSchema; + this.projectedSchema = projectedSchema; + this.nameMapping = nameMapping; + this.caseSensitive = caseSensitive; + + if (filters != null && !filters.isEmpty()) { + Expression combinedExpression = + filters.stream().reduce(Expressions.alwaysTrue(), Expressions::and); + this.rowFilter = + new FlinkSourceFilter(this.projectedSchema, combinedExpression, this.caseSensitive); + } else { + this.rowFilter = null; + } + } + + @Override + public CloseableIterator open( + FileScanTask task, InputFilesDecryptor inputFilesDecryptor) { + Schema partitionSchema = TypeUtil.select(projectedSchema, task.spec().identitySourceIds()); + + Map idToConstant = + partitionSchema.columns().isEmpty() + ? ImmutableMap.of() + : PartitionUtil.constantsMap(task, RowDataUtil::convertConstant); + + FlinkDeleteFilter deletes = + new FlinkDeleteFilter(task, tableSchema, projectedSchema, inputFilesDecryptor); + CloseableIterable iterable = + deletes.filter( + newIterable(task, deletes.requiredSchema(), idToConstant, inputFilesDecryptor)); + + // Project the RowData to remove the extra meta columns. + if (!projectedSchema.sameSchema(deletes.requiredSchema())) { + RowDataProjection rowDataProjection = + RowDataProjection.create( + deletes.requiredRowType(), + deletes.requiredSchema().asStruct(), + projectedSchema.asStruct()); + iterable = CloseableIterable.transform(iterable, rowDataProjection::wrap); + } + + return iterable.iterator(); + } + + private CloseableIterable newIterable( + FileScanTask task, + Schema schema, + Map idToConstant, + InputFilesDecryptor inputFilesDecryptor) { + CloseableIterable iter; + if (task.isDataTask()) { + throw new UnsupportedOperationException("Cannot read data task."); + } else { + switch (task.file().format()) { + case PARQUET: + iter = newParquetIterable(task, schema, idToConstant, inputFilesDecryptor); + break; + + case AVRO: + iter = newAvroIterable(task, schema, idToConstant, inputFilesDecryptor); + break; + + case ORC: + iter = newOrcIterable(task, schema, idToConstant, inputFilesDecryptor); + break; + + default: + throw new UnsupportedOperationException( + "Cannot read unknown format: " + task.file().format()); + } + } + + if (rowFilter != null) { + return CloseableIterable.filter(iter, rowFilter::filter); + } + return iter; + } + + private CloseableIterable newAvroIterable( + FileScanTask task, + Schema schema, + Map idToConstant, + InputFilesDecryptor inputFilesDecryptor) { + Avro.ReadBuilder builder = + Avro.read(inputFilesDecryptor.getInputFile(task)) + .reuseContainers() + .project(schema) + .split(task.start(), task.length()) + .createReaderFunc(readSchema -> new FlinkAvroReader(schema, readSchema, idToConstant)); + + if (nameMapping != null) { + builder.withNameMapping(NameMappingParser.fromJson(nameMapping)); + } + + return builder.build(); + } + + private CloseableIterable newParquetIterable( + FileScanTask task, + Schema schema, + Map idToConstant, + InputFilesDecryptor inputFilesDecryptor) { + Parquet.ReadBuilder builder = + Parquet.read(inputFilesDecryptor.getInputFile(task)) + .split(task.start(), task.length()) + .project(schema) + .createReaderFunc( + fileSchema -> FlinkParquetReaders.buildReader(schema, fileSchema, idToConstant)) + .filter(task.residual()) + .caseSensitive(caseSensitive) + .reuseContainers(); + + if (nameMapping != null) { + builder.withNameMapping(NameMappingParser.fromJson(nameMapping)); + } + + return builder.build(); + } + + private CloseableIterable newOrcIterable( + FileScanTask task, + Schema schema, + Map idToConstant, + InputFilesDecryptor inputFilesDecryptor) { + Schema readSchemaWithoutConstantAndMetadataFields = + TypeUtil.selectNot( + schema, Sets.union(idToConstant.keySet(), MetadataColumns.metadataFieldIds())); + + ORC.ReadBuilder builder = + ORC.read(inputFilesDecryptor.getInputFile(task)) + .project(readSchemaWithoutConstantAndMetadataFields) + .split(task.start(), task.length()) + .createReaderFunc( + readOrcSchema -> new FlinkOrcReader(schema, readOrcSchema, idToConstant)) + .filter(task.residual()) + .caseSensitive(caseSensitive); + + if (nameMapping != null) { + builder.withNameMapping(NameMappingParser.fromJson(nameMapping)); + } + + return builder.build(); + } + + private static class FlinkDeleteFilter extends DeleteFilter { + private final RowType requiredRowType; + private final RowDataWrapper asStructLike; + private final InputFilesDecryptor inputFilesDecryptor; + + FlinkDeleteFilter( + FileScanTask task, + Schema tableSchema, + Schema requestedSchema, + InputFilesDecryptor inputFilesDecryptor) { + super(task.file().path().toString(), task.deletes(), tableSchema, requestedSchema); + this.requiredRowType = FlinkSchemaUtil.convert(requiredSchema()); + this.asStructLike = new RowDataWrapper(requiredRowType, requiredSchema().asStruct()); + this.inputFilesDecryptor = inputFilesDecryptor; + } + + public RowType requiredRowType() { + return requiredRowType; + } + + @Override + protected StructLike asStructLike(RowData row) { + return asStructLike.wrap(row); + } + + @Override + protected InputFile getInputFile(String location) { + return inputFilesDecryptor.getInputFile(location); + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java new file mode 100644 index 000000000000..c958604c004a --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.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.iceberg.flink.source; + +import static org.apache.iceberg.TableProperties.DEFAULT_NAME_MAPPING; + +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.sink.RowDataTaskWriterFactory; +import org.apache.iceberg.flink.sink.TaskWriterFactory; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.PropertyUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class RowDataRewriter { + + private static final Logger LOG = LoggerFactory.getLogger(RowDataRewriter.class); + + private final Schema schema; + private final String nameMapping; + private final FileIO io; + private final boolean caseSensitive; + private final EncryptionManager encryptionManager; + private final TaskWriterFactory taskWriterFactory; + private final String tableName; + + public RowDataRewriter( + Table table, boolean caseSensitive, FileIO io, EncryptionManager encryptionManager) { + this.schema = table.schema(); + this.caseSensitive = caseSensitive; + this.io = io; + this.encryptionManager = encryptionManager; + this.nameMapping = + PropertyUtil.propertyAsString(table.properties(), DEFAULT_NAME_MAPPING, null); + this.tableName = table.name(); + + String formatString = + PropertyUtil.propertyAsString( + table.properties(), + TableProperties.DEFAULT_FILE_FORMAT, + TableProperties.DEFAULT_FILE_FORMAT_DEFAULT); + FileFormat format = FileFormat.fromString(formatString); + RowType flinkSchema = FlinkSchemaUtil.convert(table.schema()); + this.taskWriterFactory = + new RowDataTaskWriterFactory( + SerializableTable.copyOf(table), + flinkSchema, + Long.MAX_VALUE, + format, + table.properties(), + null, + false); + } + + public List rewriteDataForTasks( + DataStream dataStream, int parallelism) throws Exception { + RewriteMap map = + new RewriteMap( + schema, nameMapping, io, caseSensitive, encryptionManager, taskWriterFactory); + DataStream> ds = dataStream.map(map).setParallelism(parallelism); + return Lists.newArrayList(ds.executeAndCollect("Rewrite table :" + tableName)).stream() + .flatMap(Collection::stream) + .collect(Collectors.toList()); + } + + public static class RewriteMap extends RichMapFunction> { + + private TaskWriter writer; + private int subTaskId; + private int attemptId; + + private final Schema schema; + private final String nameMapping; + private final FileIO io; + private final boolean caseSensitive; + private final EncryptionManager encryptionManager; + private final TaskWriterFactory taskWriterFactory; + private final RowDataFileScanTaskReader rowDataReader; + + public RewriteMap( + Schema schema, + String nameMapping, + FileIO io, + boolean caseSensitive, + EncryptionManager encryptionManager, + TaskWriterFactory taskWriterFactory) { + this.schema = schema; + this.nameMapping = nameMapping; + this.io = io; + this.caseSensitive = caseSensitive; + this.encryptionManager = encryptionManager; + this.taskWriterFactory = taskWriterFactory; + this.rowDataReader = + new RowDataFileScanTaskReader( + schema, schema, nameMapping, caseSensitive, Collections.emptyList()); + } + + @Override + public void open(Configuration parameters) { + this.subTaskId = getRuntimeContext().getIndexOfThisSubtask(); + this.attemptId = getRuntimeContext().getAttemptNumber(); + // Initialize the task writer factory. + this.taskWriterFactory.initialize(subTaskId, attemptId); + } + + @Override + public List map(CombinedScanTask task) throws Exception { + // Initialize the task writer. + this.writer = taskWriterFactory.create(); + try (DataIterator iterator = + new DataIterator<>(rowDataReader, task, io, encryptionManager)) { + while (iterator.hasNext()) { + RowData rowData = iterator.next(); + writer.write(rowData); + } + return Lists.newArrayList(writer.dataFiles()); + } catch (Throwable originalThrowable) { + try { + LOG.error("Aborting commit for (subTaskId {}, attemptId {})", subTaskId, attemptId); + writer.abort(); + LOG.error("Aborted commit for (subTaskId {}, attemptId {})", subTaskId, attemptId); + } catch (Throwable inner) { + if (originalThrowable != inner) { + originalThrowable.addSuppressed(inner); + LOG.warn("Suppressing exception in catch: {}", inner.getMessage(), inner); + } + } + + if (originalThrowable instanceof Exception) { + throw originalThrowable; + } else { + throw new RuntimeException(originalThrowable); + } + } + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java new file mode 100644 index 000000000000..8f95e3e554a0 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java @@ -0,0 +1,70 @@ +/* + * 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.iceberg.flink.source; + +import java.io.Serializable; +import java.util.function.Function; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.flink.annotation.Internal; +import org.apache.flink.formats.avro.RowDataToAvroConverters; +import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.flink.FlinkSchemaUtil; + +/** + * This is not serializable because Avro {@link Schema} is not actually serializable, even though it + * implements {@link Serializable} interface. + */ +@Internal +public class RowDataToAvroGenericRecordConverter implements Function { + private final RowDataToAvroConverters.RowDataToAvroConverter converter; + private final Schema avroSchema; + + private RowDataToAvroGenericRecordConverter(RowType rowType, Schema avroSchema) { + this.converter = RowDataToAvroConverters.createConverter(rowType); + this.avroSchema = avroSchema; + } + + @Override + public GenericRecord apply(RowData rowData) { + return (GenericRecord) converter.convert(avroSchema, rowData); + } + + /** Create a converter based on Iceberg schema */ + public static RowDataToAvroGenericRecordConverter fromIcebergSchema( + String tableName, org.apache.iceberg.Schema icebergSchema) { + RowType rowType = FlinkSchemaUtil.convert(icebergSchema); + Schema avroSchema = AvroSchemaUtil.convert(icebergSchema, tableName); + return new RowDataToAvroGenericRecordConverter(rowType, avroSchema); + } + + /** Create a mapper based on Avro schema */ + public static RowDataToAvroGenericRecordConverter fromAvroSchema(Schema avroSchema) { + DataType dataType = AvroSchemaConverter.convertToDataType(avroSchema.toString()); + LogicalType logicalType = TypeConversions.fromDataToLogicalType(dataType); + RowType rowType = RowType.of(logicalType.getChildren().stream().toArray(LogicalType[]::new)); + return new RowDataToAvroGenericRecordConverter(rowType, avroSchema); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java new file mode 100644 index 000000000000..4357b1f57df6 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java @@ -0,0 +1,561 @@ +/* + * 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.iceberg.flink.source; + +import java.io.Serializable; +import java.time.Duration; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.TimeUtils; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.FlinkReadConf; +import org.apache.iceberg.flink.FlinkReadOptions; + +/** Context object with optional arguments for a Flink Scan. */ +@Internal +public class ScanContext implements Serializable { + + private static final long serialVersionUID = 1L; + + private final boolean caseSensitive; + private final boolean exposeLocality; + private final Long snapshotId; + private final String branch; + private final String tag; + private final StreamingStartingStrategy startingStrategy; + private final Long startSnapshotId; + private final Long startSnapshotTimestamp; + private final Long endSnapshotId; + private final Long asOfTimestamp; + private final String startTag; + private final String endTag; + private final Long splitSize; + private final Integer splitLookback; + private final Long splitOpenFileCost; + private final boolean isStreaming; + private final Duration monitorInterval; + + private final String nameMapping; + private final Schema schema; + private final List filters; + private final long limit; + private final boolean includeColumnStats; + private final Collection includeStatsForColumns; + private final Integer planParallelism; + private final int maxPlanningSnapshotCount; + private final int maxAllowedPlanningFailures; + + private ScanContext( + boolean caseSensitive, + Long snapshotId, + StreamingStartingStrategy startingStrategy, + Long startSnapshotTimestamp, + Long startSnapshotId, + Long endSnapshotId, + Long asOfTimestamp, + Long splitSize, + Integer splitLookback, + Long splitOpenFileCost, + boolean isStreaming, + Duration monitorInterval, + String nameMapping, + Schema schema, + List filters, + long limit, + boolean includeColumnStats, + Collection includeStatsForColumns, + boolean exposeLocality, + Integer planParallelism, + int maxPlanningSnapshotCount, + int maxAllowedPlanningFailures, + String branch, + String tag, + String startTag, + String endTag) { + this.caseSensitive = caseSensitive; + this.snapshotId = snapshotId; + this.tag = tag; + this.branch = branch; + this.startingStrategy = startingStrategy; + this.startSnapshotTimestamp = startSnapshotTimestamp; + this.startSnapshotId = startSnapshotId; + this.endSnapshotId = endSnapshotId; + this.asOfTimestamp = asOfTimestamp; + this.startTag = startTag; + this.endTag = endTag; + this.splitSize = splitSize; + this.splitLookback = splitLookback; + this.splitOpenFileCost = splitOpenFileCost; + this.isStreaming = isStreaming; + this.monitorInterval = monitorInterval; + + this.nameMapping = nameMapping; + this.schema = schema; + this.filters = filters; + this.limit = limit; + this.includeColumnStats = includeColumnStats; + this.includeStatsForColumns = includeStatsForColumns; + this.exposeLocality = exposeLocality; + this.planParallelism = planParallelism; + this.maxPlanningSnapshotCount = maxPlanningSnapshotCount; + this.maxAllowedPlanningFailures = maxAllowedPlanningFailures; + + validate(); + } + + private void validate() { + if (isStreaming) { + if (startingStrategy == StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) { + Preconditions.checkArgument( + startSnapshotId != null, + "Invalid starting snapshot id for SPECIFIC_START_SNAPSHOT_ID strategy: null"); + Preconditions.checkArgument( + startSnapshotTimestamp == null, + "Invalid starting snapshot timestamp for SPECIFIC_START_SNAPSHOT_ID strategy: not null"); + } + if (startingStrategy == StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) { + Preconditions.checkArgument( + startSnapshotTimestamp != null, + "Invalid starting snapshot timestamp for SPECIFIC_START_SNAPSHOT_TIMESTAMP strategy: null"); + Preconditions.checkArgument( + startSnapshotId == null, + "Invalid starting snapshot id for SPECIFIC_START_SNAPSHOT_ID strategy: not null"); + } + + Preconditions.checkArgument( + branch == null, + String.format( + "Cannot scan table using ref %s configured for streaming reader yet", branch)); + + Preconditions.checkArgument( + tag == null, + String.format("Cannot scan table using ref %s configured for streaming reader", tag)); + } + + Preconditions.checkArgument( + !(startTag != null && startSnapshotId() != null), + "START_SNAPSHOT_ID and START_TAG cannot both be set."); + + Preconditions.checkArgument( + !(endTag != null && endSnapshotId() != null), + "END_SNAPSHOT_ID and END_TAG cannot both be set."); + + Preconditions.checkArgument( + maxAllowedPlanningFailures >= -1, + "Cannot set maxAllowedPlanningFailures to a negative number other than -1."); + } + + public boolean caseSensitive() { + return caseSensitive; + } + + public Long snapshotId() { + return snapshotId; + } + + public String branch() { + return branch; + } + + public String tag() { + return tag; + } + + public String startTag() { + return startTag; + } + + public String endTag() { + return endTag; + } + + public StreamingStartingStrategy streamingStartingStrategy() { + return startingStrategy; + } + + public Long startSnapshotTimestamp() { + return startSnapshotTimestamp; + } + + public Long startSnapshotId() { + return startSnapshotId; + } + + public Long endSnapshotId() { + return endSnapshotId; + } + + public Long asOfTimestamp() { + return asOfTimestamp; + } + + public Long splitSize() { + return splitSize; + } + + public Integer splitLookback() { + return splitLookback; + } + + public Long splitOpenFileCost() { + return splitOpenFileCost; + } + + public boolean isStreaming() { + return isStreaming; + } + + public Duration monitorInterval() { + return monitorInterval; + } + + public String nameMapping() { + return nameMapping; + } + + public Schema project() { + return schema; + } + + public List filters() { + return filters; + } + + public long limit() { + return limit; + } + + public boolean includeColumnStats() { + return includeColumnStats; + } + + public Collection includeStatsForColumns() { + return includeStatsForColumns; + } + + public boolean exposeLocality() { + return exposeLocality; + } + + public Integer planParallelism() { + return planParallelism; + } + + public int maxPlanningSnapshotCount() { + return maxPlanningSnapshotCount; + } + + public int maxAllowedPlanningFailures() { + return maxAllowedPlanningFailures; + } + + public ScanContext copyWithAppendsBetween(Long newStartSnapshotId, long newEndSnapshotId) { + return ScanContext.builder() + .caseSensitive(caseSensitive) + .useSnapshotId(null) + .useBranch(branch) + .useTag(null) + .startSnapshotId(newStartSnapshotId) + .endSnapshotId(newEndSnapshotId) + .startTag(null) + .endTag(null) + .asOfTimestamp(null) + .splitSize(splitSize) + .splitLookback(splitLookback) + .splitOpenFileCost(splitOpenFileCost) + .streaming(isStreaming) + .monitorInterval(monitorInterval) + .nameMapping(nameMapping) + .project(schema) + .filters(filters) + .limit(limit) + .includeColumnStats(includeColumnStats) + .includeColumnStats(includeStatsForColumns) + .exposeLocality(exposeLocality) + .planParallelism(planParallelism) + .maxPlanningSnapshotCount(maxPlanningSnapshotCount) + .maxAllowedPlanningFailures(maxAllowedPlanningFailures) + .build(); + } + + public ScanContext copyWithSnapshotId(long newSnapshotId) { + return ScanContext.builder() + .caseSensitive(caseSensitive) + .useSnapshotId(newSnapshotId) + .useBranch(branch) + .useTag(tag) + .startSnapshotId(null) + .endSnapshotId(null) + .startTag(null) + .endTag(null) + .asOfTimestamp(null) + .splitSize(splitSize) + .splitLookback(splitLookback) + .splitOpenFileCost(splitOpenFileCost) + .streaming(isStreaming) + .monitorInterval(monitorInterval) + .nameMapping(nameMapping) + .project(schema) + .filters(filters) + .limit(limit) + .includeColumnStats(includeColumnStats) + .includeColumnStats(includeStatsForColumns) + .exposeLocality(exposeLocality) + .planParallelism(planParallelism) + .maxPlanningSnapshotCount(maxPlanningSnapshotCount) + .maxAllowedPlanningFailures(maxAllowedPlanningFailures) + .build(); + } + + public static Builder builder() { + return new Builder(); + } + + public static class Builder { + private boolean caseSensitive = FlinkReadOptions.CASE_SENSITIVE_OPTION.defaultValue(); + private Long snapshotId = FlinkReadOptions.SNAPSHOT_ID.defaultValue(); + private String branch = FlinkReadOptions.BRANCH.defaultValue(); + private String tag = FlinkReadOptions.TAG.defaultValue(); + private String startTag = FlinkReadOptions.START_TAG.defaultValue(); + private String endTag = FlinkReadOptions.END_TAG.defaultValue(); + private StreamingStartingStrategy startingStrategy = + FlinkReadOptions.STARTING_STRATEGY_OPTION.defaultValue(); + private Long startSnapshotTimestamp = FlinkReadOptions.START_SNAPSHOT_TIMESTAMP.defaultValue(); + private Long startSnapshotId = FlinkReadOptions.START_SNAPSHOT_ID.defaultValue(); + private Long endSnapshotId = FlinkReadOptions.END_SNAPSHOT_ID.defaultValue(); + private Long asOfTimestamp = FlinkReadOptions.AS_OF_TIMESTAMP.defaultValue(); + private Long splitSize = FlinkReadOptions.SPLIT_SIZE_OPTION.defaultValue(); + private Integer splitLookback = FlinkReadOptions.SPLIT_LOOKBACK_OPTION.defaultValue(); + private Long splitOpenFileCost = FlinkReadOptions.SPLIT_FILE_OPEN_COST_OPTION.defaultValue(); + private boolean isStreaming = FlinkReadOptions.STREAMING_OPTION.defaultValue(); + private Duration monitorInterval = + TimeUtils.parseDuration(FlinkReadOptions.MONITOR_INTERVAL_OPTION.defaultValue()); + private String nameMapping; + private Schema projectedSchema; + private List filters; + private long limit = FlinkReadOptions.LIMIT_OPTION.defaultValue(); + private boolean includeColumnStats = + FlinkReadOptions.INCLUDE_COLUMN_STATS_OPTION.defaultValue(); + private Collection includeStatsForColumns = null; + private boolean exposeLocality; + private Integer planParallelism = + FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.defaultValue(); + private int maxPlanningSnapshotCount = + FlinkReadOptions.MAX_PLANNING_SNAPSHOT_COUNT_OPTION.defaultValue(); + private int maxAllowedPlanningFailures = + FlinkReadOptions.MAX_ALLOWED_PLANNING_FAILURES_OPTION.defaultValue(); + + private Builder() {} + + public Builder caseSensitive(boolean newCaseSensitive) { + this.caseSensitive = newCaseSensitive; + return this; + } + + public Builder useSnapshotId(Long newSnapshotId) { + this.snapshotId = newSnapshotId; + return this; + } + + public Builder useTag(String newTag) { + this.tag = newTag; + return this; + } + + public Builder useBranch(String newBranch) { + this.branch = newBranch; + return this; + } + + public Builder startingStrategy(StreamingStartingStrategy newStartingStrategy) { + this.startingStrategy = newStartingStrategy; + return this; + } + + public Builder startSnapshotTimestamp(Long newStartSnapshotTimestamp) { + this.startSnapshotTimestamp = newStartSnapshotTimestamp; + return this; + } + + public Builder startSnapshotId(Long newStartSnapshotId) { + this.startSnapshotId = newStartSnapshotId; + return this; + } + + public Builder endSnapshotId(Long newEndSnapshotId) { + this.endSnapshotId = newEndSnapshotId; + return this; + } + + public Builder startTag(String newStartTag) { + this.startTag = newStartTag; + return this; + } + + public Builder endTag(String newEndTag) { + this.endTag = newEndTag; + return this; + } + + public Builder asOfTimestamp(Long newAsOfTimestamp) { + this.asOfTimestamp = newAsOfTimestamp; + return this; + } + + public Builder splitSize(Long newSplitSize) { + this.splitSize = newSplitSize; + return this; + } + + public Builder splitLookback(Integer newSplitLookback) { + this.splitLookback = newSplitLookback; + return this; + } + + public Builder splitOpenFileCost(Long newSplitOpenFileCost) { + this.splitOpenFileCost = newSplitOpenFileCost; + return this; + } + + public Builder streaming(boolean streaming) { + this.isStreaming = streaming; + return this; + } + + public Builder monitorInterval(Duration newMonitorInterval) { + this.monitorInterval = newMonitorInterval; + return this; + } + + public Builder nameMapping(String newNameMapping) { + this.nameMapping = newNameMapping; + return this; + } + + public Builder project(Schema newProjectedSchema) { + this.projectedSchema = newProjectedSchema; + return this; + } + + public Builder filters(List newFilters) { + this.filters = newFilters; + return this; + } + + public Builder limit(long newLimit) { + this.limit = newLimit; + return this; + } + + public Builder includeColumnStats(boolean newIncludeColumnStats) { + this.includeColumnStats = newIncludeColumnStats; + return this; + } + + public Builder includeColumnStats(Collection newIncludeStatsForColumns) { + this.includeStatsForColumns = newIncludeStatsForColumns; + return this; + } + + public Builder exposeLocality(boolean newExposeLocality) { + this.exposeLocality = newExposeLocality; + return this; + } + + public Builder planParallelism(Integer parallelism) { + this.planParallelism = parallelism; + return this; + } + + public Builder maxPlanningSnapshotCount(int newMaxPlanningSnapshotCount) { + this.maxPlanningSnapshotCount = newMaxPlanningSnapshotCount; + return this; + } + + public Builder maxAllowedPlanningFailures(int newMaxAllowedPlanningFailures) { + this.maxAllowedPlanningFailures = newMaxAllowedPlanningFailures; + return this; + } + + public Builder resolveConfig( + Table table, Map readOptions, ReadableConfig readableConfig) { + FlinkReadConf flinkReadConf = new FlinkReadConf(table, readOptions, readableConfig); + + return this.useSnapshotId(flinkReadConf.snapshotId()) + .useTag(flinkReadConf.tag()) + .useBranch(flinkReadConf.branch()) + .startTag(flinkReadConf.startTag()) + .endTag(flinkReadConf.endTag()) + .caseSensitive(flinkReadConf.caseSensitive()) + .asOfTimestamp(flinkReadConf.asOfTimestamp()) + .startingStrategy(flinkReadConf.startingStrategy()) + .startSnapshotTimestamp(flinkReadConf.startSnapshotTimestamp()) + .startSnapshotId(flinkReadConf.startSnapshotId()) + .endSnapshotId(flinkReadConf.endSnapshotId()) + .splitSize(flinkReadConf.splitSize()) + .splitLookback(flinkReadConf.splitLookback()) + .splitOpenFileCost(flinkReadConf.splitFileOpenCost()) + .streaming(flinkReadConf.streaming()) + .monitorInterval(flinkReadConf.monitorInterval()) + .nameMapping(flinkReadConf.nameMapping()) + .limit(flinkReadConf.limit()) + .planParallelism(flinkReadConf.workerPoolSize()) + .includeColumnStats(flinkReadConf.includeColumnStats()) + .maxPlanningSnapshotCount(flinkReadConf.maxPlanningSnapshotCount()) + .maxAllowedPlanningFailures(maxAllowedPlanningFailures); + } + + public ScanContext build() { + return new ScanContext( + caseSensitive, + snapshotId, + startingStrategy, + startSnapshotTimestamp, + startSnapshotId, + endSnapshotId, + asOfTimestamp, + splitSize, + splitLookback, + splitOpenFileCost, + isStreaming, + monitorInterval, + nameMapping, + projectedSchema, + filters, + limit, + includeColumnStats, + includeStatsForColumns, + exposeLocality, + planParallelism, + maxPlanningSnapshotCount, + maxAllowedPlanningFailures, + branch, + tag, + startTag, + endTag); + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java new file mode 100644 index 000000000000..7c3a69dbc141 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java @@ -0,0 +1,77 @@ +/* + * 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.iceberg.flink.source; + +import java.util.function.Supplier; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.config.ExecutionConfigOptions; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.hadoop.Util; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +class SourceUtil { + private SourceUtil() {} + + static boolean isLocalityEnabled( + Table table, ReadableConfig readableConfig, Boolean exposeLocality) { + Boolean localityEnabled = + exposeLocality != null + ? exposeLocality + : readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_EXPOSE_SPLIT_LOCALITY_INFO); + + if (localityEnabled != null && !localityEnabled) { + return false; + } + + return Util.mayHaveBlockLocations(table.io(), table.location()); + } + + /** + * Infer source parallelism. + * + * @param readableConfig Flink config. + * @param splitCountProvider Split count supplier. As the computation may involve expensive split + * discover, lazy evaluation is performed if inferring parallelism is enabled. + * @param limitCount limited output count. + */ + static int inferParallelism( + ReadableConfig readableConfig, long limitCount, Supplier splitCountProvider) { + int parallelism = + readableConfig.get(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM); + if (readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM)) { + int maxInferParallelism = + readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX); + Preconditions.checkState( + maxInferParallelism >= 1, + FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX.key() + + " cannot be less than 1"); + parallelism = Math.min(splitCountProvider.get(), maxInferParallelism); + } + + if (limitCount > 0) { + int limit = limitCount >= Integer.MAX_VALUE ? Integer.MAX_VALUE : (int) limitCount; + parallelism = Math.min(parallelism, limit); + } + + // parallelism must be positive. + parallelism = Math.max(1, parallelism); + return parallelism; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java new file mode 100644 index 000000000000..c27e29613fed --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java @@ -0,0 +1,269 @@ +/* + * 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.iceberg.flink.source; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.List; +import java.util.concurrent.ExecutorService; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.functions.source.RichSourceFunction; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.iceberg.util.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This is the single (non-parallel) monitoring task which takes a {@link FlinkInputFormat}, it is + * responsible for: + * + *

    + *
  1. Monitoring snapshots of the Iceberg table. + *
  2. Creating the {@link FlinkInputSplit splits} corresponding to the incremental files + *
  3. Assigning them to downstream tasks for further processing. + *
+ * + *

The splits to be read are forwarded to the downstream {@link StreamingReaderOperator} which + * can have parallelism greater than one. + */ +public class StreamingMonitorFunction extends RichSourceFunction + implements CheckpointedFunction { + + private static final Logger LOG = LoggerFactory.getLogger(StreamingMonitorFunction.class); + + private static final long INIT_LAST_SNAPSHOT_ID = -1L; + + private final TableLoader tableLoader; + private final ScanContext scanContext; + + private volatile boolean isRunning = true; + + // The checkpoint thread is not the same thread that running the function for SourceStreamTask + // now. It's necessary to + // mark this as volatile. + private volatile long lastSnapshotId = INIT_LAST_SNAPSHOT_ID; + + private transient SourceContext sourceContext; + private transient Table table; + private transient ListState lastSnapshotIdState; + private transient ExecutorService workerPool; + + public StreamingMonitorFunction(TableLoader tableLoader, ScanContext scanContext) { + Preconditions.checkArgument( + scanContext.snapshotId() == null, "Cannot set snapshot-id option for streaming reader"); + Preconditions.checkArgument( + scanContext.asOfTimestamp() == null, + "Cannot set as-of-timestamp option for streaming reader"); + Preconditions.checkArgument( + scanContext.endSnapshotId() == null, + "Cannot set end-snapshot-id option for streaming reader"); + Preconditions.checkArgument( + scanContext.endTag() == null, "Cannot set end-tag option for streaming reader"); + Preconditions.checkArgument( + scanContext.maxPlanningSnapshotCount() > 0, + "The max-planning-snapshot-count must be greater than zero"); + this.tableLoader = tableLoader; + this.scanContext = scanContext; + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + + final RuntimeContext runtimeContext = getRuntimeContext(); + ValidationException.check( + runtimeContext instanceof StreamingRuntimeContext, + "context should be instance of StreamingRuntimeContext"); + final String operatorID = ((StreamingRuntimeContext) runtimeContext).getOperatorUniqueID(); + this.workerPool = + ThreadPools.newWorkerPool( + "iceberg-worker-pool-" + operatorID, scanContext.planParallelism()); + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + // Load iceberg table from table loader. + tableLoader.open(); + table = tableLoader.loadTable(); + + // Initialize the flink state for last snapshot id. + lastSnapshotIdState = + context + .getOperatorStateStore() + .getListState(new ListStateDescriptor<>("snapshot-id-state", LongSerializer.INSTANCE)); + + // Restore the last-snapshot-id from flink's state if possible. + if (context.isRestored()) { + LOG.info("Restoring state for the {}.", getClass().getSimpleName()); + lastSnapshotId = lastSnapshotIdState.get().iterator().next(); + } else if (scanContext.startTag() != null || scanContext.startSnapshotId() != null) { + Preconditions.checkArgument( + !(scanContext.startTag() != null && scanContext.startSnapshotId() != null), + "START_SNAPSHOT_ID and START_TAG cannot both be set."); + Preconditions.checkArgument( + scanContext.branch() == null, + "Cannot scan table using ref %s configured for streaming reader yet."); + Preconditions.checkNotNull( + table.currentSnapshot(), "Don't have any available snapshot in table."); + + long startSnapshotId; + if (scanContext.startTag() != null) { + Preconditions.checkArgument( + table.snapshot(scanContext.startTag()) != null, + "Cannot find snapshot with tag %s in table.", + scanContext.startTag()); + startSnapshotId = table.snapshot(scanContext.startTag()).snapshotId(); + } else { + startSnapshotId = scanContext.startSnapshotId(); + } + + long currentSnapshotId = table.currentSnapshot().snapshotId(); + Preconditions.checkState( + SnapshotUtil.isAncestorOf(table, currentSnapshotId, startSnapshotId), + "The option start-snapshot-id %s is not an ancestor of the current snapshot.", + startSnapshotId); + + lastSnapshotId = startSnapshotId; + } + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + lastSnapshotIdState.clear(); + lastSnapshotIdState.add(lastSnapshotId); + } + + @Override + public void run(SourceContext ctx) throws Exception { + this.sourceContext = ctx; + while (isRunning) { + monitorAndForwardSplits(); + Thread.sleep(scanContext.monitorInterval().toMillis()); + } + } + + private long toSnapshotIdInclusive( + long lastConsumedSnapshotId, long currentSnapshotId, int maxPlanningSnapshotCount) { + List snapshotIds = + SnapshotUtil.snapshotIdsBetween(table, lastConsumedSnapshotId, currentSnapshotId); + if (snapshotIds.size() <= maxPlanningSnapshotCount) { + return currentSnapshotId; + } else { + // It uses reverted index since snapshotIdsBetween returns Ids that are ordered by committed + // time descending. + return snapshotIds.get(snapshotIds.size() - maxPlanningSnapshotCount); + } + } + + @VisibleForTesting + void sourceContext(SourceContext ctx) { + this.sourceContext = ctx; + } + + @VisibleForTesting + void monitorAndForwardSplits() { + // Refresh the table to get the latest committed snapshot. + table.refresh(); + + Snapshot snapshot = table.currentSnapshot(); + if (snapshot != null && snapshot.snapshotId() != lastSnapshotId) { + long snapshotId = snapshot.snapshotId(); + + ScanContext newScanContext; + if (lastSnapshotId == INIT_LAST_SNAPSHOT_ID) { + newScanContext = scanContext.copyWithSnapshotId(snapshotId); + } else { + snapshotId = + toSnapshotIdInclusive( + lastSnapshotId, snapshotId, scanContext.maxPlanningSnapshotCount()); + newScanContext = scanContext.copyWithAppendsBetween(lastSnapshotId, snapshotId); + } + + LOG.debug( + "Start discovering splits from {} (exclusive) to {} (inclusive)", + lastSnapshotId, + snapshotId); + long start = System.currentTimeMillis(); + FlinkInputSplit[] splits = + FlinkSplitPlanner.planInputSplits(table, newScanContext, workerPool); + LOG.debug( + "Discovered {} splits, time elapsed {}ms", + splits.length, + System.currentTimeMillis() - start); + + // only need to hold the checkpoint lock when emitting the splits and updating lastSnapshotId + start = System.currentTimeMillis(); + synchronized (sourceContext.getCheckpointLock()) { + for (FlinkInputSplit split : splits) { + sourceContext.collect(split); + } + + lastSnapshotId = snapshotId; + } + LOG.debug( + "Forwarded {} splits, time elapsed {}ms", + splits.length, + System.currentTimeMillis() - start); + } + } + + @Override + public void cancel() { + // this is to cover the case where cancel() is called before the run() + if (sourceContext != null) { + synchronized (sourceContext.getCheckpointLock()) { + isRunning = false; + } + } else { + isRunning = false; + } + + // Release all the resources here. + if (tableLoader != null) { + try { + tableLoader.close(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + } + + @Override + public void close() { + cancel(); + + if (workerPool != null) { + workerPool.shutdown(); + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java new file mode 100644 index 000000000000..ee6f7b63988d --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java @@ -0,0 +1,246 @@ +/* + * 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.iceberg.flink.source; + +import java.io.IOException; +import java.util.Queue; +import org.apache.flink.api.common.operators.MailboxExecutor; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.runtime.state.JavaSerializer; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; +import org.apache.flink.streaming.api.operators.StreamSourceContexts; +import org.apache.flink.streaming.api.operators.YieldingOperatorFactory; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The operator that reads the {@link FlinkInputSplit splits} received from the preceding {@link + * StreamingMonitorFunction}. Contrary to the {@link StreamingMonitorFunction} which has a + * parallelism of 1, this operator can have multiple parallelism. + * + *

As soon as a split descriptor is received, it is put in a queue, and use {@link + * MailboxExecutor} read the actual data of the split. This architecture allows the separation of + * the reading thread from the one split processing the checkpoint barriers, thus removing any + * potential back-pressure. + */ +public class StreamingReaderOperator extends AbstractStreamOperator + implements OneInputStreamOperator { + + private static final Logger LOG = LoggerFactory.getLogger(StreamingReaderOperator.class); + + // It's the same thread that is running this operator and checkpoint actions. we use this executor + // to schedule only + // one split for future reading, so that a new checkpoint could be triggered without blocking long + // time for exhausting + // all scheduled splits. + private final MailboxExecutor executor; + private FlinkInputFormat format; + + private transient SourceFunction.SourceContext sourceContext; + + private transient ListState inputSplitsState; + private transient Queue splits; + + // Splits are read by the same thread that calls processElement. Each read task is submitted to + // that thread by adding + // them to the executor. This state is used to ensure that only one read task is in that queue at + // a time, so that read + // tasks do not accumulate ahead of checkpoint tasks. When there is a read task in the queue, this + // is set to RUNNING. + // When there are no more files to read, this will be set to IDLE. + private transient SplitState currentSplitState; + + private StreamingReaderOperator( + FlinkInputFormat format, ProcessingTimeService timeService, MailboxExecutor mailboxExecutor) { + this.format = Preconditions.checkNotNull(format, "The InputFormat should not be null."); + this.processingTimeService = timeService; + this.executor = + Preconditions.checkNotNull(mailboxExecutor, "The mailboxExecutor should not be null."); + } + + @Override + public void initializeState(StateInitializationContext context) throws Exception { + super.initializeState(context); + + // TODO Replace Java serialization with Avro approach to keep state compatibility. + // See issue: https://github.com/apache/iceberg/issues/1698 + inputSplitsState = + context + .getOperatorStateStore() + .getListState(new ListStateDescriptor<>("splits", new JavaSerializer<>())); + + // Initialize the current split state to IDLE. + currentSplitState = SplitState.IDLE; + + // Recover splits state from flink state backend if possible. + splits = Lists.newLinkedList(); + if (context.isRestored()) { + int subtaskIdx = getRuntimeContext().getIndexOfThisSubtask(); + LOG.info("Restoring state for the {} (taskIdx: {}).", getClass().getSimpleName(), subtaskIdx); + + for (FlinkInputSplit split : inputSplitsState.get()) { + splits.add(split); + } + } + + this.sourceContext = + StreamSourceContexts.getSourceContext( + getOperatorConfig().getTimeCharacteristic(), + getProcessingTimeService(), + new Object(), // no actual locking needed + output, + getRuntimeContext().getExecutionConfig().getAutoWatermarkInterval(), + -1, + true); + + // Enqueue to process the recovered input splits. + enqueueProcessSplits(); + } + + @Override + public void snapshotState(StateSnapshotContext context) throws Exception { + super.snapshotState(context); + + inputSplitsState.clear(); + inputSplitsState.addAll(Lists.newArrayList(splits)); + } + + @Override + public void processElement(StreamRecord element) { + splits.add(element.getValue()); + enqueueProcessSplits(); + } + + private void enqueueProcessSplits() { + if (currentSplitState == SplitState.IDLE && !splits.isEmpty()) { + currentSplitState = SplitState.RUNNING; + executor.execute(this::processSplits, this.getClass().getSimpleName()); + } + } + + private void processSplits() throws IOException { + FlinkInputSplit split = splits.poll(); + if (split == null) { + currentSplitState = SplitState.IDLE; + return; + } + + format.open(split); + try { + RowData nextElement = null; + while (!format.reachedEnd()) { + nextElement = format.nextRecord(nextElement); + sourceContext.collect(nextElement); + } + } finally { + currentSplitState = SplitState.IDLE; + format.close(); + } + + // Re-schedule to process the next split. + enqueueProcessSplits(); + } + + @Override + public void processWatermark(Watermark mark) { + // we do nothing because we emit our own watermarks if needed. + } + + @Override + public void close() throws Exception { + super.close(); + + if (format != null) { + format.close(); + format.closeInputFormat(); + format = null; + } + + sourceContext = null; + } + + @Override + public void finish() throws Exception { + super.finish(); + output.close(); + if (sourceContext != null) { + sourceContext.emitWatermark(Watermark.MAX_WATERMARK); + sourceContext.close(); + sourceContext = null; + } + } + + static OneInputStreamOperatorFactory factory(FlinkInputFormat format) { + return new OperatorFactory(format); + } + + private enum SplitState { + IDLE, + RUNNING + } + + private static class OperatorFactory extends AbstractStreamOperatorFactory + implements YieldingOperatorFactory, + OneInputStreamOperatorFactory { + + private final FlinkInputFormat format; + + private transient MailboxExecutor mailboxExecutor; + + private OperatorFactory(FlinkInputFormat format) { + this.format = format; + } + + @Override + public void setMailboxExecutor(MailboxExecutor mailboxExecutor) { + this.mailboxExecutor = mailboxExecutor; + } + + @SuppressWarnings("unchecked") + @Override + public > O createStreamOperator( + StreamOperatorParameters parameters) { + StreamingReaderOperator operator = + new StreamingReaderOperator(format, processingTimeService, mailboxExecutor); + operator.setup( + parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); + return (O) operator; + } + + @Override + public Class getStreamOperatorClass(ClassLoader classLoader) { + return StreamingReaderOperator.class; + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java new file mode 100644 index 000000000000..11707bf82a0f --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.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.iceberg.flink.source; + +/** Starting strategy for streaming execution. */ +public enum StreamingStartingStrategy { + /** + * Do a regular table scan then switch to the incremental mode. + * + *

The incremental mode starts from the current snapshot exclusive. + */ + TABLE_SCAN_THEN_INCREMENTAL, + + /** + * Start incremental mode from the latest snapshot inclusive. + * + *

If it is an empty map, all future append snapshots should be discovered. + */ + INCREMENTAL_FROM_LATEST_SNAPSHOT, + + /** + * Start incremental mode from the earliest snapshot inclusive. + * + *

If it is an empty map, all future append snapshots should be discovered. + */ + INCREMENTAL_FROM_EARLIEST_SNAPSHOT, + + /** Start incremental mode from a snapshot with a specific id inclusive. */ + INCREMENTAL_FROM_SNAPSHOT_ID, + + /** + * Start incremental mode from a snapshot with a specific timestamp inclusive. + * + *

If the timestamp is between two snapshots, it should start from the snapshot after the + * timestamp. + */ + INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java new file mode 100644 index 000000000000..37a0f1a6055f --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.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.iceberg.flink.source.assigner; + +import java.util.ArrayDeque; +import java.util.Collection; +import java.util.PriorityQueue; +import java.util.Queue; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; +import javax.annotation.Nullable; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; +import org.apache.iceberg.flink.source.split.SerializableComparator; + +/** + * Since all methods are called in the source coordinator thread by enumerator, there is no need for + * locking. + */ +@Internal +public class DefaultSplitAssigner implements SplitAssigner { + + private final Queue pendingSplits; + private CompletableFuture availableFuture; + + public DefaultSplitAssigner(SerializableComparator comparator) { + this.pendingSplits = comparator == null ? new ArrayDeque<>() : new PriorityQueue<>(comparator); + } + + public DefaultSplitAssigner( + SerializableComparator comparator, + Collection assignerState) { + this(comparator); + // Because default assigner only tracks unassigned splits, + // there is no need to filter splits based on status (unassigned) here. + assignerState.forEach(splitState -> pendingSplits.add(splitState.split())); + } + + @Override + public synchronized GetSplitResult getNext(@Nullable String hostname) { + if (pendingSplits.isEmpty()) { + return GetSplitResult.unavailable(); + } else { + IcebergSourceSplit split = pendingSplits.poll(); + return GetSplitResult.forSplit(split); + } + } + + @Override + public void onDiscoveredSplits(Collection splits) { + addSplits(splits); + } + + @Override + public void onUnassignedSplits(Collection splits) { + addSplits(splits); + } + + private synchronized void addSplits(Collection splits) { + if (!splits.isEmpty()) { + pendingSplits.addAll(splits); + // only complete pending future if new splits are discovered + completeAvailableFuturesIfNeeded(); + } + } + + /** Simple assigner only tracks unassigned splits */ + @Override + public synchronized Collection state() { + return pendingSplits.stream() + .map(split -> new IcebergSourceSplitState(split, IcebergSourceSplitStatus.UNASSIGNED)) + .collect(Collectors.toList()); + } + + @Override + public synchronized CompletableFuture isAvailable() { + if (availableFuture == null) { + availableFuture = new CompletableFuture<>(); + } + return availableFuture; + } + + @Override + public synchronized int pendingSplitCount() { + return pendingSplits.size(); + } + + private synchronized void completeAvailableFuturesIfNeeded() { + if (availableFuture != null && !pendingSplits.isEmpty()) { + availableFuture.complete(null); + } + availableFuture = null; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java new file mode 100644 index 000000000000..72deaeb890f3 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java @@ -0,0 +1,77 @@ +/* + * 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.iceberg.flink.source.assigner; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; + +@Internal +public class GetSplitResult { + + public enum Status { + AVAILABLE, + + /** + * There are pending splits. But they can't be assigned due to constraints (like event time + * alignment) + */ + CONSTRAINED, + + /** Assigner doesn't have pending splits. */ + UNAVAILABLE + } + + private final Status status; + private final IcebergSourceSplit split; + + private GetSplitResult(Status status) { + this.status = status; + this.split = null; + } + + private GetSplitResult(IcebergSourceSplit split) { + Preconditions.checkNotNull(split, "Split cannot be null"); + this.status = Status.AVAILABLE; + this.split = split; + } + + public Status status() { + return status; + } + + public IcebergSourceSplit split() { + return split; + } + + private static final GetSplitResult UNAVAILABLE = new GetSplitResult(Status.UNAVAILABLE); + private static final GetSplitResult CONSTRAINED = new GetSplitResult(Status.CONSTRAINED); + + public static GetSplitResult unavailable() { + return UNAVAILABLE; + } + + public static GetSplitResult constrained() { + return CONSTRAINED; + } + + public static GetSplitResult forSplit(IcebergSourceSplit split) { + return new GetSplitResult(split); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java new file mode 100644 index 000000000000..e58478897aef --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java @@ -0,0 +1,46 @@ +/* + * 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.iceberg.flink.source.assigner; + +import java.util.Collection; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; +import org.apache.iceberg.flink.source.split.SerializableComparator; + +/** + * Create default assigner with a comparator that hands out splits where the order of the splits + * will be defined by the {@link SerializableComparator}. + */ +public class OrderedSplitAssignerFactory implements SplitAssignerFactory { + private final SerializableComparator comparator; + + public OrderedSplitAssignerFactory(SerializableComparator comparator) { + this.comparator = comparator; + } + + @Override + public SplitAssigner createAssigner() { + return new DefaultSplitAssigner(comparator); + } + + @Override + public SplitAssigner createAssigner(Collection assignerState) { + return new DefaultSplitAssigner(comparator, assignerState); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java new file mode 100644 index 000000000000..a2e2ff364d46 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.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.iceberg.flink.source.assigner; + +import java.util.Collection; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; + +/** Create simple assigner that hands out splits without any guarantee in order or locality. */ +public class SimpleSplitAssignerFactory implements SplitAssignerFactory { + public SimpleSplitAssignerFactory() {} + + @Override + public SplitAssigner createAssigner() { + return new DefaultSplitAssigner(null); + } + + @Override + public SplitAssigner createAssigner(Collection assignerState) { + return new DefaultSplitAssigner(null, assignerState); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java new file mode 100644 index 000000000000..ca60612f0ec9 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java @@ -0,0 +1,118 @@ +/* + * 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.iceberg.flink.source.assigner; + +import java.io.Closeable; +import java.util.Collection; +import java.util.concurrent.CompletableFuture; +import javax.annotation.Nullable; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.iceberg.flink.source.ScanContext; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; + +/** + * SplitAssigner interface is extracted out as a separate component so that we can plug in different + * split assignment strategy for different requirements. E.g. + * + *

    + *
  • Simple assigner with no ordering guarantee or locality aware optimization. + *
  • Locality aware assigner that prefer splits that are local. + *
  • Snapshot aware assigner that assign splits based on the order they are committed. + *
  • Event time alignment assigner that assign splits satisfying certain time ordering within a + * single source or across sources. + *
+ * + *

Assigner implementation needs to be thread safe. Enumerator call the assigner APIs mostly from + * the coordinator thread. But enumerator may call the {@link SplitAssigner#pendingSplitCount()} + * from the I/O threads. + */ +public interface SplitAssigner extends Closeable { + + /** + * Some assigners may need to start background threads or perform other activity such as + * registering as listeners to updates from other event sources e.g., watermark tracker. + */ + default void start() {} + + /** + * Some assigners may need to perform certain actions when their corresponding enumerators are + * closed + */ + @Override + default void close() {} + + /** + * Request a new split from the assigner when enumerator trying to assign splits to awaiting + * readers. + * + *

If enumerator wasn't able to assign the split (e.g., reader disconnected), enumerator should + * call {@link SplitAssigner#onUnassignedSplits} to return the split. + */ + GetSplitResult getNext(@Nullable String hostname); + + /** Add new splits discovered by enumerator */ + void onDiscoveredSplits(Collection splits); + + /** Forward addSplitsBack event (for failed reader) to assigner */ + void onUnassignedSplits(Collection splits); + + /** + * Some assigner (like event time alignment) may rack in-progress splits to advance watermark upon + * completed splits + */ + default void onCompletedSplits(Collection completedSplitIds) {} + + /** + * Get assigner state for checkpointing. This is a super-set API that works for all currently + * imagined assigners. + */ + Collection state(); + + /** + * Enumerator can get a notification via CompletableFuture when the assigner has more splits + * available later. Enumerator should schedule assignment in the thenAccept action of the future. + * + *

Assigner will return the same future if this method is called again before the previous + * future is completed. + * + *

The future can be completed from other thread, e.g. the coordinator thread from another + * thread for event time alignment. + * + *

If enumerator need to trigger action upon the future completion, it may want to run it in + * the coordinator thread using {@link SplitEnumeratorContext#runInCoordinatorThread(Runnable)}. + */ + CompletableFuture isAvailable(); + + /** + * Return the number of pending splits that haven't been assigned yet. + * + *

The enumerator can poll this API to publish a metric on the number of pending splits. + * + *

The enumerator can also use this information to throttle split discovery for streaming read. + * If there are already many pending splits tracked by the assigner, it is undesirable to discover + * more splits and track them in the assigner. That will increase the memory footprint and + * enumerator checkpoint size. + * + *

Throttling works better together with {@link ScanContext#maxPlanningSnapshotCount()}. + * Otherwise, the next split discovery after throttling will just discover all non-enumerated + * snapshots and splits, which defeats the purpose of throttling. + */ + int pendingSplitCount(); +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java new file mode 100644 index 000000000000..6e02a556ffcd --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.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.iceberg.flink.source.assigner; + +import java.io.Serializable; +import java.util.Collection; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; + +public interface SplitAssignerFactory extends Serializable { + + SplitAssigner createAssigner(); + + SplitAssigner createAssigner(Collection assignerState); +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java new file mode 100644 index 000000000000..03ba67a554f9 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java @@ -0,0 +1,33 @@ +/* + * 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.iceberg.flink.source.assigner; + +import org.apache.flink.annotation.Internal; + +@Internal +public enum SplitAssignerType { + SIMPLE { + @Override + public SplitAssignerFactory factory() { + return new SimpleSplitAssignerFactory(); + } + }; + + public abstract SplitAssignerFactory factory(); +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java new file mode 100644 index 000000000000..3aca390755ed --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java @@ -0,0 +1,173 @@ +/* + * 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.iceberg.flink.source.enumerator; + +import java.io.IOException; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicReference; +import javax.annotation.Nullable; +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.iceberg.flink.source.assigner.GetSplitResult; +import org.apache.iceberg.flink.source.assigner.SplitAssigner; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SplitRequestEvent; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * TODO: publish enumerator monitor metrics like number of pending metrics after FLINK-21000 is + * resolved + */ +abstract class AbstractIcebergEnumerator + implements SplitEnumerator { + private static final Logger LOG = LoggerFactory.getLogger(AbstractIcebergEnumerator.class); + + private final SplitEnumeratorContext enumeratorContext; + private final SplitAssigner assigner; + private final Map readersAwaitingSplit; + private final AtomicReference> availableFuture; + + AbstractIcebergEnumerator( + SplitEnumeratorContext enumeratorContext, SplitAssigner assigner) { + this.enumeratorContext = enumeratorContext; + this.assigner = assigner; + this.readersAwaitingSplit = new LinkedHashMap<>(); + this.availableFuture = new AtomicReference<>(); + } + + @Override + public void start() { + assigner.start(); + } + + @Override + public void close() throws IOException { + assigner.close(); + } + + @Override + public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) { + // Iceberg source uses custom split request event to piggyback finished split ids. + throw new UnsupportedOperationException( + String.format( + "Received invalid default split request event " + + "from subtask %d as Iceberg source uses custom split request event", + subtaskId)); + } + + @Override + public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { + if (sourceEvent instanceof SplitRequestEvent) { + SplitRequestEvent splitRequestEvent = (SplitRequestEvent) sourceEvent; + LOG.info("Received request split event from subtask {}", subtaskId); + assigner.onCompletedSplits(splitRequestEvent.finishedSplitIds()); + readersAwaitingSplit.put(subtaskId, splitRequestEvent.requesterHostname()); + assignSplits(); + } else { + throw new IllegalArgumentException( + String.format( + "Received unknown event from subtask %d: %s", + subtaskId, sourceEvent.getClass().getCanonicalName())); + } + } + + @Override + public void addSplitsBack(List splits, int subtaskId) { + LOG.info("Add {} splits back to the pool for failed subtask {}", splits.size(), subtaskId); + assigner.onUnassignedSplits(splits); + assignSplits(); + } + + @Override + public void addReader(int subtaskId) { + LOG.info("Added reader: {}", subtaskId); + } + + private void assignSplits() { + LOG.info("Assigning splits for {} awaiting readers", readersAwaitingSplit.size()); + Iterator> awaitingReader = + readersAwaitingSplit.entrySet().iterator(); + while (awaitingReader.hasNext()) { + Map.Entry nextAwaiting = awaitingReader.next(); + // if the reader that requested another split has failed in the meantime, remove + // it from the list of waiting readers + if (!enumeratorContext.registeredReaders().containsKey(nextAwaiting.getKey())) { + awaitingReader.remove(); + continue; + } + + int awaitingSubtask = nextAwaiting.getKey(); + String hostname = nextAwaiting.getValue(); + GetSplitResult getResult = assigner.getNext(hostname); + if (getResult.status() == GetSplitResult.Status.AVAILABLE) { + LOG.info("Assign split to subtask {}: {}", awaitingSubtask, getResult.split()); + enumeratorContext.assignSplit(getResult.split(), awaitingSubtask); + awaitingReader.remove(); + } else if (getResult.status() == GetSplitResult.Status.CONSTRAINED) { + getAvailableFutureIfNeeded(); + break; + } else if (getResult.status() == GetSplitResult.Status.UNAVAILABLE) { + if (shouldWaitForMoreSplits()) { + getAvailableFutureIfNeeded(); + break; + } else { + LOG.info("No more splits available for subtask {}", awaitingSubtask); + enumeratorContext.signalNoMoreSplits(awaitingSubtask); + awaitingReader.remove(); + } + } else { + throw new IllegalArgumentException("Unsupported status: " + getResult.status()); + } + } + } + + /** return true if enumerator should wait for splits like in the continuous enumerator case */ + protected abstract boolean shouldWaitForMoreSplits(); + + private synchronized void getAvailableFutureIfNeeded() { + if (availableFuture.get() != null) { + return; + } + + CompletableFuture future = + assigner + .isAvailable() + .thenAccept( + ignore -> + // Must run assignSplits in coordinator thread + // because the future may be completed from other threads. + // E.g., in event time alignment assigner, + // watermark advancement from another source may + // cause the available future to be completed + enumeratorContext.runInCoordinatorThread( + () -> { + LOG.debug("Executing callback of assignSplits"); + availableFuture.set(null); + assignSplits(); + })); + availableFuture.set(future); + LOG.debug("Registered callback for future available splits"); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java new file mode 100644 index 000000000000..41863ffee60b --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.enumerator; + +import java.util.Collection; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +class ContinuousEnumerationResult { + private final Collection splits; + private final IcebergEnumeratorPosition fromPosition; + private final IcebergEnumeratorPosition toPosition; + + /** + * @param splits should never be null. But it can be an empty collection + * @param fromPosition can be null + * @param toPosition should never be null. But it can have null snapshotId and snapshotTimestampMs + */ + ContinuousEnumerationResult( + Collection splits, + IcebergEnumeratorPosition fromPosition, + IcebergEnumeratorPosition toPosition) { + Preconditions.checkArgument(splits != null, "Invalid to splits collection: null"); + Preconditions.checkArgument(toPosition != null, "Invalid end position: null"); + this.splits = splits; + this.fromPosition = fromPosition; + this.toPosition = toPosition; + } + + public Collection splits() { + return splits; + } + + public IcebergEnumeratorPosition fromPosition() { + return fromPosition; + } + + public IcebergEnumeratorPosition toPosition() { + return toPosition; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java new file mode 100644 index 000000000000..b1dadfb9a69a --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java @@ -0,0 +1,177 @@ +/* + * 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.iceberg.flink.source.enumerator; + +import java.io.IOException; +import java.util.Collections; +import java.util.Objects; +import java.util.concurrent.atomic.AtomicReference; +import javax.annotation.Nullable; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.iceberg.flink.source.ScanContext; +import org.apache.iceberg.flink.source.assigner.SplitAssigner; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Internal +public class ContinuousIcebergEnumerator extends AbstractIcebergEnumerator { + + private static final Logger LOG = LoggerFactory.getLogger(ContinuousIcebergEnumerator.class); + /** + * This is hardcoded, as {@link ScanContext#maxPlanningSnapshotCount()} could be the knob to + * control the total number of snapshots worth of splits tracked by assigner. + */ + private static final int ENUMERATION_SPLIT_COUNT_HISTORY_SIZE = 3; + + private final SplitEnumeratorContext enumeratorContext; + private final SplitAssigner assigner; + private final ScanContext scanContext; + private final ContinuousSplitPlanner splitPlanner; + + /** + * snapshotId for the last enumerated snapshot. next incremental enumeration should be based off + * this as the starting position. + */ + private final AtomicReference enumeratorPosition; + + /** Track enumeration result history for split discovery throttling. */ + private final EnumerationHistory enumerationHistory; + + /** Count the consecutive failures and throw exception if the max allowed failres are reached */ + private transient int consecutiveFailures = 0; + + public ContinuousIcebergEnumerator( + SplitEnumeratorContext enumeratorContext, + SplitAssigner assigner, + ScanContext scanContext, + ContinuousSplitPlanner splitPlanner, + @Nullable IcebergEnumeratorState enumState) { + super(enumeratorContext, assigner); + + this.enumeratorContext = enumeratorContext; + this.assigner = assigner; + this.scanContext = scanContext; + this.splitPlanner = splitPlanner; + this.enumeratorPosition = new AtomicReference<>(); + this.enumerationHistory = new EnumerationHistory(ENUMERATION_SPLIT_COUNT_HISTORY_SIZE); + + if (enumState != null) { + this.enumeratorPosition.set(enumState.lastEnumeratedPosition()); + } + } + + @Override + public void start() { + super.start(); + enumeratorContext.callAsync( + this::discoverSplits, + this::processDiscoveredSplits, + 0L, + scanContext.monitorInterval().toMillis()); + } + + @Override + public void close() throws IOException { + splitPlanner.close(); + super.close(); + } + + @Override + protected boolean shouldWaitForMoreSplits() { + return true; + } + + @Override + public IcebergEnumeratorState snapshotState(long checkpointId) { + return new IcebergEnumeratorState( + enumeratorPosition.get(), assigner.state(), enumerationHistory.snapshot()); + } + + /** This method is executed in an IO thread pool. */ + private ContinuousEnumerationResult discoverSplits() { + int pendingSplitCountFromAssigner = assigner.pendingSplitCount(); + if (enumerationHistory.shouldPauseSplitDiscovery(pendingSplitCountFromAssigner)) { + // If the assigner already has many pending splits, it is better to pause split discovery. + // Otherwise, eagerly discovering more splits will just increase assigner memory footprint + // and enumerator checkpoint state size. + LOG.info( + "Pause split discovery as the assigner already has too many pending splits: {}", + pendingSplitCountFromAssigner); + return new ContinuousEnumerationResult( + Collections.emptyList(), enumeratorPosition.get(), enumeratorPosition.get()); + } else { + return splitPlanner.planSplits(enumeratorPosition.get()); + } + } + + /** This method is executed in a single coordinator thread. */ + private void processDiscoveredSplits(ContinuousEnumerationResult result, Throwable error) { + if (error == null) { + consecutiveFailures = 0; + if (!Objects.equals(result.fromPosition(), enumeratorPosition.get())) { + // Multiple discoverSplits() may be triggered with the same starting snapshot to the I/O + // thread pool. E.g., the splitDiscoveryInterval is very short (like 10 ms in some unit + // tests) or the thread pool is busy and multiple discovery actions are executed + // concurrently. Discovery result should only be accepted if the starting position + // matches the enumerator position (like compare-and-swap). + LOG.info( + "Skip {} discovered splits because the scan starting position doesn't match " + + "the current enumerator position: enumerator position = {}, scan starting position = {}", + result.splits().size(), + enumeratorPosition.get(), + result.fromPosition()); + } else { + // Sometimes, enumeration may yield no splits for a few reasons. + // - upstream paused or delayed streaming writes to the Iceberg table. + // - enumeration frequency is higher than the upstream write frequency. + if (!result.splits().isEmpty()) { + assigner.onDiscoveredSplits(result.splits()); + // EnumerationHistory makes throttling decision on split discovery + // based on the total number of splits discovered in the last a few cycles. + // Only update enumeration history when there are some discovered splits. + enumerationHistory.add(result.splits().size()); + LOG.info( + "Added {} splits discovered between ({}, {}] to the assigner", + result.splits().size(), + result.fromPosition(), + result.toPosition()); + } else { + LOG.info( + "No new splits discovered between ({}, {}]", + result.fromPosition(), + result.toPosition()); + } + // update the enumerator position even if there is no split discovered + // or the toPosition is empty (e.g. for empty table). + enumeratorPosition.set(result.toPosition()); + LOG.info("Update enumerator position to {}", result.toPosition()); + } + } else { + consecutiveFailures++; + if (scanContext.maxAllowedPlanningFailures() < 0 + || consecutiveFailures <= scanContext.maxAllowedPlanningFailures()) { + LOG.error("Failed to discover new splits", error); + } else { + throw new RuntimeException("Failed to discover new splits", error); + } + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java new file mode 100644 index 000000000000..2a1325178873 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.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.iceberg.flink.source.enumerator; + +import java.io.Closeable; +import org.apache.flink.annotation.Internal; + +/** This interface is introduced so that we can plug in different split planner for unit test */ +@Internal +public interface ContinuousSplitPlanner extends Closeable { + + /** Discover the files appended between {@code lastPosition} and current table snapshot */ + ContinuousEnumerationResult planSplits(IcebergEnumeratorPosition lastPosition); +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java new file mode 100644 index 000000000000..f0d8ca8d7057 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java @@ -0,0 +1,236 @@ +/* + * 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.iceberg.flink.source.enumerator; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.ExecutorService; +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.source.FlinkSplitPlanner; +import org.apache.iceberg.flink.source.ScanContext; +import org.apache.iceberg.flink.source.StreamingStartingStrategy; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.iceberg.util.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Internal +public class ContinuousSplitPlannerImpl implements ContinuousSplitPlanner { + private static final Logger LOG = LoggerFactory.getLogger(ContinuousSplitPlannerImpl.class); + + private final Table table; + private final ScanContext scanContext; + private final boolean isSharedPool; + private final ExecutorService workerPool; + private final TableLoader tableLoader; + + /** + * @param tableLoader A cloned tableLoader. + * @param threadName thread name prefix for worker pool to run the split planning. If null, a + * shared worker pool will be used. + */ + public ContinuousSplitPlannerImpl( + TableLoader tableLoader, ScanContext scanContext, String threadName) { + this.tableLoader = tableLoader; + this.tableLoader.open(); + this.table = tableLoader.loadTable(); + this.scanContext = scanContext; + this.isSharedPool = threadName == null; + this.workerPool = + isSharedPool + ? ThreadPools.getWorkerPool() + : ThreadPools.newWorkerPool( + "iceberg-plan-worker-pool-" + threadName, scanContext.planParallelism()); + } + + @Override + public void close() throws IOException { + if (!isSharedPool) { + workerPool.shutdown(); + } + tableLoader.close(); + } + + @Override + public ContinuousEnumerationResult planSplits(IcebergEnumeratorPosition lastPosition) { + table.refresh(); + if (lastPosition != null) { + return discoverIncrementalSplits(lastPosition); + } else { + return discoverInitialSplits(); + } + } + + private Snapshot toSnapshotInclusive( + Long lastConsumedSnapshotId, Snapshot currentSnapshot, int maxPlanningSnapshotCount) { + // snapshots are in reverse order (latest snapshot first) + List snapshots = + Lists.newArrayList( + SnapshotUtil.ancestorsBetween( + table, currentSnapshot.snapshotId(), lastConsumedSnapshotId)); + if (snapshots.size() <= maxPlanningSnapshotCount) { + return currentSnapshot; + } else { + // Because snapshots are in reverse order of commit history, this index returns + // the max allowed number of snapshots from the lastConsumedSnapshotId. + return snapshots.get(snapshots.size() - maxPlanningSnapshotCount); + } + } + + private ContinuousEnumerationResult discoverIncrementalSplits( + IcebergEnumeratorPosition lastPosition) { + Snapshot currentSnapshot = table.currentSnapshot(); + if (currentSnapshot == null) { + // empty table + Preconditions.checkArgument( + lastPosition.snapshotId() == null, + "Invalid last enumerated position for an empty table: not null"); + LOG.info("Skip incremental scan because table is empty"); + return new ContinuousEnumerationResult(Collections.emptyList(), lastPosition, lastPosition); + } else if (lastPosition.snapshotId() != null + && currentSnapshot.snapshotId() == lastPosition.snapshotId()) { + LOG.info("Current table snapshot is already enumerated: {}", currentSnapshot.snapshotId()); + return new ContinuousEnumerationResult(Collections.emptyList(), lastPosition, lastPosition); + } else { + Long lastConsumedSnapshotId = lastPosition != null ? lastPosition.snapshotId() : null; + Snapshot toSnapshotInclusive = + toSnapshotInclusive( + lastConsumedSnapshotId, currentSnapshot, scanContext.maxPlanningSnapshotCount()); + IcebergEnumeratorPosition newPosition = + IcebergEnumeratorPosition.of( + toSnapshotInclusive.snapshotId(), toSnapshotInclusive.timestampMillis()); + ScanContext incrementalScan = + scanContext.copyWithAppendsBetween( + lastPosition.snapshotId(), toSnapshotInclusive.snapshotId()); + List splits = + FlinkSplitPlanner.planIcebergSourceSplits(table, incrementalScan, workerPool); + LOG.info( + "Discovered {} splits from incremental scan: " + + "from snapshot (exclusive) is {}, to snapshot (inclusive) is {}", + splits.size(), + lastPosition, + newPosition); + return new ContinuousEnumerationResult(splits, lastPosition, newPosition); + } + } + + /** + * Discovery initial set of splits based on {@link StreamingStartingStrategy}. + *

  • {@link ContinuousEnumerationResult#splits()} should contain initial splits discovered from + * table scan for {@link StreamingStartingStrategy#TABLE_SCAN_THEN_INCREMENTAL}. For all other + * strategies, splits collection should be empty. + *
  • {@link ContinuousEnumerationResult#toPosition()} points to the starting position for the + * next incremental split discovery with exclusive behavior. Meaning files committed by the + * snapshot from the position in {@code ContinuousEnumerationResult} won't be included in the + * next incremental scan. + */ + private ContinuousEnumerationResult discoverInitialSplits() { + Optional startSnapshotOptional = startSnapshot(table, scanContext); + if (!startSnapshotOptional.isPresent()) { + return new ContinuousEnumerationResult( + Collections.emptyList(), null, IcebergEnumeratorPosition.empty()); + } + + Snapshot startSnapshot = startSnapshotOptional.get(); + LOG.info( + "Get starting snapshot id {} based on strategy {}", + startSnapshot.snapshotId(), + scanContext.streamingStartingStrategy()); + List splits; + IcebergEnumeratorPosition toPosition; + if (scanContext.streamingStartingStrategy() + == StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) { + // do a batch table scan first + splits = + FlinkSplitPlanner.planIcebergSourceSplits( + table, scanContext.copyWithSnapshotId(startSnapshot.snapshotId()), workerPool); + LOG.info( + "Discovered {} splits from initial batch table scan with snapshot Id {}", + splits.size(), + startSnapshot.snapshotId()); + // For TABLE_SCAN_THEN_INCREMENTAL, incremental mode starts exclusive from the startSnapshot + toPosition = + IcebergEnumeratorPosition.of(startSnapshot.snapshotId(), startSnapshot.timestampMillis()); + } else { + // For all other modes, starting snapshot should be consumed inclusively. + // Use parentId to achieve the inclusive behavior. It is fine if parentId is null. + splits = Collections.emptyList(); + Long parentSnapshotId = startSnapshot.parentId(); + if (parentSnapshotId != null) { + Snapshot parentSnapshot = table.snapshot(parentSnapshotId); + Long parentSnapshotTimestampMs = + parentSnapshot != null ? parentSnapshot.timestampMillis() : null; + toPosition = IcebergEnumeratorPosition.of(parentSnapshotId, parentSnapshotTimestampMs); + } else { + toPosition = IcebergEnumeratorPosition.empty(); + } + + LOG.info( + "Start incremental scan with start snapshot (inclusive): id = {}, timestamp = {}", + startSnapshot.snapshotId(), + startSnapshot.timestampMillis()); + } + + return new ContinuousEnumerationResult(splits, null, toPosition); + } + + /** + * Calculate the starting snapshot based on the {@link StreamingStartingStrategy} defined in + * {@code ScanContext}. + * + *

    If the {@link StreamingStartingStrategy} is not {@link + * StreamingStartingStrategy#TABLE_SCAN_THEN_INCREMENTAL}, the start snapshot should be consumed + * inclusively. + */ + @VisibleForTesting + static Optional startSnapshot(Table table, ScanContext scanContext) { + switch (scanContext.streamingStartingStrategy()) { + case TABLE_SCAN_THEN_INCREMENTAL: + case INCREMENTAL_FROM_LATEST_SNAPSHOT: + return Optional.ofNullable(table.currentSnapshot()); + case INCREMENTAL_FROM_EARLIEST_SNAPSHOT: + return Optional.ofNullable(SnapshotUtil.oldestAncestor(table)); + case INCREMENTAL_FROM_SNAPSHOT_ID: + Snapshot matchedSnapshotById = table.snapshot(scanContext.startSnapshotId()); + Preconditions.checkArgument( + matchedSnapshotById != null, + "Start snapshot id not found in history: " + scanContext.startSnapshotId()); + return Optional.of(matchedSnapshotById); + case INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP: + Snapshot matchedSnapshotByTimestamp = + SnapshotUtil.oldestAncestorAfter(table, scanContext.startSnapshotTimestamp()); + Preconditions.checkArgument( + matchedSnapshotByTimestamp != null, + "Cannot find a snapshot after: " + scanContext.startSnapshotTimestamp()); + return Optional.of(matchedSnapshotByTimestamp); + default: + throw new IllegalArgumentException( + "Unknown starting strategy: " + scanContext.streamingStartingStrategy()); + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java new file mode 100644 index 000000000000..ef21dad0199d --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.enumerator; + +import java.util.Arrays; +import javax.annotation.concurrent.ThreadSafe; +import org.apache.flink.annotation.VisibleForTesting; + +/** + * This enumeration history is used for split discovery throttling. It tracks the discovered split + * count per every non-empty enumeration. + */ +@ThreadSafe +class EnumerationHistory { + + private final int[] history; + // int (2B) should be enough without overflow for enumeration history + private int count; + + EnumerationHistory(int maxHistorySize) { + this.history = new int[maxHistorySize]; + } + + synchronized void restore(int[] restoredHistory) { + int startingOffset = 0; + int restoreSize = restoredHistory.length; + + if (restoredHistory.length > history.length) { + // keep the newest history + startingOffset = restoredHistory.length - history.length; + // only restore the latest history up to maxHistorySize + restoreSize = history.length; + } + + System.arraycopy(restoredHistory, startingOffset, history, 0, restoreSize); + count = restoreSize; + } + + synchronized int[] snapshot() { + int len = history.length; + if (count > len) { + int[] copy = new int[len]; + // this is like a circular buffer + int indexForOldest = count % len; + System.arraycopy(history, indexForOldest, copy, 0, len - indexForOldest); + System.arraycopy(history, 0, copy, len - indexForOldest, indexForOldest); + return copy; + } else { + return Arrays.copyOfRange(history, 0, count); + } + } + + /** Add the split count from the last enumeration result. */ + synchronized void add(int splitCount) { + int pos = count % history.length; + history[pos] = splitCount; + count += 1; + } + + @VisibleForTesting + synchronized boolean hasFullHistory() { + return count >= history.length; + } + + /** @return true if split discovery should pause because assigner has too many splits already. */ + synchronized boolean shouldPauseSplitDiscovery(int pendingSplitCountFromAssigner) { + if (count < history.length) { + // only check throttling when full history is obtained. + return false; + } else { + // if ScanContext#maxPlanningSnapshotCount() is 10, each split enumeration can + // discovery splits up to 10 snapshots. if maxHistorySize is 3, the max number of + // splits tracked in assigner shouldn't be more than 10 * (3 + 1) snapshots + // worth of splits. +1 because there could be another enumeration when the + // pending splits fall just below the 10 * 3. + int totalSplitCountFromRecentDiscovery = Arrays.stream(history).reduce(0, Integer::sum); + return pendingSplitCountFromAssigner >= totalSplitCountFromRecentDiscovery; + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java new file mode 100644 index 000000000000..96aba296f8cf --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java @@ -0,0 +1,79 @@ +/* + * 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.iceberg.flink.source.enumerator; + +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; + +class IcebergEnumeratorPosition { + private final Long snapshotId; + // Track snapshot timestamp mainly for info logging + private final Long snapshotTimestampMs; + + static IcebergEnumeratorPosition empty() { + return new IcebergEnumeratorPosition(null, null); + } + + static IcebergEnumeratorPosition of(long snapshotId, Long snapshotTimestampMs) { + return new IcebergEnumeratorPosition(snapshotId, snapshotTimestampMs); + } + + private IcebergEnumeratorPosition(Long snapshotId, Long snapshotTimestampMs) { + this.snapshotId = snapshotId; + this.snapshotTimestampMs = snapshotTimestampMs; + } + + boolean isEmpty() { + return snapshotId == null; + } + + Long snapshotId() { + return snapshotId; + } + + Long snapshotTimestampMs() { + return snapshotTimestampMs; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("snapshotId", snapshotId) + .add("snapshotTimestampMs", snapshotTimestampMs) + .toString(); + } + + @Override + public int hashCode() { + return Objects.hashCode(snapshotId, snapshotTimestampMs); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + IcebergEnumeratorPosition other = (IcebergEnumeratorPosition) o; + return Objects.equal(snapshotId, other.snapshotId()) + && Objects.equal(snapshotTimestampMs, other.snapshotTimestampMs()); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java new file mode 100644 index 000000000000..1c63807361c5 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java @@ -0,0 +1,90 @@ +/* + * 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.iceberg.flink.source.enumerator; + +import java.io.IOException; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; + +class IcebergEnumeratorPositionSerializer + implements SimpleVersionedSerializer { + + public static final IcebergEnumeratorPositionSerializer INSTANCE = + new IcebergEnumeratorPositionSerializer(); + + private static final int VERSION = 1; + + private static final ThreadLocal SERIALIZER_CACHE = + ThreadLocal.withInitial(() -> new DataOutputSerializer(128)); + + @Override + public int getVersion() { + return VERSION; + } + + @Override + public byte[] serialize(IcebergEnumeratorPosition position) throws IOException { + return serializeV1(position); + } + + @Override + public IcebergEnumeratorPosition deserialize(int version, byte[] serialized) throws IOException { + switch (version) { + case 1: + return deserializeV1(serialized); + default: + throw new IOException("Unknown version: " + version); + } + } + + private byte[] serializeV1(IcebergEnumeratorPosition position) throws IOException { + DataOutputSerializer out = SERIALIZER_CACHE.get(); + out.writeBoolean(position.snapshotId() != null); + if (position.snapshotId() != null) { + out.writeLong(position.snapshotId()); + } + out.writeBoolean(position.snapshotTimestampMs() != null); + if (position.snapshotTimestampMs() != null) { + out.writeLong(position.snapshotTimestampMs()); + } + byte[] result = out.getCopyOfBuffer(); + out.clear(); + return result; + } + + private IcebergEnumeratorPosition deserializeV1(byte[] serialized) throws IOException { + DataInputDeserializer in = new DataInputDeserializer(serialized); + Long snapshotId = null; + if (in.readBoolean()) { + snapshotId = in.readLong(); + } + + Long snapshotTimestampMs = null; + if (in.readBoolean()) { + snapshotTimestampMs = in.readLong(); + } + + if (snapshotId != null) { + return IcebergEnumeratorPosition.of(snapshotId, snapshotTimestampMs); + } else { + return IcebergEnumeratorPosition.empty(); + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java new file mode 100644 index 000000000000..024d0b101165 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java @@ -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. + */ +package org.apache.iceberg.flink.source.enumerator; + +import java.io.Serializable; +import java.util.Collection; +import javax.annotation.Nullable; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; + +/** Enumerator state for checkpointing */ +@Internal +public class IcebergEnumeratorState implements Serializable { + @Nullable private final IcebergEnumeratorPosition lastEnumeratedPosition; + private final Collection pendingSplits; + private int[] enumerationSplitCountHistory; + + public IcebergEnumeratorState(Collection pendingSplits) { + this(null, pendingSplits); + } + + public IcebergEnumeratorState( + @Nullable IcebergEnumeratorPosition lastEnumeratedPosition, + Collection pendingSplits) { + this(lastEnumeratedPosition, pendingSplits, new int[0]); + } + + public IcebergEnumeratorState( + @Nullable IcebergEnumeratorPosition lastEnumeratedPosition, + Collection pendingSplits, + int[] enumerationSplitCountHistory) { + this.lastEnumeratedPosition = lastEnumeratedPosition; + this.pendingSplits = pendingSplits; + this.enumerationSplitCountHistory = enumerationSplitCountHistory; + } + + @Nullable + public IcebergEnumeratorPosition lastEnumeratedPosition() { + return lastEnumeratedPosition; + } + + public Collection pendingSplits() { + return pendingSplits; + } + + public int[] enumerationSplitCountHistory() { + return enumerationSplitCountHistory; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java new file mode 100644 index 000000000000..95d6db2cfbc4 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.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.iceberg.flink.source.enumerator; + +import java.io.IOException; +import java.util.Collection; +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +@Internal +public class IcebergEnumeratorStateSerializer + implements SimpleVersionedSerializer { + + private static final int VERSION = 2; + + private static final ThreadLocal SERIALIZER_CACHE = + ThreadLocal.withInitial(() -> new DataOutputSerializer(1024)); + + private final IcebergEnumeratorPositionSerializer positionSerializer = + IcebergEnumeratorPositionSerializer.INSTANCE; + private final IcebergSourceSplitSerializer splitSerializer; + + public IcebergEnumeratorStateSerializer(boolean caseSensitive) { + this.splitSerializer = new IcebergSourceSplitSerializer(caseSensitive); + } + + @Override + public int getVersion() { + return VERSION; + } + + @Override + public byte[] serialize(IcebergEnumeratorState enumState) throws IOException { + return serializeV2(enumState); + } + + @Override + public IcebergEnumeratorState deserialize(int version, byte[] serialized) throws IOException { + switch (version) { + case 1: + return deserializeV1(serialized); + case 2: + return deserializeV2(serialized); + default: + throw new IOException("Unknown version: " + version); + } + } + + @VisibleForTesting + byte[] serializeV1(IcebergEnumeratorState enumState) throws IOException { + DataOutputSerializer out = SERIALIZER_CACHE.get(); + serializeEnumeratorPosition(out, enumState.lastEnumeratedPosition(), positionSerializer); + serializePendingSplits(out, enumState.pendingSplits(), splitSerializer); + byte[] result = out.getCopyOfBuffer(); + out.clear(); + return result; + } + + @VisibleForTesting + IcebergEnumeratorState deserializeV1(byte[] serialized) throws IOException { + DataInputDeserializer in = new DataInputDeserializer(serialized); + IcebergEnumeratorPosition enumeratorPosition = + deserializeEnumeratorPosition(in, positionSerializer); + Collection pendingSplits = + deserializePendingSplits(in, splitSerializer); + return new IcebergEnumeratorState(enumeratorPosition, pendingSplits); + } + + @VisibleForTesting + byte[] serializeV2(IcebergEnumeratorState enumState) throws IOException { + DataOutputSerializer out = SERIALIZER_CACHE.get(); + serializeEnumeratorPosition(out, enumState.lastEnumeratedPosition(), positionSerializer); + serializePendingSplits(out, enumState.pendingSplits(), splitSerializer); + serializeEnumerationSplitCountHistory(out, enumState.enumerationSplitCountHistory()); + byte[] result = out.getCopyOfBuffer(); + out.clear(); + return result; + } + + @VisibleForTesting + IcebergEnumeratorState deserializeV2(byte[] serialized) throws IOException { + DataInputDeserializer in = new DataInputDeserializer(serialized); + IcebergEnumeratorPosition enumeratorPosition = + deserializeEnumeratorPosition(in, positionSerializer); + Collection pendingSplits = + deserializePendingSplits(in, splitSerializer); + int[] enumerationSplitCountHistory = deserializeEnumerationSplitCountHistory(in); + return new IcebergEnumeratorState( + enumeratorPosition, pendingSplits, enumerationSplitCountHistory); + } + + private static void serializeEnumeratorPosition( + DataOutputSerializer out, + IcebergEnumeratorPosition enumeratorPosition, + IcebergEnumeratorPositionSerializer positionSerializer) + throws IOException { + out.writeBoolean(enumeratorPosition != null); + if (enumeratorPosition != null) { + out.writeInt(positionSerializer.getVersion()); + byte[] positionBytes = positionSerializer.serialize(enumeratorPosition); + out.writeInt(positionBytes.length); + out.write(positionBytes); + } + } + + private static IcebergEnumeratorPosition deserializeEnumeratorPosition( + DataInputDeserializer in, IcebergEnumeratorPositionSerializer positionSerializer) + throws IOException { + IcebergEnumeratorPosition enumeratorPosition = null; + if (in.readBoolean()) { + int version = in.readInt(); + byte[] positionBytes = new byte[in.readInt()]; + in.read(positionBytes); + enumeratorPosition = positionSerializer.deserialize(version, positionBytes); + } + return enumeratorPosition; + } + + private static void serializePendingSplits( + DataOutputSerializer out, + Collection pendingSplits, + IcebergSourceSplitSerializer splitSerializer) + throws IOException { + out.writeInt(splitSerializer.getVersion()); + out.writeInt(pendingSplits.size()); + for (IcebergSourceSplitState splitState : pendingSplits) { + byte[] splitBytes = splitSerializer.serialize(splitState.split()); + out.writeInt(splitBytes.length); + out.write(splitBytes); + out.writeUTF(splitState.status().name()); + } + } + + private static Collection deserializePendingSplits( + DataInputDeserializer in, IcebergSourceSplitSerializer splitSerializer) throws IOException { + int splitSerializerVersion = in.readInt(); + int splitCount = in.readInt(); + Collection pendingSplits = Lists.newArrayListWithCapacity(splitCount); + for (int i = 0; i < splitCount; ++i) { + byte[] splitBytes = new byte[in.readInt()]; + in.read(splitBytes); + IcebergSourceSplit split = splitSerializer.deserialize(splitSerializerVersion, splitBytes); + String statusName = in.readUTF(); + pendingSplits.add( + new IcebergSourceSplitState(split, IcebergSourceSplitStatus.valueOf(statusName))); + } + return pendingSplits; + } + + private static void serializeEnumerationSplitCountHistory( + DataOutputSerializer out, int[] enumerationSplitCountHistory) throws IOException { + out.writeInt(enumerationSplitCountHistory.length); + if (enumerationSplitCountHistory.length > 0) { + for (int enumerationSplitCount : enumerationSplitCountHistory) { + out.writeInt(enumerationSplitCount); + } + } + } + + private static int[] deserializeEnumerationSplitCountHistory(DataInputDeserializer in) + throws IOException { + int historySize = in.readInt(); + int[] history = new int[historySize]; + if (historySize > 0) { + for (int i = 0; i < historySize; ++i) { + history[i] = in.readInt(); + } + } + + return history; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java new file mode 100644 index 000000000000..4e55ea5d5fd6 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java @@ -0,0 +1,51 @@ +/* + * 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.iceberg.flink.source.enumerator; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.iceberg.flink.source.assigner.SplitAssigner; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; + +/** One-time split enumeration at the start-up for batch execution */ +@Internal +public class StaticIcebergEnumerator extends AbstractIcebergEnumerator { + private final SplitAssigner assigner; + + public StaticIcebergEnumerator( + SplitEnumeratorContext enumeratorContext, SplitAssigner assigner) { + super(enumeratorContext, assigner); + this.assigner = assigner; + } + + @Override + public void start() { + super.start(); + } + + @Override + protected boolean shouldWaitForMoreSplits() { + return false; + } + + @Override + public IcebergEnumeratorState snapshotState(long checkpointId) { + return new IcebergEnumeratorState(null, assigner.state(), new int[0]); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java new file mode 100644 index 000000000000..7b94c364c976 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java @@ -0,0 +1,171 @@ +/* + * 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.iceberg.flink.source.reader; + +import java.util.Collections; +import java.util.Set; +import javax.annotation.Nullable; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.file.src.util.Pool; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * {@link RecordsWithSplitIds} is used to pass a batch of records from fetcher to source reader. + * Batching is to improve the efficiency for records handover. + * + *

    {@link RecordsWithSplitIds} interface can encapsulate batches from multiple splits. This is + * the case for Kafka source where fetchers can retrieve records from multiple Kafka partitions at + * the same time. + * + *

    For file-based sources like Iceberg, readers always read one split/file at a time. Hence, we + * will only have a batch of records for one split here. + * + *

    This class uses array to store a batch of records from the same file (with the same + * fileOffset). + */ +class ArrayBatchRecords implements RecordsWithSplitIds> { + @Nullable private String splitId; + @Nullable private final Pool.Recycler recycler; + @Nullable private final T[] records; + private final int numberOfRecords; + private final Set finishedSplits; + private final RecordAndPosition recordAndPosition; + + // point to current read position within the records array + private int position; + + private ArrayBatchRecords( + @Nullable String splitId, + @Nullable Pool.Recycler recycler, + @Nullable T[] records, + int numberOfRecords, + int fileOffset, + long startingRecordOffset, + Set finishedSplits) { + Preconditions.checkArgument(numberOfRecords >= 0, "numberOfRecords can't be negative"); + Preconditions.checkArgument(fileOffset >= 0, "fileOffset can't be negative"); + Preconditions.checkArgument(startingRecordOffset >= 0, "numberOfRecords can't be negative"); + + this.splitId = splitId; + this.recycler = recycler; + this.records = records; + this.numberOfRecords = numberOfRecords; + this.finishedSplits = + Preconditions.checkNotNull(finishedSplits, "finishedSplits can be empty but not null"); + this.recordAndPosition = new RecordAndPosition<>(); + + recordAndPosition.set(null, fileOffset, startingRecordOffset); + this.position = 0; + } + + @Nullable + @Override + public String nextSplit() { + String nextSplit = this.splitId; + // set the splitId to null to indicate no more splits + // this class only contains record for one split + this.splitId = null; + return nextSplit; + } + + @Nullable + @Override + public RecordAndPosition nextRecordFromSplit() { + if (position < numberOfRecords) { + recordAndPosition.record(records[position]); + position++; + return recordAndPosition; + } else { + return null; + } + } + + /** + * This method is called when all records from this batch has been emitted. If recycler is set, it + * should be called to return the records array back to pool. + */ + @Override + public void recycle() { + if (recycler != null) { + recycler.recycle(records); + } + } + + @Override + public Set finishedSplits() { + return finishedSplits; + } + + @VisibleForTesting + T[] records() { + return records; + } + + @VisibleForTesting + int numberOfRecords() { + return numberOfRecords; + } + + /** + * Create a ArrayBatchRecords backed up an array with records from the same file + * + * @param splitId Iceberg source only read from one split a time. We never have multiple records + * from multiple splits. + * @param recycler Because {@link DataIterator} with {@link RowData} returns an iterator of reused + * RowData object, we need to clone RowData eagerly when constructing a batch of records. We + * can use object pool to reuse the RowData array object which can be expensive to create. + * This recycler can be provided to recycle the array object back to pool after read is + * exhausted. If the {@link DataIterator} returns an iterator of non-reused objects, we don't + * need to clone objects. It is cheap to just create the batch array. Hence, we don't need + * object pool and recycler can be set to null. + * @param records an array (maybe reused) holding a batch of records + * @param numberOfRecords actual number of records in the array + * @param fileOffset fileOffset for all records in this batch + * @param startingRecordOffset starting recordOffset + * @param record type + */ + public static ArrayBatchRecords forRecords( + String splitId, + Pool.Recycler recycler, + T[] records, + int numberOfRecords, + int fileOffset, + long startingRecordOffset) { + return new ArrayBatchRecords<>( + splitId, + recycler, + records, + numberOfRecords, + fileOffset, + startingRecordOffset, + Collections.emptySet()); + } + + /** + * Create ab ArrayBatchRecords with only finished split id + * + * @param splitId for the split that is just exhausted + */ + public static ArrayBatchRecords finishedSplit(String splitId) { + return new ArrayBatchRecords<>(null, null, null, 0, 0, 0, Collections.singleton(splitId)); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java new file mode 100644 index 000000000000..306afd1811be --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java @@ -0,0 +1,130 @@ +/* + * 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.iceberg.flink.source.reader; + +import java.io.IOException; +import java.util.NoSuchElementException; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.SourceReaderOptions; +import org.apache.flink.connector.file.src.util.Pool; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** This implementation stores record batch in array from recyclable pool */ +class ArrayPoolDataIteratorBatcher implements DataIteratorBatcher { + private final int batchSize; + private final int handoverQueueSize; + private final RecordFactory recordFactory; + + private transient Pool pool; + + ArrayPoolDataIteratorBatcher(ReadableConfig config, RecordFactory recordFactory) { + this.batchSize = config.get(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT); + this.handoverQueueSize = config.get(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY); + this.recordFactory = recordFactory; + } + + @Override + public CloseableIterator>> batch( + String splitId, DataIterator inputIterator) { + Preconditions.checkArgument(inputIterator != null, "Input data iterator can't be null"); + // lazily create pool as it is not serializable + if (pool == null) { + this.pool = createPoolOfBatches(handoverQueueSize); + } + return new ArrayPoolBatchIterator(splitId, inputIterator, pool); + } + + private Pool createPoolOfBatches(int numBatches) { + Pool poolOfBatches = new Pool<>(numBatches); + for (int batchId = 0; batchId < numBatches; batchId++) { + T[] batch = recordFactory.createBatch(batchSize); + poolOfBatches.add(batch); + } + + return poolOfBatches; + } + + private class ArrayPoolBatchIterator + implements CloseableIterator>> { + + private final String splitId; + private final DataIterator inputIterator; + private final Pool pool; + + ArrayPoolBatchIterator(String splitId, DataIterator inputIterator, Pool pool) { + this.splitId = splitId; + this.inputIterator = inputIterator; + this.pool = pool; + } + + @Override + public boolean hasNext() { + return inputIterator.hasNext(); + } + + @Override + public RecordsWithSplitIds> next() { + if (!inputIterator.hasNext()) { + throw new NoSuchElementException(); + } + + T[] batch = getCachedEntry(); + int recordCount = 0; + while (inputIterator.hasNext() && recordCount < batchSize) { + // The record produced by inputIterator can be reused like for the RowData case. + // inputIterator.next() can't be called again until the copy is made + // since the record is not consumed immediately. + T nextRecord = inputIterator.next(); + recordFactory.clone(nextRecord, batch, recordCount); + recordCount++; + if (!inputIterator.currentFileHasNext()) { + // break early so that records in the ArrayResultIterator + // have the same fileOffset. + break; + } + } + + return ArrayBatchRecords.forRecords( + splitId, + pool.recycler(), + batch, + recordCount, + inputIterator.fileOffset(), + inputIterator.recordOffset() - recordCount); + } + + @Override + public void close() throws IOException { + inputIterator.close(); + } + + private T[] getCachedEntry() { + try { + return pool.pollEntry(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted while waiting for array pool entry", e); + } + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java new file mode 100644 index 000000000000..66e59633fff2 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java @@ -0,0 +1,102 @@ +/* + * 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.iceberg.flink.source.reader; + +import java.util.List; +import org.apache.avro.generic.GenericRecord; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.source.AvroGenericRecordFileScanTaskReader; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; +import org.apache.iceberg.flink.source.RowDataToAvroGenericRecordConverter; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** Read Iceberg rows as {@link GenericRecord}. */ +public class AvroGenericRecordReaderFunction extends DataIteratorReaderFunction { + private final String tableName; + private final Schema readSchema; + private final FileIO io; + private final EncryptionManager encryption; + private final RowDataFileScanTaskReader rowDataReader; + + private transient RowDataToAvroGenericRecordConverter converter; + + /** + * Create a reader function without projection and name mapping. Column name is case-insensitive. + */ + public static AvroGenericRecordReaderFunction fromTable(Table table) { + return new AvroGenericRecordReaderFunction( + table.name(), + new Configuration(), + table.schema(), + null, + null, + false, + table.io(), + table.encryption(), + null); + } + + public AvroGenericRecordReaderFunction( + String tableName, + ReadableConfig config, + Schema tableSchema, + Schema projectedSchema, + String nameMapping, + boolean caseSensitive, + FileIO io, + EncryptionManager encryption, + List filters) { + super(new ListDataIteratorBatcher<>(config)); + this.tableName = tableName; + this.readSchema = readSchema(tableSchema, projectedSchema); + this.io = io; + this.encryption = encryption; + this.rowDataReader = + new RowDataFileScanTaskReader(tableSchema, readSchema, nameMapping, caseSensitive, filters); + } + + @Override + protected DataIterator createDataIterator(IcebergSourceSplit split) { + return new DataIterator<>( + new AvroGenericRecordFileScanTaskReader(rowDataReader, lazyConverter()), + split.task(), + io, + encryption); + } + + private RowDataToAvroGenericRecordConverter lazyConverter() { + if (converter == null) { + this.converter = RowDataToAvroGenericRecordConverter.fromIcebergSchema(tableName, readSchema); + } + return converter; + } + + private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { + Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); + return projectedSchema == null ? tableSchema : projectedSchema; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java new file mode 100644 index 000000000000..4bb6f0a98c4c --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java @@ -0,0 +1,98 @@ +/* + * 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.iceberg.flink.source.reader; + +import java.io.Serializable; +import java.util.Comparator; +import java.util.concurrent.TimeUnit; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Type.TypeID; +import org.apache.iceberg.types.Types; + +/** + * {@link SplitWatermarkExtractor} implementation which uses an Iceberg timestamp column statistics + * to get the watermarks for the {@link IcebergSourceSplit}. This watermark is emitted by the {@link + * WatermarkExtractorRecordEmitter} along with the actual records. + */ +@Internal +public class ColumnStatsWatermarkExtractor implements SplitWatermarkExtractor, Serializable { + private final int eventTimeFieldId; + private final String eventTimeFieldName; + private final TimeUnit timeUnit; + + /** + * Creates the extractor. + * + * @param schema The schema of the Table + * @param eventTimeFieldName The column which should be used as an event time + * @param timeUnit Used for converting the long value to epoch milliseconds + */ + public ColumnStatsWatermarkExtractor( + Schema schema, String eventTimeFieldName, TimeUnit timeUnit) { + Types.NestedField field = schema.findField(eventTimeFieldName); + TypeID typeID = field.type().typeId(); + Preconditions.checkArgument( + typeID.equals(TypeID.LONG) || typeID.equals(TypeID.TIMESTAMP), + "Found %s, expected a LONG or TIMESTAMP column for watermark generation.", + typeID); + this.eventTimeFieldId = field.fieldId(); + this.eventTimeFieldName = eventTimeFieldName; + // Use the timeUnit only for Long columns. + this.timeUnit = typeID.equals(TypeID.LONG) ? timeUnit : TimeUnit.MICROSECONDS; + } + + @VisibleForTesting + ColumnStatsWatermarkExtractor(int eventTimeFieldId, String eventTimeFieldName) { + this.eventTimeFieldId = eventTimeFieldId; + this.eventTimeFieldName = eventTimeFieldName; + this.timeUnit = TimeUnit.MICROSECONDS; + } + + /** + * Get the watermark for a split using column statistics. + * + * @param split The split + * @return The watermark + * @throws IllegalArgumentException if there is no statistics for the column + */ + @Override + public long extractWatermark(IcebergSourceSplit split) { + return split.task().files().stream() + .map( + scanTask -> { + Preconditions.checkArgument( + scanTask.file().lowerBounds() != null + && scanTask.file().lowerBounds().get(eventTimeFieldId) != null, + "Missing statistics for column name = %s in file = %s", + eventTimeFieldName, + eventTimeFieldId, + scanTask.file()); + return timeUnit.toMillis( + Conversions.fromByteBuffer( + Types.LongType.get(), scanTask.file().lowerBounds().get(eventTimeFieldId))); + }) + .min(Comparator.comparingLong(l -> l)) + .get(); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java new file mode 100644 index 000000000000..c376e359c600 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java @@ -0,0 +1,36 @@ +/* + * 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.iceberg.flink.source.reader; + +import java.io.Serializable; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.io.CloseableIterator; + +/** + * Batcher converts iterator of T into iterator of batched {@code + * RecordsWithSplitIds>}, as FLIP-27's {@link SplitReader#fetch()} returns + * batched records. + */ +@FunctionalInterface +public interface DataIteratorBatcher extends Serializable { + CloseableIterator>> batch( + String splitId, DataIterator inputIterator); +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java new file mode 100644 index 000000000000..bbf797ef4aa8 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.io.CloseableIterator; + +/** A {@link ReaderFunction} implementation that uses {@link DataIterator}. */ +public abstract class DataIteratorReaderFunction implements ReaderFunction { + private final DataIteratorBatcher batcher; + + public DataIteratorReaderFunction(DataIteratorBatcher batcher) { + this.batcher = batcher; + } + + protected abstract DataIterator createDataIterator(IcebergSourceSplit split); + + @Override + public CloseableIterator>> apply( + IcebergSourceSplit split) { + DataIterator inputIterator = createDataIterator(split); + inputIterator.seek(split.fileOffset(), split.recordOffset()); + return batcher.batch(split.splitId(), inputIterator); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java new file mode 100644 index 000000000000..f143b8d2df2e --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java @@ -0,0 +1,77 @@ +/* + * 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.iceberg.flink.source.reader; + +import java.util.Collection; +import java.util.Collections; +import java.util.Map; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SerializableComparator; +import org.apache.iceberg.flink.source.split.SplitRequestEvent; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +@Internal +public class IcebergSourceReader + extends SingleThreadMultiplexSourceReaderBase< + RecordAndPosition, T, IcebergSourceSplit, IcebergSourceSplit> { + + public IcebergSourceReader( + SerializableRecordEmitter emitter, + IcebergSourceReaderMetrics metrics, + ReaderFunction readerFunction, + SerializableComparator splitComparator, + SourceReaderContext context) { + super( + () -> new IcebergSourceSplitReader<>(metrics, readerFunction, splitComparator, context), + emitter, + context.getConfiguration(), + context); + } + + @Override + public void start() { + // We request a split only if we did not get splits during the checkpoint restore. + // Otherwise, reader restarts will keep requesting more and more splits. + if (getNumberOfCurrentlyAssignedSplits() == 0) { + requestSplit(Collections.emptyList()); + } + } + + @Override + protected void onSplitFinished(Map finishedSplitIds) { + requestSplit(Lists.newArrayList(finishedSplitIds.keySet())); + } + + @Override + protected IcebergSourceSplit initializedState(IcebergSourceSplit split) { + return split; + } + + @Override + protected IcebergSourceSplit toSplitType(String splitId, IcebergSourceSplit splitState) { + return splitState; + } + + private void requestSplit(Collection finishedSplitIds) { + context.sendSourceEventToCoordinator(new SplitRequestEvent(finishedSplitIds)); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java new file mode 100644 index 000000000000..2a3e1dd86b95 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java @@ -0,0 +1,61 @@ +/* + * 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.iceberg.flink.source.reader; + +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.MetricGroup; + +public class IcebergSourceReaderMetrics { + private final Counter assignedSplits; + private final Counter assignedBytes; + private final Counter finishedSplits; + private final Counter finishedBytes; + private final Counter splitReaderFetchCalls; + + public IcebergSourceReaderMetrics(MetricGroup metrics, String fullTableName) { + MetricGroup readerMetrics = + metrics.addGroup("IcebergSourceReader").addGroup("table", fullTableName); + + this.assignedSplits = readerMetrics.counter("assignedSplits"); + this.assignedBytes = readerMetrics.counter("assignedBytes"); + this.finishedSplits = readerMetrics.counter("finishedSplits"); + this.finishedBytes = readerMetrics.counter("finishedBytes"); + this.splitReaderFetchCalls = readerMetrics.counter("splitReaderFetchCalls"); + } + + public void incrementAssignedSplits(long count) { + assignedSplits.inc(count); + } + + public void incrementAssignedBytes(long count) { + assignedBytes.inc(count); + } + + public void incrementFinishedSplits(long count) { + finishedSplits.inc(count); + } + + public void incrementFinishedBytes(long count) { + finishedBytes.inc(count); + } + + public void incrementSplitReaderFetchCalls(long count) { + splitReaderFetchCalls.inc(count); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java new file mode 100644 index 000000000000..4e270dfa3d13 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java @@ -0,0 +1,147 @@ +/* + * 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.iceberg.flink.source.reader; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.ArrayDeque; +import java.util.Collections; +import java.util.List; +import java.util.Queue; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.connector.base.source.reader.RecordsBySplits; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SerializableComparator; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class IcebergSourceSplitReader implements SplitReader, IcebergSourceSplit> { + private static final Logger LOG = LoggerFactory.getLogger(IcebergSourceSplitReader.class); + + private final IcebergSourceReaderMetrics metrics; + private final ReaderFunction openSplitFunction; + private final SerializableComparator splitComparator; + private final int indexOfSubtask; + private final Queue splits; + + private CloseableIterator>> currentReader; + private IcebergSourceSplit currentSplit; + private String currentSplitId; + + IcebergSourceSplitReader( + IcebergSourceReaderMetrics metrics, + ReaderFunction openSplitFunction, + SerializableComparator splitComparator, + SourceReaderContext context) { + this.metrics = metrics; + this.openSplitFunction = openSplitFunction; + this.splitComparator = splitComparator; + this.indexOfSubtask = context.getIndexOfSubtask(); + this.splits = new ArrayDeque<>(); + } + + @Override + public RecordsWithSplitIds> fetch() throws IOException { + metrics.incrementSplitReaderFetchCalls(1); + if (currentReader == null) { + IcebergSourceSplit nextSplit = splits.poll(); + if (nextSplit != null) { + currentSplit = nextSplit; + currentSplitId = nextSplit.splitId(); + currentReader = openSplitFunction.apply(currentSplit); + } else { + // return an empty result, which will lead to split fetch to be idle. + // SplitFetcherManager will then close idle fetcher. + return new RecordsBySplits(Collections.emptyMap(), Collections.emptySet()); + } + } + + if (currentReader.hasNext()) { + // Because Iterator#next() doesn't support checked exception, + // we need to wrap and unwrap the checked IOException with UncheckedIOException + try { + return currentReader.next(); + } catch (UncheckedIOException e) { + throw e.getCause(); + } + } else { + return finishSplit(); + } + } + + @Override + public void handleSplitsChanges(SplitsChange splitsChange) { + if (!(splitsChange instanceof SplitsAddition)) { + throw new UnsupportedOperationException( + String.format("Unsupported split change: %s", splitsChange.getClass())); + } + + if (splitComparator != null) { + List newSplits = Lists.newArrayList(splitsChange.splits()); + newSplits.sort(splitComparator); + LOG.info("Add {} splits to reader: {}", newSplits.size(), newSplits); + splits.addAll(newSplits); + } else { + LOG.info("Add {} splits to reader", splitsChange.splits().size()); + splits.addAll(splitsChange.splits()); + } + metrics.incrementAssignedSplits(splitsChange.splits().size()); + metrics.incrementAssignedBytes(calculateBytes(splitsChange)); + } + + @Override + public void wakeUp() {} + + @Override + public void close() throws Exception { + currentSplitId = null; + if (currentReader != null) { + currentReader.close(); + } + } + + private long calculateBytes(IcebergSourceSplit split) { + return split.task().files().stream().map(FileScanTask::length).reduce(0L, Long::sum); + } + + private long calculateBytes(SplitsChange splitsChanges) { + return splitsChanges.splits().stream().map(this::calculateBytes).reduce(0L, Long::sum); + } + + private ArrayBatchRecords finishSplit() throws IOException { + if (currentReader != null) { + currentReader.close(); + currentReader = null; + } + + ArrayBatchRecords finishRecords = ArrayBatchRecords.finishedSplit(currentSplitId); + LOG.info("Split reader {} finished split: {}", indexOfSubtask, currentSplitId); + metrics.incrementFinishedSplits(1); + metrics.incrementFinishedBytes(calculateBytes(currentSplit)); + currentSplitId = null; + return finishRecords; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java new file mode 100644 index 000000000000..1acb3df76102 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java @@ -0,0 +1,85 @@ +/* + * 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.iceberg.flink.source.reader; + +import java.util.Collections; +import java.util.List; +import java.util.Set; +import javax.annotation.Nullable; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +class ListBatchRecords implements RecordsWithSplitIds> { + private String splitId; + private final List records; + private final Set finishedSplits; + private final RecordAndPosition recordAndPosition; + + // point to current read position within the records list + private int position; + + ListBatchRecords( + String splitId, + List records, + int fileOffset, + long startingRecordOffset, + Set finishedSplits) { + this.splitId = splitId; + this.records = records; + this.finishedSplits = + Preconditions.checkNotNull(finishedSplits, "finishedSplits can be empty but not null"); + this.recordAndPosition = new RecordAndPosition<>(); + this.recordAndPosition.set(null, fileOffset, startingRecordOffset); + + this.position = 0; + } + + @Nullable + @Override + public String nextSplit() { + String nextSplit = this.splitId; + // set the splitId to null to indicate no more splits + // this class only contains record for one split + this.splitId = null; + return nextSplit; + } + + @Nullable + @Override + public RecordAndPosition nextRecordFromSplit() { + if (position < records.size()) { + recordAndPosition.record(records.get(position)); + position++; + return recordAndPosition; + } else { + return null; + } + } + + @Override + public Set finishedSplits() { + return finishedSplits; + } + + public static ListBatchRecords forRecords( + String splitId, List records, int fileOffset, long startingRecordOffset) { + return new ListBatchRecords<>( + splitId, records, fileOffset, startingRecordOffset, Collections.emptySet()); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java new file mode 100644 index 000000000000..365416239d37 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java @@ -0,0 +1,94 @@ +/* + * 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.iceberg.flink.source.reader; + +import java.io.IOException; +import java.util.List; +import java.util.NoSuchElementException; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +/** + * FlinkRecordReaderFunction essentially cloned objects already. So there is no need to use array + * pool to clone objects. Simply create a new ArrayList for each batch. + */ +class ListDataIteratorBatcher implements DataIteratorBatcher { + + private final int batchSize; + + ListDataIteratorBatcher(ReadableConfig config) { + this.batchSize = config.get(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT); + } + + @Override + public CloseableIterator>> batch( + String splitId, DataIterator dataIterator) { + return new ListBatchIterator(splitId, dataIterator); + } + + private class ListBatchIterator + implements CloseableIterator>> { + + private final String splitId; + private final DataIterator inputIterator; + + ListBatchIterator(String splitId, DataIterator inputIterator) { + this.splitId = splitId; + this.inputIterator = inputIterator; + } + + @Override + public boolean hasNext() { + return inputIterator.hasNext(); + } + + @Override + public RecordsWithSplitIds> next() { + if (!inputIterator.hasNext()) { + throw new NoSuchElementException(); + } + + final List batch = Lists.newArrayListWithCapacity(batchSize); + int recordCount = 0; + while (inputIterator.hasNext() && recordCount < batchSize) { + T nextRecord = inputIterator.next(); + batch.add(nextRecord); + recordCount++; + if (!inputIterator.currentFileHasNext()) { + // break early so that records have the same fileOffset. + break; + } + } + + return ListBatchRecords.forRecords( + splitId, batch, inputIterator.fileOffset(), inputIterator.recordOffset() - recordCount); + } + + @Override + public void close() throws IOException { + if (inputIterator != null) { + inputIterator.close(); + } + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java new file mode 100644 index 000000000000..fb4466913b90 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java @@ -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. + */ +package org.apache.iceberg.flink.source.reader; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.DataTaskReader; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** Reading metadata tables (like snapshots, manifests, etc.) */ +@Internal +public class MetaDataReaderFunction extends DataIteratorReaderFunction { + private final Schema readSchema; + private final FileIO io; + private final EncryptionManager encryption; + + public MetaDataReaderFunction( + ReadableConfig config, + Schema tableSchema, + Schema projectedSchema, + FileIO io, + EncryptionManager encryption) { + super( + new ArrayPoolDataIteratorBatcher<>( + config, + new RowDataRecordFactory( + FlinkSchemaUtil.convert(readSchema(tableSchema, projectedSchema))))); + this.readSchema = readSchema(tableSchema, projectedSchema); + this.io = io; + this.encryption = encryption; + } + + @Override + public DataIterator createDataIterator(IcebergSourceSplit split) { + return new DataIterator<>(new DataTaskReader(readSchema), split.task(), io, encryption); + } + + private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { + Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); + return projectedSchema == null ? tableSchema : projectedSchema; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java new file mode 100644 index 000000000000..1ea91f10b4e7 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java @@ -0,0 +1,31 @@ +/* + * 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.iceberg.flink.source.reader; + +import java.io.Serializable; +import java.util.function.Function; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.io.CloseableIterator; + +@FunctionalInterface +public interface ReaderFunction + extends Serializable, + Function< + IcebergSourceSplit, CloseableIterator>>> {} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java new file mode 100644 index 000000000000..6ac92592b6aa --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java @@ -0,0 +1,78 @@ +/* + * 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.iceberg.flink.source.reader; + +import org.apache.flink.annotation.Internal; + +/** + * A record along with the reader position to be stored in the checkpoint. + * + *

    The position defines the point in the reader AFTER the record. Record processing and updating + * checkpointed state happens atomically. The position points to where the reader should resume + * after this record is processed. + * + *

    This mutable object is useful in cases where only one instance of a {@code RecordAndPosition} + * is needed at a time. Then the same instance of RecordAndPosition can be reused. + */ +@Internal +public class RecordAndPosition { + private T record; + private int fileOffset; + private long recordOffset; + + public RecordAndPosition(T record, int fileOffset, long recordOffset) { + this.record = record; + this.fileOffset = fileOffset; + this.recordOffset = recordOffset; + } + + public RecordAndPosition() {} + + // ------------------------------------------------------------------------ + + public T record() { + return record; + } + + public int fileOffset() { + return fileOffset; + } + + public long recordOffset() { + return recordOffset; + } + + /** Updates the record and position in this object. */ + public void set(T newRecord, int newFileOffset, long newRecordOffset) { + this.record = newRecord; + this.fileOffset = newFileOffset; + this.recordOffset = newRecordOffset; + } + + /** Sets the next record of a sequence. This increments the {@code recordOffset} by one. */ + public void record(T nextRecord) { + this.record = nextRecord; + this.recordOffset++; + } + + @Override + public String toString() { + return String.format("%s @ %d + %d", record, fileOffset, recordOffset); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java new file mode 100644 index 000000000000..ef92e2e6b81f --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java @@ -0,0 +1,34 @@ +/* + * 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.iceberg.flink.source.reader; + +import java.io.Serializable; + +/** + * In FLIP-27 source, SplitReader#fetch() returns a batch of records. Since DataIterator for RowData + * returns an iterator of reused RowData objects, RecordFactory is needed to (1) create object array + * that is recyclable via pool. (2) clone RowData element from DataIterator to the batch array. + */ +interface RecordFactory extends Serializable { + /** Create a batch of records */ + T[] createBatch(int batchSize); + + /** Clone record into the specified position of the batch array */ + void clone(T from, T[] batch, int position); +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java new file mode 100644 index 000000000000..5d0a00954e7a --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java @@ -0,0 +1,79 @@ +/* + * 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.iceberg.flink.source.reader; + +import java.util.List; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +public class RowDataReaderFunction extends DataIteratorReaderFunction { + private final Schema tableSchema; + private final Schema readSchema; + private final String nameMapping; + private final boolean caseSensitive; + private final FileIO io; + private final EncryptionManager encryption; + private final List filters; + + public RowDataReaderFunction( + ReadableConfig config, + Schema tableSchema, + Schema projectedSchema, + String nameMapping, + boolean caseSensitive, + FileIO io, + EncryptionManager encryption, + List filters) { + super( + new ArrayPoolDataIteratorBatcher<>( + config, + new RowDataRecordFactory( + FlinkSchemaUtil.convert(readSchema(tableSchema, projectedSchema))))); + this.tableSchema = tableSchema; + this.readSchema = readSchema(tableSchema, projectedSchema); + this.nameMapping = nameMapping; + this.caseSensitive = caseSensitive; + this.io = io; + this.encryption = encryption; + this.filters = filters; + } + + @Override + public DataIterator createDataIterator(IcebergSourceSplit split) { + return new DataIterator<>( + new RowDataFileScanTaskReader(tableSchema, readSchema, nameMapping, caseSensitive, filters), + split.task(), + io, + encryption); + } + + private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { + Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); + return projectedSchema == null ? tableSchema : projectedSchema; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java new file mode 100644 index 000000000000..1e265b2663ce --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java @@ -0,0 +1,61 @@ +/* + * 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.iceberg.flink.source.reader; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.InternalSerializers; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.flink.data.RowDataUtil; + +class RowDataRecordFactory implements RecordFactory { + private final RowType rowType; + private final TypeSerializer[] fieldSerializers; + + RowDataRecordFactory(RowType rowType) { + this.rowType = rowType; + this.fieldSerializers = createFieldSerializers(rowType); + } + + static TypeSerializer[] createFieldSerializers(RowType rowType) { + return rowType.getChildren().stream() + .map(InternalSerializers::create) + .toArray(TypeSerializer[]::new); + } + + @Override + public RowData[] createBatch(int batchSize) { + RowData[] arr = new RowData[batchSize]; + for (int i = 0; i < batchSize; ++i) { + arr[i] = new GenericRowData(rowType.getFieldCount()); + } + return arr; + } + + @Override + public void clone(RowData from, RowData[] batch, int position) { + // Set the return value from RowDataUtil.clone back to the array. + // Clone method returns same clone target object (reused) if it is a GenericRowData. + // Clone method will allocate a new GenericRowData object + // if the target object is NOT a GenericRowData. + // So we should always set the clone return value back to the array. + batch[position] = RowDataUtil.clone(from, batch[position], rowType, fieldSerializers); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java new file mode 100644 index 000000000000..a6e2c1dae243 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.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.iceberg.flink.source.reader; + +import java.io.Serializable; +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.base.source.reader.RecordEmitter; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; + +@Internal +@FunctionalInterface +public interface SerializableRecordEmitter + extends RecordEmitter, T, IcebergSourceSplit>, Serializable { + static SerializableRecordEmitter defaultEmitter() { + return (element, output, split) -> { + output.collect(element.record()); + split.updatePosition(element.fileOffset(), element.recordOffset()); + }; + } + + static SerializableRecordEmitter emitterWithWatermark(SplitWatermarkExtractor extractor) { + return new WatermarkExtractorRecordEmitter<>(extractor); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java new file mode 100644 index 000000000000..d1c50ac8ca52 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java @@ -0,0 +1,28 @@ +/* + * 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.iceberg.flink.source.reader; + +import java.io.Serializable; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; + +/** The interface used to extract watermarks from splits. */ +public interface SplitWatermarkExtractor extends Serializable { + /** Get the watermark for a split. */ + long extractWatermark(IcebergSourceSplit split); +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java new file mode 100644 index 000000000000..02ef57d344b1 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import org.apache.flink.api.common.eventtime.Watermark; +import org.apache.flink.api.connector.source.SourceOutput; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Emitter which emits the watermarks, records and updates the split position. + * + *

    The Emitter emits watermarks at the beginning of every split provided by the {@link + * SplitWatermarkExtractor}. + */ +class WatermarkExtractorRecordEmitter implements SerializableRecordEmitter { + private static final Logger LOG = LoggerFactory.getLogger(WatermarkExtractorRecordEmitter.class); + private final SplitWatermarkExtractor timeExtractor; + private String lastSplitId = null; + private long watermark; + + WatermarkExtractorRecordEmitter(SplitWatermarkExtractor timeExtractor) { + this.timeExtractor = timeExtractor; + } + + @Override + public void emitRecord( + RecordAndPosition element, SourceOutput output, IcebergSourceSplit split) { + if (!split.splitId().equals(lastSplitId)) { + long newWatermark = timeExtractor.extractWatermark(split); + if (newWatermark < watermark) { + LOG.info( + "Received a new split with lower watermark. Previous watermark = {}, current watermark = {}, previous split = {}, current split = {}", + watermark, + newWatermark, + lastSplitId, + split.splitId()); + } else { + watermark = newWatermark; + output.emitWatermark(new Watermark(watermark)); + LOG.debug("Watermark = {} emitted based on split = {}", watermark, lastSplitId); + } + + lastSplitId = split.splitId(); + } + + output.collect(element.record()); + split.updatePosition(element.fileOffset(), element.recordOffset()); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java new file mode 100644 index 000000000000..e4bfbf1452e2 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java @@ -0,0 +1,177 @@ +/* + * 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.iceberg.flink.source.split; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Collection; +import java.util.List; +import java.util.stream.Collectors; +import javax.annotation.Nullable; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.util.InstantiationUtil; +import org.apache.iceberg.BaseCombinedScanTask; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.FileScanTaskParser; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +@Internal +public class IcebergSourceSplit implements SourceSplit, Serializable { + private static final long serialVersionUID = 1L; + private static final ThreadLocal SERIALIZER_CACHE = + ThreadLocal.withInitial(() -> new DataOutputSerializer(1024)); + + private final CombinedScanTask task; + + private int fileOffset; + private long recordOffset; + + // The splits are frequently serialized into checkpoints. + // Caching the byte representation makes repeated serialization cheap. + @Nullable private transient byte[] serializedBytesCache; + + private IcebergSourceSplit(CombinedScanTask task, int fileOffset, long recordOffset) { + this.task = task; + this.fileOffset = fileOffset; + this.recordOffset = recordOffset; + } + + public static IcebergSourceSplit fromCombinedScanTask(CombinedScanTask combinedScanTask) { + return fromCombinedScanTask(combinedScanTask, 0, 0L); + } + + public static IcebergSourceSplit fromCombinedScanTask( + CombinedScanTask combinedScanTask, int fileOffset, long recordOffset) { + return new IcebergSourceSplit(combinedScanTask, fileOffset, recordOffset); + } + + public CombinedScanTask task() { + return task; + } + + public int fileOffset() { + return fileOffset; + } + + public long recordOffset() { + return recordOffset; + } + + @Override + public String splitId() { + return MoreObjects.toStringHelper(this).add("files", toString(task.files())).toString(); + } + + public void updatePosition(int newFileOffset, long newRecordOffset) { + // invalidate the cache after position change + serializedBytesCache = null; + fileOffset = newFileOffset; + recordOffset = newRecordOffset; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("files", toString(task.files())) + .add("fileOffset", fileOffset) + .add("recordOffset", recordOffset) + .toString(); + } + + private String toString(Collection files) { + return Iterables.toString( + files.stream() + .map( + fileScanTask -> + MoreObjects.toStringHelper(fileScanTask) + .add("file", fileScanTask.file().path().toString()) + .add("start", fileScanTask.start()) + .add("length", fileScanTask.length()) + .toString()) + .collect(Collectors.toList())); + } + + byte[] serializeV1() throws IOException { + if (serializedBytesCache == null) { + serializedBytesCache = InstantiationUtil.serializeObject(this); + } + + return serializedBytesCache; + } + + static IcebergSourceSplit deserializeV1(byte[] serialized) throws IOException { + try { + return InstantiationUtil.deserializeObject( + serialized, IcebergSourceSplit.class.getClassLoader()); + } catch (ClassNotFoundException e) { + throw new RuntimeException("Failed to deserialize the split.", e); + } + } + + byte[] serializeV2() throws IOException { + if (serializedBytesCache == null) { + DataOutputSerializer out = SERIALIZER_CACHE.get(); + Collection fileScanTasks = task.tasks(); + Preconditions.checkArgument( + fileOffset >= 0 && fileOffset < fileScanTasks.size(), + "Invalid file offset: %s. Should be within the range of [0, %s)", + fileOffset, + fileScanTasks.size()); + + out.writeInt(fileOffset); + out.writeLong(recordOffset); + out.writeInt(fileScanTasks.size()); + + for (FileScanTask fileScanTask : fileScanTasks) { + String taskJson = FileScanTaskParser.toJson(fileScanTask); + out.writeUTF(taskJson); + } + + serializedBytesCache = out.getCopyOfBuffer(); + out.clear(); + } + + return serializedBytesCache; + } + + static IcebergSourceSplit deserializeV2(byte[] serialized, boolean caseSensitive) + throws IOException { + DataInputDeserializer in = new DataInputDeserializer(serialized); + int fileOffset = in.readInt(); + long recordOffset = in.readLong(); + int taskCount = in.readInt(); + + List tasks = Lists.newArrayListWithCapacity(taskCount); + for (int i = 0; i < taskCount; ++i) { + String taskJson = in.readUTF(); + FileScanTask task = FileScanTaskParser.fromJson(taskJson, caseSensitive); + tasks.add(task); + } + + CombinedScanTask combinedScanTask = new BaseCombinedScanTask(tasks); + return IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, fileOffset, recordOffset); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java new file mode 100644 index 000000000000..8c089819e731 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java @@ -0,0 +1,60 @@ +/* + * 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.iceberg.flink.source.split; + +import java.io.IOException; +import org.apache.flink.annotation.Internal; +import org.apache.flink.core.io.SimpleVersionedSerializer; + +@Internal +public class IcebergSourceSplitSerializer implements SimpleVersionedSerializer { + private static final int VERSION = 2; + + private final boolean caseSensitive; + + public IcebergSourceSplitSerializer(boolean caseSensitive) { + this.caseSensitive = caseSensitive; + } + + @Override + public int getVersion() { + return VERSION; + } + + @Override + public byte[] serialize(IcebergSourceSplit split) throws IOException { + return split.serializeV2(); + } + + @Override + public IcebergSourceSplit deserialize(int version, byte[] serialized) throws IOException { + switch (version) { + case 1: + return IcebergSourceSplit.deserializeV1(serialized); + case 2: + return IcebergSourceSplit.deserializeV2(serialized, caseSensitive); + default: + throw new IOException( + String.format( + "Failed to deserialize IcebergSourceSplit. " + + "Encountered unsupported version: %d. Supported version are [1]", + version)); + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java new file mode 100644 index 000000000000..d9061e049e00 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.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.iceberg.flink.source.split; + +public class IcebergSourceSplitState { + private final IcebergSourceSplit split; + private final IcebergSourceSplitStatus status; + + public IcebergSourceSplitState(IcebergSourceSplit split, IcebergSourceSplitStatus status) { + this.split = split; + this.status = status; + } + + public IcebergSourceSplit split() { + return split; + } + + public IcebergSourceSplitStatus status() { + return status; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java new file mode 100644 index 000000000000..d4a84a165e1a --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java @@ -0,0 +1,25 @@ +/* + * 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.iceberg.flink.source.split; + +public enum IcebergSourceSplitStatus { + UNASSIGNED, + ASSIGNED, + COMPLETED +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java new file mode 100644 index 000000000000..319648ca275c --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java @@ -0,0 +1,24 @@ +/* + * 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.iceberg.flink.source.split; + +import java.io.Serializable; +import java.util.Comparator; + +public interface SerializableComparator extends Comparator, Serializable {} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java new file mode 100644 index 000000000000..56ee92014d12 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.split; + +import org.apache.iceberg.flink.source.reader.SplitWatermarkExtractor; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * Provides implementations of {@link org.apache.iceberg.flink.source.split.SerializableComparator} + * which could be used for ordering splits. These are used by the {@link + * org.apache.iceberg.flink.source.assigner.OrderedSplitAssignerFactory} and the {@link + * org.apache.iceberg.flink.source.reader.IcebergSourceReader} + */ +public class SplitComparators { + private SplitComparators() {} + + /** Comparator which orders the splits based on the file sequence number of the data files */ + public static SerializableComparator fileSequenceNumber() { + return (IcebergSourceSplit o1, IcebergSourceSplit o2) -> { + Preconditions.checkArgument( + o1.task().files().size() == 1 && o2.task().files().size() == 1, + "Could not compare combined task. Please use 'split-open-file-cost' to prevent combining multiple files to a split"); + + Long seq1 = o1.task().files().iterator().next().file().fileSequenceNumber(); + Long seq2 = o2.task().files().iterator().next().file().fileSequenceNumber(); + + Preconditions.checkNotNull( + seq1, + "Invalid file sequence number: null. Doesn't support splits written with V1 format: %s", + o1); + Preconditions.checkNotNull( + seq2, + "Invalid file sequence number: null. Doesn't support splits written with V1 format: %s", + o2); + + int temp = Long.compare(seq1, seq2); + if (temp != 0) { + return temp; + } else { + return o1.splitId().compareTo(o2.splitId()); + } + }; + } + + /** Comparator which orders the splits based on watermark of the splits */ + public static SerializableComparator watermark( + SplitWatermarkExtractor watermarkExtractor) { + return (IcebergSourceSplit o1, IcebergSourceSplit o2) -> { + long watermark1 = watermarkExtractor.extractWatermark(o1); + long watermark2 = watermarkExtractor.extractWatermark(o2); + + int temp = Long.compare(watermark1, watermark2); + if (temp != 0) { + return temp; + } else { + return o1.splitId().compareTo(o2.splitId()); + } + }; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java new file mode 100644 index 000000000000..eabd757aa638 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.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.iceberg.flink.source.split; + +import java.util.Collection; +import java.util.Collections; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SourceEvent; + +/** We can remove this class once FLINK-21364 is resolved. */ +@Internal +public class SplitRequestEvent implements SourceEvent { + private static final long serialVersionUID = 1L; + + private final Collection finishedSplitIds; + private final String requesterHostname; + + public SplitRequestEvent() { + this(Collections.emptyList()); + } + + public SplitRequestEvent(Collection finishedSplitIds) { + this(finishedSplitIds, null); + } + + public SplitRequestEvent(Collection finishedSplitIds, String requesterHostname) { + this.finishedSplitIds = finishedSplitIds; + this.requesterHostname = requesterHostname; + } + + public Collection finishedSplitIds() { + return finishedSplitIds; + } + + public String requesterHostname() { + return requesterHostname; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java new file mode 100644 index 000000000000..f0b9bf64fb1a --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java @@ -0,0 +1,246 @@ +/* + * 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.iceberg.flink.util; + +import java.util.List; +import java.util.Map; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.TableChange; +import org.apache.flink.table.catalog.UniqueConstraint; +import org.apache.iceberg.Table; +import org.apache.iceberg.Transaction; +import org.apache.iceberg.UpdateProperties; +import org.apache.iceberg.UpdateSchema; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.types.Type; + +public class FlinkAlterTableUtil { + private FlinkAlterTableUtil() {} + + public static void commitChanges( + Table table, + String setLocation, + String setSnapshotId, + String pickSnapshotId, + Map setProperties) { + commitManageSnapshots(table, setSnapshotId, pickSnapshotId); + + Transaction transaction = table.newTransaction(); + + if (setLocation != null) { + transaction.updateLocation().setLocation(setLocation).commit(); + } + + if (!setProperties.isEmpty()) { + UpdateProperties updateProperties = transaction.updateProperties(); + setProperties.forEach( + (k, v) -> { + if (v == null) { + updateProperties.remove(k); + } else { + updateProperties.set(k, v); + } + }); + updateProperties.commit(); + } + + transaction.commitTransaction(); + } + + public static void commitChanges( + Table table, + String setLocation, + String setSnapshotId, + String pickSnapshotId, + List schemaChanges, + List propertyChanges) { + commitManageSnapshots(table, setSnapshotId, pickSnapshotId); + + Transaction transaction = table.newTransaction(); + + if (setLocation != null) { + transaction.updateLocation().setLocation(setLocation).commit(); + } + + if (!schemaChanges.isEmpty()) { + UpdateSchema updateSchema = transaction.updateSchema(); + FlinkAlterTableUtil.applySchemaChanges(updateSchema, schemaChanges); + updateSchema.commit(); + } + + if (!propertyChanges.isEmpty()) { + UpdateProperties updateProperties = transaction.updateProperties(); + FlinkAlterTableUtil.applyPropertyChanges(updateProperties, propertyChanges); + updateProperties.commit(); + } + + transaction.commitTransaction(); + } + + public static void commitManageSnapshots( + Table table, String setSnapshotId, String cherrypickSnapshotId) { + // don't allow setting the snapshot and picking a commit at the same time because order is + // ambiguous and choosing one order leads to different results + Preconditions.checkArgument( + setSnapshotId == null || cherrypickSnapshotId == null, + "Cannot set the current snapshot ID and cherry-pick snapshot changes"); + + if (setSnapshotId != null) { + long newSnapshotId = Long.parseLong(setSnapshotId); + table.manageSnapshots().setCurrentSnapshot(newSnapshotId).commit(); + } + + // if updating the table snapshot, perform that update first in case it fails + if (cherrypickSnapshotId != null) { + long newSnapshotId = Long.parseLong(cherrypickSnapshotId); + table.manageSnapshots().cherrypick(newSnapshotId).commit(); + } + } + + /** + * Applies a list of Flink table changes to an {@link UpdateSchema} operation. + * + * @param pendingUpdate an uncommitted UpdateSchema operation to configure + * @param schemaChanges a list of Flink table changes + */ + public static void applySchemaChanges( + UpdateSchema pendingUpdate, List schemaChanges) { + for (TableChange change : schemaChanges) { + if (change instanceof TableChange.AddColumn) { + TableChange.AddColumn addColumn = (TableChange.AddColumn) change; + Column flinkColumn = addColumn.getColumn(); + Preconditions.checkArgument( + FlinkCompatibilityUtil.isPhysicalColumn(flinkColumn), + "Unsupported table change: Adding computed column %s.", + flinkColumn.getName()); + Type icebergType = FlinkSchemaUtil.convert(flinkColumn.getDataType().getLogicalType()); + if (flinkColumn.getDataType().getLogicalType().isNullable()) { + pendingUpdate.addColumn(flinkColumn.getName(), icebergType); + } else { + pendingUpdate.addRequiredColumn(flinkColumn.getName(), icebergType); + } + } else if (change instanceof TableChange.ModifyColumn) { + TableChange.ModifyColumn modifyColumn = (TableChange.ModifyColumn) change; + applyModifyColumn(pendingUpdate, modifyColumn); + } else if (change instanceof TableChange.DropColumn) { + TableChange.DropColumn dropColumn = (TableChange.DropColumn) change; + pendingUpdate.deleteColumn(dropColumn.getColumnName()); + } else if (change instanceof TableChange.AddWatermark) { + throw new UnsupportedOperationException("Unsupported table change: AddWatermark."); + } else if (change instanceof TableChange.ModifyWatermark) { + throw new UnsupportedOperationException("Unsupported table change: ModifyWatermark."); + } else if (change instanceof TableChange.DropWatermark) { + throw new UnsupportedOperationException("Unsupported table change: DropWatermark."); + } else if (change instanceof TableChange.AddUniqueConstraint) { + TableChange.AddUniqueConstraint addPk = (TableChange.AddUniqueConstraint) change; + applyUniqueConstraint(pendingUpdate, addPk.getConstraint()); + } else if (change instanceof TableChange.ModifyUniqueConstraint) { + TableChange.ModifyUniqueConstraint modifyPk = (TableChange.ModifyUniqueConstraint) change; + applyUniqueConstraint(pendingUpdate, modifyPk.getNewConstraint()); + } else if (change instanceof TableChange.DropConstraint) { + throw new UnsupportedOperationException("Unsupported table change: DropConstraint."); + } else { + throw new UnsupportedOperationException("Cannot apply unknown table change: " + change); + } + } + } + + /** + * Applies a list of Flink table property changes to an {@link UpdateProperties} operation. + * + * @param pendingUpdate an uncommitted UpdateProperty operation to configure + * @param propertyChanges a list of Flink table changes + */ + public static void applyPropertyChanges( + UpdateProperties pendingUpdate, List propertyChanges) { + for (TableChange change : propertyChanges) { + if (change instanceof TableChange.SetOption) { + TableChange.SetOption setOption = (TableChange.SetOption) change; + pendingUpdate.set(setOption.getKey(), setOption.getValue()); + } else if (change instanceof TableChange.ResetOption) { + TableChange.ResetOption resetOption = (TableChange.ResetOption) change; + pendingUpdate.remove(resetOption.getKey()); + } else { + throw new UnsupportedOperationException( + "The given table change is not a property change: " + change); + } + } + } + + private static void applyModifyColumn( + UpdateSchema pendingUpdate, TableChange.ModifyColumn modifyColumn) { + if (modifyColumn instanceof TableChange.ModifyColumnName) { + TableChange.ModifyColumnName modifyName = (TableChange.ModifyColumnName) modifyColumn; + pendingUpdate.renameColumn(modifyName.getOldColumnName(), modifyName.getNewColumnName()); + } else if (modifyColumn instanceof TableChange.ModifyColumnPosition) { + TableChange.ModifyColumnPosition modifyPosition = + (TableChange.ModifyColumnPosition) modifyColumn; + applyModifyColumnPosition(pendingUpdate, modifyPosition); + } else if (modifyColumn instanceof TableChange.ModifyPhysicalColumnType) { + TableChange.ModifyPhysicalColumnType modifyType = + (TableChange.ModifyPhysicalColumnType) modifyColumn; + Type type = FlinkSchemaUtil.convert(modifyType.getNewType().getLogicalType()); + String columnName = modifyType.getOldColumn().getName(); + pendingUpdate.updateColumn(columnName, type.asPrimitiveType()); + if (modifyType.getNewColumn().getDataType().getLogicalType().isNullable()) { + pendingUpdate.makeColumnOptional(columnName); + } else { + pendingUpdate.requireColumn(columnName); + } + } else if (modifyColumn instanceof TableChange.ModifyColumnComment) { + TableChange.ModifyColumnComment modifyComment = + (TableChange.ModifyColumnComment) modifyColumn; + pendingUpdate.updateColumnDoc( + modifyComment.getOldColumn().getName(), modifyComment.getNewComment()); + } else { + throw new UnsupportedOperationException( + "Cannot apply unknown modify-column change: " + modifyColumn); + } + } + + private static void applyModifyColumnPosition( + UpdateSchema pendingUpdate, TableChange.ModifyColumnPosition modifyColumnPosition) { + TableChange.ColumnPosition newPosition = modifyColumnPosition.getNewPosition(); + if (newPosition instanceof TableChange.First) { + pendingUpdate.moveFirst(modifyColumnPosition.getOldColumn().getName()); + } else if (newPosition instanceof TableChange.After) { + TableChange.After after = (TableChange.After) newPosition; + pendingUpdate.moveAfter(modifyColumnPosition.getOldColumn().getName(), after.column()); + } else { + throw new UnsupportedOperationException( + "Cannot apply unknown modify-column-position change: " + modifyColumnPosition); + } + } + + private static void applyUniqueConstraint( + UpdateSchema pendingUpdate, UniqueConstraint constraint) { + switch (constraint.getType()) { + case PRIMARY_KEY: + pendingUpdate.setIdentifierFields(constraint.getColumns()); + break; + case UNIQUE_KEY: + throw new UnsupportedOperationException( + "Unsupported table change: setting unique key constraints."); + default: + throw new UnsupportedOperationException( + "Cannot apply unknown unique constraint: " + constraint.getType().name()); + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java new file mode 100644 index 000000000000..f02af894e82b --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java @@ -0,0 +1,47 @@ +/* + * 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.iceberg.flink.util; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.table.api.TableColumn; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.logical.RowType; + +/** + * This is a small util class that try to hide calls to Flink Internal or PublicEvolve interfaces as + * Flink can change those APIs during minor version release. + */ +public class FlinkCompatibilityUtil { + + private FlinkCompatibilityUtil() {} + + public static TypeInformation toTypeInfo(RowType rowType) { + return InternalTypeInfo.of(rowType); + } + + public static boolean isPhysicalColumn(TableColumn column) { + return column.isPhysical(); + } + + public static boolean isPhysicalColumn(Column column) { + return column.isPhysical(); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java new file mode 100644 index 000000000000..00d74d8d345c --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java @@ -0,0 +1,33 @@ +/* + * 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.iceberg.flink.util; + +import org.apache.flink.streaming.api.datastream.DataStream; + +public class FlinkPackage { + /** Choose {@link DataStream} class because it is one of the core Flink API. */ + private static final String VERSION = DataStream.class.getPackage().getImplementationVersion(); + + private FlinkPackage() {} + + /** Returns Flink version string like x.y.z */ + public static String version() { + return VERSION; + } +} diff --git a/flink/v1.17/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink/v1.17/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory new file mode 100644 index 000000000000..29a9955a7e20 --- /dev/null +++ b/flink/v1.17/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -0,0 +1,16 @@ +# 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. + +org.apache.iceberg.flink.FlinkDynamicTableFactory diff --git a/flink/v1.17/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink/v1.17/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory new file mode 100644 index 000000000000..2b6bfa3cd579 --- /dev/null +++ b/flink/v1.17/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory @@ -0,0 +1,16 @@ +# 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. + +org.apache.iceberg.flink.FlinkCatalogFactory diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java new file mode 100644 index 000000000000..47319ec9bc60 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java @@ -0,0 +1,90 @@ +/* + * 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.iceberg.flink; + +import org.junit.Test; + +public abstract class AvroGenericRecordConverterBase { + protected abstract void testConverter(DataGenerator dataGenerator) throws Exception; + + @Test + public void testPrimitiveTypes() throws Exception { + testConverter(new DataGenerators.Primitives()); + } + + @Test + public void testStructOfPrimitive() throws Exception { + testConverter(new DataGenerators.StructOfPrimitive()); + } + + @Test + public void testStructOfArray() throws Exception { + testConverter(new DataGenerators.StructOfArray()); + } + + @Test + public void testStructOfMap() throws Exception { + testConverter(new DataGenerators.StructOfMap()); + } + + @Test + public void testStructOfStruct() throws Exception { + testConverter(new DataGenerators.StructOfStruct()); + } + + @Test + public void testArrayOfPrimitive() throws Exception { + testConverter(new DataGenerators.ArrayOfPrimitive()); + } + + @Test + public void testArrayOfArray() throws Exception { + testConverter(new DataGenerators.ArrayOfArray()); + } + + @Test + public void testArrayOfMap() throws Exception { + testConverter(new DataGenerators.ArrayOfMap()); + } + + @Test + public void testArrayOfStruct() throws Exception { + testConverter(new DataGenerators.ArrayOfStruct()); + } + + @Test + public void testMapOfPrimitives() throws Exception { + testConverter(new DataGenerators.MapOfPrimitives()); + } + + @Test + public void testMapOfArray() throws Exception { + testConverter(new DataGenerators.MapOfArray()); + } + + @Test + public void testMapOfMap() throws Exception { + testConverter(new DataGenerators.MapOfMap()); + } + + @Test + public void testMapOfStruct() throws Exception { + testConverter(new DataGenerators.MapOfStruct()); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java new file mode 100644 index 000000000000..b1e3b20ff7ac --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java @@ -0,0 +1,42 @@ +/* + * 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.iceberg.flink; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.GenericRecord; + +/** + * This interface defines test data generator. Different implementations for primitive and complex + * nested fields are defined in {@link DataGenerators}. + */ +public interface DataGenerator { + Schema icebergSchema(); + + RowType flinkRowType(); + + org.apache.avro.Schema avroSchema(); + + GenericRecord generateIcebergGenericRecord(); + + GenericRowData generateFlinkRowData(); + + org.apache.avro.generic.GenericRecord generateAvroGenericRecord(); +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java new file mode 100644 index 000000000000..e2cd411d7069 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java @@ -0,0 +1,1172 @@ +/* + * 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.iceberg.flink; + +import static org.apache.iceberg.types.Types.NestedField.required; + +import com.fasterxml.jackson.databind.node.IntNode; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.util.Arrays; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import org.apache.avro.LogicalTypes; +import org.apache.avro.SchemaBuilder; +import org.apache.avro.generic.GenericData; +import org.apache.avro.util.Utf8; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; +import org.joda.time.DateTime; +import org.joda.time.DateTimeZone; +import org.joda.time.Days; + +/** + * Util class to generate test data with extensive coverage different field types: from primitives + * to complex nested types. + */ +public class DataGenerators { + + public static class Primitives implements DataGenerator { + private static final DateTime JODA_DATETIME_EPOC = + new DateTime(1970, 1, 1, 0, 0, 0, 0, DateTimeZone.UTC); + private static final DateTime JODA_DATETIME_20220110 = + new DateTime(2022, 1, 10, 0, 0, 0, 0, DateTimeZone.UTC); + private static final int DAYS_BTW_EPOC_AND_20220110 = + Days.daysBetween(JODA_DATETIME_EPOC, JODA_DATETIME_20220110).getDays(); + private static final int HOUR_8_IN_MILLI = (int) TimeUnit.HOURS.toMillis(8); + + private static final LocalDate JAVA_LOCAL_DATE_20220110 = LocalDate.of(2022, 1, 10); + private static final LocalTime JAVA_LOCAL_TIME_HOUR8 = LocalTime.of(8, 0); + private static final OffsetDateTime JAVA_OFFSET_DATE_TIME_20220110 = + OffsetDateTime.of(2022, 1, 10, 0, 0, 0, 0, ZoneOffset.UTC); + private static final LocalDateTime JAVA_LOCAL_DATE_TIME_20220110 = + LocalDateTime.of(2022, 1, 10, 0, 0, 0); + private static final BigDecimal BIG_DECIMAL_NEGATIVE = new BigDecimal("-1.50"); + private static final byte[] FIXED_BYTES = "012345689012345".getBytes(StandardCharsets.UTF_8); + + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + // primitive types + Types.NestedField.optional(2, "boolean_field", Types.BooleanType.get()), + Types.NestedField.optional(3, "int_field", Types.IntegerType.get()), + Types.NestedField.optional(4, "long_field", Types.LongType.get()), + Types.NestedField.optional(5, "float_field", Types.FloatType.get()), + Types.NestedField.optional(6, "double_field", Types.DoubleType.get()), + Types.NestedField.required(7, "string_field", Types.StringType.get()), + Types.NestedField.required(8, "date_field", Types.DateType.get()), + Types.NestedField.required(9, "time_field", Types.TimeType.get()), + Types.NestedField.required(10, "ts_with_zone_field", Types.TimestampType.withZone()), + Types.NestedField.required( + 11, "ts_without_zone_field", Types.TimestampType.withoutZone()), + Types.NestedField.required(12, "uuid_field", Types.UUIDType.get()), + Types.NestedField.required(13, "binary_field", Types.BinaryType.get()), + Types.NestedField.required(14, "decimal_field", Types.DecimalType.of(9, 2)), + Types.NestedField.required(15, "fixed_field", Types.FixedType.ofLength(16))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + /** + * Fix up Avro Schema that is converted from Iceberg Schema. + * + * @param schemaConvertedFromIceberg Avro Schema converted from Iceberg schema via {@link + * AvroSchemaUtil#convert(Schema, String)} + */ + private org.apache.avro.Schema fixupAvroSchemaConvertedFromIcebergSchema( + org.apache.avro.Schema schemaConvertedFromIceberg) { + List fixedFields = + schemaConvertedFromIceberg.getFields().stream() + .map( + field -> { + org.apache.avro.Schema.Field updatedField = field; + if (field.name().equals("time_field")) { + // Iceberg's AvroSchemaUtil uses timestamp-micros with Long value for time + // field, while AvroToRowDataConverters#convertToTime() always looks for + // Integer value assuming millis. The root problem is that + // AvroToRowDataConverters#createConverter() uses LogicalTypeRoot to + // determine converter and LogicalTypeRoot lost the timestamp precision + // carried by LogicalType like Time(6). + org.apache.avro.Schema fieldSchema = + LogicalTypes.timeMillis() + .addToSchema( + org.apache.avro.Schema.create(org.apache.avro.Schema.Type.INT)); + updatedField = new org.apache.avro.Schema.Field("time_field", fieldSchema); + } + + return new org.apache.avro.Schema.Field(updatedField, updatedField.schema()); + }) + .collect(Collectors.toList()); + return org.apache.avro.Schema.createRecord( + schemaConvertedFromIceberg.getName(), + schemaConvertedFromIceberg.getDoc(), + schemaConvertedFromIceberg.getNamespace(), + schemaConvertedFromIceberg.isError(), + fixedFields); + } + + private final org.apache.avro.Schema avroSchema = + fixupAvroSchemaConvertedFromIcebergSchema(AvroSchemaUtil.convert(icebergSchema, "table")); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField("boolean_field", false); + genericRecord.setField("int_field", Integer.MAX_VALUE); + genericRecord.setField("long_field", Long.MAX_VALUE); + genericRecord.setField("float_field", Float.MAX_VALUE); + genericRecord.setField("double_field", Double.MAX_VALUE); + genericRecord.setField("string_field", "str"); + + genericRecord.setField("date_field", JAVA_LOCAL_DATE_20220110); + genericRecord.setField("time_field", JAVA_LOCAL_TIME_HOUR8); + genericRecord.setField("ts_with_zone_field", JAVA_OFFSET_DATE_TIME_20220110); + genericRecord.setField("ts_without_zone_field", JAVA_LOCAL_DATE_TIME_20220110); + + byte[] uuidBytes = new byte[16]; + for (int i = 0; i < 16; ++i) { + uuidBytes[i] = (byte) i; + } + + genericRecord.setField("uuid_field", UUID.nameUUIDFromBytes(uuidBytes)); + + byte[] binaryBytes = new byte[7]; + for (int i = 0; i < 7; ++i) { + binaryBytes[i] = (byte) i; + } + genericRecord.setField("binary_field", ByteBuffer.wrap(binaryBytes)); + + genericRecord.setField("decimal_field", BIG_DECIMAL_NEGATIVE); + genericRecord.setField("fixed_field", FIXED_BYTES); + + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + byte[] uuidBytes = new byte[16]; + for (int i = 0; i < 16; ++i) { + uuidBytes[i] = (byte) i; + } + + byte[] binaryBytes = new byte[7]; + for (int i = 0; i < 7; ++i) { + binaryBytes[i] = (byte) i; + } + + return GenericRowData.of( + StringData.fromString("row_id_value"), + false, + Integer.MAX_VALUE, + Long.MAX_VALUE, + Float.MAX_VALUE, + Double.MAX_VALUE, + StringData.fromString("str"), + DAYS_BTW_EPOC_AND_20220110, + HOUR_8_IN_MILLI, + // Although Avro logical type for timestamp fields are in micro seconds, + // AvroToRowDataConverters only looks for long value in milliseconds. + TimestampData.fromEpochMillis(JODA_DATETIME_20220110.getMillis()), + TimestampData.fromEpochMillis(JODA_DATETIME_20220110.getMillis()), + uuidBytes, + binaryBytes, + DecimalData.fromBigDecimal(BIG_DECIMAL_NEGATIVE, 9, 2), + FIXED_BYTES); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", new Utf8("row_id_value")); + genericRecord.put("boolean_field", false); + genericRecord.put("int_field", Integer.MAX_VALUE); + genericRecord.put("long_field", Long.MAX_VALUE); + genericRecord.put("float_field", Float.MAX_VALUE); + genericRecord.put("double_field", Double.MAX_VALUE); + genericRecord.put("string_field", new Utf8("str")); + + genericRecord.put("date_field", DAYS_BTW_EPOC_AND_20220110); + genericRecord.put("time_field", HOUR_8_IN_MILLI); + // Although Avro logical type for timestamp fields are in micro seconds, + // AvroToRowDataConverters only looks for long value in milliseconds. + genericRecord.put("ts_with_zone_field", JODA_DATETIME_20220110.getMillis()); + genericRecord.put("ts_without_zone_field", JODA_DATETIME_20220110.getMillis()); + + byte[] uuidBytes = new byte[16]; + for (int i = 0; i < 16; ++i) { + uuidBytes[i] = (byte) i; + } + genericRecord.put("uuid_field", ByteBuffer.wrap(uuidBytes)); + + byte[] binaryBytes = new byte[7]; + for (int i = 0; i < 7; ++i) { + binaryBytes[i] = (byte) i; + } + genericRecord.put("binary_field", ByteBuffer.wrap(binaryBytes)); + + BigDecimal bigDecimal = new BigDecimal("-1.50"); + // unscaledValue().toByteArray() is to match the behavior of RowDataToAvroConverters from + // Flink for decimal type + genericRecord.put("decimal_field", ByteBuffer.wrap(bigDecimal.unscaledValue().toByteArray())); + + genericRecord.put("fixed_field", ByteBuffer.wrap(FIXED_BYTES)); + + return genericRecord; + } + } + + public static class StructOfPrimitive implements DataGenerator { + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.required( + 2, + "struct_of_primitive", + Types.StructType.of( + required(101, "id", Types.IntegerType.get()), + required(102, "name", Types.StringType.get())))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + private final org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(icebergSchema, "table"); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + Schema structSchema = + new Schema(icebergSchema.findField("struct_of_primitive").type().asStructType().fields()); + GenericRecord struct = GenericRecord.create(structSchema); + struct.setField("id", 1); + struct.setField("name", "Jane"); + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField("struct_of_primitive", struct); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + return GenericRowData.of( + StringData.fromString("row_id_value"), + GenericRowData.of(1, StringData.fromString("Jane"))); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.Schema structSchema = avroSchema.getField("struct_of_primitive").schema(); + org.apache.avro.generic.GenericRecord struct = new GenericData.Record(structSchema); + struct.put("id", 1); + struct.put("name", "Jane"); + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", "row_id_value"); + genericRecord.put("struct_of_primitive", struct); + return genericRecord; + } + } + + public static class StructOfArray implements DataGenerator { + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.required( + 2, + "struct_of_array", + Types.StructType.of( + required(101, "id", Types.IntegerType.get()), + required( + 102, "names", Types.ListType.ofRequired(201, Types.StringType.get()))))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + private final org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(icebergSchema, "table"); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + Schema structSchema = + new Schema(icebergSchema.findField("struct_of_array").type().asStructType().fields()); + GenericRecord struct = GenericRecord.create(structSchema); + struct.setField("id", 1); + struct.setField("names", Arrays.asList("Jane", "Joe")); + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField("struct_of_array", struct); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + StringData[] names = {StringData.fromString("Jane"), StringData.fromString("Joe")}; + return GenericRowData.of( + StringData.fromString("row_id_value"), GenericRowData.of(1, new GenericArrayData(names))); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.Schema structSchema = avroSchema.getField("struct_of_array").schema(); + org.apache.avro.generic.GenericRecord struct = new GenericData.Record(structSchema); + struct.put("id", 1); + struct.put("names", Arrays.asList("Jane", "Joe")); + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", "row_id_value"); + genericRecord.put("struct_of_array", struct); + return genericRecord; + } + } + + public static class StructOfMap implements DataGenerator { + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.required( + 2, + "struct_of_map", + Types.StructType.of( + required(101, "id", Types.IntegerType.get()), + required( + 102, + "names", + Types.MapType.ofRequired( + 201, 202, Types.StringType.get(), Types.StringType.get()))))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + private final org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(icebergSchema, "table"); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + Schema structSchema = + new Schema(icebergSchema.findField("struct_of_map").type().asStructType().fields()); + GenericRecord struct = GenericRecord.create(structSchema); + struct.setField("id", 1); + struct.setField("names", ImmutableMap.of("Jane", "female", "Joe", "male")); + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField("struct_of_map", struct); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + return GenericRowData.of( + StringData.fromString("row_id_value"), + GenericRowData.of( + 1, + new GenericMapData( + ImmutableMap.of( + StringData.fromString("Jane"), + StringData.fromString("female"), + StringData.fromString("Joe"), + StringData.fromString("male"))))); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.Schema structSchema = avroSchema.getField("struct_of_map").schema(); + org.apache.avro.generic.GenericRecord struct = new GenericData.Record(structSchema); + struct.put("id", 1); + struct.put("names", ImmutableMap.of("Jane", new Utf8("female"), "Joe", new Utf8("male"))); + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", "row_id_value"); + genericRecord.put("struct_of_map", struct); + return genericRecord; + } + } + + public static class StructOfStruct implements DataGenerator { + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.required( + 2, + "struct_of_struct", + Types.StructType.of( + required(101, "id", Types.IntegerType.get()), + required( + 102, + "person_struct", + Types.StructType.of( + Types.NestedField.required(201, "name", Types.StringType.get()), + Types.NestedField.required(202, "address", Types.StringType.get())))))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + private final org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(icebergSchema, "table"); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + Schema structSchema = + new Schema(icebergSchema.findField("struct_of_struct").type().asStructType().fields()); + Schema personSchema = + new Schema(structSchema.findField("person_struct").type().asStructType().fields()); + GenericRecord person = GenericRecord.create(personSchema); + person.setField("name", "Jane"); + person.setField("address", "Apple Park"); + GenericRecord struct = GenericRecord.create(structSchema); + struct.setField("id", 1); + struct.setField("person_struct", person); + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField("struct_of_struct", struct); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + return GenericRowData.of( + StringData.fromString("row_id_value"), + GenericRowData.of( + 1, + GenericRowData.of( + StringData.fromString("Jane"), StringData.fromString("Apple Park")))); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.Schema structSchema = avroSchema.getField("struct_of_struct").schema(); + org.apache.avro.Schema personSchema = structSchema.getField("person_struct").schema(); + org.apache.avro.generic.GenericRecord person = new GenericData.Record(personSchema); + person.put("name", "Jane"); + person.put("address", "Apple Park"); + org.apache.avro.generic.GenericRecord struct = new GenericData.Record(structSchema); + struct.put("id", 1); + struct.put("person_struct", person); + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", "row_id_value"); + genericRecord.put("struct_of_struct", struct); + return genericRecord; + } + } + + public static class ArrayOfPrimitive implements DataGenerator { + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.required( + 2, "array_of_int", Types.ListType.ofOptional(101, Types.IntegerType.get()))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + private final org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(icebergSchema, "table"); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField("array_of_int", Arrays.asList(1, 2, 3)); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + Integer[] arr = {1, 2, 3}; + return GenericRowData.of(StringData.fromString("row_id_value"), new GenericArrayData(arr)); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", "row_id_value"); + genericRecord.put("array_of_int", Arrays.asList(1, 2, 3)); + return genericRecord; + } + } + + public static class ArrayOfArray implements DataGenerator { + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.required( + 2, + "array_of_array", + Types.ListType.ofRequired( + 101, Types.ListType.ofRequired(201, Types.IntegerType.get())))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + private final org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(icebergSchema, "table"); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField( + "array_of_array", Arrays.asList(Arrays.asList(1, 2, 3), Arrays.asList(4, 5, 6))); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + // non-primitive + Integer[] array1 = {1, 2, 3}; + Integer[] array2 = {4, 5, 6}; + GenericArrayData[] arrayOfArrays = { + new GenericArrayData(array1), new GenericArrayData(array2) + }; + return GenericRowData.of( + StringData.fromString("row_id_value"), new GenericArrayData(arrayOfArrays)); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", "row_id_value"); + genericRecord.put( + "array_of_array", Arrays.asList(Arrays.asList(1, 2, 3), Arrays.asList(4, 5, 6))); + return genericRecord; + } + } + + public static class ArrayOfMap implements DataGenerator { + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.required( + 2, + "array_of_map", + Types.ListType.ofRequired( + 101, + Types.MapType.ofRequired( + 201, 202, Types.StringType.get(), Types.IntegerType.get())))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + private final org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(icebergSchema, "table"); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField( + "array_of_map", + Arrays.asList( + ImmutableMap.of("Jane", 1, "Joe", 2), ImmutableMap.of("Alice", 3, "Bob", 4))); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + GenericMapData[] array = { + new GenericMapData( + ImmutableMap.of(StringData.fromString("Jane"), 1, StringData.fromString("Joe"), 2)), + new GenericMapData( + ImmutableMap.of(StringData.fromString("Alice"), 3, StringData.fromString("Bob"), 4)) + }; + return GenericRowData.of(StringData.fromString("row_id_value"), new GenericArrayData(array)); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", "row_id_value"); + genericRecord.put( + "array_of_map", + Arrays.asList( + ImmutableMap.of("Jane", 1, "Joe", 2), ImmutableMap.of("Alice", 3, "Bob", 4))); + return genericRecord; + } + } + + public static class ArrayOfStruct implements DataGenerator { + private final Types.StructType structType = + Types.StructType.of( + required(201, "id", Types.IntegerType.get()), + required(202, "name", Types.StringType.get())); + private final Schema structIcebergSchema = new Schema(structType.fields()); + private final org.apache.avro.Schema structAvroSchema = + AvroSchemaUtil.convert(structIcebergSchema, "struct"); + + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.optional( + 2, "array_of_struct", Types.ListType.ofRequired(101, structType))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + private final org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(icebergSchema, "table"); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + GenericRecord struct1 = GenericRecord.create(structIcebergSchema); + struct1.setField("id", 1); + struct1.setField("name", "Jane"); + GenericRecord struct2 = GenericRecord.create(structIcebergSchema); + struct2.setField("id", 2); + struct2.setField("name", "Joe"); + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField("array_of_struct", Arrays.asList(struct1, struct2)); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + GenericRowData[] structArray = { + GenericRowData.of(1, StringData.fromString("Jane")), + GenericRowData.of(2, StringData.fromString("Joe")) + }; + return GenericRowData.of( + StringData.fromString("row_id_value"), new GenericArrayData(structArray)); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.generic.GenericRecord struct1 = new GenericData.Record(structAvroSchema); + struct1.put("id", 1); + struct1.put("name", "Jane"); + org.apache.avro.generic.GenericRecord struct2 = new GenericData.Record(structAvroSchema); + struct2.put("id", 2); + struct2.put("name", "Joe"); + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", "row_id_value"); + genericRecord.put("array_of_struct", Arrays.asList(struct1, struct2)); + return genericRecord; + } + } + + public static class MapOfPrimitives implements DataGenerator { + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.optional( + 2, + "map_of_primitives", + Types.MapType.ofRequired( + 101, 102, Types.StringType.get(), Types.IntegerType.get()))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + private final org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(icebergSchema, "table"); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField("map_of_primitives", ImmutableMap.of("Jane", 1, "Joe", 2)); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + return GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericMapData( + ImmutableMap.of(StringData.fromString("Jane"), 1, StringData.fromString("Joe"), 2))); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", "row_id_value"); + genericRecord.put("map_of_primitives", ImmutableMap.of("Jane", 1, "Joe", 2)); + return genericRecord; + } + } + + public static class MapOfArray implements DataGenerator { + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.required( + 2, + "map_of_array", + Types.MapType.ofRequired( + 101, + 102, + Types.StringType.get(), + Types.ListType.ofRequired(201, Types.IntegerType.get())))); + + private final RowType rowType = FlinkSchemaUtil.convert(icebergSchema); + + private final org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(icebergSchema, "table"); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return rowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField( + "map_of_array", + ImmutableMap.of( + "Jane", Arrays.asList(1, 2, 3), + "Joe", Arrays.asList(4, 5, 6))); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + Integer[] janeArray = {1, 2, 3}; + Integer[] joeArray = {4, 5, 6}; + return GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericMapData( + ImmutableMap.of( + StringData.fromString("Jane"), + new GenericArrayData(janeArray), + StringData.fromString("Joe"), + new GenericArrayData(joeArray)))); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", "row_id_value"); + genericRecord.put( + "map_of_array", + ImmutableMap.of( + "Jane", Arrays.asList(1, 2, 3), + "Joe", Arrays.asList(4, 5, 6))); + return genericRecord; + } + } + + public static class MapOfMap implements DataGenerator { + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.required( + 2, + "map_of_map", + Types.MapType.ofRequired( + 101, + 102, + Types.StringType.get(), + Types.MapType.ofRequired( + 301, 302, Types.StringType.get(), Types.IntegerType.get())))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + private final org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(icebergSchema, "table"); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField( + "map_of_map", + ImmutableMap.of( + "female", ImmutableMap.of("Jane", 1, "Alice", 2), + "male", ImmutableMap.of("Joe", 3, "Bob", 4))); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + return GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericMapData( + ImmutableMap.of( + StringData.fromString("female"), + new GenericMapData( + ImmutableMap.of( + StringData.fromString("Jane"), 1, StringData.fromString("Alice"), 2)), + StringData.fromString("male"), + new GenericMapData( + ImmutableMap.of( + StringData.fromString("Joe"), 3, StringData.fromString("Bob"), 4))))); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", "row_id_value"); + genericRecord.put( + "map_of_map", + ImmutableMap.of( + "female", ImmutableMap.of("Jane", 1, "Alice", 2), + "male", ImmutableMap.of("Joe", 3, "Bob", 4))); + return genericRecord; + } + } + + public static class MapOfStruct implements DataGenerator { + private org.apache.avro.Schema createAvroSchemaIdField() { + org.apache.avro.Schema schema = SchemaBuilder.builder().intType(); + // this is needed to match the converter generated schema props + schema.addProp("field-id", IntNode.valueOf(201)); + return schema; + } + + private org.apache.avro.Schema createAvroSchemaNameField() { + org.apache.avro.Schema schema = SchemaBuilder.builder().stringType(); + // this is needed to match the converter generated schema props + schema.addProp("field-id", IntNode.valueOf(202)); + return schema; + } + + private final Types.StructType structType = + Types.StructType.of( + required(201, "id", Types.IntegerType.get()), + required(202, "name", Types.StringType.get())); + private final Schema structIcebergSchema = new Schema(structType.fields()); + + private final org.apache.avro.Schema structAvroSchema = + SchemaBuilder.builder() + .record("struct") + .fields() + .name("id") + .type(createAvroSchemaIdField()) + .noDefault() + .name("name") + .type(createAvroSchemaNameField()) + .noDefault() + .endRecord(); + + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.required( + 2, + "map_of_struct", + Types.MapType.ofRequired(101, 102, Types.StringType.get(), structType))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + // Can't use AvroSchemaUtil.convert otherwise the nested schema will have generated name like + // `r102` not the specified name like `struct`. + org.apache.avro.Schema avroSchema = + SchemaBuilder.builder() + .record("table") + .fields() + .requiredString("row_id") + .name("map_of_struct") + .type(SchemaBuilder.builder().map().values(structAvroSchema)) + .noDefault() + .endRecord(); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + GenericRecord struct1 = GenericRecord.create(structIcebergSchema); + struct1.setField("id", 1); + struct1.setField("name", "Jane"); + GenericRecord struct2 = GenericRecord.create(structIcebergSchema); + struct2.setField("id", 2); + struct2.setField("name", "Joe"); + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField( + "map_of_struct", ImmutableMap.of("struct1", struct1, "struct2", struct2)); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + return GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericMapData( + ImmutableMap.of( + StringData.fromString("struct1"), + GenericRowData.of(1, StringData.fromString("Jane")), + StringData.fromString("struct2"), + GenericRowData.of(2, StringData.fromString("Joe"))))); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.generic.GenericRecord struct1 = new GenericData.Record(structAvroSchema); + struct1.put("id", 1); + struct1.put("name", new Utf8("Jane")); + org.apache.avro.generic.GenericRecord struct2 = new GenericData.Record(structAvroSchema); + struct2.put("id", 2); + struct2.put("name", new Utf8("Joe")); + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", new Utf8("row_id_value")); + genericRecord.put("map_of_struct", ImmutableMap.of("struct1", struct1, "struct2", struct2)); + return genericRecord; + } + } + + public static class MapOfStructStruct implements DataGenerator { + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.optional( + 2, + "map", + Types.MapType.ofOptional( + 101, + 102, + Types.StructType.of( + Types.NestedField.required(201, "key", Types.LongType.get()), + Types.NestedField.optional(202, "keyData", Types.StringType.get())), + Types.StructType.of( + Types.NestedField.required(203, "value", Types.LongType.get()), + Types.NestedField.optional(204, "valueData", Types.StringType.get()))))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + throw new UnsupportedOperationException( + "Not applicable as Avro Map only support string key type"); + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + throw new UnsupportedOperationException("Not implemented yet"); + } + + @Override + public GenericRowData generateFlinkRowData() { + return GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericMapData( + ImmutableMap.of( + GenericRowData.of(1L, StringData.fromString("key_data")), + GenericRowData.of(1L, StringData.fromString("value_data"))))); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + throw new UnsupportedOperationException("Avro Map only support string key type"); + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/FlinkCatalogTestBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/FlinkCatalogTestBase.java new file mode 100644 index 000000000000..74c5d343e996 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/FlinkCatalogTestBase.java @@ -0,0 +1,155 @@ +/* + * 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.iceberg.flink; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import org.apache.flink.util.ArrayUtils; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SupportsNamespaces; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.relocated.com.google.common.base.Joiner; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public abstract class FlinkCatalogTestBase extends FlinkTestBase { + + protected static final String DATABASE = "db"; + private static TemporaryFolder hiveWarehouse = new TemporaryFolder(); + private static TemporaryFolder hadoopWarehouse = new TemporaryFolder(); + + @BeforeClass + public static void createWarehouse() throws IOException { + hiveWarehouse.create(); + hadoopWarehouse.create(); + } + + @AfterClass + public static void dropWarehouse() { + hiveWarehouse.delete(); + hadoopWarehouse.delete(); + } + + @Before + public void before() { + sql("CREATE CATALOG %s WITH %s", catalogName, toWithClause(config)); + } + + @After + public void clean() { + dropCatalog(catalogName, true); + } + + @Parameterized.Parameters(name = "catalogName = {0} baseNamespace = {1}") + public static Iterable parameters() { + return Lists.newArrayList( + new Object[] {"testhive", Namespace.empty()}, + new Object[] {"testhadoop", Namespace.empty()}, + new Object[] {"testhadoop_basenamespace", Namespace.of("l0", "l1")}); + } + + protected final String catalogName; + protected final Namespace baseNamespace; + protected final Catalog validationCatalog; + protected final SupportsNamespaces validationNamespaceCatalog; + protected final Map config = Maps.newHashMap(); + + protected final String flinkDatabase; + protected final Namespace icebergNamespace; + protected final boolean isHadoopCatalog; + + public FlinkCatalogTestBase(String catalogName, Namespace baseNamespace) { + this.catalogName = catalogName; + this.baseNamespace = baseNamespace; + this.isHadoopCatalog = catalogName.startsWith("testhadoop"); + this.validationCatalog = + isHadoopCatalog + ? new HadoopCatalog(hiveConf, "file:" + hadoopWarehouse.getRoot()) + : catalog; + this.validationNamespaceCatalog = (SupportsNamespaces) validationCatalog; + + config.put("type", "iceberg"); + if (!baseNamespace.isEmpty()) { + config.put(FlinkCatalogFactory.BASE_NAMESPACE, baseNamespace.toString()); + } + if (isHadoopCatalog) { + config.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hadoop"); + } else { + config.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hive"); + config.put(CatalogProperties.URI, getURI(hiveConf)); + } + config.put(CatalogProperties.WAREHOUSE_LOCATION, String.format("file://%s", warehouseRoot())); + + this.flinkDatabase = catalogName + "." + DATABASE; + this.icebergNamespace = + Namespace.of(ArrayUtils.concat(baseNamespace.levels(), new String[] {DATABASE})); + } + + protected String warehouseRoot() { + if (isHadoopCatalog) { + return hadoopWarehouse.getRoot().getAbsolutePath(); + } else { + return hiveWarehouse.getRoot().getAbsolutePath(); + } + } + + protected String getFullQualifiedTableName(String tableName) { + final List levels = Lists.newArrayList(icebergNamespace.levels()); + levels.add(tableName); + return Joiner.on('.').join(levels); + } + + static String getURI(HiveConf conf) { + return conf.get(HiveConf.ConfVars.METASTOREURIS.varname); + } + + static String toWithClause(Map props) { + StringBuilder builder = new StringBuilder(); + builder.append("("); + int propCount = 0; + for (Map.Entry entry : props.entrySet()) { + if (propCount > 0) { + builder.append(","); + } + builder + .append("'") + .append(entry.getKey()) + .append("'") + .append("=") + .append("'") + .append(entry.getValue()) + .append("'"); + propCount++; + } + builder.append(")"); + return builder.toString(); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java new file mode 100644 index 000000000000..8076e0ec76f8 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java @@ -0,0 +1,129 @@ +/* + * 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.iceberg.flink; + +import java.util.List; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.test.util.TestBaseUtils; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.hive.HiveCatalog; +import org.apache.iceberg.hive.TestHiveMetastore; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.assertj.core.api.Assertions; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.rules.TemporaryFolder; + +public abstract class FlinkTestBase extends TestBaseUtils { + + @ClassRule + public static MiniClusterWithClientResource miniClusterResource = + MiniClusterResource.createWithClassloaderCheckDisabled(); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + private static TestHiveMetastore metastore = null; + protected static HiveConf hiveConf = null; + protected static HiveCatalog catalog = null; + + private volatile TableEnvironment tEnv = null; + + @BeforeClass + public static void startMetastore() { + FlinkTestBase.metastore = new TestHiveMetastore(); + metastore.start(); + FlinkTestBase.hiveConf = metastore.hiveConf(); + FlinkTestBase.catalog = + (HiveCatalog) + CatalogUtil.loadCatalog( + HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf); + } + + @AfterClass + public static void stopMetastore() throws Exception { + metastore.stop(); + FlinkTestBase.catalog = null; + } + + protected TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + if (tEnv == null) { + EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build(); + + TableEnvironment env = TableEnvironment.create(settings); + env.getConfig() + .getConfiguration() + .set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); + tEnv = env; + } + } + } + return tEnv; + } + + protected static TableResult exec(TableEnvironment env, String query, Object... args) { + return env.executeSql(String.format(query, args)); + } + + protected TableResult exec(String query, Object... args) { + return exec(getTableEnv(), query, args); + } + + protected List sql(String query, Object... args) { + TableResult tableResult = exec(query, args); + try (CloseableIterator iter = tableResult.collect()) { + return Lists.newArrayList(iter); + } catch (Exception e) { + throw new RuntimeException("Failed to collect table result", e); + } + } + + protected void assertSameElements(Iterable expected, Iterable actual) { + Assertions.assertThat(actual).isNotNull().containsExactlyInAnyOrderElementsOf(expected); + } + + protected void assertSameElements(String message, Iterable expected, Iterable actual) { + Assertions.assertThat(actual) + .isNotNull() + .as(message) + .containsExactlyInAnyOrderElementsOf(expected); + } + + /** + * We can not drop currently used catalog after FLINK-29677, so we have make sure that we do not + * use the current catalog before dropping it. This method switches to the 'default_catalog' and + * drops the one requested. + * + * @param catalogName The catalog to drop + * @param ifExists If we should use the 'IF EXISTS' when dropping the catalog + */ + protected void dropCatalog(String catalogName, boolean ifExists) { + sql("USE CATALOG default_catalog"); + sql("DROP CATALOG %s %s", ifExists ? "IF EXISTS" : "", catalogName); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java new file mode 100644 index 000000000000..d8e1325254d9 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java @@ -0,0 +1,104 @@ +/* + * 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.iceberg.flink; + +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.UUID; +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.extension.AfterAllCallback; +import org.junit.jupiter.api.extension.AfterEachCallback; +import org.junit.jupiter.api.extension.BeforeAllCallback; +import org.junit.jupiter.api.extension.BeforeEachCallback; +import org.junit.jupiter.api.extension.ExtensionContext; + +public class HadoopCatalogExtension + implements BeforeAllCallback, BeforeEachCallback, AfterAllCallback, AfterEachCallback { + protected final String database; + protected final String tableName; + + protected Path temporaryFolder; + protected Catalog catalog; + protected CatalogLoader catalogLoader; + protected String warehouse; + protected TableLoader tableLoader; + + public HadoopCatalogExtension(String database, String tableName) { + this.database = database; + this.tableName = tableName; + } + + @Override + public void beforeAll(ExtensionContext context) throws Exception { + this.temporaryFolder = Files.createTempDirectory("junit5_hadoop_catalog-"); + } + + @Override + public void afterAll(ExtensionContext context) throws Exception { + FileUtils.deleteDirectory(temporaryFolder.toFile()); + } + + @Override + public void beforeEach(ExtensionContext context) throws Exception { + Assertions.assertThat(temporaryFolder).exists().isDirectory(); + this.warehouse = "file:" + temporaryFolder + "/" + UUID.randomUUID(); + this.catalogLoader = + CatalogLoader.hadoop( + "hadoop", + new Configuration(), + ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouse)); + this.catalog = catalogLoader.loadCatalog(); + this.tableLoader = + TableLoader.fromCatalog(catalogLoader, TableIdentifier.of(database, tableName)); + } + + @Override + public void afterEach(ExtensionContext context) throws Exception { + try { + catalog.dropTable(TableIdentifier.of(database, tableName)); + ((HadoopCatalog) catalog).close(); + tableLoader.close(); + } catch (Exception e) { + throw new RuntimeException("Failed to close catalog resource"); + } + } + + public TableLoader tableLoader() { + return tableLoader; + } + + public Catalog catalog() { + return catalog; + } + + public CatalogLoader catalogLoader() { + return catalogLoader; + } + + public String warehouse() { + return warehouse; + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java new file mode 100644 index 000000000000..2b4694d3c268 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java @@ -0,0 +1,90 @@ +/* + * 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.iceberg.flink; + +import java.io.File; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.Assert; +import org.junit.rules.ExternalResource; +import org.junit.rules.TemporaryFolder; + +public class HadoopCatalogResource extends ExternalResource { + protected final TemporaryFolder temporaryFolder; + protected final String database; + protected final String tableName; + + protected Catalog catalog; + protected CatalogLoader catalogLoader; + protected String warehouse; + protected TableLoader tableLoader; + + public HadoopCatalogResource(TemporaryFolder temporaryFolder, String database, String tableName) { + this.temporaryFolder = temporaryFolder; + this.database = database; + this.tableName = tableName; + } + + @Override + protected void before() throws Throwable { + File warehouseFile = temporaryFolder.newFolder(); + Assert.assertTrue(warehouseFile.delete()); + // before variables + this.warehouse = "file:" + warehouseFile; + this.catalogLoader = + CatalogLoader.hadoop( + "hadoop", + new Configuration(), + ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouse)); + this.catalog = catalogLoader.loadCatalog(); + this.tableLoader = + TableLoader.fromCatalog(catalogLoader, TableIdentifier.of(database, tableName)); + } + + @Override + protected void after() { + try { + catalog.dropTable(TableIdentifier.of(database, tableName)); + ((HadoopCatalog) catalog).close(); + tableLoader.close(); + } catch (Exception e) { + throw new RuntimeException("Failed to close catalog resource"); + } + } + + public TableLoader tableLoader() { + return tableLoader; + } + + public Catalog catalog() { + return catalog; + } + + public CatalogLoader catalogLoader() { + return catalogLoader; + } + + public String warehouse() { + return warehouse; + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java new file mode 100644 index 000000000000..2da6f74b91d8 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink; + +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.junit.rules.TemporaryFolder; + +public class HadoopTableResource extends HadoopCatalogResource { + private final Schema schema; + private final PartitionSpec partitionSpec; + + private Table table; + + public HadoopTableResource( + TemporaryFolder temporaryFolder, String database, String tableName, Schema schema) { + this(temporaryFolder, database, tableName, schema, null); + } + + public HadoopTableResource( + TemporaryFolder temporaryFolder, + String database, + String tableName, + Schema schema, + PartitionSpec partitionSpec) { + super(temporaryFolder, database, tableName); + this.schema = schema; + this.partitionSpec = partitionSpec; + } + + @Override + protected void before() throws Throwable { + super.before(); + if (partitionSpec == null) { + this.table = catalog.createTable(TableIdentifier.of(database, tableName), schema); + } else { + this.table = + catalog.createTable(TableIdentifier.of(database, tableName), schema, partitionSpec); + } + tableLoader.open(); + } + + public Table table() { + return table; + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java new file mode 100644 index 000000000000..45af9241b743 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java @@ -0,0 +1,53 @@ +/* + * 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.iceberg.flink; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.test.util.MiniClusterWithClientResource; + +public class MiniClusterResource { + + private static final int DEFAULT_TM_NUM = 1; + private static final int DEFAULT_PARALLELISM = 4; + + public static final Configuration DISABLE_CLASSLOADER_CHECK_CONFIG = + new Configuration() + // disable classloader check as Avro may cache class/object in the serializers. + .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); + + private MiniClusterResource() {} + + /** + * It will start a mini cluster with classloader.check-leaked-classloader=false, so that we won't + * break the unit tests because of the class loader leak issue. In our iceberg integration tests, + * there're some that will assert the results after finished the flink jobs, so actually we may + * access the class loader that has been closed by the flink task managers if we enable the switch + * classloader.check-leaked-classloader by default. + */ + public static MiniClusterWithClientResource createWithClassloaderCheckDisabled() { + return new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(DEFAULT_TM_NUM) + .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) + .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) + .build()); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java new file mode 100644 index 000000000000..9a73b80e077d --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java @@ -0,0 +1,53 @@ +/* + * 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.iceberg.flink; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.test.junit5.MiniClusterExtension; + +public class MiniFlinkClusterExtension { + + private static final int DEFAULT_TM_NUM = 1; + private static final int DEFAULT_PARALLELISM = 4; + + public static final Configuration DISABLE_CLASSLOADER_CHECK_CONFIG = + new Configuration() + // disable classloader check as Avro may cache class/object in the serializers. + .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); + + private MiniFlinkClusterExtension() {} + + /** + * It will start a mini cluster with classloader.check-leaked-classloader=false, so that we won't + * break the unit tests because of the class loader leak issue. In our iceberg integration tests, + * there're some that will assert the results after finished the flink jobs, so actually we may + * access the class loader that has been closed by the flink task managers if we enable the switch + * classloader.check-leaked-classloader by default. + */ + public static MiniClusterExtension createWithClassloaderCheckDisabled() { + return new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(DEFAULT_TM_NUM) + .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) + .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) + .build()); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java new file mode 100644 index 000000000000..e532fb62615c --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java @@ -0,0 +1,135 @@ +/* + * 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.iceberg.flink; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.time.temporal.ChronoUnit; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +public class RowDataConverter { + private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC); + private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate(); + + private RowDataConverter() {} + + public static RowData convert(Schema iSchema, Record record) { + return convert(iSchema.asStruct(), record); + } + + private static RowData convert(Types.StructType struct, Record record) { + GenericRowData rowData = new GenericRowData(struct.fields().size()); + List fields = struct.fields(); + for (int i = 0; i < fields.size(); i += 1) { + Types.NestedField field = fields.get(i); + + Type fieldType = field.type(); + rowData.setField(i, convert(fieldType, record.get(i))); + } + return rowData; + } + + private static Object convert(Type type, Object object) { + if (object == null) { + return null; + } + + switch (type.typeId()) { + case BOOLEAN: + case INTEGER: + case LONG: + case FLOAT: + case DOUBLE: + case FIXED: + return object; + case DATE: + return (int) ChronoUnit.DAYS.between(EPOCH_DAY, (LocalDate) object); + case TIME: + // Iceberg's time is in microseconds, while flink's time is in milliseconds. + LocalTime localTime = (LocalTime) object; + return (int) TimeUnit.NANOSECONDS.toMillis(localTime.toNanoOfDay()); + case TIMESTAMP: + if (((Types.TimestampType) type).shouldAdjustToUTC()) { + return TimestampData.fromInstant(((OffsetDateTime) object).toInstant()); + } else { + return TimestampData.fromLocalDateTime((LocalDateTime) object); + } + case STRING: + return StringData.fromString((String) object); + case UUID: + UUID uuid = (UUID) object; + ByteBuffer bb = ByteBuffer.allocate(16); + bb.putLong(uuid.getMostSignificantBits()); + bb.putLong(uuid.getLeastSignificantBits()); + return bb.array(); + case BINARY: + ByteBuffer buffer = (ByteBuffer) object; + return Arrays.copyOfRange( + buffer.array(), + buffer.arrayOffset() + buffer.position(), + buffer.arrayOffset() + buffer.remaining()); + case DECIMAL: + Types.DecimalType decimalType = (Types.DecimalType) type; + return DecimalData.fromBigDecimal( + (BigDecimal) object, decimalType.precision(), decimalType.scale()); + case STRUCT: + return convert(type.asStructType(), (Record) object); + case LIST: + List list = (List) object; + Object[] convertedArray = new Object[list.size()]; + for (int i = 0; i < convertedArray.length; i++) { + convertedArray[i] = convert(type.asListType().elementType(), list.get(i)); + } + return new GenericArrayData(convertedArray); + case MAP: + Map convertedMap = Maps.newLinkedHashMap(); + Map map = (Map) object; + for (Map.Entry entry : map.entrySet()) { + convertedMap.put( + convert(type.asMapType().keyType(), entry.getKey()), + convert(type.asMapType().valueType(), entry.getValue())); + } + return new GenericMapData(convertedMap); + default: + throw new UnsupportedOperationException("Not a supported type: " + type); + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java new file mode 100644 index 000000000000..f48764f772b4 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java @@ -0,0 +1,443 @@ +/* + * 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.iceberg.flink; + +import static org.apache.iceberg.hadoop.HadoopOutputFile.fromPath; + +import java.io.IOException; +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.IcebergGenerics; +import org.apache.iceberg.data.InternalRecordWrapper; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.deletes.EqualityDeleteWriter; +import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.deletes.PositionDeleteWriter; +import org.apache.iceberg.encryption.EncryptedOutputFile; +import org.apache.iceberg.flink.sink.FlinkAppenderFactory; +import org.apache.iceberg.hadoop.HadoopInputFile; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Pair; +import org.apache.iceberg.util.StructLikeSet; +import org.apache.iceberg.util.StructLikeWrapper; +import org.awaitility.Awaitility; +import org.junit.Assert; + +public class SimpleDataUtil { + + private SimpleDataUtil() {} + + public static final Schema SCHEMA = + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get())); + + public static final TableSchema FLINK_SCHEMA = + TableSchema.builder().field("id", DataTypes.INT()).field("data", DataTypes.STRING()).build(); + + public static final RowType ROW_TYPE = (RowType) FLINK_SCHEMA.toRowDataType().getLogicalType(); + + public static final Record RECORD = GenericRecord.create(SCHEMA); + + public static Table createTable( + String path, Map properties, boolean partitioned) { + PartitionSpec spec; + if (partitioned) { + spec = PartitionSpec.builderFor(SCHEMA).identity("data").build(); + } else { + spec = PartitionSpec.unpartitioned(); + } + return new HadoopTables().create(SCHEMA, spec, properties, path); + } + + public static Record createRecord(Integer id, String data) { + Record record = RECORD.copy(); + record.setField("id", id); + record.setField("data", data); + return record; + } + + public static RowData createRowData(Integer id, String data) { + return GenericRowData.of(id, StringData.fromString(data)); + } + + public static RowData createInsert(Integer id, String data) { + return GenericRowData.ofKind(RowKind.INSERT, id, StringData.fromString(data)); + } + + public static RowData createDelete(Integer id, String data) { + return GenericRowData.ofKind(RowKind.DELETE, id, StringData.fromString(data)); + } + + public static RowData createUpdateBefore(Integer id, String data) { + return GenericRowData.ofKind(RowKind.UPDATE_BEFORE, id, StringData.fromString(data)); + } + + public static RowData createUpdateAfter(Integer id, String data) { + return GenericRowData.ofKind(RowKind.UPDATE_AFTER, id, StringData.fromString(data)); + } + + public static DataFile writeFile( + Table table, + Schema schema, + PartitionSpec spec, + Configuration conf, + String location, + String filename, + List rows) + throws IOException { + return writeFile(table, schema, spec, conf, location, filename, rows, null); + } + + /** Write the list of {@link RowData} to the given path and with the given partition data */ + public static DataFile writeFile( + Table table, + Schema schema, + PartitionSpec spec, + Configuration conf, + String location, + String filename, + List rows, + StructLike partition) + throws IOException { + Path path = new Path(location, filename); + FileFormat fileFormat = FileFormat.fromFileName(filename); + Preconditions.checkNotNull(fileFormat, "Cannot determine format for file: %s", filename); + + RowType flinkSchema = FlinkSchemaUtil.convert(schema); + FileAppenderFactory appenderFactory = + new FlinkAppenderFactory( + table, schema, flinkSchema, ImmutableMap.of(), spec, null, null, null); + + FileAppender appender = appenderFactory.newAppender(fromPath(path, conf), fileFormat); + try (FileAppender closeableAppender = appender) { + closeableAppender.addAll(rows); + } + + DataFiles.Builder builder = + DataFiles.builder(spec) + .withInputFile(HadoopInputFile.fromPath(path, conf)) + .withMetrics(appender.metrics()); + + if (partition != null) { + builder = builder.withPartition(partition); + } + + return builder.build(); + } + + public static DeleteFile writeEqDeleteFile( + Table table, + FileFormat format, + String filename, + FileAppenderFactory appenderFactory, + List deletes) + throws IOException { + EncryptedOutputFile outputFile = + table + .encryption() + .encrypt(fromPath(new Path(table.location(), filename), new Configuration())); + + EqualityDeleteWriter eqWriter = + appenderFactory.newEqDeleteWriter(outputFile, format, null); + try (EqualityDeleteWriter writer = eqWriter) { + writer.write(deletes); + } + return eqWriter.toDeleteFile(); + } + + public static DeleteFile writePosDeleteFile( + Table table, + FileFormat format, + String filename, + FileAppenderFactory appenderFactory, + List> positions) + throws IOException { + EncryptedOutputFile outputFile = + table + .encryption() + .encrypt(fromPath(new Path(table.location(), filename), new Configuration())); + + PositionDeleteWriter posWriter = + appenderFactory.newPosDeleteWriter(outputFile, format, null); + PositionDelete posDelete = PositionDelete.create(); + try (PositionDeleteWriter writer = posWriter) { + for (Pair p : positions) { + writer.write(posDelete.set(p.first(), p.second(), null)); + } + } + return posWriter.toDeleteFile(); + } + + private static List convertToRecords(List rows) { + List records = Lists.newArrayList(); + for (RowData row : rows) { + Integer id = row.isNullAt(0) ? null : row.getInt(0); + String data = row.isNullAt(1) ? null : row.getString(1).toString(); + records.add(createRecord(id, data)); + } + return records; + } + + public static void assertTableRows(String tablePath, List expected, String branch) + throws IOException { + assertTableRecords(tablePath, convertToRecords(expected), branch); + } + + public static void assertTableRows(Table table, List expected) throws IOException { + assertTableRecords(table, convertToRecords(expected), SnapshotRef.MAIN_BRANCH); + } + + public static void assertTableRows(Table table, List expected, String branch) + throws IOException { + assertTableRecords(table, convertToRecords(expected), branch); + } + + /** Get all rows for a table */ + public static List tableRecords(Table table) throws IOException { + table.refresh(); + List records = Lists.newArrayList(); + try (CloseableIterable iterable = IcebergGenerics.read(table).build()) { + for (Record record : iterable) { + records.add(record); + } + } + return records; + } + + public static boolean equalsRecords(List expected, List actual, Schema schema) { + if (expected.size() != actual.size()) { + return false; + } + Types.StructType type = schema.asStruct(); + StructLikeSet expectedSet = StructLikeSet.create(type); + expectedSet.addAll(expected); + StructLikeSet actualSet = StructLikeSet.create(type); + actualSet.addAll(actual); + return expectedSet.equals(actualSet); + } + + public static void assertRecordsEqual(List expected, List actual, Schema schema) { + Assert.assertEquals(expected.size(), actual.size()); + Types.StructType type = schema.asStruct(); + StructLikeSet expectedSet = StructLikeSet.create(type); + expectedSet.addAll(expected); + StructLikeSet actualSet = StructLikeSet.create(type); + actualSet.addAll(actual); + Assert.assertEquals(expectedSet, actualSet); + } + + /** + * Assert table contains the expected list of records after waiting up to the configured {@code + * timeout} + */ + public static void assertTableRecords(Table table, List expected, Duration timeout) { + Awaitility.await("expected list of records should be produced") + .atMost(timeout) + .untilAsserted( + () -> { + equalsRecords(expected, tableRecords(table), table.schema()); + assertRecordsEqual(expected, tableRecords(table), table.schema()); + }); + } + + public static void assertTableRecords(Table table, List expected) throws IOException { + assertTableRecords(table, expected, SnapshotRef.MAIN_BRANCH); + } + + public static void assertTableRecords(Table table, List expected, String branch) + throws IOException { + table.refresh(); + Snapshot snapshot = latestSnapshot(table, branch); + + if (snapshot == null) { + Assert.assertEquals(expected, ImmutableList.of()); + return; + } + + Types.StructType type = table.schema().asStruct(); + StructLikeSet expectedSet = StructLikeSet.create(type); + expectedSet.addAll(expected); + + try (CloseableIterable iterable = + IcebergGenerics.read(table).useSnapshot(snapshot.snapshotId()).build()) { + StructLikeSet actualSet = StructLikeSet.create(type); + + for (Record record : iterable) { + actualSet.add(record); + } + + Assert.assertEquals("Should produce the expected record", expectedSet, actualSet); + } + } + + // Returns the latest snapshot of the given branch in the table + public static Snapshot latestSnapshot(Table table, String branch) { + // For the main branch, currentSnapshot() is used to validate that the API behavior has + // not changed since that was the API used for validation prior to addition of branches. + if (branch.equals(SnapshotRef.MAIN_BRANCH)) { + return table.currentSnapshot(); + } + + return table.snapshot(branch); + } + + public static void assertTableRecords(String tablePath, List expected) + throws IOException { + Preconditions.checkArgument(expected != null, "expected records shouldn't be null"); + assertTableRecords(new HadoopTables().load(tablePath), expected, SnapshotRef.MAIN_BRANCH); + } + + public static void assertTableRecords(String tablePath, List expected, String branch) + throws IOException { + Preconditions.checkArgument(expected != null, "expected records shouldn't be null"); + assertTableRecords(new HadoopTables().load(tablePath), expected, branch); + } + + public static StructLikeSet expectedRowSet(Table table, Record... records) { + StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); + InternalRecordWrapper wrapper = new InternalRecordWrapper(table.schema().asStruct()); + for (Record record : records) { + set.add(wrapper.copyFor(record)); + } + return set; + } + + public static StructLikeSet actualRowSet(Table table, String... columns) throws IOException { + return actualRowSet(table, null, columns); + } + + public static StructLikeSet actualRowSet(Table table, Long snapshotId, String... columns) + throws IOException { + table.refresh(); + StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); + InternalRecordWrapper wrapper = new InternalRecordWrapper(table.schema().asStruct()); + try (CloseableIterable reader = + IcebergGenerics.read(table) + .useSnapshot(snapshotId == null ? table.currentSnapshot().snapshotId() : snapshotId) + .select(columns) + .build()) { + reader.forEach(record -> set.add(wrapper.copyFor(record))); + } + return set; + } + + public static List partitionDataFiles(Table table, Map partitionValues) + throws IOException { + table.refresh(); + Types.StructType partitionType = table.spec().partitionType(); + + Record partitionRecord = GenericRecord.create(partitionType).copy(partitionValues); + StructLikeWrapper expectedWrapper = + StructLikeWrapper.forType(partitionType).set(partitionRecord); + + List dataFiles = Lists.newArrayList(); + try (CloseableIterable fileScanTasks = table.newScan().planFiles()) { + for (FileScanTask scanTask : fileScanTasks) { + StructLikeWrapper wrapper = + StructLikeWrapper.forType(partitionType).set(scanTask.file().partition()); + + if (expectedWrapper.equals(wrapper)) { + dataFiles.add(scanTask.file()); + } + } + } + + return dataFiles; + } + + public static Map> snapshotToDataFiles(Table table) throws IOException { + table.refresh(); + + Map> result = Maps.newHashMap(); + Snapshot current = table.currentSnapshot(); + while (current != null) { + TableScan tableScan = table.newScan(); + if (current.parentId() != null) { + // Collect the data files that was added only in current snapshot. + tableScan = tableScan.appendsBetween(current.parentId(), current.snapshotId()); + } else { + // Collect the data files that was added in the oldest snapshot. + tableScan = tableScan.useSnapshot(current.snapshotId()); + } + try (CloseableIterable scanTasks = tableScan.planFiles()) { + result.put( + current.snapshotId(), + ImmutableList.copyOf(Iterables.transform(scanTasks, FileScanTask::file))); + } + + // Continue to traverse the parent snapshot if exists. + if (current.parentId() == null) { + break; + } + // Iterate to the parent snapshot. + current = table.snapshot(current.parentId()); + } + return result; + } + + public static List matchingPartitions( + List dataFiles, PartitionSpec partitionSpec, Map partitionValues) { + Types.StructType partitionType = partitionSpec.partitionType(); + Record partitionRecord = GenericRecord.create(partitionType).copy(partitionValues); + StructLikeWrapper expected = StructLikeWrapper.forType(partitionType).set(partitionRecord); + return dataFiles.stream() + .filter( + df -> { + StructLikeWrapper wrapper = + StructLikeWrapper.forType(partitionType).set(df.partition()); + return wrapper.equals(expected); + }) + .collect(Collectors.toList()); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestBase.java new file mode 100644 index 000000000000..4fc0207f269e --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestBase.java @@ -0,0 +1,130 @@ +/* + * 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.iceberg.flink; + +import java.nio.file.Path; +import java.util.List; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.test.util.TestBaseUtils; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.hive.HiveCatalog; +import org.apache.iceberg.hive.TestHiveMetastore; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +public abstract class TestBase extends TestBaseUtils { + + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + @TempDir Path temporaryDirectory; + + private static TestHiveMetastore metastore = null; + protected static HiveConf hiveConf = null; + protected static HiveCatalog catalog = null; + + private volatile TableEnvironment tEnv = null; + + @BeforeAll + public static void startMetastore() { + TestBase.metastore = new TestHiveMetastore(); + metastore.start(); + TestBase.hiveConf = metastore.hiveConf(); + TestBase.catalog = + (HiveCatalog) + CatalogUtil.loadCatalog( + HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf); + } + + @AfterAll + public static void stopMetastore() throws Exception { + metastore.stop(); + TestBase.catalog = null; + } + + protected TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + if (tEnv == null) { + EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build(); + + TableEnvironment env = TableEnvironment.create(settings); + env.getConfig() + .getConfiguration() + .set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); + tEnv = env; + } + } + } + return tEnv; + } + + protected static TableResult exec(TableEnvironment env, String query, Object... args) { + return env.executeSql(String.format(query, args)); + } + + protected TableResult exec(String query, Object... args) { + return exec(getTableEnv(), query, args); + } + + protected List sql(String query, Object... args) { + TableResult tableResult = exec(query, args); + try (CloseableIterator iter = tableResult.collect()) { + return Lists.newArrayList(iter); + } catch (Exception e) { + throw new RuntimeException("Failed to collect table result", e); + } + } + + protected void assertSameElements(Iterable expected, Iterable actual) { + Assertions.assertThat(actual).isNotNull().containsExactlyInAnyOrderElementsOf(expected); + } + + protected void assertSameElements(String message, Iterable expected, Iterable actual) { + Assertions.assertThat(actual) + .isNotNull() + .as(message) + .containsExactlyInAnyOrderElementsOf(expected); + } + + /** + * We can not drop currently used catalog after FLINK-29677, so we have make sure that we do not + * use the current catalog before dropping it. This method switches to the 'default_catalog' and + * drops the one requested. + * + * @param catalogName The catalog to drop + * @param ifExists If we should use the 'IF EXISTS' when dropping the catalog + */ + protected void dropCatalog(String catalogName, boolean ifExists) { + sql("USE CATALOG default_catalog"); + sql("DROP CATALOG %s %s", ifExists ? "IF EXISTS" : "", catalogName); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java new file mode 100644 index 000000000000..384ac5c52d00 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java @@ -0,0 +1,118 @@ +/* + * 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.iceberg.flink; + +import static org.apache.iceberg.CatalogProperties.URI; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.util.Map; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +/** Test for {@link CatalogLoader}. */ +public class TestCatalogLoader extends FlinkTestBase { + + private static File warehouse = null; + private static final TableIdentifier IDENTIFIER = TableIdentifier.of("default", "my_table"); + private static final Schema SCHEMA = + new Schema(Types.NestedField.required(1, "f1", Types.StringType.get())); + + @BeforeClass + public static void createWarehouse() throws IOException { + warehouse = File.createTempFile("warehouse", null); + Assert.assertTrue(warehouse.delete()); + hiveConf.set("my_key", "my_value"); + } + + @AfterClass + public static void dropWarehouse() throws IOException { + if (warehouse != null && warehouse.exists()) { + Path warehousePath = new Path(warehouse.getAbsolutePath()); + FileSystem fs = warehousePath.getFileSystem(hiveConf); + Assert.assertTrue("Failed to delete " + warehousePath, fs.delete(warehousePath, true)); + } + } + + @Test + public void testHadoopCatalogLoader() throws IOException, ClassNotFoundException { + Map properties = Maps.newHashMap(); + properties.put(CatalogProperties.WAREHOUSE_LOCATION, "file:" + warehouse); + CatalogLoader loader = CatalogLoader.hadoop("my_catalog", hiveConf, properties); + validateCatalogLoader(loader); + } + + @Test + public void testHiveCatalogLoader() throws IOException, ClassNotFoundException { + CatalogLoader loader = CatalogLoader.hive("my_catalog", hiveConf, Maps.newHashMap()); + validateCatalogLoader(loader); + } + + @Test + public void testRESTCatalogLoader() { + Map properties = Maps.newHashMap(); + properties.put(URI, "http://localhost/"); + CatalogLoader.rest("my_catalog", hiveConf, Maps.newHashMap()); + } + + private static void validateCatalogLoader(CatalogLoader loader) + throws IOException, ClassNotFoundException { + Table table = javaSerAndDeSer(loader).loadCatalog().createTable(IDENTIFIER, SCHEMA); + validateHadoopConf(table); + } + + private static void validateHadoopConf(Table table) { + FileIO io = table.io(); + Assertions.assertThat(io) + .as("FileIO should be a HadoopFileIO") + .isInstanceOf(HadoopFileIO.class); + HadoopFileIO hadoopIO = (HadoopFileIO) io; + Assert.assertEquals("my_value", hadoopIO.conf().get("my_key")); + } + + @SuppressWarnings("unchecked") + private static T javaSerAndDeSer(T object) throws IOException, ClassNotFoundException { + ByteArrayOutputStream bytes = new ByteArrayOutputStream(); + try (ObjectOutputStream out = new ObjectOutputStream(bytes)) { + out.writeObject(object); + } + + try (ObjectInputStream in = + new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { + return (T) in.readObject(); + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java new file mode 100644 index 000000000000..147d2a173ddc --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java @@ -0,0 +1,116 @@ +/* + * 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.iceberg.flink; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +/** Test for {@link TableLoader}. */ +public class TestCatalogTableLoader extends TestBase { + + private static File warehouse = null; + private static final TableIdentifier IDENTIFIER = TableIdentifier.of("default", "my_table"); + private static final Schema SCHEMA = + new Schema(Types.NestedField.required(1, "f1", Types.StringType.get())); + + @BeforeAll + public static void createWarehouse() throws IOException { + warehouse = File.createTempFile("warehouse", null); + Assertions.assertThat(warehouse.delete()).isTrue(); + hiveConf.set("my_key", "my_value"); + } + + @AfterAll + public static void dropWarehouse() throws IOException { + if (warehouse != null && warehouse.exists()) { + Path warehousePath = new Path(warehouse.getAbsolutePath()); + FileSystem fs = warehousePath.getFileSystem(hiveConf); + Assertions.assertThat(fs.delete(warehousePath, true)) + .as("Failed to delete " + warehousePath) + .isTrue(); + } + } + + @Test + public void testHadoopTableLoader() throws IOException, ClassNotFoundException { + String location = "file:" + warehouse + "/my_table"; + new HadoopTables(hiveConf).create(SCHEMA, location); + validateTableLoader(TableLoader.fromHadoopTable(location, hiveConf)); + } + + @Test + public void testHiveCatalogTableLoader() throws IOException, ClassNotFoundException { + CatalogLoader loader = CatalogLoader.hive("my_catalog", hiveConf, Maps.newHashMap()); + javaSerdes(loader).loadCatalog().createTable(IDENTIFIER, SCHEMA); + + CatalogLoader catalogLoader = CatalogLoader.hive("my_catalog", hiveConf, Maps.newHashMap()); + validateTableLoader(TableLoader.fromCatalog(catalogLoader, IDENTIFIER)); + } + + private static void validateTableLoader(TableLoader loader) + throws IOException, ClassNotFoundException { + TableLoader copied = javaSerdes(loader); + copied.open(); + try { + validateHadoopConf(copied.loadTable()); + } finally { + copied.close(); + } + } + + private static void validateHadoopConf(Table table) { + FileIO io = table.io(); + Assertions.assertThat(io) + .as("FileIO should be a HadoopFileIO") + .isInstanceOf(HadoopFileIO.class); + HadoopFileIO hadoopIO = (HadoopFileIO) io; + Assertions.assertThat(hadoopIO.conf().get("my_key")).isEqualTo("my_value"); + } + + @SuppressWarnings("unchecked") + private static T javaSerdes(T object) throws IOException, ClassNotFoundException { + ByteArrayOutputStream bytes = new ByteArrayOutputStream(); + try (ObjectOutputStream out = new ObjectOutputStream(bytes)) { + out.writeObject(object); + } + + try (ObjectInputStream in = + new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { + return (T) in.readObject(); + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java new file mode 100644 index 000000000000..8e9066e391c9 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java @@ -0,0 +1,305 @@ +/* + * 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.iceberg.flink; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import org.apache.flink.types.Row; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.BaseTable; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.source.BoundedTableFactory; +import org.apache.iceberg.flink.source.ChangeLogTableTestBase; +import org.apache.iceberg.relocated.com.google.common.base.Joiner; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.util.StructLikeSet; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +/** + * In this test case, we mainly cover the impact of primary key selection, multiple operations + * within a single transaction, and multiple operations between different txn on the correctness of + * the data. + */ +@RunWith(Parameterized.class) +public class TestChangeLogTable extends ChangeLogTableTestBase { + private static final Configuration CONF = new Configuration(); + private static final String SOURCE_TABLE = "default_catalog.default_database.source_change_logs"; + + private static final String CATALOG_NAME = "test_catalog"; + private static final String DATABASE_NAME = "test_db"; + private static final String TABLE_NAME = "test_table"; + private static String warehouse; + + private final boolean partitioned; + + @Parameterized.Parameters(name = "PartitionedTable={0}") + public static Iterable parameters() { + return ImmutableList.of(new Object[] {true}, new Object[] {false}); + } + + public TestChangeLogTable(boolean partitioned) { + this.partitioned = partitioned; + } + + @BeforeClass + public static void createWarehouse() throws IOException { + File warehouseFile = TEMPORARY_FOLDER.newFolder(); + Assert.assertTrue("The warehouse should be deleted", warehouseFile.delete()); + warehouse = String.format("file:%s", warehouseFile); + } + + @Before + public void before() { + sql( + "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", + CATALOG_NAME, warehouse); + sql("USE CATALOG %s", CATALOG_NAME); + sql("CREATE DATABASE %s", DATABASE_NAME); + sql("USE %s", DATABASE_NAME); + // Set the table.exec.sink.upsert-materialize=NONE, so that downstream operators will receive + // the + // records with the same order as the source operator, bypassing Flink's inferred shuffle. + getTableEnv().getConfig().set("table.exec.sink.upsert-materialize", "NONE"); + } + + @After + @Override + public void clean() { + sql("DROP TABLE IF EXISTS %s", TABLE_NAME); + sql("DROP DATABASE IF EXISTS %s", DATABASE_NAME); + dropCatalog(CATALOG_NAME, true); + BoundedTableFactory.clearDataSets(); + } + + @Test + public void testSqlChangeLogOnIdKey() throws Exception { + List> inputRowsPerCheckpoint = + ImmutableList.of( + ImmutableList.of( + insertRow(1, "aaa"), + deleteRow(1, "aaa"), + insertRow(1, "bbb"), + insertRow(2, "aaa"), + deleteRow(2, "aaa"), + insertRow(2, "bbb")), + ImmutableList.of( + updateBeforeRow(2, "bbb"), + updateAfterRow(2, "ccc"), + deleteRow(2, "ccc"), + insertRow(2, "ddd")), + ImmutableList.of( + deleteRow(1, "bbb"), + insertRow(1, "ccc"), + deleteRow(1, "ccc"), + insertRow(1, "ddd"))); + + List> expectedRecordsPerCheckpoint = + ImmutableList.of( + ImmutableList.of(insertRow(1, "bbb"), insertRow(2, "bbb")), + ImmutableList.of(insertRow(1, "bbb"), insertRow(2, "ddd")), + ImmutableList.of(insertRow(1, "ddd"), insertRow(2, "ddd"))); + + testSqlChangeLog( + TABLE_NAME, ImmutableList.of("id"), inputRowsPerCheckpoint, expectedRecordsPerCheckpoint); + } + + @Test + public void testChangeLogOnDataKey() throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of( + insertRow(1, "aaa"), + deleteRow(1, "aaa"), + insertRow(2, "bbb"), + insertRow(1, "bbb"), + insertRow(2, "aaa")), + ImmutableList.of( + updateBeforeRow(2, "aaa"), updateAfterRow(1, "ccc"), insertRow(1, "aaa")), + ImmutableList.of(deleteRow(1, "bbb"), insertRow(2, "aaa"), insertRow(2, "ccc"))); + + List> expectedRecords = + ImmutableList.of( + ImmutableList.of(insertRow(1, "bbb"), insertRow(2, "aaa")), + ImmutableList.of(insertRow(1, "aaa"), insertRow(1, "bbb"), insertRow(1, "ccc")), + ImmutableList.of( + insertRow(1, "aaa"), + insertRow(1, "ccc"), + insertRow(2, "aaa"), + insertRow(2, "ccc"))); + + testSqlChangeLog(TABLE_NAME, ImmutableList.of("data"), elementsPerCheckpoint, expectedRecords); + } + + @Test + public void testChangeLogOnIdDataKey() throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of( + insertRow(1, "aaa"), + deleteRow(1, "aaa"), + insertRow(2, "bbb"), + insertRow(1, "bbb"), + insertRow(2, "aaa")), + ImmutableList.of( + updateBeforeRow(2, "aaa"), updateAfterRow(1, "ccc"), insertRow(1, "aaa")), + ImmutableList.of(deleteRow(1, "bbb"), insertRow(2, "aaa"))); + + List> expectedRecords = + ImmutableList.of( + ImmutableList.of(insertRow(1, "bbb"), insertRow(2, "aaa"), insertRow(2, "bbb")), + ImmutableList.of( + insertRow(1, "aaa"), insertRow(1, "bbb"), insertRow(1, "ccc"), insertRow(2, "bbb")), + ImmutableList.of( + insertRow(1, "aaa"), + insertRow(1, "ccc"), + insertRow(2, "aaa"), + insertRow(2, "bbb"))); + + testSqlChangeLog( + TABLE_NAME, ImmutableList.of("data", "id"), elementsPerCheckpoint, expectedRecords); + } + + @Test + public void testPureInsertOnIdKey() throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of(insertRow(1, "aaa"), insertRow(2, "bbb")), + ImmutableList.of(insertRow(3, "ccc"), insertRow(4, "ddd")), + ImmutableList.of(insertRow(5, "eee"), insertRow(6, "fff"))); + + List> expectedRecords = + ImmutableList.of( + ImmutableList.of(insertRow(1, "aaa"), insertRow(2, "bbb")), + ImmutableList.of( + insertRow(1, "aaa"), insertRow(2, "bbb"), insertRow(3, "ccc"), insertRow(4, "ddd")), + ImmutableList.of( + insertRow(1, "aaa"), + insertRow(2, "bbb"), + insertRow(3, "ccc"), + insertRow(4, "ddd"), + insertRow(5, "eee"), + insertRow(6, "fff"))); + + testSqlChangeLog(TABLE_NAME, ImmutableList.of("data"), elementsPerCheckpoint, expectedRecords); + } + + private static Record record(int id, String data) { + return SimpleDataUtil.createRecord(id, data); + } + + private Table createTable(String tableName, List key, boolean isPartitioned) { + String partitionByCause = isPartitioned ? "PARTITIONED BY (data)" : ""; + sql( + "CREATE TABLE %s(id INT, data VARCHAR, PRIMARY KEY(%s) NOT ENFORCED) %s", + tableName, Joiner.on(',').join(key), partitionByCause); + + // Upgrade the iceberg table to format v2. + CatalogLoader loader = + CatalogLoader.hadoop( + "my_catalog", CONF, ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouse)); + Table table = loader.loadCatalog().loadTable(TableIdentifier.of(DATABASE_NAME, TABLE_NAME)); + TableOperations ops = ((BaseTable) table).operations(); + TableMetadata meta = ops.current(); + ops.commit(meta, meta.upgradeToFormatVersion(2)); + + return table; + } + + private void testSqlChangeLog( + String tableName, + List key, + List> inputRowsPerCheckpoint, + List> expectedRecordsPerCheckpoint) + throws Exception { + String dataId = BoundedTableFactory.registerDataSet(inputRowsPerCheckpoint); + sql( + "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" + + " WITH ('connector'='BoundedSource', 'data-id'='%s')", + SOURCE_TABLE, dataId); + + Assert.assertEquals( + "Should have the expected rows", + listJoin(inputRowsPerCheckpoint), + sql("SELECT * FROM %s", SOURCE_TABLE)); + + Table table = createTable(tableName, key, partitioned); + sql("INSERT INTO %s SELECT * FROM %s", tableName, SOURCE_TABLE); + + table.refresh(); + List snapshots = findValidSnapshots(table); + int expectedSnapshotNum = expectedRecordsPerCheckpoint.size(); + Assert.assertEquals( + "Should have the expected snapshot number", expectedSnapshotNum, snapshots.size()); + + for (int i = 0; i < expectedSnapshotNum; i++) { + long snapshotId = snapshots.get(i).snapshotId(); + List expectedRows = expectedRecordsPerCheckpoint.get(i); + Assert.assertEquals( + "Should have the expected records for the checkpoint#" + i, + expectedRowSet(table, expectedRows), + actualRowSet(table, snapshotId)); + } + + if (expectedSnapshotNum > 0) { + Assert.assertEquals( + "Should have the expected rows in the final table", + Sets.newHashSet(expectedRecordsPerCheckpoint.get(expectedSnapshotNum - 1)), + Sets.newHashSet(sql("SELECT * FROM %s", tableName))); + } + } + + private List findValidSnapshots(Table table) { + List validSnapshots = Lists.newArrayList(); + for (Snapshot snapshot : table.snapshots()) { + if (snapshot.allManifests(table.io()).stream() + .anyMatch(m -> snapshot.snapshotId() == m.snapshotId())) { + validSnapshots.add(snapshot); + } + } + return validSnapshots; + } + + private static StructLikeSet expectedRowSet(Table table, List rows) { + Record[] records = new Record[rows.size()]; + for (int i = 0; i < records.length; i++) { + records[i] = record((int) rows.get(i).getField(0), (String) rows.get(i).getField(1)); + } + return SimpleDataUtil.expectedRowSet(table, records); + } + + private static StructLikeSet actualRowSet(Table table, long snapshotId) throws IOException { + return SimpleDataUtil.actualRowSet(table, snapshotId, "*"); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java new file mode 100644 index 000000000000..e9372adda4c1 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java @@ -0,0 +1,207 @@ +/* + * 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.iceberg.flink; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Map; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; +import org.junit.Test; + +public class TestDataFileSerialization { + + private static final Schema DATE_SCHEMA = + new Schema( + required(1, "id", Types.LongType.get()), + optional(2, "data", Types.StringType.get()), + required(3, "date", Types.StringType.get()), + optional(4, "double", Types.DoubleType.get())); + + private static final PartitionSpec PARTITION_SPEC = + PartitionSpec.builderFor(DATE_SCHEMA).identity("date").build(); + + private static final Map COLUMN_SIZES = Maps.newHashMap(); + private static final Map VALUE_COUNTS = Maps.newHashMap(); + private static final Map NULL_VALUE_COUNTS = Maps.newHashMap(); + private static final Map NAN_VALUE_COUNTS = Maps.newHashMap(); + private static final Map LOWER_BOUNDS = Maps.newHashMap(); + private static final Map UPPER_BOUNDS = Maps.newHashMap(); + + static { + COLUMN_SIZES.put(1, 2L); + COLUMN_SIZES.put(2, 3L); + VALUE_COUNTS.put(1, 5L); + VALUE_COUNTS.put(2, 3L); + VALUE_COUNTS.put(4, 2L); + NULL_VALUE_COUNTS.put(1, 0L); + NULL_VALUE_COUNTS.put(2, 2L); + NAN_VALUE_COUNTS.put(4, 1L); + LOWER_BOUNDS.put(1, longToBuffer(0L)); + UPPER_BOUNDS.put(1, longToBuffer(4L)); + } + + private static final Metrics METRICS = + new Metrics( + 5L, null, VALUE_COUNTS, NULL_VALUE_COUNTS, NAN_VALUE_COUNTS, LOWER_BOUNDS, UPPER_BOUNDS); + + private static final DataFile DATA_FILE = + DataFiles.builder(PARTITION_SPEC) + .withPath("/path/to/data-1.parquet") + .withFileSizeInBytes(1234) + .withPartitionPath("date=2018-06-08") + .withMetrics(METRICS) + .withSplitOffsets(ImmutableList.of(4L)) + .withEncryptionKeyMetadata(ByteBuffer.allocate(4).putInt(34)) + .withSortOrder(SortOrder.unsorted()) + .build(); + + private static final DeleteFile POS_DELETE_FILE = + FileMetadata.deleteFileBuilder(PARTITION_SPEC) + .ofPositionDeletes() + .withPath("/path/to/pos-delete.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("date=2018-06-08") + .withMetrics(METRICS) + .withEncryptionKeyMetadata(ByteBuffer.allocate(4).putInt(35)) + .withRecordCount(23) + .build(); + + private static final DeleteFile EQ_DELETE_FILE = + FileMetadata.deleteFileBuilder(PARTITION_SPEC) + .ofEqualityDeletes(2, 3) + .withPath("/path/to/equality-delete.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("date=2018-06-08") + .withMetrics(METRICS) + .withEncryptionKeyMetadata(ByteBuffer.allocate(4).putInt(35)) + .withRecordCount(23) + .withSortOrder(SortOrder.unsorted()) + .build(); + + @Test + public void testJavaSerialization() throws Exception { + ByteArrayOutputStream bytes = new ByteArrayOutputStream(); + try (ObjectOutputStream out = new ObjectOutputStream(bytes)) { + out.writeObject(DATA_FILE); + out.writeObject(DATA_FILE.copy()); + + out.writeObject(POS_DELETE_FILE); + out.writeObject(POS_DELETE_FILE.copy()); + + out.writeObject(EQ_DELETE_FILE); + out.writeObject(EQ_DELETE_FILE.copy()); + } + + try (ObjectInputStream in = + new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { + for (int i = 0; i < 2; i += 1) { + Object obj = in.readObject(); + Assertions.assertThat(obj).as("Should be a DataFile").isInstanceOf(DataFile.class); + TestHelpers.assertEquals(DATA_FILE, (DataFile) obj); + } + + for (int i = 0; i < 2; i += 1) { + Object obj = in.readObject(); + Assertions.assertThat(obj) + .as("Should be a position DeleteFile") + .isInstanceOf(DeleteFile.class); + TestHelpers.assertEquals(POS_DELETE_FILE, (DeleteFile) obj); + } + + for (int i = 0; i < 2; i += 1) { + Object obj = in.readObject(); + Assertions.assertThat(obj) + .as("Should be a equality DeleteFile") + .isInstanceOf(DeleteFile.class); + TestHelpers.assertEquals(EQ_DELETE_FILE, (DeleteFile) obj); + } + } + } + + @Test + public void testDataFileKryoSerialization() throws IOException { + KryoSerializer kryo = new KryoSerializer<>(DataFile.class, new ExecutionConfig()); + + DataOutputSerializer outputView = new DataOutputSerializer(1024); + + kryo.serialize(DATA_FILE, outputView); + kryo.serialize(DATA_FILE.copy(), outputView); + + DataInputDeserializer inputView = new DataInputDeserializer(outputView.getCopyOfBuffer()); + DataFile dataFile1 = kryo.deserialize(inputView); + DataFile dataFile2 = kryo.deserialize(inputView); + + TestHelpers.assertEquals(DATA_FILE, dataFile1); + TestHelpers.assertEquals(DATA_FILE, dataFile2); + } + + @Test + public void testDeleteFileKryoSerialization() throws IOException { + KryoSerializer kryo = new KryoSerializer<>(DeleteFile.class, new ExecutionConfig()); + + DataOutputSerializer outputView = new DataOutputSerializer(1024); + + kryo.serialize(POS_DELETE_FILE, outputView); + kryo.serialize(POS_DELETE_FILE.copy(), outputView); + + kryo.serialize(EQ_DELETE_FILE, outputView); + kryo.serialize(EQ_DELETE_FILE.copy(), outputView); + + DataInputDeserializer inputView = new DataInputDeserializer(outputView.getCopyOfBuffer()); + + DeleteFile posDeleteFile1 = kryo.deserialize(inputView); + DeleteFile posDeleteFile2 = kryo.deserialize(inputView); + + TestHelpers.assertEquals(POS_DELETE_FILE, posDeleteFile1); + TestHelpers.assertEquals(POS_DELETE_FILE, posDeleteFile2); + + DeleteFile eqDeleteFile1 = kryo.deserialize(inputView); + DeleteFile eqDeleteFile2 = kryo.deserialize(inputView); + + TestHelpers.assertEquals(EQ_DELETE_FILE, eqDeleteFile1); + TestHelpers.assertEquals(EQ_DELETE_FILE, eqDeleteFile2); + } + + private static ByteBuffer longToBuffer(long value) { + return ByteBuffer.allocate(8).order(ByteOrder.LITTLE_ENDIAN).putLong(0, value); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java new file mode 100644 index 000000000000..884ea2d1d3b1 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink; + +import static org.apache.iceberg.types.Types.NestedField.required; + +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.types.Types; + +public class TestFixtures { + + private TestFixtures() {} + + public static final Schema SCHEMA = + new Schema( + required(1, "data", Types.StringType.get()), + required(2, "id", Types.LongType.get()), + required(3, "dt", Types.StringType.get())); + + public static final PartitionSpec SPEC = + PartitionSpec.builderFor(SCHEMA).identity("dt").bucket("id", 1).build(); + + public static final RowType ROW_TYPE = FlinkSchemaUtil.convert(SCHEMA); + + public static final String DATABASE = "default"; + public static final String TABLE = "t"; + public static final String SINK_TABLE = "t_sink"; + + public static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DATABASE, TABLE); + + public static final Schema TS_SCHEMA = + new Schema( + required(1, "ts", Types.TimestampType.withoutZone()), + required(2, "str", Types.StringType.get())); + + public static final PartitionSpec TS_SPEC = + PartitionSpec.builderFor(TS_SCHEMA).hour("ts").build(); + + public static final RowType TS_ROW_TYPE = FlinkSchemaUtil.convert(TS_SCHEMA); +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java new file mode 100644 index 000000000000..2fbd7627efab --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink; + +import java.io.File; +import java.util.concurrent.TimeUnit; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableDescriptor; +import org.apache.flink.table.api.TableEnvironment; +import org.assertj.core.api.Assertions; +import org.awaitility.Awaitility; +import org.junit.Test; + +public class TestFlinkAnonymousTable extends FlinkTestBase { + + @Test + public void testWriteAnonymousTable() throws Exception { + File warehouseDir = TEMPORARY_FOLDER.newFolder(); + TableEnvironment tEnv = getTableEnv(); + Table table = + tEnv.from( + TableDescriptor.forConnector("datagen") + .schema(Schema.newBuilder().column("f0", DataTypes.STRING()).build()) + .option("number-of-rows", "3") + .build()); + + TableDescriptor descriptor = + TableDescriptor.forConnector("iceberg") + .schema(Schema.newBuilder().column("f0", DataTypes.STRING()).build()) + .option("catalog-name", "hadoop_test") + .option("catalog-type", "hadoop") + .option("catalog-database", "test_db") + .option("catalog-table", "test") + .option("warehouse", warehouseDir.getAbsolutePath()) + .build(); + + table.insertInto(descriptor).execute(); + Awaitility.await() + .atMost(3, TimeUnit.SECONDS) + .untilAsserted( + () -> + Assertions.assertThat( + warehouseDir.toPath().resolve("test_db").resolve("test").toFile()) + .exists()); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java new file mode 100644 index 000000000000..47b47cb6262d --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java @@ -0,0 +1,293 @@ +/* + * 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.iceberg.flink; + +import java.io.File; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException; +import org.apache.flink.types.Row; +import org.apache.iceberg.Schema; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; +import org.junit.After; +import org.junit.Assert; +import org.junit.Assume; +import org.junit.Test; + +public class TestFlinkCatalogDatabase extends FlinkCatalogTestBase { + + public TestFlinkCatalogDatabase(String catalogName, Namespace baseNamespace) { + super(catalogName, baseNamespace); + } + + @After + @Override + public void clean() { + sql("DROP TABLE IF EXISTS %s.tl", flinkDatabase); + sql("DROP DATABASE IF EXISTS %s", flinkDatabase); + super.clean(); + } + + @Test + public void testCreateNamespace() { + Assert.assertFalse( + "Database should not already exist", + validationNamespaceCatalog.namespaceExists(icebergNamespace)); + + sql("CREATE DATABASE %s", flinkDatabase); + + Assert.assertTrue( + "Database should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace)); + + sql("CREATE DATABASE IF NOT EXISTS %s", flinkDatabase); + Assert.assertTrue( + "Database should still exist", + validationNamespaceCatalog.namespaceExists(icebergNamespace)); + + sql("DROP DATABASE IF EXISTS %s", flinkDatabase); + Assert.assertFalse( + "Database should be dropped", validationNamespaceCatalog.namespaceExists(icebergNamespace)); + + sql("CREATE DATABASE IF NOT EXISTS %s", flinkDatabase); + Assert.assertTrue( + "Database should be created", validationNamespaceCatalog.namespaceExists(icebergNamespace)); + } + + @Test + public void testDropEmptyDatabase() { + Assert.assertFalse( + "Namespace should not already exist", + validationNamespaceCatalog.namespaceExists(icebergNamespace)); + + sql("CREATE DATABASE %s", flinkDatabase); + + Assert.assertTrue( + "Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace)); + + sql("DROP DATABASE %s", flinkDatabase); + + Assert.assertFalse( + "Namespace should have been dropped", + validationNamespaceCatalog.namespaceExists(icebergNamespace)); + } + + @Test + public void testDropNonEmptyNamespace() { + Assume.assumeFalse( + "Hadoop catalog throws IOException: Directory is not empty.", isHadoopCatalog); + + Assert.assertFalse( + "Namespace should not already exist", + validationNamespaceCatalog.namespaceExists(icebergNamespace)); + + sql("CREATE DATABASE %s", flinkDatabase); + + validationCatalog.createTable( + TableIdentifier.of(icebergNamespace, "tl"), + new Schema(Types.NestedField.optional(0, "id", Types.LongType.get()))); + + Assert.assertTrue( + "Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace)); + Assert.assertTrue( + "Table should exist", + validationCatalog.tableExists(TableIdentifier.of(icebergNamespace, "tl"))); + + Assertions.assertThatThrownBy(() -> sql("DROP DATABASE %s", flinkDatabase)) + .cause() + .isInstanceOf(DatabaseNotEmptyException.class) + .hasMessage( + String.format("Database %s in catalog %s is not empty.", DATABASE, catalogName)); + + sql("DROP TABLE %s.tl", flinkDatabase); + } + + @Test + public void testListTables() { + Assert.assertFalse( + "Namespace should not already exist", + validationNamespaceCatalog.namespaceExists(icebergNamespace)); + + sql("CREATE DATABASE %s", flinkDatabase); + sql("USE CATALOG %s", catalogName); + sql("USE %s", DATABASE); + + Assert.assertTrue( + "Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace)); + + Assert.assertEquals("Should not list any tables", 0, sql("SHOW TABLES").size()); + + validationCatalog.createTable( + TableIdentifier.of(icebergNamespace, "tl"), + new Schema(Types.NestedField.optional(0, "id", Types.LongType.get()))); + + List tables = sql("SHOW TABLES"); + Assert.assertEquals("Only 1 table", 1, tables.size()); + Assert.assertEquals("Table name should match", "tl", tables.get(0).getField(0)); + } + + @Test + public void testListNamespace() { + Assert.assertFalse( + "Namespace should not already exist", + validationNamespaceCatalog.namespaceExists(icebergNamespace)); + + sql("CREATE DATABASE %s", flinkDatabase); + sql("USE CATALOG %s", catalogName); + + Assert.assertTrue( + "Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace)); + + List databases = sql("SHOW DATABASES"); + + if (isHadoopCatalog) { + Assert.assertEquals("Should have 1 database", 1, databases.size()); + Assert.assertEquals("Should have db database", "db", databases.get(0).getField(0)); + + if (!baseNamespace.isEmpty()) { + // test namespace not belongs to this catalog + validationNamespaceCatalog.createNamespace( + Namespace.of(baseNamespace.level(0), "UNKNOWN_NAMESPACE")); + databases = sql("SHOW DATABASES"); + Assert.assertEquals("Should have 1 database", 1, databases.size()); + Assert.assertEquals( + "Should have db and default database", "db", databases.get(0).getField(0)); + } + } else { + // If there are multiple classes extends FlinkTestBase, TestHiveMetastore may loose the + // creation for default + // database. See HiveMetaStore.HMSHandler.init. + Assert.assertTrue( + "Should have db database", + databases.stream().anyMatch(d -> Objects.equals(d.getField(0), "db"))); + } + } + + @Test + public void testCreateNamespaceWithMetadata() { + Assume.assumeFalse("HadoopCatalog does not support namespace metadata", isHadoopCatalog); + + Assert.assertFalse( + "Namespace should not already exist", + validationNamespaceCatalog.namespaceExists(icebergNamespace)); + + sql("CREATE DATABASE %s WITH ('prop'='value')", flinkDatabase); + + Assert.assertTrue( + "Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace)); + + Map nsMetadata = + validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace); + + Assert.assertEquals( + "Namespace should have expected prop value", "value", nsMetadata.get("prop")); + } + + @Test + public void testCreateNamespaceWithComment() { + Assume.assumeFalse("HadoopCatalog does not support namespace metadata", isHadoopCatalog); + + Assert.assertFalse( + "Namespace should not already exist", + validationNamespaceCatalog.namespaceExists(icebergNamespace)); + + sql("CREATE DATABASE %s COMMENT 'namespace doc'", flinkDatabase); + + Assert.assertTrue( + "Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace)); + + Map nsMetadata = + validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace); + + Assert.assertEquals( + "Namespace should have expected comment", "namespace doc", nsMetadata.get("comment")); + } + + @Test + public void testCreateNamespaceWithLocation() throws Exception { + Assume.assumeFalse("HadoopCatalog does not support namespace metadata", isHadoopCatalog); + + Assert.assertFalse( + "Namespace should not already exist", + validationNamespaceCatalog.namespaceExists(icebergNamespace)); + + File location = TEMPORARY_FOLDER.newFile(); + Assert.assertTrue(location.delete()); + + sql("CREATE DATABASE %s WITH ('location'='%s')", flinkDatabase, location); + + Assert.assertTrue( + "Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace)); + + Map nsMetadata = + validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace); + + Assert.assertEquals( + "Namespace should have expected location", + "file:" + location.getPath(), + nsMetadata.get("location")); + } + + @Test + public void testSetProperties() { + Assume.assumeFalse("HadoopCatalog does not support namespace metadata", isHadoopCatalog); + + Assert.assertFalse( + "Namespace should not already exist", + validationNamespaceCatalog.namespaceExists(icebergNamespace)); + + sql("CREATE DATABASE %s", flinkDatabase); + + Assert.assertTrue( + "Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace)); + + Map defaultMetadata = + validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace); + Assert.assertFalse( + "Default metadata should not have custom property", defaultMetadata.containsKey("prop")); + + sql("ALTER DATABASE %s SET ('prop'='value')", flinkDatabase); + + Map nsMetadata = + validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace); + + Assert.assertEquals( + "Namespace should have expected prop value", "value", nsMetadata.get("prop")); + } + + @Test + public void testHadoopNotSupportMeta() { + Assume.assumeTrue("HadoopCatalog does not support namespace metadata", isHadoopCatalog); + + Assert.assertFalse( + "Namespace should not already exist", + validationNamespaceCatalog.namespaceExists(icebergNamespace)); + + Assertions.assertThatThrownBy( + () -> sql("CREATE DATABASE %s WITH ('prop'='value')", flinkDatabase)) + .cause() + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage( + String.format( + "Cannot create namespace %s: metadata is not supported", icebergNamespace)); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java new file mode 100644 index 000000000000..ba08b76dd59d --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java @@ -0,0 +1,117 @@ +/* + * 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.iceberg.flink; + +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.hive.HiveCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.assertj.core.api.Assertions; +import org.junit.Before; +import org.junit.Test; + +public class TestFlinkCatalogFactory { + + private Map props; + + @Before + public void before() { + props = Maps.newHashMap(); + props.put("type", "iceberg"); + props.put(CatalogProperties.WAREHOUSE_LOCATION, "/tmp/location"); + } + + @Test + public void testCreateCatalogHive() { + String catalogName = "hiveCatalog"; + props.put( + FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HIVE); + + Catalog catalog = + FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration()) + .loadCatalog(); + + Assertions.assertThat(catalog).isNotNull().isInstanceOf(HiveCatalog.class); + } + + @Test + public void testCreateCatalogHadoop() { + String catalogName = "hadoopCatalog"; + props.put( + FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HADOOP); + + Catalog catalog = + FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration()) + .loadCatalog(); + + Assertions.assertThat(catalog).isNotNull().isInstanceOf(HadoopCatalog.class); + } + + @Test + public void testCreateCatalogCustom() { + String catalogName = "customCatalog"; + props.put(CatalogProperties.CATALOG_IMPL, CustomHadoopCatalog.class.getName()); + + Catalog catalog = + FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration()) + .loadCatalog(); + + Assertions.assertThat(catalog).isNotNull().isInstanceOf(CustomHadoopCatalog.class); + } + + @Test + public void testCreateCatalogCustomWithHiveCatalogTypeSet() { + String catalogName = "customCatalog"; + props.put(CatalogProperties.CATALOG_IMPL, CustomHadoopCatalog.class.getName()); + props.put( + FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HIVE); + + Assertions.assertThatThrownBy( + () -> FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith( + "Cannot create catalog customCatalog, both catalog-type and catalog-impl are set"); + } + + @Test + public void testLoadCatalogUnknown() { + String catalogName = "unknownCatalog"; + props.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "fooType"); + + Assertions.assertThatThrownBy( + () -> FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration())) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageStartingWith("Unknown catalog-type: fooType"); + } + + public static class CustomHadoopCatalog extends HadoopCatalog { + + public CustomHadoopCatalog() {} + + public CustomHadoopCatalog(Configuration conf, String warehouseLocation) { + setConf(conf); + initialize( + "custom", ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation)); + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java new file mode 100644 index 000000000000..8f5ddde91851 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java @@ -0,0 +1,692 @@ +/* + * 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.iceberg.flink; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Arrays; +import java.util.Collections; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.api.constraints.UniqueConstraint; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.iceberg.BaseTable; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DataOperations; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; +import org.junit.After; +import org.junit.Assert; +import org.junit.Assume; +import org.junit.Before; +import org.junit.Test; + +public class TestFlinkCatalogTable extends FlinkCatalogTestBase { + + public TestFlinkCatalogTable(String catalogName, Namespace baseNamespace) { + super(catalogName, baseNamespace); + } + + @Override + @Before + public void before() { + super.before(); + sql("CREATE DATABASE %s", flinkDatabase); + sql("USE CATALOG %s", catalogName); + sql("USE %s", DATABASE); + } + + @After + public void cleanNamespaces() { + sql("DROP TABLE IF EXISTS %s.tl", flinkDatabase); + sql("DROP TABLE IF EXISTS %s.tl2", flinkDatabase); + sql("DROP DATABASE IF EXISTS %s", flinkDatabase); + super.clean(); + } + + @Test + public void testGetTable() { + sql("CREATE TABLE tl(id BIGINT, strV STRING)"); + + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, "tl")); + Schema iSchema = + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "strV", Types.StringType.get())); + Assert.assertEquals( + "Should load the expected iceberg schema", iSchema.toString(), table.schema().toString()); + } + + @Test + public void testRenameTable() { + Assume.assumeFalse("HadoopCatalog does not support rename table", isHadoopCatalog); + + final Schema tableSchema = + new Schema(Types.NestedField.optional(0, "id", Types.LongType.get())); + validationCatalog.createTable(TableIdentifier.of(icebergNamespace, "tl"), tableSchema); + sql("ALTER TABLE tl RENAME TO tl2"); + + Assertions.assertThatThrownBy(() -> getTableEnv().from("tl")) + .isInstanceOf(ValidationException.class) + .hasMessage("Table `tl` was not found."); + + Schema actualSchema = FlinkSchemaUtil.convert(getTableEnv().from("tl2").getSchema()); + Assert.assertEquals(tableSchema.asStruct(), actualSchema.asStruct()); + } + + @Test + public void testCreateTable() throws TableNotExistException { + sql("CREATE TABLE tl(id BIGINT)"); + + Table table = table("tl"); + Assert.assertEquals( + new Schema(Types.NestedField.optional(1, "id", Types.LongType.get())).asStruct(), + table.schema().asStruct()); + + CatalogTable catalogTable = catalogTable("tl"); + Assert.assertEquals( + TableSchema.builder().field("id", DataTypes.BIGINT()).build(), catalogTable.getSchema()); + } + + @Test + public void testCreateTableWithPrimaryKey() throws Exception { + sql("CREATE TABLE tl(id BIGINT, data STRING, key STRING PRIMARY KEY NOT ENFORCED)"); + + Table table = table("tl"); + Assert.assertEquals( + "Should have the expected row key.", + Sets.newHashSet(table.schema().findField("key").fieldId()), + table.schema().identifierFieldIds()); + + CatalogTable catalogTable = catalogTable("tl"); + Optional uniqueConstraintOptional = catalogTable.getSchema().getPrimaryKey(); + Assert.assertTrue( + "Should have the expected unique constraint", uniqueConstraintOptional.isPresent()); + Assert.assertEquals( + "Should have the expected columns", + ImmutableList.of("key"), + uniqueConstraintOptional.get().getColumns()); + } + + @Test + public void testCreateTableWithMultiColumnsInPrimaryKey() throws Exception { + sql( + "CREATE TABLE tl(id BIGINT, data STRING, CONSTRAINT pk_constraint PRIMARY KEY(data, id) NOT ENFORCED)"); + + Table table = table("tl"); + Assert.assertEquals( + "Should have the expected RowKey", + Sets.newHashSet( + table.schema().findField("id").fieldId(), table.schema().findField("data").fieldId()), + table.schema().identifierFieldIds()); + + CatalogTable catalogTable = catalogTable("tl"); + Optional uniqueConstraintOptional = catalogTable.getSchema().getPrimaryKey(); + Assert.assertTrue( + "Should have the expected unique constraint", uniqueConstraintOptional.isPresent()); + Assert.assertEquals( + "Should have the expected columns", + ImmutableSet.of("data", "id"), + ImmutableSet.copyOf(uniqueConstraintOptional.get().getColumns())); + } + + @Test + public void testCreateTableIfNotExists() { + sql("CREATE TABLE tl(id BIGINT)"); + + // Assert that table does exist. + assertThat(table("tl")).isNotNull(); + + sql("DROP TABLE tl"); + Assertions.assertThatThrownBy(() -> table("tl")) + .isInstanceOf(NoSuchTableException.class) + .hasMessage("Table does not exist: " + getFullQualifiedTableName("tl")); + + sql("CREATE TABLE IF NOT EXISTS tl(id BIGINT)"); + assertThat(table("tl").properties()).doesNotContainKey("key"); + + table("tl").updateProperties().set("key", "value").commit(); + assertThat(table("tl").properties()).containsEntry("key", "value"); + + sql("CREATE TABLE IF NOT EXISTS tl(id BIGINT)"); + assertThat(table("tl").properties()).containsEntry("key", "value"); + } + + @Test + public void testCreateTableLike() throws TableNotExistException { + sql("CREATE TABLE tl(id BIGINT)"); + sql("CREATE TABLE tl2 LIKE tl"); + + Table table = table("tl2"); + Assert.assertEquals( + new Schema(Types.NestedField.optional(1, "id", Types.LongType.get())).asStruct(), + table.schema().asStruct()); + + CatalogTable catalogTable = catalogTable("tl2"); + Assert.assertEquals( + TableSchema.builder().field("id", DataTypes.BIGINT()).build(), catalogTable.getSchema()); + } + + @Test + public void testCreateTableLocation() { + Assume.assumeFalse( + "HadoopCatalog does not support creating table with location", isHadoopCatalog); + + sql("CREATE TABLE tl(id BIGINT) WITH ('location'='file:///tmp/location')"); + + Table table = table("tl"); + Assert.assertEquals( + new Schema(Types.NestedField.optional(1, "id", Types.LongType.get())).asStruct(), + table.schema().asStruct()); + Assert.assertEquals("file:///tmp/location", table.location()); + } + + @Test + public void testCreatePartitionTable() throws TableNotExistException { + sql("CREATE TABLE tl(id BIGINT, dt STRING) PARTITIONED BY(dt)"); + + Table table = table("tl"); + Assert.assertEquals( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct(), + table.schema().asStruct()); + Assert.assertEquals( + PartitionSpec.builderFor(table.schema()).identity("dt").build(), table.spec()); + + CatalogTable catalogTable = catalogTable("tl"); + Assert.assertEquals( + TableSchema.builder() + .field("id", DataTypes.BIGINT()) + .field("dt", DataTypes.STRING()) + .build(), + catalogTable.getSchema()); + Assert.assertEquals(Collections.singletonList("dt"), catalogTable.getPartitionKeys()); + } + + @Test + public void testCreateTableWithFormatV2ThroughTableProperty() throws Exception { + sql("CREATE TABLE tl(id BIGINT) WITH ('format-version'='2')"); + + Table table = table("tl"); + Assert.assertEquals( + "should create table using format v2", + 2, + ((BaseTable) table).operations().current().formatVersion()); + } + + @Test + public void testUpgradeTableWithFormatV2ThroughTableProperty() throws Exception { + sql("CREATE TABLE tl(id BIGINT) WITH ('format-version'='1')"); + + Table table = table("tl"); + TableOperations ops = ((BaseTable) table).operations(); + Assert.assertEquals("should create table using format v1", 1, ops.refresh().formatVersion()); + + sql("ALTER TABLE tl SET('format-version'='2')"); + Assert.assertEquals("should update table to use format v2", 2, ops.refresh().formatVersion()); + } + + @Test + public void testDowngradeTableToFormatV1ThroughTablePropertyFails() throws Exception { + sql("CREATE TABLE tl(id BIGINT) WITH ('format-version'='2')"); + + Table table = table("tl"); + TableOperations ops = ((BaseTable) table).operations(); + Assert.assertEquals("should create table using format v2", 2, ops.refresh().formatVersion()); + + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl SET('format-version'='1')")) + .rootCause() + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot downgrade v2 table to v1"); + } + + @Test + public void testLoadTransformPartitionTable() throws TableNotExistException { + Schema schema = new Schema(Types.NestedField.optional(0, "id", Types.LongType.get())); + validationCatalog.createTable( + TableIdentifier.of(icebergNamespace, "tl"), + schema, + PartitionSpec.builderFor(schema).bucket("id", 100).build()); + + CatalogTable catalogTable = catalogTable("tl"); + Assert.assertEquals( + TableSchema.builder().field("id", DataTypes.BIGINT()).build(), catalogTable.getSchema()); + Assert.assertEquals(Collections.emptyList(), catalogTable.getPartitionKeys()); + } + + @Test + public void testAlterTableProperties() throws TableNotExistException { + sql("CREATE TABLE tl(id BIGINT) WITH ('oldK'='oldV')"); + Map properties = Maps.newHashMap(); + properties.put("oldK", "oldV"); + + // new + sql("ALTER TABLE tl SET('newK'='newV')"); + properties.put("newK", "newV"); + assertThat(table("tl").properties()).containsAllEntriesOf(properties); + + // update old + sql("ALTER TABLE tl SET('oldK'='oldV2')"); + properties.put("oldK", "oldV2"); + assertThat(table("tl").properties()).containsAllEntriesOf(properties); + + // remove property + sql("ALTER TABLE tl RESET('oldK')"); + properties.remove("oldK"); + assertThat(table("tl").properties()).containsAllEntriesOf(properties); + } + + @Test + public void testAlterTableAddColumn() { + sql("CREATE TABLE tl(id BIGINT)"); + Schema schemaBefore = table("tl").schema(); + Assert.assertEquals( + new Schema(Types.NestedField.optional(1, "id", Types.LongType.get())).asStruct(), + schemaBefore.asStruct()); + + sql("ALTER TABLE tl ADD (dt STRING)"); + Schema schemaAfter1 = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct(), + schemaAfter1.asStruct()); + + // Add multiple columns + sql("ALTER TABLE tl ADD (col1 STRING, col2 BIGINT)"); + Schema schemaAfter2 = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get()), + Types.NestedField.optional(3, "col1", Types.StringType.get()), + Types.NestedField.optional(4, "col2", Types.LongType.get())) + .asStruct(), + schemaAfter2.asStruct()); + + // Adding a required field should fail because Iceberg's SchemaUpdate does not allow + // incompatible changes. + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl ADD (pk STRING NOT NULL)")) + .hasRootCauseInstanceOf(IllegalArgumentException.class) + .hasRootCauseMessage("Incompatible change: cannot add required column: pk"); + + // Adding an existing field should fail due to Flink's internal validation. + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl ADD (id STRING)")) + .isInstanceOf(ValidationException.class) + .hasMessageContaining("Try to add a column `id` which already exists in the table."); + } + + @Test + public void testAlterTableDropColumn() { + sql("CREATE TABLE tl(id BIGINT, dt STRING, col1 STRING, col2 BIGINT)"); + Schema schemaBefore = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get()), + Types.NestedField.optional(3, "col1", Types.StringType.get()), + Types.NestedField.optional(4, "col2", Types.LongType.get())) + .asStruct(), + schemaBefore.asStruct()); + + sql("ALTER TABLE tl DROP (dt)"); + Schema schemaAfter1 = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(3, "col1", Types.StringType.get()), + Types.NestedField.optional(4, "col2", Types.LongType.get())) + .asStruct(), + schemaAfter1.asStruct()); + + // Drop multiple columns + sql("ALTER TABLE tl DROP (col1, col2)"); + Schema schemaAfter2 = table("tl").schema(); + Assert.assertEquals( + new Schema(Types.NestedField.optional(1, "id", Types.LongType.get())).asStruct(), + schemaAfter2.asStruct()); + + // Dropping an non-existing field should fail due to Flink's internal validation. + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl DROP (foo)")) + .isInstanceOf(ValidationException.class) + .hasMessageContaining("The column `foo` does not exist in the base table."); + + // Dropping an already-deleted field should fail due to Flink's internal validation. + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl DROP (dt)")) + .isInstanceOf(ValidationException.class) + .hasMessageContaining("The column `dt` does not exist in the base table."); + } + + @Test + public void testAlterTableModifyColumnName() { + sql("CREATE TABLE tl(id BIGINT, dt STRING)"); + Schema schemaBefore = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct(), + schemaBefore.asStruct()); + + sql("ALTER TABLE tl RENAME dt TO data"); + Schema schemaAfter = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get())) + .asStruct(), + schemaAfter.asStruct()); + } + + @Test + public void testAlterTableModifyColumnType() { + sql("CREATE TABLE tl(id INTEGER, dt STRING)"); + Schema schemaBefore = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct(), + schemaBefore.asStruct()); + + // Promote type from Integer to Long + sql("ALTER TABLE tl MODIFY (id BIGINT)"); + Schema schemaAfter = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct(), + schemaAfter.asStruct()); + + // Type change that doesn't follow the type-promotion rule should fail due to Iceberg's + // validation. + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (dt INTEGER)")) + .isInstanceOf(TableException.class) + .hasRootCauseInstanceOf(IllegalArgumentException.class) + .hasRootCauseMessage("Cannot change column type: dt: string -> int"); + } + + @Test + public void testAlterTableModifyColumnNullability() { + sql("CREATE TABLE tl(id INTEGER NOT NULL, dt STRING)"); + Schema schemaBefore = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct(), + schemaBefore.asStruct()); + + // Changing nullability from optional to required should fail + // because Iceberg's SchemaUpdate does not allow incompatible changes. + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (dt STRING NOT NULL)")) + .isInstanceOf(TableException.class) + .hasRootCauseInstanceOf(IllegalArgumentException.class) + .hasRootCauseMessage("Cannot change column nullability: dt: optional -> required"); + + // Set nullability from required to optional + sql("ALTER TABLE tl MODIFY (id INTEGER)"); + Schema schemaAfter = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct(), + schemaAfter.asStruct()); + } + + @Test + public void testAlterTableModifyColumnPosition() { + sql("CREATE TABLE tl(id BIGINT, dt STRING)"); + Schema schemaBefore = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct(), + schemaBefore.asStruct()); + + sql("ALTER TABLE tl MODIFY (dt STRING FIRST)"); + Schema schemaAfter = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.optional(2, "dt", Types.StringType.get()), + Types.NestedField.optional(1, "id", Types.LongType.get())) + .asStruct(), + schemaAfter.asStruct()); + + sql("ALTER TABLE tl MODIFY (dt STRING AFTER id)"); + Schema schemaAfterAfter = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct(), + schemaAfterAfter.asStruct()); + + // Modifying the position of a non-existing column should fail due to Flink's internal + // validation. + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (non_existing STRING FIRST)")) + .isInstanceOf(ValidationException.class) + .hasMessageContaining( + "Try to modify a column `non_existing` which does not exist in the table."); + + // Moving a column after a non-existing column should fail due to Flink's internal validation. + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (dt STRING AFTER non_existing)")) + .isInstanceOf(ValidationException.class) + .hasMessageContaining( + "Referenced column `non_existing` by 'AFTER' does not exist in the table."); + } + + @Test + public void testAlterTableModifyColumnComment() { + sql("CREATE TABLE tl(id BIGINT, dt STRING)"); + Schema schemaBefore = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct(), + schemaBefore.asStruct()); + + sql("ALTER TABLE tl MODIFY (dt STRING COMMENT 'comment for dt field')"); + Schema schemaAfter = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get(), "comment for dt field")) + .asStruct(), + schemaAfter.asStruct()); + } + + @Test + public void testAlterTableConstraint() { + sql("CREATE TABLE tl(id BIGINT NOT NULL, dt STRING NOT NULL, col1 STRING)"); + Schema schemaBefore = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.required(2, "dt", Types.StringType.get()), + Types.NestedField.optional(3, "col1", Types.StringType.get())) + .asStruct(), + schemaBefore.asStruct()); + Assert.assertEquals(ImmutableSet.of(), schemaBefore.identifierFieldNames()); + + sql("ALTER TABLE tl ADD (PRIMARY KEY (id) NOT ENFORCED)"); + Schema schemaAfterAdd = table("tl").schema(); + Assert.assertEquals(ImmutableSet.of("id"), schemaAfterAdd.identifierFieldNames()); + + sql("ALTER TABLE tl MODIFY (PRIMARY KEY (dt) NOT ENFORCED)"); + Schema schemaAfterModify = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.required(2, "dt", Types.StringType.get()), + Types.NestedField.optional(3, "col1", Types.StringType.get())) + .asStruct(), + schemaAfterModify.asStruct()); + Assert.assertEquals(ImmutableSet.of("dt"), schemaAfterModify.identifierFieldNames()); + + // Composite primary key + sql("ALTER TABLE tl MODIFY (PRIMARY KEY (id, dt) NOT ENFORCED)"); + Schema schemaAfterComposite = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.required(2, "dt", Types.StringType.get()), + Types.NestedField.optional(3, "col1", Types.StringType.get())) + .asStruct(), + schemaAfterComposite.asStruct()); + Assert.assertEquals(ImmutableSet.of("id", "dt"), schemaAfterComposite.identifierFieldNames()); + + // Setting an optional field as primary key should fail + // because Iceberg's SchemaUpdate does not allow incompatible changes. + Assertions.assertThatThrownBy( + () -> sql("ALTER TABLE tl MODIFY (PRIMARY KEY (col1) NOT ENFORCED)")) + .isInstanceOf(TableException.class) + .hasRootCauseInstanceOf(IllegalArgumentException.class) + .hasRootCauseMessage("Cannot add field col1 as an identifier field: not a required field"); + + // Setting a composite key containing an optional field should fail + // because Iceberg's SchemaUpdate does not allow incompatible changes. + Assertions.assertThatThrownBy( + () -> sql("ALTER TABLE tl MODIFY (PRIMARY KEY (id, col1) NOT ENFORCED)")) + .isInstanceOf(TableException.class) + .hasRootCauseInstanceOf(IllegalArgumentException.class) + .hasRootCauseMessage("Cannot add field col1 as an identifier field: not a required field"); + + // Dropping constraints is not supported yet + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl DROP PRIMARY KEY")) + .isInstanceOf(TableException.class) + .hasRootCauseInstanceOf(UnsupportedOperationException.class) + .hasRootCauseMessage("Unsupported table change: DropConstraint."); + } + + @Test + public void testRelocateTable() { + Assume.assumeFalse("HadoopCatalog does not support relocate table", isHadoopCatalog); + + sql("CREATE TABLE tl(id BIGINT)"); + sql("ALTER TABLE tl SET('location'='file:///tmp/location')"); + Assert.assertEquals("file:///tmp/location", table("tl").location()); + } + + @Test + public void testSetCurrentAndCherryPickSnapshotId() { + sql("CREATE TABLE tl(c1 INT, c2 STRING, c3 STRING) PARTITIONED BY (c1)"); + + Table table = table("tl"); + + DataFile fileA = + DataFiles.builder(table.spec()) + .withPath("/path/to/data-a.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=0") // easy way to set partition data for now + .withRecordCount(1) + .build(); + DataFile fileB = + DataFiles.builder(table.spec()) + .withPath("/path/to/data-b.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=1") // easy way to set partition data for now + .withRecordCount(1) + .build(); + DataFile replacementFile = + DataFiles.builder(table.spec()) + .withPath("/path/to/data-a-replacement.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=0") // easy way to set partition data for now + .withRecordCount(1) + .build(); + + table.newAppend().appendFile(fileA).commit(); + long snapshotId = table.currentSnapshot().snapshotId(); + + // stage an overwrite that replaces FILE_A + table.newReplacePartitions().addFile(replacementFile).stageOnly().commit(); + + Snapshot staged = Iterables.getLast(table.snapshots()); + Assert.assertEquals( + "Should find the staged overwrite snapshot", DataOperations.OVERWRITE, staged.operation()); + + // add another append so that the original commit can't be fast-forwarded + table.newAppend().appendFile(fileB).commit(); + + // test cherry pick + sql("ALTER TABLE tl SET('cherry-pick-snapshot-id'='%s')", staged.snapshotId()); + validateTableFiles(table, fileB, replacementFile); + + // test set current snapshot + sql("ALTER TABLE tl SET('current-snapshot-id'='%s')", snapshotId); + validateTableFiles(table, fileA); + } + + private void validateTableFiles(Table tbl, DataFile... expectedFiles) { + tbl.refresh(); + Set expectedFilePaths = + Arrays.stream(expectedFiles).map(DataFile::path).collect(Collectors.toSet()); + Set actualFilePaths = + StreamSupport.stream(tbl.newScan().planFiles().spliterator(), false) + .map(FileScanTask::file) + .map(ContentFile::path) + .collect(Collectors.toSet()); + Assert.assertEquals("Files should match", expectedFilePaths, actualFilePaths); + } + + private Table table(String name) { + return validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, name)); + } + + private CatalogTable catalogTable(String name) throws TableNotExistException { + return (CatalogTable) + getTableEnv() + .getCatalog(getTableEnv().getCurrentCatalog()) + .get() + .getTable(new ObjectPath(DATABASE, name)); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java new file mode 100644 index 000000000000..0008e4320c8a --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java @@ -0,0 +1,121 @@ +/* + * 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.iceberg.flink; + +import java.util.List; +import org.apache.flink.table.catalog.CatalogPartitionSpec; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.assertj.core.api.Assertions; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runners.Parameterized; + +public class TestFlinkCatalogTablePartitions extends FlinkCatalogTestBase { + + private String tableName = "test_table"; + + private final FileFormat format; + + @Parameterized.Parameters( + name = "catalogName={0}, baseNamespace={1}, format={2}, cacheEnabled={3}") + public static Iterable parameters() { + List parameters = Lists.newArrayList(); + for (FileFormat format : + new FileFormat[] {FileFormat.ORC, FileFormat.AVRO, FileFormat.PARQUET}) { + for (Boolean cacheEnabled : new Boolean[] {true, false}) { + for (Object[] catalogParams : FlinkCatalogTestBase.parameters()) { + String catalogName = (String) catalogParams[0]; + Namespace baseNamespace = (Namespace) catalogParams[1]; + parameters.add(new Object[] {catalogName, baseNamespace, format, cacheEnabled}); + } + } + } + return parameters; + } + + public TestFlinkCatalogTablePartitions( + String catalogName, Namespace baseNamespace, FileFormat format, boolean cacheEnabled) { + super(catalogName, baseNamespace); + this.format = format; + config.put(CatalogProperties.CACHE_ENABLED, String.valueOf(cacheEnabled)); + } + + @Override + @Before + public void before() { + super.before(); + sql("CREATE DATABASE %s", flinkDatabase); + sql("USE CATALOG %s", catalogName); + sql("USE %s", DATABASE); + } + + @After + public void cleanNamespaces() { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); + sql("DROP DATABASE IF EXISTS %s", flinkDatabase); + super.clean(); + } + + @Test + public void testListPartitionsWithUnpartitionedTable() { + sql( + "CREATE TABLE %s (id INT, data VARCHAR) with ('write.format.default'='%s')", + tableName, format.name()); + sql("INSERT INTO %s SELECT 1,'a'", tableName); + + ObjectPath objectPath = new ObjectPath(DATABASE, tableName); + FlinkCatalog flinkCatalog = (FlinkCatalog) getTableEnv().getCatalog(catalogName).get(); + Assertions.assertThatThrownBy(() -> flinkCatalog.listPartitions(objectPath)) + .isInstanceOf(TableNotPartitionedException.class) + .hasMessageStartingWith("Table db.test_table in catalog") + .hasMessageEndingWith("is not partitioned."); + } + + @Test + public void testListPartitionsWithPartitionedTable() + throws TableNotExistException, TableNotPartitionedException { + sql( + "CREATE TABLE %s (id INT, data VARCHAR) PARTITIONED BY (data) " + + "with ('write.format.default'='%s')", + tableName, format.name()); + sql("INSERT INTO %s SELECT 1,'a'", tableName); + sql("INSERT INTO %s SELECT 2,'b'", tableName); + + ObjectPath objectPath = new ObjectPath(DATABASE, tableName); + FlinkCatalog flinkCatalog = (FlinkCatalog) getTableEnv().getCatalog(catalogName).get(); + List list = flinkCatalog.listPartitions(objectPath); + Assert.assertEquals("Should have 2 partition", 2, list.size()); + + List expected = Lists.newArrayList(); + CatalogPartitionSpec partitionSpec1 = new CatalogPartitionSpec(ImmutableMap.of("data", "a")); + CatalogPartitionSpec partitionSpec2 = new CatalogPartitionSpec(ImmutableMap.of("data", "b")); + expected.add(partitionSpec1); + expected.add(partitionSpec2); + Assert.assertEquals("Should produce the expected catalog partition specs.", list, expected); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java new file mode 100644 index 000000000000..4b6ac25ab8e3 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java @@ -0,0 +1,61 @@ +/* + * 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.iceberg.flink; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.time.Duration; +import java.util.Map; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.Test; + +public class TestFlinkConfParser { + + @Test + public void testDurationConf() { + Map writeOptions = ImmutableMap.of("write-prop", "111s"); + + ConfigOption configOption = + ConfigOptions.key("conf-prop").durationType().noDefaultValue(); + Configuration flinkConf = new Configuration(); + flinkConf.setString(configOption.key(), "222s"); + + Table table = mock(Table.class); + when(table.properties()).thenReturn(ImmutableMap.of("table-prop", "333s")); + + FlinkConfParser confParser = new FlinkConfParser(table, writeOptions, flinkConf); + Duration defaultVal = Duration.ofMillis(999); + + Duration result = + confParser.durationConf().option("write-prop").defaultValue(defaultVal).parse(); + assertThat(result).isEqualTo(Duration.ofSeconds(111)); + + result = confParser.durationConf().flinkConfig(configOption).defaultValue(defaultVal).parse(); + assertThat(result).isEqualTo(Duration.ofSeconds(222)); + + result = confParser.durationConf().tableProperty("table-prop").defaultValue(defaultVal).parse(); + assertThat(result).isEqualTo(Duration.ofSeconds(333)); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java new file mode 100644 index 000000000000..c89ea4f53054 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java @@ -0,0 +1,467 @@ +/* + * 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.iceberg.flink; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.Expressions; +import org.apache.flink.table.api.TableColumn; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.expressions.ApiExpressionUtils; +import org.apache.flink.table.expressions.CallExpression; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.FieldReferenceExpression; +import org.apache.flink.table.expressions.ResolvedExpression; +import org.apache.flink.table.expressions.UnresolvedCallExpression; +import org.apache.flink.table.expressions.UnresolvedReferenceExpression; +import org.apache.flink.table.expressions.ValueLiteralExpression; +import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor; +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +import org.apache.iceberg.expressions.And; +import org.apache.iceberg.expressions.BoundLiteralPredicate; +import org.apache.iceberg.expressions.Not; +import org.apache.iceberg.expressions.Or; +import org.apache.iceberg.expressions.UnboundPredicate; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.util.DateTimeUtil; +import org.apache.iceberg.util.Pair; +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Test; + +public class TestFlinkFilters { + + private static final TableSchema TABLE_SCHEMA = + TableSchema.builder() + .field("field1", DataTypes.INT()) + .field("field2", DataTypes.BIGINT()) + .field("field3", DataTypes.FLOAT()) + .field("field4", DataTypes.DOUBLE()) + .field("field5", DataTypes.STRING()) + .field("field6", DataTypes.BOOLEAN()) + .field("field7", DataTypes.BINARY(2)) + .field("field8", DataTypes.DECIMAL(10, 2)) + .field("field9", DataTypes.DATE()) + .field("field10", DataTypes.TIME()) + .field("field11", DataTypes.TIMESTAMP()) + .field("field12", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()) + .build(); + + // A map list of fields and values used to verify the conversion of flink expression to iceberg + // expression + private static final List> FIELD_VALUE_LIST = + ImmutableList.of( + Pair.of("field1", 1), + Pair.of("field2", 2L), + Pair.of("field3", 3F), + Pair.of("field4", 4D), + Pair.of("field5", "iceberg"), + Pair.of("field6", true), + Pair.of("field7", new byte[] {'a', 'b'}), + Pair.of("field8", BigDecimal.valueOf(10.12)), + Pair.of("field9", DateTimeUtil.daysFromDate(LocalDate.now())), + Pair.of("field10", DateTimeUtil.microsFromTime(LocalTime.now())), + Pair.of("field11", DateTimeUtil.microsFromTimestamp(LocalDateTime.now())), + Pair.of("field12", DateTimeUtil.microsFromInstant(Instant.now()))); + + @Test + public void testFlinkDataTypeEqual() { + matchLiteral("field1", 1, 1); + matchLiteral("field2", 10L, 10L); + matchLiteral("field3", 1.2F, 1.2F); + matchLiteral("field4", 3.4D, 3.4D); + matchLiteral("field5", "abcd", "abcd"); + matchLiteral("field6", true, true); + matchLiteral("field7", new byte[] {'a', 'b'}, ByteBuffer.wrap(new byte[] {'a', 'b'})); + matchLiteral("field8", BigDecimal.valueOf(10.12), BigDecimal.valueOf(10.12)); + + LocalDate date = LocalDate.parse("2020-12-23"); + matchLiteral("field9", date, DateTimeUtil.daysFromDate(date)); + + LocalTime time = LocalTime.parse("12:13:14"); + matchLiteral("field10", time, DateTimeUtil.microsFromTime(time)); + + LocalDateTime dateTime = LocalDateTime.parse("2020-12-23T12:13:14"); + matchLiteral("field11", dateTime, DateTimeUtil.microsFromTimestamp(dateTime)); + + Instant instant = Instant.parse("2020-12-23T12:13:14.00Z"); + matchLiteral("field12", instant, DateTimeUtil.microsFromInstant(instant)); + } + + @Test + public void testEquals() { + for (Pair pair : FIELD_VALUE_LIST) { + UnboundPredicate expected = + org.apache.iceberg.expressions.Expressions.equal(pair.first(), pair.second()); + + Optional actual = + FlinkFilters.convert( + resolve(Expressions.$(pair.first()).isEqual(Expressions.lit(pair.second())))); + Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertPredicatesMatch(expected, actual.get()); + + Optional actual1 = + FlinkFilters.convert( + resolve(Expressions.lit(pair.second()).isEqual(Expressions.$(pair.first())))); + Assert.assertTrue("Conversion should succeed", actual1.isPresent()); + assertPredicatesMatch(expected, actual1.get()); + } + } + + @Test + public void testEqualsNaN() { + UnboundPredicate expected = org.apache.iceberg.expressions.Expressions.isNaN("field3"); + + Optional actual = + FlinkFilters.convert(resolve(Expressions.$("field3").isEqual(Expressions.lit(Float.NaN)))); + Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertPredicatesMatch(expected, actual.get()); + + Optional actual1 = + FlinkFilters.convert(resolve(Expressions.lit(Float.NaN).isEqual(Expressions.$("field3")))); + Assert.assertTrue("Conversion should succeed", actual1.isPresent()); + assertPredicatesMatch(expected, actual1.get()); + } + + @Test + public void testNotEquals() { + for (Pair pair : FIELD_VALUE_LIST) { + UnboundPredicate expected = + org.apache.iceberg.expressions.Expressions.notEqual(pair.first(), pair.second()); + + Optional actual = + FlinkFilters.convert( + resolve(Expressions.$(pair.first()).isNotEqual(Expressions.lit(pair.second())))); + Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertPredicatesMatch(expected, actual.get()); + + Optional actual1 = + FlinkFilters.convert( + resolve(Expressions.lit(pair.second()).isNotEqual(Expressions.$(pair.first())))); + Assert.assertTrue("Conversion should succeed", actual1.isPresent()); + assertPredicatesMatch(expected, actual1.get()); + } + } + + @Test + public void testNotEqualsNaN() { + UnboundPredicate expected = org.apache.iceberg.expressions.Expressions.notNaN("field3"); + + Optional actual = + FlinkFilters.convert( + resolve(Expressions.$("field3").isNotEqual(Expressions.lit(Float.NaN)))); + Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertPredicatesMatch(expected, actual.get()); + + Optional actual1 = + FlinkFilters.convert( + resolve(Expressions.lit(Float.NaN).isNotEqual(Expressions.$("field3")))); + Assert.assertTrue("Conversion should succeed", actual1.isPresent()); + assertPredicatesMatch(expected, actual1.get()); + } + + @Test + public void testGreaterThan() { + UnboundPredicate expected = + org.apache.iceberg.expressions.Expressions.greaterThan("field1", 1); + + Optional actual = + FlinkFilters.convert(resolve(Expressions.$("field1").isGreater(Expressions.lit(1)))); + Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertPredicatesMatch(expected, actual.get()); + + Optional actual1 = + FlinkFilters.convert(resolve(Expressions.lit(1).isLess(Expressions.$("field1")))); + Assert.assertTrue("Conversion should succeed", actual1.isPresent()); + assertPredicatesMatch(expected, actual1.get()); + } + + @Test + public void testGreaterThanEquals() { + UnboundPredicate expected = + org.apache.iceberg.expressions.Expressions.greaterThanOrEqual("field1", 1); + + Optional actual = + FlinkFilters.convert(resolve(Expressions.$("field1").isGreaterOrEqual(Expressions.lit(1)))); + Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertPredicatesMatch(expected, actual.get()); + + Optional actual1 = + FlinkFilters.convert(resolve(Expressions.lit(1).isLessOrEqual(Expressions.$("field1")))); + Assert.assertTrue("Conversion should succeed", actual1.isPresent()); + assertPredicatesMatch(expected, actual1.get()); + } + + @Test + public void testLessThan() { + UnboundPredicate expected = + org.apache.iceberg.expressions.Expressions.lessThan("field1", 1); + + Optional actual = + FlinkFilters.convert(resolve(Expressions.$("field1").isLess(Expressions.lit(1)))); + Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertPredicatesMatch(expected, actual.get()); + + Optional actual1 = + FlinkFilters.convert(resolve(Expressions.lit(1).isGreater(Expressions.$("field1")))); + Assert.assertTrue("Conversion should succeed", actual1.isPresent()); + assertPredicatesMatch(expected, actual1.get()); + } + + @Test + public void testLessThanEquals() { + UnboundPredicate expected = + org.apache.iceberg.expressions.Expressions.lessThanOrEqual("field1", 1); + + Optional actual = + FlinkFilters.convert(resolve(Expressions.$("field1").isLessOrEqual(Expressions.lit(1)))); + Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertPredicatesMatch(expected, actual.get()); + + Optional actual1 = + FlinkFilters.convert(resolve(Expressions.lit(1).isGreaterOrEqual(Expressions.$("field1")))); + Assert.assertTrue("Conversion should succeed", actual1.isPresent()); + assertPredicatesMatch(expected, actual1.get()); + } + + @Test + public void testIsNull() { + Expression expr = resolve(Expressions.$("field1").isNull()); + Optional actual = FlinkFilters.convert(expr); + Assert.assertTrue("Conversion should succeed", actual.isPresent()); + UnboundPredicate expected = org.apache.iceberg.expressions.Expressions.isNull("field1"); + assertPredicatesMatch(expected, actual.get()); + } + + @Test + public void testIsNotNull() { + Expression expr = resolve(Expressions.$("field1").isNotNull()); + Optional actual = FlinkFilters.convert(expr); + Assert.assertTrue("Conversion should succeed", actual.isPresent()); + UnboundPredicate expected = + org.apache.iceberg.expressions.Expressions.notNull("field1"); + assertPredicatesMatch(expected, actual.get()); + } + + @Test + public void testAnd() { + Expression expr = + resolve( + Expressions.$("field1") + .isEqual(Expressions.lit(1)) + .and(Expressions.$("field2").isEqual(Expressions.lit(2L)))); + Optional actual = FlinkFilters.convert(expr); + Assert.assertTrue("Conversion should succeed", actual.isPresent()); + And and = (And) actual.get(); + And expected = + (And) + org.apache.iceberg.expressions.Expressions.and( + org.apache.iceberg.expressions.Expressions.equal("field1", 1), + org.apache.iceberg.expressions.Expressions.equal("field2", 2L)); + + assertPredicatesMatch(expected.left(), and.left()); + assertPredicatesMatch(expected.right(), and.right()); + } + + @Test + public void testOr() { + Expression expr = + resolve( + Expressions.$("field1") + .isEqual(Expressions.lit(1)) + .or(Expressions.$("field2").isEqual(Expressions.lit(2L)))); + Optional actual = FlinkFilters.convert(expr); + Assert.assertTrue("Conversion should succeed", actual.isPresent()); + Or or = (Or) actual.get(); + Or expected = + (Or) + org.apache.iceberg.expressions.Expressions.or( + org.apache.iceberg.expressions.Expressions.equal("field1", 1), + org.apache.iceberg.expressions.Expressions.equal("field2", 2L)); + + assertPredicatesMatch(expected.left(), or.left()); + assertPredicatesMatch(expected.right(), or.right()); + } + + @Test + public void testNot() { + Expression expr = + resolve( + ApiExpressionUtils.unresolvedCall( + BuiltInFunctionDefinitions.NOT, + Expressions.$("field1").isEqual(Expressions.lit(1)))); + Optional actual = FlinkFilters.convert(expr); + Assert.assertTrue("Conversion should succeed", actual.isPresent()); + Not not = (Not) actual.get(); + Not expected = + (Not) + org.apache.iceberg.expressions.Expressions.not( + org.apache.iceberg.expressions.Expressions.equal("field1", 1)); + + Assert.assertEquals("Predicate operation should match", expected.op(), not.op()); + assertPredicatesMatch(expected.child(), not.child()); + } + + @Test + public void testLike() { + UnboundPredicate expected = + org.apache.iceberg.expressions.Expressions.startsWith("field5", "abc"); + Expression expr = + resolve( + ApiExpressionUtils.unresolvedCall( + BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("abc%"))); + Optional actual = FlinkFilters.convert(expr); + Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertPredicatesMatch(expected, actual.get()); + + expr = + resolve( + ApiExpressionUtils.unresolvedCall( + BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("%abc"))); + actual = FlinkFilters.convert(expr); + Assert.assertFalse("Conversion should failed", actual.isPresent()); + + expr = + resolve( + ApiExpressionUtils.unresolvedCall( + BuiltInFunctionDefinitions.LIKE, + Expressions.$("field5"), + Expressions.lit("%abc%"))); + actual = FlinkFilters.convert(expr); + Assert.assertFalse("Conversion should failed", actual.isPresent()); + + expr = + resolve( + ApiExpressionUtils.unresolvedCall( + BuiltInFunctionDefinitions.LIKE, + Expressions.$("field5"), + Expressions.lit("abc%d"))); + actual = FlinkFilters.convert(expr); + Assert.assertFalse("Conversion should failed", actual.isPresent()); + + expr = + resolve( + ApiExpressionUtils.unresolvedCall( + BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("%"))); + actual = FlinkFilters.convert(expr); + Assert.assertFalse("Conversion should failed", actual.isPresent()); + + expr = + resolve( + ApiExpressionUtils.unresolvedCall( + BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("a_"))); + actual = FlinkFilters.convert(expr); + Assert.assertFalse("Conversion should failed", actual.isPresent()); + + expr = + resolve( + ApiExpressionUtils.unresolvedCall( + BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("a%b"))); + actual = FlinkFilters.convert(expr); + Assert.assertFalse("Conversion should failed", actual.isPresent()); + } + + @SuppressWarnings("unchecked") + private void matchLiteral(String fieldName, Object flinkLiteral, T icebergLiteral) { + Expression expr = resolve(Expressions.$(fieldName).isEqual(Expressions.lit(flinkLiteral))); + Optional actual = FlinkFilters.convert(expr); + Assert.assertTrue("Conversion should succeed", actual.isPresent()); + org.apache.iceberg.expressions.Expression expression = actual.get(); + Assertions.assertThat(expression) + .as("The expression should be a UnboundPredicate") + .isInstanceOf(UnboundPredicate.class); + UnboundPredicate unboundPredicate = (UnboundPredicate) expression; + + org.apache.iceberg.expressions.Expression expression1 = + unboundPredicate.bind(FlinkSchemaUtil.convert(TABLE_SCHEMA).asStruct(), false); + Assertions.assertThat(expression1) + .as("The expression should be a BoundLiteralPredicate") + .isInstanceOf(BoundLiteralPredicate.class); + + BoundLiteralPredicate predicate = (BoundLiteralPredicate) expression1; + Assert.assertTrue("Should match the literal", predicate.test(icebergLiteral)); + } + + private static Expression resolve(Expression originalExpression) { + return originalExpression.accept( + new ApiExpressionDefaultVisitor() { + @Override + public Expression visit(UnresolvedReferenceExpression unresolvedReference) { + String name = unresolvedReference.getName(); + Optional field = TABLE_SCHEMA.getTableColumn(name); + if (field.isPresent()) { + int index = TABLE_SCHEMA.getTableColumns().indexOf(field.get()); + return new FieldReferenceExpression(name, field.get().getType(), 0, index); + } else { + return null; + } + } + + @Override + public Expression visit(UnresolvedCallExpression unresolvedCall) { + List children = + unresolvedCall.getChildren().stream() + .map(e -> (ResolvedExpression) e.accept(this)) + .collect(Collectors.toList()); + return new CallExpression( + unresolvedCall.getFunctionDefinition(), children, DataTypes.STRING()); + } + + @Override + public Expression visit(ValueLiteralExpression valueLiteral) { + return valueLiteral; + } + + @Override + protected Expression defaultMethod(Expression expression) { + throw new UnsupportedOperationException( + String.format("unsupported expression: %s", expression)); + } + }); + } + + private void assertPredicatesMatch( + org.apache.iceberg.expressions.Expression expected, + org.apache.iceberg.expressions.Expression actual) { + Assertions.assertThat(expected) + .as("The expected expression should be a UnboundPredicate") + .isInstanceOf(UnboundPredicate.class); + Assertions.assertThat(actual) + .as("The actual expression should be a UnboundPredicate") + .isInstanceOf(UnboundPredicate.class); + UnboundPredicate predicateExpected = (UnboundPredicate) expected; + UnboundPredicate predicateActual = (UnboundPredicate) actual; + Assert.assertEquals( + "Predicate operation should match", predicateExpected.op(), predicateActual.op()); + Assert.assertEquals( + "Predicate literal should match", predicateExpected.literal(), predicateActual.literal()); + Assert.assertEquals( + "Predicate name should match", + predicateExpected.ref().name(), + predicateActual.ref().name()); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java new file mode 100644 index 000000000000..8f238587d30d --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java @@ -0,0 +1,105 @@ +/* + * 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.iceberg.flink; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestFlinkHiveCatalog extends FlinkTestBase { + + @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); + + @Test + public void testCreateCatalogWithWarehouseLocation() throws IOException { + Map props = Maps.newHashMap(); + props.put("type", "iceberg"); + props.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hive"); + props.put(CatalogProperties.URI, FlinkCatalogTestBase.getURI(hiveConf)); + + File warehouseDir = tempFolder.newFolder(); + props.put(CatalogProperties.WAREHOUSE_LOCATION, "file://" + warehouseDir.getAbsolutePath()); + + checkSQLQuery(props, warehouseDir); + } + + @Test + public void testCreateCatalogWithHiveConfDir() throws IOException { + // Dump the hive conf into a local file. + File hiveConfDir = tempFolder.newFolder(); + File hiveSiteXML = new File(hiveConfDir, "hive-site.xml"); + File warehouseDir = tempFolder.newFolder(); + try (FileOutputStream fos = new FileOutputStream(hiveSiteXML)) { + Configuration newConf = new Configuration(hiveConf); + // Set another new directory which is different with the hive metastore's warehouse path. + newConf.set( + HiveConf.ConfVars.METASTOREWAREHOUSE.varname, "file://" + warehouseDir.getAbsolutePath()); + newConf.writeXml(fos); + } + Assert.assertTrue("hive-site.xml should be created now.", Files.exists(hiveSiteXML.toPath())); + + // Construct the catalog attributions. + Map props = Maps.newHashMap(); + props.put("type", "iceberg"); + props.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hive"); + props.put(CatalogProperties.URI, FlinkCatalogTestBase.getURI(hiveConf)); + // Set the 'hive-conf-dir' instead of 'warehouse' + props.put(FlinkCatalogFactory.HIVE_CONF_DIR, hiveConfDir.getAbsolutePath()); + + checkSQLQuery(props, warehouseDir); + } + + private void checkSQLQuery(Map catalogProperties, File warehouseDir) + throws IOException { + sql( + "CREATE CATALOG test_catalog WITH %s", + FlinkCatalogTestBase.toWithClause(catalogProperties)); + sql("USE CATALOG test_catalog"); + sql("CREATE DATABASE test_db"); + sql("USE test_db"); + sql("CREATE TABLE test_table(c1 INT, c2 STRING)"); + sql("INSERT INTO test_table SELECT 1, 'a'"); + + Path databasePath = warehouseDir.toPath().resolve("test_db.db"); + Assert.assertTrue("Database path should exist", Files.exists(databasePath)); + + Path tablePath = databasePath.resolve("test_table"); + Assert.assertTrue("Table path should exist", Files.exists(tablePath)); + + Path dataPath = tablePath.resolve("data"); + Assert.assertTrue("Table data path should exist", Files.exists(dataPath)); + Assert.assertEquals( + "Should have a .crc file and a .parquet file", 2, Files.list(dataPath).count()); + + sql("DROP TABLE test_table"); + sql("DROP DATABASE test_db"); + dropCatalog("test_catalog", false); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java new file mode 100644 index 000000000000..4ac32c08ebef --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java @@ -0,0 +1,416 @@ +/* + * 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.iceberg.flink; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.types.logical.BinaryType; +import org.apache.flink.table.types.logical.CharType; +import org.apache.flink.table.types.logical.LocalZonedTimestampType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.TimeType; +import org.apache.flink.table.types.logical.TimestampType; +import org.apache.flink.table.types.logical.VarBinaryType; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Test; + +public class TestFlinkSchemaUtil { + + @Test + public void testConvertFlinkSchemaToIcebergSchema() { + TableSchema flinkSchema = + TableSchema.builder() + .field("id", DataTypes.INT().notNull()) + .field("name", DataTypes.STRING()) /* optional by default */ + .field("salary", DataTypes.DOUBLE().notNull()) + .field( + "locations", + DataTypes.MAP( + DataTypes.STRING(), + DataTypes.ROW( + DataTypes.FIELD("posX", DataTypes.DOUBLE().notNull(), "X field"), + DataTypes.FIELD("posY", DataTypes.DOUBLE().notNull(), "Y field")))) + .field("strArray", DataTypes.ARRAY(DataTypes.STRING()).nullable()) + .field("intArray", DataTypes.ARRAY(DataTypes.INT()).nullable()) + .field("char", DataTypes.CHAR(10).notNull()) + .field("varchar", DataTypes.VARCHAR(10).notNull()) + .field("boolean", DataTypes.BOOLEAN().nullable()) + .field("tinyint", DataTypes.TINYINT()) + .field("smallint", DataTypes.SMALLINT()) + .field("bigint", DataTypes.BIGINT()) + .field("varbinary", DataTypes.VARBINARY(10)) + .field("binary", DataTypes.BINARY(10)) + .field("time", DataTypes.TIME()) + .field("timestampWithoutZone", DataTypes.TIMESTAMP()) + .field("timestampWithZone", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()) + .field("date", DataTypes.DATE()) + .field("decimal", DataTypes.DECIMAL(2, 2)) + .field("decimal2", DataTypes.DECIMAL(38, 2)) + .field("decimal3", DataTypes.DECIMAL(10, 1)) + .field("multiset", DataTypes.MULTISET(DataTypes.STRING().notNull())) + .build(); + + Schema icebergSchema = + new Schema( + Types.NestedField.required(0, "id", Types.IntegerType.get(), null), + Types.NestedField.optional(1, "name", Types.StringType.get(), null), + Types.NestedField.required(2, "salary", Types.DoubleType.get(), null), + Types.NestedField.optional( + 3, + "locations", + Types.MapType.ofOptional( + 24, + 25, + Types.StringType.get(), + Types.StructType.of( + Types.NestedField.required(22, "posX", Types.DoubleType.get(), "X field"), + Types.NestedField.required( + 23, "posY", Types.DoubleType.get(), "Y field")))), + Types.NestedField.optional( + 4, "strArray", Types.ListType.ofOptional(26, Types.StringType.get())), + Types.NestedField.optional( + 5, "intArray", Types.ListType.ofOptional(27, Types.IntegerType.get())), + Types.NestedField.required(6, "char", Types.StringType.get()), + Types.NestedField.required(7, "varchar", Types.StringType.get()), + Types.NestedField.optional(8, "boolean", Types.BooleanType.get()), + Types.NestedField.optional(9, "tinyint", Types.IntegerType.get()), + Types.NestedField.optional(10, "smallint", Types.IntegerType.get()), + Types.NestedField.optional(11, "bigint", Types.LongType.get()), + Types.NestedField.optional(12, "varbinary", Types.BinaryType.get()), + Types.NestedField.optional(13, "binary", Types.FixedType.ofLength(10)), + Types.NestedField.optional(14, "time", Types.TimeType.get()), + Types.NestedField.optional( + 15, "timestampWithoutZone", Types.TimestampType.withoutZone()), + Types.NestedField.optional(16, "timestampWithZone", Types.TimestampType.withZone()), + Types.NestedField.optional(17, "date", Types.DateType.get()), + Types.NestedField.optional(18, "decimal", Types.DecimalType.of(2, 2)), + Types.NestedField.optional(19, "decimal2", Types.DecimalType.of(38, 2)), + Types.NestedField.optional(20, "decimal3", Types.DecimalType.of(10, 1)), + Types.NestedField.optional( + 21, + "multiset", + Types.MapType.ofRequired(28, 29, Types.StringType.get(), Types.IntegerType.get()))); + + checkSchema(flinkSchema, icebergSchema); + } + + @Test + public void testMapField() { + TableSchema flinkSchema = + TableSchema.builder() + .field( + "map_int_long", + DataTypes.MAP(DataTypes.INT(), DataTypes.BIGINT()).notNull()) /* Required */ + .field( + "map_int_array_string", + DataTypes.MAP(DataTypes.ARRAY(DataTypes.INT()), DataTypes.STRING())) + .field( + "map_decimal_string", DataTypes.MAP(DataTypes.DECIMAL(10, 2), DataTypes.STRING())) + .field( + "map_fields_fields", + DataTypes.MAP( + DataTypes.ROW( + DataTypes.FIELD("field_int", DataTypes.INT(), "doc - int"), + DataTypes.FIELD("field_string", DataTypes.STRING(), "doc - string")) + .notNull(), /* Required */ + DataTypes.ROW( + DataTypes.FIELD( + "field_array", + DataTypes.ARRAY(DataTypes.STRING()), + "doc - array")) + .notNull() /* Required */) + .notNull() /* Required */) + .build(); + + Schema icebergSchema = + new Schema( + Types.NestedField.required( + 0, + "map_int_long", + Types.MapType.ofOptional(4, 5, Types.IntegerType.get(), Types.LongType.get()), + null), + Types.NestedField.optional( + 1, + "map_int_array_string", + Types.MapType.ofOptional( + 7, + 8, + Types.ListType.ofOptional(6, Types.IntegerType.get()), + Types.StringType.get()), + null), + Types.NestedField.optional( + 2, + "map_decimal_string", + Types.MapType.ofOptional( + 9, 10, Types.DecimalType.of(10, 2), Types.StringType.get())), + Types.NestedField.required( + 3, + "map_fields_fields", + Types.MapType.ofRequired( + 15, + 16, + Types.StructType.of( + Types.NestedField.optional( + 11, "field_int", Types.IntegerType.get(), "doc - int"), + Types.NestedField.optional( + 12, "field_string", Types.StringType.get(), "doc - string")), + Types.StructType.of( + Types.NestedField.optional( + 14, + "field_array", + Types.ListType.ofOptional(13, Types.StringType.get()), + "doc - array"))))); + + checkSchema(flinkSchema, icebergSchema); + } + + @Test + public void testStructField() { + TableSchema flinkSchema = + TableSchema.builder() + .field( + "struct_int_string_decimal", + DataTypes.ROW( + DataTypes.FIELD("field_int", DataTypes.INT()), + DataTypes.FIELD("field_string", DataTypes.STRING()), + DataTypes.FIELD("field_decimal", DataTypes.DECIMAL(19, 2)), + DataTypes.FIELD( + "field_struct", + DataTypes.ROW( + DataTypes.FIELD("inner_struct_int", DataTypes.INT()), + DataTypes.FIELD( + "inner_struct_float_array", + DataTypes.ARRAY(DataTypes.FLOAT()))) + .notNull()) /* Row is required */) + .notNull()) /* Required */ + .field( + "struct_map_int_int", + DataTypes.ROW( + DataTypes.FIELD( + "field_map", DataTypes.MAP(DataTypes.INT(), DataTypes.INT()))) + .nullable()) /* Optional */ + .build(); + + Schema icebergSchema = + new Schema( + Types.NestedField.required( + 0, + "struct_int_string_decimal", + Types.StructType.of( + Types.NestedField.optional(5, "field_int", Types.IntegerType.get()), + Types.NestedField.optional(6, "field_string", Types.StringType.get()), + Types.NestedField.optional(7, "field_decimal", Types.DecimalType.of(19, 2)), + Types.NestedField.required( + 8, + "field_struct", + Types.StructType.of( + Types.NestedField.optional( + 3, "inner_struct_int", Types.IntegerType.get()), + Types.NestedField.optional( + 4, + "inner_struct_float_array", + Types.ListType.ofOptional(2, Types.FloatType.get())))))), + Types.NestedField.optional( + 1, + "struct_map_int_int", + Types.StructType.of( + Types.NestedField.optional( + 11, + "field_map", + Types.MapType.ofOptional( + 9, 10, Types.IntegerType.get(), Types.IntegerType.get()))))); + + checkSchema(flinkSchema, icebergSchema); + } + + @Test + public void testListField() { + TableSchema flinkSchema = + TableSchema.builder() + .field( + "list_struct_fields", + DataTypes.ARRAY(DataTypes.ROW(DataTypes.FIELD("field_int", DataTypes.INT()))) + .notNull()) /* Required */ + .field( + "list_optional_struct_fields", + DataTypes.ARRAY( + DataTypes.ROW( + DataTypes.FIELD( + "field_timestamp_with_local_time_zone", + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()))) + .nullable()) /* Optional */ + .field( + "list_map_fields", + DataTypes.ARRAY( + DataTypes.MAP( + DataTypes.ARRAY( + DataTypes.INT().notNull()), /* Key of map must be required */ + DataTypes.ROW( + DataTypes.FIELD("field_0", DataTypes.INT(), "doc - int"))) + .notNull()) + .notNull()) /* Required */ + .build(); + + Schema icebergSchema = + new Schema( + Types.NestedField.required( + 0, + "list_struct_fields", + Types.ListType.ofOptional( + 4, + Types.StructType.of( + Types.NestedField.optional(3, "field_int", Types.IntegerType.get())))), + Types.NestedField.optional( + 1, + "list_optional_struct_fields", + Types.ListType.ofOptional( + 6, + Types.StructType.of( + Types.NestedField.optional( + 5, + "field_timestamp_with_local_time_zone", + Types.TimestampType.withZone())))), + Types.NestedField.required( + 2, + "list_map_fields", + Types.ListType.ofRequired( + 11, + Types.MapType.ofOptional( + 9, + 10, + Types.ListType.ofRequired(7, Types.IntegerType.get()), + Types.StructType.of( + Types.NestedField.optional( + 8, "field_0", Types.IntegerType.get(), "doc - int")))))); + + checkSchema(flinkSchema, icebergSchema); + } + + private void checkSchema(TableSchema flinkSchema, Schema icebergSchema) { + Assert.assertEquals(icebergSchema.asStruct(), FlinkSchemaUtil.convert(flinkSchema).asStruct()); + // The conversion is not a 1:1 mapping, so we just check iceberg types. + Assert.assertEquals( + icebergSchema.asStruct(), + FlinkSchemaUtil.convert(FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(icebergSchema))) + .asStruct()); + } + + @Test + public void testInconsistentTypes() { + checkInconsistentType( + Types.UUIDType.get(), new BinaryType(16), new BinaryType(16), Types.FixedType.ofLength(16)); + checkInconsistentType( + Types.StringType.get(), + new VarCharType(VarCharType.MAX_LENGTH), + new CharType(100), + Types.StringType.get()); + checkInconsistentType( + Types.BinaryType.get(), + new VarBinaryType(VarBinaryType.MAX_LENGTH), + new VarBinaryType(100), + Types.BinaryType.get()); + checkInconsistentType( + Types.TimeType.get(), new TimeType(), new TimeType(3), Types.TimeType.get()); + checkInconsistentType( + Types.TimestampType.withoutZone(), + new TimestampType(6), + new TimestampType(3), + Types.TimestampType.withoutZone()); + checkInconsistentType( + Types.TimestampType.withZone(), + new LocalZonedTimestampType(6), + new LocalZonedTimestampType(3), + Types.TimestampType.withZone()); + } + + private void checkInconsistentType( + Type icebergType, + LogicalType flinkExpectedType, + LogicalType flinkType, + Type icebergExpectedType) { + Assert.assertEquals(flinkExpectedType, FlinkSchemaUtil.convert(icebergType)); + Assert.assertEquals( + Types.StructType.of(Types.NestedField.optional(0, "f0", icebergExpectedType)), + FlinkSchemaUtil.convert(FlinkSchemaUtil.toSchema(RowType.of(flinkType))).asStruct()); + } + + @Test + public void testConvertFlinkSchemaBaseOnIcebergSchema() { + Schema baseSchema = + new Schema( + Lists.newArrayList( + Types.NestedField.required(101, "int", Types.IntegerType.get()), + Types.NestedField.optional(102, "string", Types.StringType.get())), + Sets.newHashSet(101)); + + TableSchema flinkSchema = + TableSchema.builder() + .field("int", DataTypes.INT().notNull()) + .field("string", DataTypes.STRING().nullable()) + .primaryKey("int") + .build(); + Schema convertedSchema = FlinkSchemaUtil.convert(baseSchema, flinkSchema); + Assert.assertEquals(baseSchema.asStruct(), convertedSchema.asStruct()); + Assert.assertEquals(ImmutableSet.of(101), convertedSchema.identifierFieldIds()); + } + + @Test + public void testConvertFlinkSchemaWithPrimaryKeys() { + Schema icebergSchema = + new Schema( + Lists.newArrayList( + Types.NestedField.required(1, "int", Types.IntegerType.get()), + Types.NestedField.required(2, "string", Types.StringType.get())), + Sets.newHashSet(1, 2)); + + TableSchema tableSchema = FlinkSchemaUtil.toSchema(icebergSchema); + Assert.assertTrue(tableSchema.getPrimaryKey().isPresent()); + Assert.assertEquals( + ImmutableSet.of("int", "string"), + ImmutableSet.copyOf(tableSchema.getPrimaryKey().get().getColumns())); + } + + @Test + public void testConvertFlinkSchemaWithNestedColumnInPrimaryKeys() { + Schema icebergSchema = + new Schema( + Lists.newArrayList( + Types.NestedField.required( + 1, + "struct", + Types.StructType.of( + Types.NestedField.required(2, "inner", Types.IntegerType.get())))), + Sets.newHashSet(2)); + + Assertions.assertThatThrownBy(() -> FlinkSchemaUtil.toSchema(icebergSchema)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Could not create a PRIMARY KEY") + .hasMessageContaining("Column 'struct.inner' does not exist."); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java new file mode 100644 index 000000000000..754062798928 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java @@ -0,0 +1,380 @@ +/* + * 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.iceberg.flink; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.Expressions; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.internal.TableEnvironmentImpl; +import org.apache.flink.table.operations.ModifyOperation; +import org.apache.flink.table.planner.delegation.PlannerBase; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.types.Row; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.source.BoundedTableFactory; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.junit.After; +import org.junit.Assert; +import org.junit.Assume; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestFlinkTableSink extends FlinkCatalogTestBase { + + @ClassRule + public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = + MiniClusterResource.createWithClassloaderCheckDisabled(); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + private static final String SOURCE_TABLE = "default_catalog.default_database.bounded_source"; + private static final String TABLE_NAME = "test_table"; + private TableEnvironment tEnv; + private Table icebergTable; + + private final FileFormat format; + private final boolean isStreamingJob; + + @Parameterized.Parameters( + name = "catalogName={0}, baseNamespace={1}, format={2}, isStreaming={3}") + public static Iterable parameters() { + List parameters = Lists.newArrayList(); + for (FileFormat format : + new FileFormat[] {FileFormat.ORC, FileFormat.AVRO, FileFormat.PARQUET}) { + for (Boolean isStreaming : new Boolean[] {true, false}) { + for (Object[] catalogParams : FlinkCatalogTestBase.parameters()) { + String catalogName = (String) catalogParams[0]; + Namespace baseNamespace = (Namespace) catalogParams[1]; + parameters.add(new Object[] {catalogName, baseNamespace, format, isStreaming}); + } + } + } + return parameters; + } + + public TestFlinkTableSink( + String catalogName, Namespace baseNamespace, FileFormat format, Boolean isStreamingJob) { + super(catalogName, baseNamespace); + this.format = format; + this.isStreamingJob = isStreamingJob; + } + + @Override + protected TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + EnvironmentSettings.Builder settingsBuilder = EnvironmentSettings.newInstance(); + if (isStreamingJob) { + settingsBuilder.inStreamingMode(); + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + env.enableCheckpointing(400); + env.setMaxParallelism(2); + env.setParallelism(2); + tEnv = StreamTableEnvironment.create(env, settingsBuilder.build()); + } else { + settingsBuilder.inBatchMode(); + tEnv = TableEnvironment.create(settingsBuilder.build()); + } + } + } + return tEnv; + } + + @Override + @Before + public void before() { + super.before(); + sql("CREATE DATABASE %s", flinkDatabase); + sql("USE CATALOG %s", catalogName); + sql("USE %s", DATABASE); + sql( + "CREATE TABLE %s (id int, data varchar) with ('write.format.default'='%s')", + TABLE_NAME, format.name()); + icebergTable = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); + } + + @Override + @After + public void clean() { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME); + sql("DROP DATABASE IF EXISTS %s", flinkDatabase); + BoundedTableFactory.clearDataSets(); + super.clean(); + } + + @Test + public void testInsertFromSourceTable() throws Exception { + // Register the rows into a temporary table. + getTableEnv() + .createTemporaryView( + "sourceTable", + getTableEnv() + .fromValues( + SimpleDataUtil.FLINK_SCHEMA.toRowDataType(), + Expressions.row(1, "hello"), + Expressions.row(2, "world"), + Expressions.row(3, (String) null), + Expressions.row(null, "bar"))); + + // Redirect the records from source table to destination table. + sql("INSERT INTO %s SELECT id,data from sourceTable", TABLE_NAME); + + // Assert the table records as expected. + SimpleDataUtil.assertTableRecords( + icebergTable, + Lists.newArrayList( + SimpleDataUtil.createRecord(1, "hello"), + SimpleDataUtil.createRecord(2, "world"), + SimpleDataUtil.createRecord(3, null), + SimpleDataUtil.createRecord(null, "bar"))); + } + + @Test + public void testOverwriteTable() throws Exception { + Assume.assumeFalse( + "Flink unbounded streaming does not support overwrite operation", isStreamingJob); + + sql("INSERT INTO %s SELECT 1, 'a'", TABLE_NAME); + SimpleDataUtil.assertTableRecords( + icebergTable, Lists.newArrayList(SimpleDataUtil.createRecord(1, "a"))); + + sql("INSERT OVERWRITE %s SELECT 2, 'b'", TABLE_NAME); + SimpleDataUtil.assertTableRecords( + icebergTable, Lists.newArrayList(SimpleDataUtil.createRecord(2, "b"))); + } + + @Test + public void testWriteParallelism() throws Exception { + List dataSet = + IntStream.range(1, 1000) + .mapToObj(i -> ImmutableList.of(Row.of(i, "aaa"), Row.of(i, "bbb"), Row.of(i, "ccc"))) + .flatMap(List::stream) + .collect(Collectors.toList()); + String dataId = BoundedTableFactory.registerDataSet(ImmutableList.of(dataSet)); + sql( + "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" + + " WITH ('connector'='BoundedSource', 'data-id'='%s')", + SOURCE_TABLE, dataId); + + PlannerBase planner = (PlannerBase) ((TableEnvironmentImpl) getTableEnv()).getPlanner(); + String insertSQL = + String.format( + "INSERT INTO %s /*+ OPTIONS('write-parallelism'='1') */ SELECT * FROM %s", + TABLE_NAME, SOURCE_TABLE); + ModifyOperation operation = (ModifyOperation) planner.getParser().parse(insertSQL).get(0); + Transformation dummySink = planner.translate(Collections.singletonList(operation)).get(0); + Transformation committer = dummySink.getInputs().get(0); + Transformation writer = committer.getInputs().get(0); + + Assert.assertEquals("Should have the expected 1 parallelism.", 1, writer.getParallelism()); + + writer + .getInputs() + .forEach( + input -> + Assert.assertEquals( + "Should have the expected parallelism.", + isStreamingJob ? 2 : 4, + input.getParallelism())); + } + + @Test + public void testReplacePartitions() throws Exception { + Assume.assumeFalse( + "Flink unbounded streaming does not support overwrite operation", isStreamingJob); + String tableName = "test_partition"; + sql( + "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH ('write.format.default'='%s')", + tableName, format.name()); + + try { + Table partitionedTable = + validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, tableName)); + + sql("INSERT INTO %s SELECT 1, 'a'", tableName); + sql("INSERT INTO %s SELECT 2, 'b'", tableName); + sql("INSERT INTO %s SELECT 3, 'c'", tableName); + + SimpleDataUtil.assertTableRecords( + partitionedTable, + Lists.newArrayList( + SimpleDataUtil.createRecord(1, "a"), + SimpleDataUtil.createRecord(2, "b"), + SimpleDataUtil.createRecord(3, "c"))); + + sql("INSERT OVERWRITE %s SELECT 4, 'b'", tableName); + sql("INSERT OVERWRITE %s SELECT 5, 'a'", tableName); + + SimpleDataUtil.assertTableRecords( + partitionedTable, + Lists.newArrayList( + SimpleDataUtil.createRecord(5, "a"), + SimpleDataUtil.createRecord(4, "b"), + SimpleDataUtil.createRecord(3, "c"))); + + sql("INSERT OVERWRITE %s PARTITION (data='a') SELECT 6", tableName); + + SimpleDataUtil.assertTableRecords( + partitionedTable, + Lists.newArrayList( + SimpleDataUtil.createRecord(6, "a"), + SimpleDataUtil.createRecord(4, "b"), + SimpleDataUtil.createRecord(3, "c"))); + } finally { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); + } + } + + @Test + public void testInsertIntoPartition() throws Exception { + String tableName = "test_insert_into_partition"; + sql( + "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH ('write.format.default'='%s')", + tableName, format.name()); + + try { + Table partitionedTable = + validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, tableName)); + + // Full partition. + sql("INSERT INTO %s PARTITION (data='a') SELECT 1", tableName); + sql("INSERT INTO %s PARTITION (data='a') SELECT 2", tableName); + sql("INSERT INTO %s PARTITION (data='b') SELECT 3", tableName); + + SimpleDataUtil.assertTableRecords( + partitionedTable, + Lists.newArrayList( + SimpleDataUtil.createRecord(1, "a"), + SimpleDataUtil.createRecord(2, "a"), + SimpleDataUtil.createRecord(3, "b"))); + + // Partial partition. + sql("INSERT INTO %s SELECT 4, 'c'", tableName); + sql("INSERT INTO %s SELECT 5, 'd'", tableName); + + SimpleDataUtil.assertTableRecords( + partitionedTable, + Lists.newArrayList( + SimpleDataUtil.createRecord(1, "a"), + SimpleDataUtil.createRecord(2, "a"), + SimpleDataUtil.createRecord(3, "b"), + SimpleDataUtil.createRecord(4, "c"), + SimpleDataUtil.createRecord(5, "d"))); + } finally { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); + } + } + + @Test + public void testHashDistributeMode() throws Exception { + String tableName = "test_hash_distribution_mode"; + Map tableProps = + ImmutableMap.of( + "write.format.default", + format.name(), + TableProperties.WRITE_DISTRIBUTION_MODE, + DistributionMode.HASH.modeName()); + + // Initialize a BoundedSource table to precisely emit those rows in only one checkpoint. + List dataSet = + IntStream.range(1, 1000) + .mapToObj(i -> ImmutableList.of(Row.of(i, "aaa"), Row.of(i, "bbb"), Row.of(i, "ccc"))) + .flatMap(List::stream) + .collect(Collectors.toList()); + String dataId = BoundedTableFactory.registerDataSet(ImmutableList.of(dataSet)); + sql( + "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" + + " WITH ('connector'='BoundedSource', 'data-id'='%s')", + SOURCE_TABLE, dataId); + Assert.assertEquals( + "Should have the expected rows in source table.", + Sets.newHashSet(dataSet), + Sets.newHashSet(sql("SELECT * FROM %s", SOURCE_TABLE))); + + sql( + "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH %s", + tableName, toWithClause(tableProps)); + + try { + // Insert data set. + sql("INSERT INTO %s SELECT * FROM %s", tableName, SOURCE_TABLE); + + Assert.assertEquals( + "Should have the expected rows in sink table.", + Sets.newHashSet(dataSet), + Sets.newHashSet(sql("SELECT * FROM %s", tableName))); + + // Sometimes we will have more than one checkpoint if we pass the auto checkpoint interval, + // thus producing multiple snapshots. Here we assert that each snapshot has only 1 file per + // partition. + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, tableName)); + Map> snapshotToDataFiles = SimpleDataUtil.snapshotToDataFiles(table); + for (List dataFiles : snapshotToDataFiles.values()) { + if (dataFiles.isEmpty()) { + continue; + } + + Assert.assertEquals( + "There should be 1 data file in partition 'aaa'", + 1, + SimpleDataUtil.matchingPartitions( + dataFiles, table.spec(), ImmutableMap.of("data", "aaa")) + .size()); + Assert.assertEquals( + "There should be 1 data file in partition 'bbb'", + 1, + SimpleDataUtil.matchingPartitions( + dataFiles, table.spec(), ImmutableMap.of("data", "bbb")) + .size()); + Assert.assertEquals( + "There should be 1 data file in partition 'ccc'", + 1, + SimpleDataUtil.matchingPartitions( + dataFiles, table.spec(), ImmutableMap.of("data", "ccc")) + .size()); + } + } finally { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java new file mode 100644 index 000000000000..a25ebab6c4c2 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java @@ -0,0 +1,346 @@ +/* + * 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.iceberg.flink; + +import java.time.LocalDate; +import java.util.List; +import java.util.Map; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.types.Row; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestFlinkUpsert extends FlinkCatalogTestBase { + + @ClassRule + public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = + MiniClusterResource.createWithClassloaderCheckDisabled(); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + private final boolean isStreamingJob; + private final Map tableUpsertProps = Maps.newHashMap(); + private TableEnvironment tEnv; + + public TestFlinkUpsert( + String catalogName, Namespace baseNamespace, FileFormat format, Boolean isStreamingJob) { + super(catalogName, baseNamespace); + this.isStreamingJob = isStreamingJob; + tableUpsertProps.put(TableProperties.FORMAT_VERSION, "2"); + tableUpsertProps.put(TableProperties.UPSERT_ENABLED, "true"); + tableUpsertProps.put(TableProperties.DEFAULT_FILE_FORMAT, format.name()); + } + + @Parameterized.Parameters( + name = "catalogName={0}, baseNamespace={1}, format={2}, isStreaming={3}") + public static Iterable parameters() { + List parameters = Lists.newArrayList(); + for (FileFormat format : + new FileFormat[] {FileFormat.PARQUET, FileFormat.AVRO, FileFormat.ORC}) { + for (Boolean isStreaming : new Boolean[] {true, false}) { + // Only test with one catalog as this is a file operation concern. + // FlinkCatalogTestBase requires the catalog name start with testhadoop if using hadoop + // catalog. + String catalogName = "testhadoop"; + Namespace baseNamespace = Namespace.of("default"); + parameters.add(new Object[] {catalogName, baseNamespace, format, isStreaming}); + } + } + return parameters; + } + + @Override + protected TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + EnvironmentSettings.Builder settingsBuilder = EnvironmentSettings.newInstance(); + if (isStreamingJob) { + settingsBuilder.inStreamingMode(); + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + env.enableCheckpointing(400); + env.setMaxParallelism(2); + env.setParallelism(2); + tEnv = StreamTableEnvironment.create(env, settingsBuilder.build()); + } else { + settingsBuilder.inBatchMode(); + tEnv = TableEnvironment.create(settingsBuilder.build()); + } + } + } + return tEnv; + } + + @Override + @Before + public void before() { + super.before(); + sql("CREATE DATABASE IF NOT EXISTS %s", flinkDatabase); + sql("USE CATALOG %s", catalogName); + sql("USE %s", DATABASE); + } + + @Override + @After + public void clean() { + sql("DROP DATABASE IF EXISTS %s", flinkDatabase); + super.clean(); + } + + @Test + public void testUpsertAndQuery() { + String tableName = "test_upsert_query"; + LocalDate dt20220301 = LocalDate.of(2022, 3, 1); + LocalDate dt20220302 = LocalDate.of(2022, 3, 2); + + sql( + "CREATE TABLE %s(id INT NOT NULL, name STRING NOT NULL, dt DATE, PRIMARY KEY(id,dt) NOT ENFORCED) " + + "PARTITIONED BY (dt) WITH %s", + tableName, toWithClause(tableUpsertProps)); + + try { + sql( + "INSERT INTO %s VALUES " + + "(1, 'Bill', DATE '2022-03-01')," + + "(1, 'Jane', DATE '2022-03-01')," + + "(2, 'Jane', DATE '2022-03-01')", + tableName); + + sql( + "INSERT INTO %s VALUES " + + "(2, 'Bill', DATE '2022-03-01')," + + "(1, 'Jane', DATE '2022-03-02')," + + "(2, 'Jane', DATE '2022-03-02')", + tableName); + + List rowsOn20220301 = + Lists.newArrayList(Row.of(1, "Jane", dt20220301), Row.of(2, "Bill", dt20220301)); + TestHelpers.assertRows( + sql("SELECT * FROM %s WHERE dt < '2022-03-02'", tableName), rowsOn20220301); + + List rowsOn20220302 = + Lists.newArrayList(Row.of(1, "Jane", dt20220302), Row.of(2, "Jane", dt20220302)); + TestHelpers.assertRows( + sql("SELECT * FROM %s WHERE dt = '2022-03-02'", tableName), rowsOn20220302); + + TestHelpers.assertRows( + sql("SELECT * FROM %s", tableName), + Lists.newArrayList(Iterables.concat(rowsOn20220301, rowsOn20220302))); + } finally { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); + } + } + + @Test + public void testUpsertOptions() { + String tableName = "test_upsert_options"; + LocalDate dt20220301 = LocalDate.of(2022, 3, 1); + LocalDate dt20220302 = LocalDate.of(2022, 3, 2); + + Map optionsUpsertProps = Maps.newHashMap(tableUpsertProps); + optionsUpsertProps.remove(TableProperties.UPSERT_ENABLED); + sql( + "CREATE TABLE %s(id INT NOT NULL, name STRING NOT NULL, dt DATE, PRIMARY KEY(id,dt) NOT ENFORCED) " + + "PARTITIONED BY (dt) WITH %s", + tableName, toWithClause(optionsUpsertProps)); + + try { + sql( + "INSERT INTO %s /*+ OPTIONS('upsert-enabled'='true')*/ VALUES " + + "(1, 'Bill', DATE '2022-03-01')," + + "(1, 'Jane', DATE '2022-03-01')," + + "(2, 'Jane', DATE '2022-03-01')", + tableName); + + sql( + "INSERT INTO %s /*+ OPTIONS('upsert-enabled'='true')*/ VALUES " + + "(2, 'Bill', DATE '2022-03-01')," + + "(1, 'Jane', DATE '2022-03-02')," + + "(2, 'Jane', DATE '2022-03-02')", + tableName); + + List rowsOn20220301 = + Lists.newArrayList(Row.of(1, "Jane", dt20220301), Row.of(2, "Bill", dt20220301)); + TestHelpers.assertRows( + sql("SELECT * FROM %s WHERE dt < '2022-03-02'", tableName), rowsOn20220301); + + List rowsOn20220302 = + Lists.newArrayList(Row.of(1, "Jane", dt20220302), Row.of(2, "Jane", dt20220302)); + TestHelpers.assertRows( + sql("SELECT * FROM %s WHERE dt = '2022-03-02'", tableName), rowsOn20220302); + + TestHelpers.assertRows( + sql("SELECT * FROM %s", tableName), + Lists.newArrayList(Iterables.concat(rowsOn20220301, rowsOn20220302))); + } finally { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); + } + } + + @Test + public void testPrimaryKeyEqualToPartitionKey() { + // This is an SQL based reproduction of TestFlinkIcebergSinkV2#testUpsertOnDataKey + String tableName = "upsert_on_id_key"; + try { + sql( + "CREATE TABLE %s(id INT NOT NULL, name STRING NOT NULL, PRIMARY KEY(id) NOT ENFORCED) " + + "PARTITIONED BY (id) WITH %s", + tableName, toWithClause(tableUpsertProps)); + + sql("INSERT INTO %s VALUES " + "(1, 'Bill')," + "(1, 'Jane')," + "(2, 'Bill')", tableName); + + TestHelpers.assertRows( + sql("SELECT * FROM %s", tableName), + Lists.newArrayList(Row.of(1, "Jane"), Row.of(2, "Bill"))); + + sql("INSERT INTO %s VALUES " + "(1, 'Bill')," + "(2, 'Jane')", tableName); + + TestHelpers.assertRows( + sql("SELECT * FROM %s", tableName), + Lists.newArrayList(Row.of(1, "Bill"), Row.of(2, "Jane"))); + + sql("INSERT INTO %s VALUES " + "(3, 'Bill')," + "(4, 'Jane')", tableName); + + TestHelpers.assertRows( + sql("SELECT * FROM %s", tableName), + Lists.newArrayList( + Row.of(1, "Bill"), Row.of(2, "Jane"), Row.of(3, "Bill"), Row.of(4, "Jane"))); + } finally { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); + } + } + + @Test + public void testPrimaryKeyFieldsAtBeginningOfSchema() { + String tableName = "upsert_on_pk_at_schema_start"; + LocalDate dt = LocalDate.of(2022, 3, 1); + try { + sql( + "CREATE TABLE %s(id INT, dt DATE NOT NULL, name STRING NOT NULL, PRIMARY KEY(id,dt) NOT ENFORCED) " + + "PARTITIONED BY (dt) WITH %s", + tableName, toWithClause(tableUpsertProps)); + + sql( + "INSERT INTO %s VALUES " + + "(1, DATE '2022-03-01', 'Andy')," + + "(1, DATE '2022-03-01', 'Bill')," + + "(2, DATE '2022-03-01', 'Jane')", + tableName); + + TestHelpers.assertRows( + sql("SELECT * FROM %s", tableName), + Lists.newArrayList(Row.of(1, dt, "Bill"), Row.of(2, dt, "Jane"))); + + sql( + "INSERT INTO %s VALUES " + + "(1, DATE '2022-03-01', 'Jane')," + + "(2, DATE '2022-03-01', 'Bill')", + tableName); + + TestHelpers.assertRows( + sql("SELECT * FROM %s", tableName), + Lists.newArrayList(Row.of(1, dt, "Jane"), Row.of(2, dt, "Bill"))); + + sql( + "INSERT INTO %s VALUES " + + "(3, DATE '2022-03-01', 'Duke')," + + "(4, DATE '2022-03-01', 'Leon')", + tableName); + + TestHelpers.assertRows( + sql("SELECT * FROM %s", tableName), + Lists.newArrayList( + Row.of(1, dt, "Jane"), + Row.of(2, dt, "Bill"), + Row.of(3, dt, "Duke"), + Row.of(4, dt, "Leon"))); + } finally { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); + } + } + + @Test + public void testPrimaryKeyFieldsAtEndOfTableSchema() { + // This is the same test case as testPrimaryKeyFieldsAtBeginningOfSchema, but the primary key + // fields + // are located at the end of the flink schema. + String tableName = "upsert_on_pk_at_schema_end"; + LocalDate dt = LocalDate.of(2022, 3, 1); + try { + sql( + "CREATE TABLE %s(name STRING NOT NULL, id INT, dt DATE NOT NULL, PRIMARY KEY(id,dt) NOT ENFORCED) " + + "PARTITIONED BY (dt) WITH %s", + tableName, toWithClause(tableUpsertProps)); + + sql( + "INSERT INTO %s VALUES " + + "('Andy', 1, DATE '2022-03-01')," + + "('Bill', 1, DATE '2022-03-01')," + + "('Jane', 2, DATE '2022-03-01')", + tableName); + + TestHelpers.assertRows( + sql("SELECT * FROM %s", tableName), + Lists.newArrayList(Row.of("Bill", 1, dt), Row.of("Jane", 2, dt))); + + sql( + "INSERT INTO %s VALUES " + + "('Jane', 1, DATE '2022-03-01')," + + "('Bill', 2, DATE '2022-03-01')", + tableName); + + TestHelpers.assertRows( + sql("SELECT * FROM %s", tableName), + Lists.newArrayList(Row.of("Jane", 1, dt), Row.of("Bill", 2, dt))); + + sql( + "INSERT INTO %s VALUES " + + "('Duke', 3, DATE '2022-03-01')," + + "('Leon', 4, DATE '2022-03-01')", + tableName); + + TestHelpers.assertRows( + sql("SELECT * FROM %s", tableName), + Lists.newArrayList( + Row.of("Jane", 1, dt), + Row.of("Bill", 2, dt), + Row.of("Duke", 3, dt), + Row.of("Leon", 4, dt))); + } finally { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java new file mode 100644 index 000000000000..3e574b841c8d --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java @@ -0,0 +1,611 @@ +/* + * 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.iceberg.flink; + +import java.io.IOException; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.OffsetDateTime; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.function.Consumer; +import java.util.stream.Collectors; +import org.apache.avro.generic.GenericData; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.data.conversion.DataStructureConverter; +import org.apache.flink.table.data.conversion.DataStructureConverters; +import org.apache.flink.table.runtime.typeutils.InternalSerializers; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.flink.types.Row; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.data.RowDataUtil; +import org.apache.iceberg.flink.source.FlinkInputFormat; +import org.apache.iceberg.flink.source.FlinkInputSplit; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Streams; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.DateTimeUtil; +import org.assertj.core.api.Assertions; +import org.junit.Assert; + +public class TestHelpers { + private TestHelpers() {} + + public static T roundTripKryoSerialize(Class clazz, T table) throws IOException { + KryoSerializer kryo = new KryoSerializer<>(clazz, new ExecutionConfig()); + + DataOutputSerializer outputView = new DataOutputSerializer(1024); + kryo.serialize(table, outputView); + + DataInputDeserializer inputView = new DataInputDeserializer(outputView.getCopyOfBuffer()); + return kryo.deserialize(inputView); + } + + public static RowData copyRowData(RowData from, RowType rowType) { + TypeSerializer[] fieldSerializers = + rowType.getChildren().stream() + .map((LogicalType type) -> InternalSerializers.create(type)) + .toArray(TypeSerializer[]::new); + return RowDataUtil.clone(from, null, rowType, fieldSerializers); + } + + public static void readRowData(FlinkInputFormat input, Consumer visitor) + throws IOException { + for (FlinkInputSplit s : input.createInputSplits(0)) { + input.open(s); + try { + while (!input.reachedEnd()) { + RowData row = input.nextRecord(null); + visitor.accept(row); + } + } finally { + input.close(); + } + } + } + + public static List readRowData(FlinkInputFormat inputFormat, RowType rowType) + throws IOException { + List results = Lists.newArrayList(); + readRowData(inputFormat, row -> results.add(copyRowData(row, rowType))); + return results; + } + + public static List readRows(FlinkInputFormat inputFormat, RowType rowType) + throws IOException { + return convertRowDataToRow(readRowData(inputFormat, rowType), rowType); + } + + public static List convertRowDataToRow(List rowDataList, RowType rowType) { + DataStructureConverter converter = + DataStructureConverters.getConverter(TypeConversions.fromLogicalToDataType(rowType)); + return rowDataList.stream() + .map(converter::toExternal) + .map(Row.class::cast) + .collect(Collectors.toList()); + } + + public static void assertRecords(List results, List expectedRecords, Schema schema) { + List expected = Lists.newArrayList(); + @SuppressWarnings("unchecked") + DataStructureConverter converter = + (DataStructureConverter) + DataStructureConverters.getConverter( + TypeConversions.fromLogicalToDataType(FlinkSchemaUtil.convert(schema))); + expectedRecords.forEach( + r -> expected.add(converter.toExternal(RowDataConverter.convert(schema, r)))); + assertRows(results, expected); + } + + public static void assertRows(List results, List expected, RowType rowType) { + assertRows(convertRowDataToRow(results, rowType), convertRowDataToRow(expected, rowType)); + } + + public static void assertRows(List results, List expected) { + Assertions.assertThat(results).containsExactlyInAnyOrderElementsOf(expected); + } + + public static void assertRowData(Schema schema, StructLike expected, RowData actual) { + assertRowData(schema.asStruct(), FlinkSchemaUtil.convert(schema), expected, actual); + } + + public static void assertRowData( + Types.StructType structType, + LogicalType rowType, + StructLike expectedRecord, + RowData actualRowData) { + if (expectedRecord == null && actualRowData == null) { + return; + } + + Assert.assertTrue( + "expected Record and actual RowData should be both null or not null", + expectedRecord != null && actualRowData != null); + + List types = Lists.newArrayList(); + for (Types.NestedField field : structType.fields()) { + types.add(field.type()); + } + + for (int i = 0; i < types.size(); i += 1) { + LogicalType logicalType = ((RowType) rowType).getTypeAt(i); + Object expected = expectedRecord.get(i, Object.class); + // The RowData.createFieldGetter won't return null for the required field. But in the + // projection case, if we are + // projecting a nested required field from an optional struct, then we should give a null for + // the projected field + // if the outer struct value is null. So we need to check the nullable for actualRowData here. + // For more details + // please see issue #2738. + Object actual = + actualRowData.isNullAt(i) + ? null + : RowData.createFieldGetter(logicalType, i).getFieldOrNull(actualRowData); + assertEquals(types.get(i), logicalType, expected, actual); + } + } + + private static void assertEquals( + Type type, LogicalType logicalType, Object expected, Object actual) { + + if (expected == null && actual == null) { + return; + } + + Assert.assertTrue( + "expected and actual should be both null or not null", expected != null && actual != null); + + switch (type.typeId()) { + case BOOLEAN: + Assert.assertEquals("boolean value should be equal", expected, actual); + break; + case INTEGER: + Assert.assertEquals("int value should be equal", expected, actual); + break; + case LONG: + Assert.assertEquals("long value should be equal", expected, actual); + break; + case FLOAT: + Assert.assertEquals("float value should be equal", expected, actual); + break; + case DOUBLE: + Assert.assertEquals("double value should be equal", expected, actual); + break; + case STRING: + Assertions.assertThat(expected) + .as("Should expect a CharSequence") + .isInstanceOf(CharSequence.class); + Assert.assertEquals("string should be equal", String.valueOf(expected), actual.toString()); + break; + case DATE: + Assertions.assertThat(expected).as("Should expect a Date").isInstanceOf(LocalDate.class); + LocalDate date = DateTimeUtil.dateFromDays((int) actual); + Assert.assertEquals("date should be equal", expected, date); + break; + case TIME: + Assertions.assertThat(expected) + .as("Should expect a LocalTime") + .isInstanceOf(LocalTime.class); + int milliseconds = (int) (((LocalTime) expected).toNanoOfDay() / 1000_000); + Assert.assertEquals("time millis should be equal", milliseconds, actual); + break; + case TIMESTAMP: + if (((Types.TimestampType) type).shouldAdjustToUTC()) { + Assertions.assertThat(expected) + .as("Should expect a OffsetDataTime") + .isInstanceOf(OffsetDateTime.class); + OffsetDateTime ts = (OffsetDateTime) expected; + Assert.assertEquals( + "OffsetDataTime should be equal", + ts.toLocalDateTime(), + ((TimestampData) actual).toLocalDateTime()); + } else { + Assertions.assertThat(expected) + .as("Should expect a LocalDataTime") + .isInstanceOf(LocalDateTime.class); + LocalDateTime ts = (LocalDateTime) expected; + Assert.assertEquals( + "LocalDataTime should be equal", ts, ((TimestampData) actual).toLocalDateTime()); + } + break; + case BINARY: + Assertions.assertThat(expected) + .as("Should expect a ByteBuffer") + .isInstanceOf(ByteBuffer.class); + Assert.assertEquals("binary should be equal", expected, ByteBuffer.wrap((byte[]) actual)); + break; + case DECIMAL: + Assertions.assertThat(expected) + .as("Should expect a BigDecimal") + .isInstanceOf(BigDecimal.class); + BigDecimal bd = (BigDecimal) expected; + Assert.assertEquals( + "decimal value should be equal", bd, ((DecimalData) actual).toBigDecimal()); + break; + case LIST: + Assertions.assertThat(expected) + .as("Should expect a Collection") + .isInstanceOf(Collection.class); + Collection expectedArrayData = (Collection) expected; + ArrayData actualArrayData = (ArrayData) actual; + LogicalType elementType = ((ArrayType) logicalType).getElementType(); + Assert.assertEquals( + "array length should be equal", expectedArrayData.size(), actualArrayData.size()); + assertArrayValues( + type.asListType().elementType(), elementType, expectedArrayData, actualArrayData); + break; + case MAP: + Assertions.assertThat(expected).as("Should expect a Map").isInstanceOf(Map.class); + assertMapValues(type.asMapType(), logicalType, (Map) expected, (MapData) actual); + break; + case STRUCT: + Assertions.assertThat(expected).as("Should expect a Record").isInstanceOf(StructLike.class); + assertRowData(type.asStructType(), logicalType, (StructLike) expected, (RowData) actual); + break; + case UUID: + Assertions.assertThat(expected).as("Should expect a UUID").isInstanceOf(UUID.class); + ByteBuffer bb = ByteBuffer.wrap((byte[]) actual); + long firstLong = bb.getLong(); + long secondLong = bb.getLong(); + Assert.assertEquals( + "UUID should be equal", + expected.toString(), + new UUID(firstLong, secondLong).toString()); + break; + case FIXED: + Assertions.assertThat(expected).as("Should expect byte[]").isInstanceOf(byte[].class); + Assert.assertArrayEquals("binary should be equal", (byte[]) expected, (byte[]) actual); + break; + default: + throw new IllegalArgumentException("Not a supported type: " + type); + } + } + + public static void assertEquals(Schema schema, List records, List rows) { + Streams.forEachPair( + records.stream(), rows.stream(), (record, row) -> assertEquals(schema, record, row)); + } + + public static void assertEquals(Schema schema, GenericData.Record record, Row row) { + List fields = schema.asStruct().fields(); + Assert.assertEquals(fields.size(), record.getSchema().getFields().size()); + Assert.assertEquals(fields.size(), row.getArity()); + RowType rowType = FlinkSchemaUtil.convert(schema); + for (int i = 0; i < fields.size(); ++i) { + Type fieldType = fields.get(i).type(); + Object expectedValue = record.get(i); + Object actualValue = row.getField(i); + LogicalType logicalType = rowType.getTypeAt(i); + assertAvroEquals(fieldType, logicalType, expectedValue, actualValue); + } + } + + private static void assertEquals(Types.StructType struct, GenericData.Record record, Row row) { + List fields = struct.fields(); + for (int i = 0; i < fields.size(); i += 1) { + Type fieldType = fields.get(i).type(); + Object expectedValue = record.get(i); + Object actualValue = row.getField(i); + assertAvroEquals(fieldType, null, expectedValue, actualValue); + } + } + + private static void assertAvroEquals( + Type type, LogicalType logicalType, Object expected, Object actual) { + + if (expected == null && actual == null) { + return; + } + + Assert.assertTrue( + "expected and actual should be both null or not null", expected != null && actual != null); + + switch (type.typeId()) { + case BOOLEAN: + case INTEGER: + case LONG: + case FLOAT: + case DOUBLE: + Assertions.assertThat(expected) + .as("Should expect a " + type.typeId().javaClass()) + .isInstanceOf(type.typeId().javaClass()); + Assertions.assertThat(actual) + .as("Should expect a " + type.typeId().javaClass()) + .isInstanceOf(type.typeId().javaClass()); + Assert.assertEquals(type.typeId() + " value should be equal", expected, actual); + break; + case STRING: + Assertions.assertThat(expected) + .as("Should expect a CharSequence") + .isInstanceOf(CharSequence.class); + Assertions.assertThat(actual) + .as("Should expect a CharSequence") + .isInstanceOf(CharSequence.class); + Assert.assertEquals("string should be equal", expected.toString(), actual.toString()); + break; + case DATE: + Assertions.assertThat(expected).as("Should expect a Date").isInstanceOf(LocalDate.class); + LocalDate date = DateTimeUtil.dateFromDays((int) actual); + Assert.assertEquals("date should be equal", expected, date); + break; + case TIME: + Assertions.assertThat(expected) + .as("Should expect a LocalTime") + .isInstanceOf(LocalTime.class); + int milliseconds = (int) (((LocalTime) expected).toNanoOfDay() / 1000_000); + Assert.assertEquals("time millis should be equal", milliseconds, actual); + break; + case TIMESTAMP: + if (((Types.TimestampType) type).shouldAdjustToUTC()) { + Assertions.assertThat(expected) + .as("Should expect a OffsetDataTime") + .isInstanceOf(OffsetDateTime.class); + OffsetDateTime ts = (OffsetDateTime) expected; + Assert.assertEquals( + "OffsetDataTime should be equal", + ts.toLocalDateTime(), + ((TimestampData) actual).toLocalDateTime()); + } else { + Assertions.assertThat(expected) + .as("Should expect a LocalDataTime") + .isInstanceOf(LocalDateTime.class); + LocalDateTime ts = (LocalDateTime) expected; + Assert.assertEquals( + "LocalDataTime should be equal", ts, ((TimestampData) actual).toLocalDateTime()); + } + break; + case BINARY: + Assertions.assertThat(expected) + .as("Should expect a ByteBuffer") + .isInstanceOf(ByteBuffer.class); + Assert.assertEquals("binary should be equal", expected, ByteBuffer.wrap((byte[]) actual)); + break; + case DECIMAL: + Assertions.assertThat(expected) + .as("Should expect a BigDecimal") + .isInstanceOf(BigDecimal.class); + BigDecimal bd = (BigDecimal) expected; + Assert.assertEquals( + "decimal value should be equal", bd, ((DecimalData) actual).toBigDecimal()); + break; + case LIST: + Assertions.assertThat(expected) + .as("Should expect a Collection") + .isInstanceOf(Collection.class); + Collection expectedArrayData = (Collection) expected; + ArrayData actualArrayData; + try { + actualArrayData = (ArrayData) actual; + } catch (ClassCastException e) { + actualArrayData = new GenericArrayData((Object[]) actual); + } + LogicalType elementType = ((ArrayType) logicalType).getElementType(); + Assert.assertEquals( + "array length should be equal", expectedArrayData.size(), actualArrayData.size()); + assertArrayValues( + type.asListType().elementType(), elementType, expectedArrayData, actualArrayData); + break; + case MAP: + Assertions.assertThat(expected).as("Should expect a Map").isInstanceOf(Map.class); + MapData actualMap; + try { + actualMap = (MapData) actual; + } catch (ClassCastException e) { + actualMap = new GenericMapData((Map) actual); + } + assertMapValues(type.asMapType(), logicalType, (Map) expected, actualMap); + break; + case STRUCT: + Assertions.assertThat(expected) + .as("Should expect a Record") + .isInstanceOf(GenericData.Record.class); + assertEquals( + type.asNestedType().asStructType(), (GenericData.Record) expected, (Row) actual); + break; + case UUID: + Assertions.assertThat(expected).as("Should expect a UUID").isInstanceOf(UUID.class); + ByteBuffer bb = ByteBuffer.wrap((byte[]) actual); + long firstLong = bb.getLong(); + long secondLong = bb.getLong(); + Assert.assertEquals( + "UUID should be equal", + expected.toString(), + new UUID(firstLong, secondLong).toString()); + break; + case FIXED: + Assertions.assertThat(expected).as("Should expect byte[]").isInstanceOf(byte[].class); + Assert.assertArrayEquals("binary should be equal", (byte[]) expected, (byte[]) actual); + break; + default: + throw new IllegalArgumentException("Not a supported type: " + type); + } + } + + private static void assertArrayValues( + Type type, LogicalType logicalType, Collection expectedArray, ArrayData actualArray) { + List expectedElements = Lists.newArrayList(expectedArray); + for (int i = 0; i < expectedArray.size(); i += 1) { + if (expectedElements.get(i) == null) { + Assert.assertTrue(actualArray.isNullAt(i)); + continue; + } + + Object expected = expectedElements.get(i); + + assertEquals( + type, + logicalType, + expected, + ArrayData.createElementGetter(logicalType).getElementOrNull(actualArray, i)); + } + } + + private static void assertMapValues( + Types.MapType mapType, LogicalType type, Map expected, MapData actual) { + Assert.assertEquals("map size should be equal", expected.size(), actual.size()); + + ArrayData actualKeyArrayData = actual.keyArray(); + ArrayData actualValueArrayData = actual.valueArray(); + LogicalType actualKeyType = ((MapType) type).getKeyType(); + LogicalType actualValueType = ((MapType) type).getValueType(); + Type keyType = mapType.keyType(); + Type valueType = mapType.valueType(); + + ArrayData.ElementGetter keyGetter = ArrayData.createElementGetter(actualKeyType); + ArrayData.ElementGetter valueGetter = ArrayData.createElementGetter(actualValueType); + + for (Map.Entry entry : expected.entrySet()) { + Object matchedActualKey = null; + int matchedKeyIndex = 0; + for (int i = 0; i < actual.size(); i += 1) { + try { + Object key = keyGetter.getElementOrNull(actualKeyArrayData, i); + assertEquals(keyType, actualKeyType, entry.getKey(), key); + matchedActualKey = key; + matchedKeyIndex = i; + break; + } catch (AssertionError e) { + // not found + } + } + Assert.assertNotNull("Should have a matching key", matchedActualKey); + final int valueIndex = matchedKeyIndex; + assertEquals( + valueType, + actualValueType, + entry.getValue(), + valueGetter.getElementOrNull(actualValueArrayData, valueIndex)); + } + } + + public static void assertEquals(ManifestFile expected, ManifestFile actual) { + if (expected == actual) { + return; + } + Assert.assertTrue("Should not be null.", expected != null && actual != null); + Assert.assertEquals("Path must match", expected.path(), actual.path()); + Assert.assertEquals("Length must match", expected.length(), actual.length()); + Assert.assertEquals("Spec id must match", expected.partitionSpecId(), actual.partitionSpecId()); + Assert.assertEquals("ManifestContent must match", expected.content(), actual.content()); + Assert.assertEquals( + "SequenceNumber must match", expected.sequenceNumber(), actual.sequenceNumber()); + Assert.assertEquals( + "MinSequenceNumber must match", expected.minSequenceNumber(), actual.minSequenceNumber()); + Assert.assertEquals("Snapshot id must match", expected.snapshotId(), actual.snapshotId()); + Assert.assertEquals( + "Added files flag must match", expected.hasAddedFiles(), actual.hasAddedFiles()); + Assert.assertEquals( + "Added files count must match", expected.addedFilesCount(), actual.addedFilesCount()); + Assert.assertEquals( + "Added rows count must match", expected.addedRowsCount(), actual.addedRowsCount()); + Assert.assertEquals( + "Existing files flag must match", expected.hasExistingFiles(), actual.hasExistingFiles()); + Assert.assertEquals( + "Existing files count must match", + expected.existingFilesCount(), + actual.existingFilesCount()); + Assert.assertEquals( + "Existing rows count must match", expected.existingRowsCount(), actual.existingRowsCount()); + Assert.assertEquals( + "Deleted files flag must match", expected.hasDeletedFiles(), actual.hasDeletedFiles()); + Assert.assertEquals( + "Deleted files count must match", expected.deletedFilesCount(), actual.deletedFilesCount()); + Assert.assertEquals( + "Deleted rows count must match", expected.deletedRowsCount(), actual.deletedRowsCount()); + + List expectedSummaries = expected.partitions(); + List actualSummaries = actual.partitions(); + Assert.assertEquals( + "PartitionFieldSummary size does not match", + expectedSummaries.size(), + actualSummaries.size()); + for (int i = 0; i < expectedSummaries.size(); i++) { + Assert.assertEquals( + "Null flag in partition must match", + expectedSummaries.get(i).containsNull(), + actualSummaries.get(i).containsNull()); + Assert.assertEquals( + "NaN flag in partition must match", + expectedSummaries.get(i).containsNaN(), + actualSummaries.get(i).containsNaN()); + Assert.assertEquals( + "Lower bounds in partition must match", + expectedSummaries.get(i).lowerBound(), + actualSummaries.get(i).lowerBound()); + Assert.assertEquals( + "Upper bounds in partition must match", + expectedSummaries.get(i).upperBound(), + actualSummaries.get(i).upperBound()); + } + } + + public static void assertEquals(ContentFile expected, ContentFile actual) { + if (expected == actual) { + return; + } + Assert.assertTrue("Shouldn't be null.", expected != null && actual != null); + Assert.assertEquals("SpecId", expected.specId(), actual.specId()); + Assert.assertEquals("Content", expected.content(), actual.content()); + Assert.assertEquals("Path", expected.path(), actual.path()); + Assert.assertEquals("Format", expected.format(), actual.format()); + Assert.assertEquals("Partition size", expected.partition().size(), actual.partition().size()); + for (int i = 0; i < expected.partition().size(); i++) { + Assert.assertEquals( + "Partition data at index " + i, + expected.partition().get(i, Object.class), + actual.partition().get(i, Object.class)); + } + Assert.assertEquals("Record count", expected.recordCount(), actual.recordCount()); + Assert.assertEquals("File size in bytes", expected.fileSizeInBytes(), actual.fileSizeInBytes()); + Assert.assertEquals("Column sizes", expected.columnSizes(), actual.columnSizes()); + Assert.assertEquals("Value counts", expected.valueCounts(), actual.valueCounts()); + Assert.assertEquals("Null value counts", expected.nullValueCounts(), actual.nullValueCounts()); + Assert.assertEquals("Lower bounds", expected.lowerBounds(), actual.lowerBounds()); + Assert.assertEquals("Upper bounds", expected.upperBounds(), actual.upperBounds()); + Assert.assertEquals("Key metadata", expected.keyMetadata(), actual.keyMetadata()); + Assert.assertEquals("Split offsets", expected.splitOffsets(), actual.splitOffsets()); + Assert.assertEquals( + "Equality field id list", actual.equalityFieldIds(), expected.equalityFieldIds()); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java new file mode 100644 index 000000000000..4f71b5fe8d7c --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java @@ -0,0 +1,350 @@ +/* + * 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.iceberg.flink; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Map; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.catalog.Catalog; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.types.Row; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.thrift.TException; +import org.assertj.core.api.Assertions; +import org.junit.After; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestIcebergConnector extends FlinkTestBase { + + private static final String TABLE_NAME = "test_table"; + + @ClassRule public static final TemporaryFolder WAREHOUSE = new TemporaryFolder(); + + private final String catalogName; + private final Map properties; + private final boolean isStreaming; + private volatile TableEnvironment tEnv; + + @Parameterized.Parameters(name = "catalogName = {0}, properties = {1}, isStreaming={2}") + public static Iterable parameters() { + return Lists.newArrayList( + // Create iceberg table in the hadoop catalog and default database. + new Object[] { + "testhadoop", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hadoop"), + true + }, + new Object[] { + "testhadoop", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hadoop", + "catalog-table", "not_existing_table"), + true + }, + new Object[] { + "testhadoop", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hadoop"), + false + }, + // Create iceberg table in the hadoop catalog and not_existing_db. + new Object[] { + "testhadoop", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hadoop", + "catalog-database", "not_existing_db"), + true + }, + new Object[] { + "testhadoop", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hadoop", + "catalog-database", "not_existing_db", + "catalog-table", "not_existing_table"), + true + }, + new Object[] { + "testhadoop", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hadoop", + "catalog-database", "not_existing_db"), + false + }, + // Create iceberg table in the hive catalog and default database. + new Object[] { + "testhive", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hive"), + true + }, + new Object[] { + "testhive", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hive", + "catalog-table", "not_existing_table"), + true + }, + new Object[] { + "testhive", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hive"), + false + }, + // Create iceberg table in the hive catalog and not_existing_db. + new Object[] { + "testhive", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hive", + "catalog-database", "not_existing_db"), + true + }, + new Object[] { + "testhive", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hive", + "catalog-database", "not_existing_db", + "catalog-table", "not_existing_table"), + true + }, + new Object[] { + "testhive", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hive", + "catalog-database", "not_existing_db"), + false + }); + } + + public TestIcebergConnector( + String catalogName, Map properties, boolean isStreaming) { + this.catalogName = catalogName; + this.properties = properties; + this.isStreaming = isStreaming; + } + + @Override + protected TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + if (tEnv == null) { + EnvironmentSettings.Builder settingsBuilder = EnvironmentSettings.newInstance(); + if (isStreaming) { + settingsBuilder.inStreamingMode(); + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + env.enableCheckpointing(400); + env.setMaxParallelism(2); + env.setParallelism(2); + tEnv = StreamTableEnvironment.create(env, settingsBuilder.build()); + } else { + settingsBuilder.inBatchMode(); + tEnv = TableEnvironment.create(settingsBuilder.build()); + } + // Set only one parallelism. + tEnv.getConfig() + .getConfiguration() + .set(CoreOptions.DEFAULT_PARALLELISM, 1) + .set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); + } + } + } + return tEnv; + } + + @After + public void after() throws TException { + sql("DROP TABLE IF EXISTS %s", TABLE_NAME); + + // Clean the created orphan databases and tables from hive-metastore. + if (isHiveCatalog()) { + HiveMetaStoreClient metaStoreClient = new HiveMetaStoreClient(hiveConf); + try { + metaStoreClient.dropTable(databaseName(), tableName()); + if (!isDefaultDatabaseName()) { + try { + metaStoreClient.dropDatabase(databaseName()); + } catch (Exception ignored) { + // Ignore + } + } + } finally { + metaStoreClient.close(); + } + } + } + + private void testCreateConnectorTable() { + Map tableProps = createTableProps(); + + // Create table under the flink's current database. + sql("CREATE TABLE %s (id BIGINT, data STRING) WITH %s", TABLE_NAME, toWithClause(tableProps)); + sql("INSERT INTO %s VALUES (1, 'AAA'), (2, 'BBB'), (3, 'CCC')", TABLE_NAME); + Assert.assertEquals( + "Should have expected rows", + Sets.newHashSet(Row.of(1L, "AAA"), Row.of(2L, "BBB"), Row.of(3L, "CCC")), + Sets.newHashSet(sql("SELECT * FROM %s", TABLE_NAME))); + + FlinkCatalogFactory factory = new FlinkCatalogFactory(); + Catalog flinkCatalog = factory.createCatalog(catalogName, tableProps, new Configuration()); + Assert.assertTrue( + "Should have created the expected database", flinkCatalog.databaseExists(databaseName())); + Assert.assertTrue( + "Should have created the expected table", + flinkCatalog.tableExists(new ObjectPath(databaseName(), tableName()))); + + // Drop and create it again. + sql("DROP TABLE %s", TABLE_NAME); + sql("CREATE TABLE %s (id BIGINT, data STRING) WITH %s", TABLE_NAME, toWithClause(tableProps)); + Assert.assertEquals( + "Should have expected rows", + Sets.newHashSet(Row.of(1L, "AAA"), Row.of(2L, "BBB"), Row.of(3L, "CCC")), + Sets.newHashSet(sql("SELECT * FROM %s", TABLE_NAME))); + } + + @Test + public void testCreateTableUnderDefaultDatabase() { + testCreateConnectorTable(); + } + + @Test + public void testCatalogDatabaseConflictWithFlinkDatabase() { + sql("CREATE DATABASE IF NOT EXISTS `%s`", databaseName()); + sql("USE `%s`", databaseName()); + + try { + testCreateConnectorTable(); + // Ensure that the table was created under the specific database. + Assertions.assertThatThrownBy( + () -> sql("CREATE TABLE `default_catalog`.`%s`.`%s`", databaseName(), TABLE_NAME)) + .isInstanceOf(org.apache.flink.table.api.TableException.class) + .hasMessageStartingWith("Could not execute CreateTable in path"); + } finally { + sql("DROP TABLE IF EXISTS `%s`.`%s`", databaseName(), TABLE_NAME); + if (!isDefaultDatabaseName()) { + sql("DROP DATABASE `%s`", databaseName()); + } + } + } + + @Test + public void testConnectorTableInIcebergCatalog() { + // Create the catalog properties + Map catalogProps = Maps.newHashMap(); + catalogProps.put("type", "iceberg"); + if (isHiveCatalog()) { + catalogProps.put("catalog-type", "hive"); + catalogProps.put(CatalogProperties.URI, FlinkCatalogTestBase.getURI(hiveConf)); + } else { + catalogProps.put("catalog-type", "hadoop"); + } + catalogProps.put(CatalogProperties.WAREHOUSE_LOCATION, createWarehouse()); + + // Create the table properties + Map tableProps = createTableProps(); + + // Create a connector table in an iceberg catalog. + sql("CREATE CATALOG `test_catalog` WITH %s", toWithClause(catalogProps)); + try { + Assertions.assertThatThrownBy( + () -> + sql( + "CREATE TABLE `test_catalog`.`%s`.`%s` (id BIGINT, data STRING) WITH %s", + FlinkCatalogFactory.DEFAULT_DATABASE_NAME, + TABLE_NAME, + toWithClause(tableProps))) + .cause() + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot create the table with 'connector'='iceberg' table property in an iceberg catalog, " + + "Please create table with 'connector'='iceberg' property in a non-iceberg catalog or " + + "create table without 'connector'='iceberg' related properties in an iceberg table."); + } finally { + sql("DROP CATALOG IF EXISTS `test_catalog`"); + } + } + + private Map createTableProps() { + Map tableProps = Maps.newHashMap(properties); + tableProps.put("catalog-name", catalogName); + tableProps.put(CatalogProperties.WAREHOUSE_LOCATION, createWarehouse()); + if (isHiveCatalog()) { + tableProps.put(CatalogProperties.URI, FlinkCatalogTestBase.getURI(hiveConf)); + } + return tableProps; + } + + private boolean isHiveCatalog() { + return "testhive".equalsIgnoreCase(catalogName); + } + + private boolean isDefaultDatabaseName() { + return FlinkCatalogFactory.DEFAULT_DATABASE_NAME.equalsIgnoreCase(databaseName()); + } + + private String tableName() { + return properties.getOrDefault("catalog-table", TABLE_NAME); + } + + private String databaseName() { + return properties.getOrDefault("catalog-database", "default_database"); + } + + private String toWithClause(Map props) { + return FlinkCatalogTestBase.toWithClause(props); + } + + private static String createWarehouse() { + try { + return String.format("file://%s", WAREHOUSE.newFolder().getAbsolutePath()); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java new file mode 100644 index 000000000000..6bd94e9ca61c --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java @@ -0,0 +1,174 @@ +/* + * 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.iceberg.flink; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.GenericManifestFile; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.ManifestWriter; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestManifestFileSerialization { + + private static final Schema SCHEMA = + new Schema( + required(1, "id", Types.LongType.get()), + optional(2, "data", Types.StringType.get()), + required(3, "date", Types.StringType.get()), + required(4, "double", Types.DoubleType.get())); + + private static final PartitionSpec SPEC = + PartitionSpec.builderFor(SCHEMA).identity("double").build(); + + private static final DataFile FILE_A = + DataFiles.builder(SPEC) + .withPath("/path/to/data-1.parquet") + .withFileSizeInBytes(0) + .withPartition(org.apache.iceberg.TestHelpers.Row.of(1D)) + .withPartitionPath("double=1") + .withMetrics( + new Metrics( + 5L, + null, // no column sizes + ImmutableMap.of(1, 5L, 2, 3L), // value count + ImmutableMap.of(1, 0L, 2, 2L), // null count + ImmutableMap.of(), // nan count + ImmutableMap.of(1, longToBuffer(0L)), // lower bounds + ImmutableMap.of(1, longToBuffer(4L)) // upper bounds + )) + .build(); + + private static final DataFile FILE_B = + DataFiles.builder(SPEC) + .withPath("/path/to/data-2.parquet") + .withFileSizeInBytes(0) + .withPartition(org.apache.iceberg.TestHelpers.Row.of(Double.NaN)) + .withPartitionPath("double=NaN") + .withMetrics( + new Metrics( + 1L, + null, // no column sizes + ImmutableMap.of(1, 1L, 4, 1L), // value count + ImmutableMap.of(1, 0L, 2, 0L), // null count + ImmutableMap.of(4, 1L), // nan count + ImmutableMap.of(1, longToBuffer(0L)), // lower bounds + ImmutableMap.of(1, longToBuffer(1L)) // upper bounds + )) + .build(); + + private static final FileIO FILE_IO = new HadoopFileIO(new Configuration()); + + @Rule public TemporaryFolder temp = new TemporaryFolder(); + + @Test + public void testKryoSerialization() throws IOException { + KryoSerializer kryo = + new KryoSerializer<>(ManifestFile.class, new ExecutionConfig()); + + DataOutputSerializer outputView = new DataOutputSerializer(1024); + + ManifestFile manifest = writeManifest(FILE_A, FILE_B); + + kryo.serialize(manifest, outputView); + kryo.serialize(manifest.copy(), outputView); + kryo.serialize(GenericManifestFile.copyOf(manifest).build(), outputView); + + DataInputDeserializer inputView = new DataInputDeserializer(outputView.getCopyOfBuffer()); + ManifestFile m1 = kryo.deserialize(inputView); + ManifestFile m2 = kryo.deserialize(inputView); + ManifestFile m3 = kryo.deserialize(inputView); + + TestHelpers.assertEquals(manifest, m1); + TestHelpers.assertEquals(manifest, m2); + TestHelpers.assertEquals(manifest, m3); + } + + @Test + public void testJavaSerialization() throws Exception { + ByteArrayOutputStream bytes = new ByteArrayOutputStream(); + + ManifestFile manifest = writeManifest(FILE_A, FILE_B); + + try (ObjectOutputStream out = new ObjectOutputStream(bytes)) { + out.writeObject(manifest); + out.writeObject(manifest.copy()); + out.writeObject(GenericManifestFile.copyOf(manifest).build()); + } + + try (ObjectInputStream in = + new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { + for (int i = 0; i < 3; i += 1) { + Object obj = in.readObject(); + Assertions.assertThat(obj).as("Should be a ManifestFile").isInstanceOf(ManifestFile.class); + TestHelpers.assertEquals(manifest, (ManifestFile) obj); + } + } + } + + private ManifestFile writeManifest(DataFile... files) throws IOException { + File manifestFile = temp.newFile("input.m0.avro"); + Assert.assertTrue(manifestFile.delete()); + OutputFile outputFile = FILE_IO.newOutputFile(manifestFile.getCanonicalPath()); + + ManifestWriter writer = ManifestFiles.write(SPEC, outputFile); + try { + for (DataFile file : files) { + writer.add(file); + } + } finally { + writer.close(); + } + + return writer.toManifestFile(); + } + + private static ByteBuffer longToBuffer(long value) { + return ByteBuffer.allocate(8).order(ByteOrder.LITTLE_ENDIAN).putLong(0, value); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java new file mode 100644 index 000000000000..c78fa51215dd --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java @@ -0,0 +1,93 @@ +/* + * 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.iceberg.flink; + +import java.util.Iterator; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.RecordWrapperTest; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.data.InternalRecordWrapper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.data.RandomRowData; +import org.apache.iceberg.util.StructLikeWrapper; +import org.assertj.core.api.Assertions; +import org.junit.Assert; + +public class TestRowDataWrapper extends RecordWrapperTest { + + /** + * Flink's time type has been truncated to millis seconds, so we need a customized assert method + * to check the values. + */ + @Override + public void testTime() { + generateAndValidate( + new Schema(TIME.fields()), + (message, expectedWrapper, actualWrapper) -> { + for (int pos = 0; pos < TIME.fields().size(); pos++) { + Object expected = expectedWrapper.get().get(pos, Object.class); + Object actual = actualWrapper.get().get(pos, Object.class); + if (expected == actual) { + return; + } + + Assertions.assertThat(actual).isNotNull(); + Assertions.assertThat(expected).isNotNull(); + + int expectedMilliseconds = (int) ((long) expected / 1000_000); + int actualMilliseconds = (int) ((long) actual / 1000_000); + Assert.assertEquals(message, expectedMilliseconds, actualMilliseconds); + } + }); + } + + @Override + protected void generateAndValidate(Schema schema, RecordWrapperTest.AssertMethod assertMethod) { + int numRecords = 100; + Iterable recordList = RandomGenericData.generate(schema, numRecords, 101L); + Iterable rowDataList = RandomRowData.generate(schema, numRecords, 101L); + + InternalRecordWrapper recordWrapper = new InternalRecordWrapper(schema.asStruct()); + RowDataWrapper rowDataWrapper = + new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); + + Iterator actual = recordList.iterator(); + Iterator expected = rowDataList.iterator(); + + StructLikeWrapper actualWrapper = StructLikeWrapper.forType(schema.asStruct()); + StructLikeWrapper expectedWrapper = StructLikeWrapper.forType(schema.asStruct()); + for (int i = 0; i < numRecords; i++) { + Assert.assertTrue("Should have more records", actual.hasNext()); + Assert.assertTrue("Should have more RowData", expected.hasNext()); + + StructLike recordStructLike = recordWrapper.wrap(actual.next()); + StructLike rowDataStructLike = rowDataWrapper.wrap(expected.next()); + + assertMethod.assertEquals( + "Should have expected StructLike values", + actualWrapper.set(recordStructLike), + expectedWrapper.set(rowDataStructLike)); + } + + Assert.assertFalse("Shouldn't have more record", actual.hasNext()); + Assert.assertFalse("Shouldn't have more RowData", expected.hasNext()); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java new file mode 100644 index 000000000000..4ad302dde436 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink; + +import java.io.File; +import org.apache.iceberg.Table; +import org.apache.iceberg.TestTables; + +public class TestTableLoader implements TableLoader { + private File dir; + + public static TableLoader of(String dir) { + return new TestTableLoader(dir); + } + + public TestTableLoader(String dir) { + this.dir = new File(dir); + } + + @Override + public void open() {} + + @Override + public boolean isOpen() { + return true; + } + + @Override + public Table loadTable() { + return TestTables.load(dir, "test"); + } + + @Override + @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) + public TableLoader clone() { + return new TestTableLoader(dir.getAbsolutePath()); + } + + @Override + public void close() {} +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java new file mode 100644 index 000000000000..27124d93fef4 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java @@ -0,0 +1,110 @@ +/* + * 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.iceberg.flink; + +import static org.apache.iceberg.flink.TestHelpers.roundTripKryoSerialize; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.io.File; +import java.io.IOException; +import java.util.Map; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.MetadataTableUtils; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.Transaction; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestTableSerialization { + private static final HadoopTables TABLES = new HadoopTables(); + + private static final Schema SCHEMA = + new Schema( + required(1, "id", Types.LongType.get()), + optional(2, "data", Types.StringType.get()), + required(3, "date", Types.StringType.get()), + optional(4, "double", Types.DoubleType.get())); + + private static final PartitionSpec SPEC = + PartitionSpec.builderFor(SCHEMA).identity("date").build(); + + private static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); + + @Rule public TemporaryFolder temp = new TemporaryFolder(); + private Table table; + + @Before + public void initTable() throws IOException { + Map props = ImmutableMap.of("k1", "v1", "k2", "v2"); + + File tableLocation = temp.newFolder(); + Assert.assertTrue(tableLocation.delete()); + + this.table = TABLES.create(SCHEMA, SPEC, SORT_ORDER, props, tableLocation.toString()); + } + + @Test + public void testSerializableTableKryoSerialization() throws IOException { + SerializableTable serializableTable = (SerializableTable) SerializableTable.copyOf(table); + TestHelpers.assertSerializedAndLoadedMetadata( + table, roundTripKryoSerialize(SerializableTable.class, serializableTable)); + } + + @Test + public void testSerializableMetadataTableKryoSerialization() throws IOException { + for (MetadataTableType type : MetadataTableType.values()) { + TableOperations ops = ((HasTableOperations) table).operations(); + Table metadataTable = + MetadataTableUtils.createMetadataTableInstance(ops, table.name(), "meta", type); + SerializableTable serializableMetadataTable = + (SerializableTable) SerializableTable.copyOf(metadataTable); + + TestHelpers.assertSerializedAndLoadedMetadata( + metadataTable, + roundTripKryoSerialize(SerializableTable.class, serializableMetadataTable)); + } + } + + @Test + public void testSerializableTransactionTableKryoSerialization() throws IOException { + Transaction txn = table.newTransaction(); + + txn.updateProperties().set("k1", "v1").commit(); + + Table txnTable = txn.table(); + SerializableTable serializableTxnTable = (SerializableTable) SerializableTable.copyOf(txnTable); + + TestHelpers.assertSerializedMetadata( + txnTable, roundTripKryoSerialize(SerializableTable.class, serializableTxnTable)); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java new file mode 100644 index 000000000000..07e5ca051da5 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java @@ -0,0 +1,499 @@ +/* + * 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.iceberg.flink.actions; + +import static org.apache.iceberg.flink.SimpleDataUtil.RECORD; + +import java.io.File; +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.types.Row; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileContent; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.RewriteDataFilesActionResult; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.flink.FlinkCatalogTestBase; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Pair; +import org.assertj.core.api.Assertions; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestRewriteDataFilesAction extends FlinkCatalogTestBase { + + private static final String TABLE_NAME_UNPARTITIONED = "test_table_unpartitioned"; + private static final String TABLE_NAME_PARTITIONED = "test_table_partitioned"; + private static final String TABLE_NAME_WITH_PK = "test_table_with_pk"; + private final FileFormat format; + private Table icebergTableUnPartitioned; + private Table icebergTablePartitioned; + private Table icebergTableWithPk; + + public TestRewriteDataFilesAction( + String catalogName, Namespace baseNamespace, FileFormat format) { + super(catalogName, baseNamespace); + this.format = format; + } + + @Override + protected TableEnvironment getTableEnv() { + super.getTableEnv().getConfig().getConfiguration().set(CoreOptions.DEFAULT_PARALLELISM, 1); + return super.getTableEnv(); + } + + @Parameterized.Parameters(name = "catalogName={0}, baseNamespace={1}, format={2}") + public static Iterable parameters() { + List parameters = Lists.newArrayList(); + for (FileFormat format : + new FileFormat[] {FileFormat.AVRO, FileFormat.ORC, FileFormat.PARQUET}) { + for (Object[] catalogParams : FlinkCatalogTestBase.parameters()) { + String catalogName = (String) catalogParams[0]; + Namespace baseNamespace = (Namespace) catalogParams[1]; + parameters.add(new Object[] {catalogName, baseNamespace, format}); + } + } + return parameters; + } + + @Rule public TemporaryFolder temp = new TemporaryFolder(); + + @Override + @Before + public void before() { + super.before(); + sql("CREATE DATABASE %s", flinkDatabase); + sql("USE CATALOG %s", catalogName); + sql("USE %s", DATABASE); + sql( + "CREATE TABLE %s (id int, data varchar) with ('write.format.default'='%s')", + TABLE_NAME_UNPARTITIONED, format.name()); + icebergTableUnPartitioned = + validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME_UNPARTITIONED)); + + sql( + "CREATE TABLE %s (id int, data varchar,spec varchar) " + + " PARTITIONED BY (data,spec) with ('write.format.default'='%s')", + TABLE_NAME_PARTITIONED, format.name()); + icebergTablePartitioned = + validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME_PARTITIONED)); + + sql( + "CREATE TABLE %s (id int, data varchar, PRIMARY KEY(`id`) NOT ENFORCED) with ('write.format.default'='%s', 'format-version'='2')", + TABLE_NAME_WITH_PK, format.name()); + icebergTableWithPk = + validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME_WITH_PK)); + } + + @Override + @After + public void clean() { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME_UNPARTITIONED); + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME_PARTITIONED); + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME_WITH_PK); + sql("DROP DATABASE IF EXISTS %s", flinkDatabase); + super.clean(); + } + + @Test + public void testRewriteDataFilesEmptyTable() throws Exception { + Assert.assertNull("Table must be empty", icebergTableUnPartitioned.currentSnapshot()); + Actions.forTable(icebergTableUnPartitioned).rewriteDataFiles().execute(); + Assert.assertNull("Table must stay empty", icebergTableUnPartitioned.currentSnapshot()); + } + + @Test + public void testRewriteDataFilesUnpartitionedTable() throws Exception { + sql("INSERT INTO %s SELECT 1, 'hello'", TABLE_NAME_UNPARTITIONED); + sql("INSERT INTO %s SELECT 2, 'world'", TABLE_NAME_UNPARTITIONED); + + icebergTableUnPartitioned.refresh(); + + CloseableIterable tasks = icebergTableUnPartitioned.newScan().planFiles(); + List dataFiles = + Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); + Assert.assertEquals("Should have 2 data files before rewrite", 2, dataFiles.size()); + + RewriteDataFilesActionResult result = + Actions.forTable(icebergTableUnPartitioned).rewriteDataFiles().execute(); + + Assert.assertEquals("Action should rewrite 2 data files", 2, result.deletedDataFiles().size()); + Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFiles().size()); + + icebergTableUnPartitioned.refresh(); + + CloseableIterable tasks1 = icebergTableUnPartitioned.newScan().planFiles(); + List dataFiles1 = + Lists.newArrayList(CloseableIterable.transform(tasks1, FileScanTask::file)); + Assert.assertEquals("Should have 1 data files after rewrite", 1, dataFiles1.size()); + + // Assert the table records as expected. + SimpleDataUtil.assertTableRecords( + icebergTableUnPartitioned, + Lists.newArrayList( + SimpleDataUtil.createRecord(1, "hello"), SimpleDataUtil.createRecord(2, "world"))); + } + + @Test + public void testRewriteDataFilesPartitionedTable() throws Exception { + sql("INSERT INTO %s SELECT 1, 'hello' ,'a'", TABLE_NAME_PARTITIONED); + sql("INSERT INTO %s SELECT 2, 'hello' ,'a'", TABLE_NAME_PARTITIONED); + sql("INSERT INTO %s SELECT 3, 'world' ,'b'", TABLE_NAME_PARTITIONED); + sql("INSERT INTO %s SELECT 4, 'world' ,'b'", TABLE_NAME_PARTITIONED); + + icebergTablePartitioned.refresh(); + + CloseableIterable tasks = icebergTablePartitioned.newScan().planFiles(); + List dataFiles = + Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); + Assert.assertEquals("Should have 4 data files before rewrite", 4, dataFiles.size()); + + RewriteDataFilesActionResult result = + Actions.forTable(icebergTablePartitioned).rewriteDataFiles().execute(); + + Assert.assertEquals("Action should rewrite 4 data files", 4, result.deletedDataFiles().size()); + Assert.assertEquals("Action should add 2 data file", 2, result.addedDataFiles().size()); + + icebergTablePartitioned.refresh(); + + CloseableIterable tasks1 = icebergTablePartitioned.newScan().planFiles(); + List dataFiles1 = + Lists.newArrayList(CloseableIterable.transform(tasks1, FileScanTask::file)); + Assert.assertEquals("Should have 2 data files after rewrite", 2, dataFiles1.size()); + + // Assert the table records as expected. + Schema schema = + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get()), + Types.NestedField.optional(3, "spec", Types.StringType.get())); + + Record record = GenericRecord.create(schema); + SimpleDataUtil.assertTableRecords( + icebergTablePartitioned, + Lists.newArrayList( + record.copy("id", 1, "data", "hello", "spec", "a"), + record.copy("id", 2, "data", "hello", "spec", "a"), + record.copy("id", 3, "data", "world", "spec", "b"), + record.copy("id", 4, "data", "world", "spec", "b"))); + } + + @Test + public void testRewriteDataFilesWithFilter() throws Exception { + sql("INSERT INTO %s SELECT 1, 'hello' ,'a'", TABLE_NAME_PARTITIONED); + sql("INSERT INTO %s SELECT 2, 'hello' ,'a'", TABLE_NAME_PARTITIONED); + sql("INSERT INTO %s SELECT 3, 'world' ,'a'", TABLE_NAME_PARTITIONED); + sql("INSERT INTO %s SELECT 4, 'world' ,'b'", TABLE_NAME_PARTITIONED); + sql("INSERT INTO %s SELECT 5, 'world' ,'b'", TABLE_NAME_PARTITIONED); + + icebergTablePartitioned.refresh(); + + CloseableIterable tasks = icebergTablePartitioned.newScan().planFiles(); + List dataFiles = + Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); + Assert.assertEquals("Should have 5 data files before rewrite", 5, dataFiles.size()); + + RewriteDataFilesActionResult result = + Actions.forTable(icebergTablePartitioned) + .rewriteDataFiles() + .filter(Expressions.equal("spec", "a")) + .filter(Expressions.startsWith("data", "he")) + .execute(); + + Assert.assertEquals("Action should rewrite 2 data files", 2, result.deletedDataFiles().size()); + Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFiles().size()); + + icebergTablePartitioned.refresh(); + + CloseableIterable tasks1 = icebergTablePartitioned.newScan().planFiles(); + List dataFiles1 = + Lists.newArrayList(CloseableIterable.transform(tasks1, FileScanTask::file)); + Assert.assertEquals("Should have 4 data files after rewrite", 4, dataFiles1.size()); + + // Assert the table records as expected. + Schema schema = + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get()), + Types.NestedField.optional(3, "spec", Types.StringType.get())); + + Record record = GenericRecord.create(schema); + SimpleDataUtil.assertTableRecords( + icebergTablePartitioned, + Lists.newArrayList( + record.copy("id", 1, "data", "hello", "spec", "a"), + record.copy("id", 2, "data", "hello", "spec", "a"), + record.copy("id", 3, "data", "world", "spec", "a"), + record.copy("id", 4, "data", "world", "spec", "b"), + record.copy("id", 5, "data", "world", "spec", "b"))); + } + + @Test + public void testRewriteLargeTableHasResiduals() throws IOException { + // all records belong to the same partition + List records1 = Lists.newArrayList(); + List records2 = Lists.newArrayList(); + List expected = Lists.newArrayList(); + for (int i = 0; i < 100; i++) { + int id = i; + String data = String.valueOf(i % 3); + if (i % 2 == 0) { + records1.add("(" + id + ",'" + data + "')"); + } else { + records2.add("(" + id + ",'" + data + "')"); + } + Record record = RECORD.copy(); + record.setField("id", id); + record.setField("data", data); + expected.add(record); + } + + sql("INSERT INTO %s values " + StringUtils.join(records1, ","), TABLE_NAME_UNPARTITIONED); + sql("INSERT INTO %s values " + StringUtils.join(records2, ","), TABLE_NAME_UNPARTITIONED); + + icebergTableUnPartitioned.refresh(); + + CloseableIterable tasks = + icebergTableUnPartitioned + .newScan() + .ignoreResiduals() + .filter(Expressions.equal("data", "0")) + .planFiles(); + for (FileScanTask task : tasks) { + Assert.assertEquals("Residuals must be ignored", Expressions.alwaysTrue(), task.residual()); + } + List dataFiles = + Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); + Assert.assertEquals("Should have 2 data files before rewrite", 2, dataFiles.size()); + + Actions actions = Actions.forTable(icebergTableUnPartitioned); + + RewriteDataFilesActionResult result = + actions.rewriteDataFiles().filter(Expressions.equal("data", "0")).execute(); + Assert.assertEquals("Action should rewrite 2 data files", 2, result.deletedDataFiles().size()); + Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFiles().size()); + + // Assert the table records as expected. + SimpleDataUtil.assertTableRecords(icebergTableUnPartitioned, expected); + } + + /** + * a test case to test avoid repeate compress + * + *

    If datafile cannot be combined to CombinedScanTask with other DataFiles, the size of the + * CombinedScanTask list size is 1, so we remove these CombinedScanTasks to avoid compressed + * repeatedly. + * + *

    In this test case,we generated 3 data files and set targetSizeInBytes greater than the + * largest file size so that it cannot be combined a CombinedScanTask with other datafiles. The + * datafile with the largest file size will not be compressed. + * + * @throws IOException IOException + */ + @Test + public void testRewriteAvoidRepeateCompress() throws IOException { + List expected = Lists.newArrayList(); + Schema schema = icebergTableUnPartitioned.schema(); + GenericAppenderFactory genericAppenderFactory = new GenericAppenderFactory(schema); + File file = temp.newFile(); + int count = 0; + try (FileAppender fileAppender = + genericAppenderFactory.newAppender(Files.localOutput(file), format)) { + long filesize = 20000; + for (; fileAppender.length() < filesize; count++) { + Record record = SimpleDataUtil.createRecord(count, UUID.randomUUID().toString()); + fileAppender.add(record); + expected.add(record); + } + } + + DataFile dataFile = + DataFiles.builder(icebergTableUnPartitioned.spec()) + .withPath(file.getAbsolutePath()) + .withFileSizeInBytes(file.length()) + .withFormat(format) + .withRecordCount(count) + .build(); + + icebergTableUnPartitioned.newAppend().appendFile(dataFile).commit(); + + sql("INSERT INTO %s SELECT 1,'a' ", TABLE_NAME_UNPARTITIONED); + sql("INSERT INTO %s SELECT 2,'b' ", TABLE_NAME_UNPARTITIONED); + + icebergTableUnPartitioned.refresh(); + + CloseableIterable tasks = icebergTableUnPartitioned.newScan().planFiles(); + List dataFiles = + Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); + Assert.assertEquals("Should have 3 data files before rewrite", 3, dataFiles.size()); + + Actions actions = Actions.forTable(icebergTableUnPartitioned); + + long targetSizeInBytes = file.length() + 10; + RewriteDataFilesActionResult result = + actions + .rewriteDataFiles() + .targetSizeInBytes(targetSizeInBytes) + .splitOpenFileCost(1) + .execute(); + Assert.assertEquals("Action should rewrite 2 data files", 2, result.deletedDataFiles().size()); + Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFiles().size()); + + icebergTableUnPartitioned.refresh(); + + CloseableIterable tasks1 = icebergTableUnPartitioned.newScan().planFiles(); + List dataFilesRewrote = + Lists.newArrayList(CloseableIterable.transform(tasks1, FileScanTask::file)); + Assert.assertEquals("Should have 2 data files after rewrite", 2, dataFilesRewrote.size()); + + // the biggest file do not be rewrote + List rewroteDataFileNames = + dataFilesRewrote.stream().map(ContentFile::path).collect(Collectors.toList()); + Assert.assertTrue(rewroteDataFileNames.contains(file.getAbsolutePath())); + + // Assert the table records as expected. + expected.add(SimpleDataUtil.createRecord(1, "a")); + expected.add(SimpleDataUtil.createRecord(2, "b")); + SimpleDataUtil.assertTableRecords(icebergTableUnPartitioned, expected); + } + + @Test + public void testRewriteNoConflictWithEqualityDeletes() throws IOException { + // Add 2 data files + sql("INSERT INTO %s SELECT 1, 'hello'", TABLE_NAME_WITH_PK); + sql("INSERT INTO %s SELECT 2, 'world'", TABLE_NAME_WITH_PK); + + // Load 2 stale tables to pass to rewrite actions + // Since the first rewrite will refresh stale1, we need another stale2 for the second rewrite + Table stale1 = + validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME_WITH_PK)); + Table stale2 = + validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME_WITH_PK)); + + // Add 1 data file and 1 equality-delete file + sql("INSERT INTO %s /*+ OPTIONS('upsert-enabled'='true')*/ SELECT 1, 'hi'", TABLE_NAME_WITH_PK); + + icebergTableWithPk.refresh(); + Assert.assertEquals( + "The latest sequence number should be greater than that of the stale snapshot", + stale1.currentSnapshot().sequenceNumber() + 1, + icebergTableWithPk.currentSnapshot().sequenceNumber()); + + CloseableIterable tasks = icebergTableWithPk.newScan().planFiles(); + List dataFiles = + Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); + Set deleteFiles = + Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::deletes)).stream() + .flatMap(Collection::stream) + .collect(Collectors.toSet()); + Assert.assertEquals("Should have 3 data files before rewrite", 3, dataFiles.size()); + Assert.assertEquals("Should have 1 delete file before rewrite", 1, deleteFiles.size()); + Assert.assertSame( + "The 1 delete file should be an equality-delete file", + Iterables.getOnlyElement(deleteFiles).content(), + FileContent.EQUALITY_DELETES); + shouldHaveDataAndFileSequenceNumbers( + TABLE_NAME_WITH_PK, + ImmutableList.of(Pair.of(1L, 1L), Pair.of(2L, 2L), Pair.of(3L, 3L), Pair.of(3L, 3L))); + + Assertions.assertThatThrownBy( + () -> + Actions.forTable(stale1) + .rewriteDataFiles() + .useStartingSequenceNumber(false) + .execute(), + "Rewrite using new sequence number should fail") + .isInstanceOf(ValidationException.class); + + // Rewrite using the starting sequence number should succeed + RewriteDataFilesActionResult result = + Actions.forTable(stale2).rewriteDataFiles().useStartingSequenceNumber(true).execute(); + + // Should not rewrite files from the new commit + Assert.assertEquals("Action should rewrite 2 data files", 2, result.deletedDataFiles().size()); + Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFiles().size()); + // The 2 older files with file-sequence-number <= 2 should be rewritten into a new file. + // The new file is the one with file-sequence-number == 4. + // The new file should use rewrite's starting-sequence-number 2 as its data-sequence-number. + shouldHaveDataAndFileSequenceNumbers( + TABLE_NAME_WITH_PK, ImmutableList.of(Pair.of(3L, 3L), Pair.of(3L, 3L), Pair.of(2L, 4L))); + + // Assert the table records as expected. + SimpleDataUtil.assertTableRecords( + icebergTableWithPk, + Lists.newArrayList( + SimpleDataUtil.createRecord(1, "hi"), SimpleDataUtil.createRecord(2, "world"))); + } + + /** + * Assert that data files and delete files in the table should have expected data sequence numbers + * and file sequence numbers + * + * @param tableName table name + * @param expectedSequenceNumbers list of {@link Pair}'s. Each {@link Pair} contains + * (expectedDataSequenceNumber, expectedFileSequenceNumber) of a file. + */ + private void shouldHaveDataAndFileSequenceNumbers( + String tableName, List> expectedSequenceNumbers) { + // "status < 2" for added or existing entries + List liveEntries = sql("SELECT * FROM %s$entries WHERE status < 2", tableName); + + List> actualSequenceNumbers = + liveEntries.stream() + .map( + row -> + Pair.of( + row.getFieldAs("sequence_number"), row.getFieldAs("file_sequence_number"))) + .collect(Collectors.toList()); + Assertions.assertThat(actualSequenceNumbers).hasSameElementsAs(expectedSequenceNumbers); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java new file mode 100644 index 000000000000..cc58d9817ac6 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java @@ -0,0 +1,38 @@ +/* + * 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.iceberg.flink.data; + +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.RowDataConverter; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; + +public class RandomRowData { + private RandomRowData() {} + + public static Iterable generate(Schema schema, int numRecords, long seed) { + return convert(schema, RandomGenericData.generate(schema, numRecords, seed)); + } + + public static Iterable convert(Schema schema, Iterable records) { + return Iterables.transform(records, record -> RowDataConverter.convert(schema, record)); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java new file mode 100644 index 000000000000..74b1da6007e6 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java @@ -0,0 +1,50 @@ +/* + * 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.iceberg.flink.data; + +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.conversion.DataStructureConverter; +import org.apache.flink.table.data.conversion.DataStructureConverters; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.flink.types.Row; + +public class RowDataToRowMapper extends RichMapFunction { + + private final RowType rowType; + + private transient DataStructureConverter converter; + + public RowDataToRowMapper(RowType rowType) { + this.rowType = rowType; + } + + @Override + public void open(Configuration parameters) throws Exception { + this.converter = + DataStructureConverters.getConverter(TypeConversions.fromLogicalToDataType(rowType)); + } + + @Override + public Row map(RowData value) throws Exception { + return (Row) converter.toExternal(value); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java new file mode 100644 index 000000000000..e8aab824ea2d --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java @@ -0,0 +1,184 @@ +/* + * 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.iceberg.flink.data; + +import java.io.File; +import java.io.IOException; +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Time; +import java.util.Iterator; +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.data.DataTest; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.avro.DataReader; +import org.apache.iceberg.data.avro.DataWriter; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.DateTimeUtil; +import org.junit.Assert; +import org.junit.Test; + +public class TestFlinkAvroReaderWriter extends DataTest { + + private static final int NUM_RECORDS = 100; + + private static final Schema SCHEMA_NUM_TYPE = + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "int", Types.IntegerType.get()), + Types.NestedField.optional(3, "float", Types.FloatType.get()), + Types.NestedField.optional(4, "double", Types.DoubleType.get()), + Types.NestedField.optional(5, "date", Types.DateType.get()), + Types.NestedField.optional(6, "time", Types.TimeType.get()), + Types.NestedField.optional(7, "timestamp", Types.TimestampType.withoutZone()), + Types.NestedField.optional(8, "bigint", Types.LongType.get()), + Types.NestedField.optional(9, "decimal", Types.DecimalType.of(4, 2))); + + @Override + protected void writeAndValidate(Schema schema) throws IOException { + List expectedRecords = RandomGenericData.generate(schema, NUM_RECORDS, 1991L); + writeAndValidate(schema, expectedRecords, NUM_RECORDS); + } + + private void writeAndValidate(Schema schema, List expectedRecords, int numRecord) + throws IOException { + RowType flinkSchema = FlinkSchemaUtil.convert(schema); + List expectedRows = Lists.newArrayList(RandomRowData.convert(schema, expectedRecords)); + + File recordsFile = temp.newFile(); + Assert.assertTrue("Delete should succeed", recordsFile.delete()); + + // Write the expected records into AVRO file, then read them into RowData and assert with the + // expected Record list. + try (FileAppender writer = + Avro.write(Files.localOutput(recordsFile)) + .schema(schema) + .createWriterFunc(DataWriter::create) + .build()) { + writer.addAll(expectedRecords); + } + + try (CloseableIterable reader = + Avro.read(Files.localInput(recordsFile)) + .project(schema) + .createReaderFunc(FlinkAvroReader::new) + .build()) { + Iterator expected = expectedRecords.iterator(); + Iterator rows = reader.iterator(); + for (int i = 0; i < numRecord; i++) { + Assert.assertTrue("Should have expected number of records", rows.hasNext()); + TestHelpers.assertRowData(schema.asStruct(), flinkSchema, expected.next(), rows.next()); + } + Assert.assertFalse("Should not have extra records", rows.hasNext()); + } + + File rowDataFile = temp.newFile(); + Assert.assertTrue("Delete should succeed", rowDataFile.delete()); + + // Write the expected RowData into AVRO file, then read them into Record and assert with the + // expected RowData list. + try (FileAppender writer = + Avro.write(Files.localOutput(rowDataFile)) + .schema(schema) + .createWriterFunc(ignore -> new FlinkAvroWriter(flinkSchema)) + .build()) { + writer.addAll(expectedRows); + } + + try (CloseableIterable reader = + Avro.read(Files.localInput(rowDataFile)) + .project(schema) + .createReaderFunc(DataReader::create) + .build()) { + Iterator expected = expectedRows.iterator(); + Iterator records = reader.iterator(); + for (int i = 0; i < numRecord; i += 1) { + Assert.assertTrue("Should have expected number of records", records.hasNext()); + TestHelpers.assertRowData(schema.asStruct(), flinkSchema, records.next(), expected.next()); + } + Assert.assertFalse("Should not have extra records", records.hasNext()); + } + } + + private Record recordNumType( + int id, + int intV, + float floatV, + double doubleV, + long date, + long time, + long timestamp, + long bigint, + double decimal) { + Record record = GenericRecord.create(SCHEMA_NUM_TYPE); + record.setField("id", id); + record.setField("int", intV); + record.setField("float", floatV); + record.setField("double", doubleV); + record.setField( + "date", DateTimeUtil.dateFromDays((int) new Date(date).toLocalDate().toEpochDay())); + record.setField("time", new Time(time).toLocalTime()); + record.setField("timestamp", DateTimeUtil.timestampFromMicros(timestamp * 1000)); + record.setField("bigint", bigint); + record.setField("decimal", BigDecimal.valueOf(decimal)); + return record; + } + + @Test + public void testNumericTypes() throws IOException { + + List expected = + ImmutableList.of( + recordNumType( + 2, + Integer.MAX_VALUE, + Float.MAX_VALUE, + Double.MAX_VALUE, + Long.MAX_VALUE, + 1643811742000L, + 1643811742000L, + 1643811742000L, + 10.24d), + recordNumType( + 2, + Integer.MIN_VALUE, + Float.MIN_VALUE, + Double.MIN_VALUE, + Long.MIN_VALUE, + 1643811742000L, + 1643811742000L, + 1643811742000L, + 10.24d)); + + writeAndValidate(SCHEMA_NUM_TYPE, expected, 2); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java new file mode 100644 index 000000000000..fdffc0e01c20 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.data; + +import java.io.File; +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.DataTest; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.orc.GenericOrcReader; +import org.apache.iceberg.data.orc.GenericOrcWriter; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.Assert; + +public class TestFlinkOrcReaderWriter extends DataTest { + private static final int NUM_RECORDS = 100; + + @Override + protected void writeAndValidate(Schema schema) throws IOException { + RowType flinkSchema = FlinkSchemaUtil.convert(schema); + List expectedRecords = RandomGenericData.generate(schema, NUM_RECORDS, 1990L); + List expectedRows = Lists.newArrayList(RandomRowData.convert(schema, expectedRecords)); + + File recordsFile = temp.newFile(); + Assert.assertTrue("Delete should succeed", recordsFile.delete()); + + // Write the expected records into ORC file, then read them into RowData and assert with the + // expected Record list. + try (FileAppender writer = + ORC.write(Files.localOutput(recordsFile)) + .schema(schema) + .createWriterFunc(GenericOrcWriter::buildWriter) + .build()) { + writer.addAll(expectedRecords); + } + + try (CloseableIterable reader = + ORC.read(Files.localInput(recordsFile)) + .project(schema) + .createReaderFunc(type -> new FlinkOrcReader(schema, type)) + .build()) { + Iterator expected = expectedRecords.iterator(); + Iterator rows = reader.iterator(); + for (int i = 0; i < NUM_RECORDS; i++) { + Assert.assertTrue("Should have expected number of records", rows.hasNext()); + TestHelpers.assertRowData(schema.asStruct(), flinkSchema, expected.next(), rows.next()); + } + Assert.assertFalse("Should not have extra records", rows.hasNext()); + } + + File rowDataFile = temp.newFile(); + Assert.assertTrue("Delete should succeed", rowDataFile.delete()); + + // Write the expected RowData into ORC file, then read them into Record and assert with the + // expected RowData list. + RowType rowType = FlinkSchemaUtil.convert(schema); + try (FileAppender writer = + ORC.write(Files.localOutput(rowDataFile)) + .schema(schema) + .createWriterFunc((iSchema, typeDesc) -> FlinkOrcWriter.buildWriter(rowType, iSchema)) + .build()) { + writer.addAll(expectedRows); + } + + try (CloseableIterable reader = + ORC.read(Files.localInput(rowDataFile)) + .project(schema) + .createReaderFunc(type -> GenericOrcReader.buildReader(schema, type)) + .build()) { + Iterator expected = expectedRows.iterator(); + Iterator records = reader.iterator(); + for (int i = 0; i < NUM_RECORDS; i += 1) { + Assert.assertTrue("Should have expected number of records", records.hasNext()); + TestHelpers.assertRowData(schema.asStruct(), flinkSchema, records.next(), expected.next()); + } + Assert.assertFalse("Should not have extra records", records.hasNext()); + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java new file mode 100644 index 000000000000..30a2a7bb51ce --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java @@ -0,0 +1,138 @@ +/* + * 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.iceberg.flink.data; + +import static org.apache.iceberg.types.Types.NestedField.optional; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Iterator; +import java.util.List; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.GenericRecordBuilder; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.data.DataTest; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.apache.parquet.avro.AvroParquetWriter; +import org.apache.parquet.hadoop.ParquetWriter; +import org.junit.Assert; +import org.junit.Test; + +public class TestFlinkParquetReader extends DataTest { + private static final int NUM_RECORDS = 100; + + @Test + public void testTwoLevelList() throws IOException { + Schema schema = + new Schema( + optional(1, "arraybytes", Types.ListType.ofRequired(3, Types.BinaryType.get())), + optional(2, "topbytes", Types.BinaryType.get())); + org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(schema.asStruct()); + + File testFile = temp.newFile(); + Assert.assertTrue(testFile.delete()); + + ParquetWriter writer = + AvroParquetWriter.builder(new Path(testFile.toURI())) + .withDataModel(GenericData.get()) + .withSchema(avroSchema) + .config("parquet.avro.add-list-element-records", "true") + .config("parquet.avro.write-old-list-structure", "true") + .build(); + + GenericRecordBuilder recordBuilder = new GenericRecordBuilder(avroSchema); + List expectedByteList = Lists.newArrayList(); + byte[] expectedByte = {0x00, 0x01}; + ByteBuffer expectedBinary = ByteBuffer.wrap(expectedByte); + expectedByteList.add(expectedBinary); + recordBuilder.set("arraybytes", expectedByteList); + recordBuilder.set("topbytes", expectedBinary); + GenericData.Record expectedRecord = recordBuilder.build(); + + writer.write(expectedRecord); + writer.close(); + + try (CloseableIterable reader = + Parquet.read(Files.localInput(testFile)) + .project(schema) + .createReaderFunc(type -> FlinkParquetReaders.buildReader(schema, type)) + .build()) { + Iterator rows = reader.iterator(); + Assert.assertTrue("Should have at least one row", rows.hasNext()); + RowData rowData = rows.next(); + Assert.assertArrayEquals(rowData.getArray(0).getBinary(0), expectedByte); + Assert.assertArrayEquals(rowData.getBinary(1), expectedByte); + Assert.assertFalse("Should not have more than one row", rows.hasNext()); + } + } + + private void writeAndValidate(Iterable iterable, Schema schema) throws IOException { + File testFile = temp.newFile(); + Assert.assertTrue("Delete should succeed", testFile.delete()); + + try (FileAppender writer = + Parquet.write(Files.localOutput(testFile)) + .schema(schema) + .createWriterFunc(GenericParquetWriter::buildWriter) + .build()) { + writer.addAll(iterable); + } + + try (CloseableIterable reader = + Parquet.read(Files.localInput(testFile)) + .project(schema) + .createReaderFunc(type -> FlinkParquetReaders.buildReader(schema, type)) + .build()) { + Iterator expected = iterable.iterator(); + Iterator rows = reader.iterator(); + LogicalType rowType = FlinkSchemaUtil.convert(schema); + for (int i = 0; i < NUM_RECORDS; i += 1) { + Assert.assertTrue("Should have expected number of rows", rows.hasNext()); + TestHelpers.assertRowData(schema.asStruct(), rowType, expected.next(), rows.next()); + } + Assert.assertFalse("Should not have extra rows", rows.hasNext()); + } + } + + @Override + protected void writeAndValidate(Schema schema) throws IOException { + writeAndValidate(RandomGenericData.generate(schema, NUM_RECORDS, 19981), schema); + writeAndValidate( + RandomGenericData.generateDictionaryEncodableRecords(schema, NUM_RECORDS, 21124), schema); + writeAndValidate( + RandomGenericData.generateFallbackRecords(schema, NUM_RECORDS, 21124, NUM_RECORDS / 20), + schema); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java new file mode 100644 index 000000000000..7b868eafc311 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java @@ -0,0 +1,93 @@ +/* + * 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.iceberg.flink.data; + +import java.io.File; +import java.io.IOException; +import java.util.Iterator; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.DataTest; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.parquet.GenericParquetReaders; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.parquet.Parquet; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.rules.TemporaryFolder; + +public class TestFlinkParquetWriter extends DataTest { + private static final int NUM_RECORDS = 100; + + @Rule public TemporaryFolder temp = new TemporaryFolder(); + + private void writeAndValidate(Iterable iterable, Schema schema) throws IOException { + File testFile = temp.newFile(); + Assert.assertTrue("Delete should succeed", testFile.delete()); + + LogicalType logicalType = FlinkSchemaUtil.convert(schema); + + try (FileAppender writer = + Parquet.write(Files.localOutput(testFile)) + .schema(schema) + .createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(logicalType, msgType)) + .build()) { + writer.addAll(iterable); + } + + try (CloseableIterable reader = + Parquet.read(Files.localInput(testFile)) + .project(schema) + .createReaderFunc(msgType -> GenericParquetReaders.buildReader(schema, msgType)) + .build()) { + Iterator expected = iterable.iterator(); + Iterator actual = reader.iterator(); + LogicalType rowType = FlinkSchemaUtil.convert(schema); + for (int i = 0; i < NUM_RECORDS; i += 1) { + Assert.assertTrue("Should have expected number of rows", actual.hasNext()); + TestHelpers.assertRowData(schema.asStruct(), rowType, actual.next(), expected.next()); + } + Assert.assertFalse("Should not have extra rows", actual.hasNext()); + } + } + + @Override + protected void writeAndValidate(Schema schema) throws IOException { + writeAndValidate(RandomRowData.generate(schema, NUM_RECORDS, 19981), schema); + + writeAndValidate( + RandomRowData.convert( + schema, + RandomGenericData.generateDictionaryEncodableRecords(schema, NUM_RECORDS, 21124)), + schema); + + writeAndValidate( + RandomRowData.convert( + schema, + RandomGenericData.generateFallbackRecords( + schema, NUM_RECORDS, 21124, NUM_RECORDS / 20)), + schema); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java new file mode 100644 index 000000000000..3cd25c8fa983 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java @@ -0,0 +1,593 @@ +/* + * 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.iceberg.flink.data; + +import java.util.List; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.DataGenerator; +import org.apache.iceberg.flink.DataGenerators; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.StructProjection; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +public class TestRowDataProjection { + @Test + public void testNullRootRowData() { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + RowDataProjection projection = RowDataProjection.create(schema, schema.select("id")); + + Assertions.assertThatThrownBy(() -> projection.wrap(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid row data: null"); + } + + @Test + public void testFullProjection() { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + generateAndValidate(schema, schema); + + GenericRowData rowData = GenericRowData.of(1L, StringData.fromString("a")); + GenericRowData copyRowData = GenericRowData.of(1L, StringData.fromString("a")); + GenericRowData otherRowData = GenericRowData.of(2L, StringData.fromString("b")); + testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); + } + + @Test + public void testReorderedFullProjection() { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + Schema reordered = + new Schema( + Types.NestedField.optional(1, "data", Types.StringType.get()), + Types.NestedField.required(0, "id", Types.LongType.get())); + + generateAndValidate(schema, reordered); + + GenericRowData rowData = GenericRowData.of(1L, StringData.fromString("a")); + GenericRowData copyRowData = GenericRowData.of(1L, StringData.fromString("a")); + GenericRowData otherRowData = GenericRowData.of(2L, StringData.fromString("b")); + testEqualsAndHashCode(schema, reordered, rowData, copyRowData, otherRowData); + } + + @Test + public void testBasicProjection() { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); + Schema dataOnly = new Schema(Types.NestedField.optional(1, "data", Types.StringType.get())); + generateAndValidate(schema, idOnly); + generateAndValidate(schema, dataOnly); + + GenericRowData rowData = GenericRowData.of(1L, StringData.fromString("a")); + GenericRowData copyRowData = GenericRowData.of(1L, StringData.fromString("a")); + GenericRowData otherRowData = GenericRowData.of(2L, StringData.fromString("b")); + testEqualsAndHashCode(schema, idOnly, rowData, copyRowData, otherRowData); + testEqualsAndHashCode(schema, dataOnly, rowData, copyRowData, otherRowData); + } + + @Test + public void testEmptyProjection() { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + generateAndValidate(schema, schema.select()); + + GenericRowData rowData = GenericRowData.of(1L, StringData.fromString("a")); + GenericRowData copyRowData = GenericRowData.of(1L, StringData.fromString("a")); + GenericRowData otherRowData = GenericRowData.of(2L, StringData.fromString("b")); + testEqualsAndHashCode(schema, schema.select(), rowData, copyRowData, otherRowData, true); + } + + @Test + public void testRename() { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + Schema renamed = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "renamed", Types.StringType.get())); + generateAndValidate(schema, renamed); + + GenericRowData rowData = GenericRowData.of(1L, StringData.fromString("a")); + GenericRowData copyRowData = GenericRowData.of(1L, StringData.fromString("a")); + GenericRowData otherRowData = GenericRowData.of(2L, StringData.fromString("b")); + testEqualsAndHashCode(schema, renamed, rowData, copyRowData, otherRowData); + } + + @Test + public void testNestedProjection() { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional( + 3, + "location", + Types.StructType.of( + Types.NestedField.required(1, "lat", Types.FloatType.get()), + Types.NestedField.required(2, "long", Types.FloatType.get())))); + + GenericRowData rowData = GenericRowData.of(1L, GenericRowData.of(1.0f, 1.0f)); + GenericRowData copyRowData = GenericRowData.of(1L, GenericRowData.of(1.0f, 1.0f)); + GenericRowData otherRowData = GenericRowData.of(2L, GenericRowData.of(2.0f, 2.0f)); + + GenericRowData rowDataNullStruct = GenericRowData.of(1L, null); + GenericRowData copyRowDataNullStruct = GenericRowData.of(1L, null); + GenericRowData otherRowDataNullStruct = GenericRowData.of(2L, null); + + // Project id only. + Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); + Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, idOnly); + testEqualsAndHashCode(schema, idOnly, rowData, copyRowData, otherRowData); + testEqualsAndHashCode( + schema, idOnly, rowDataNullStruct, copyRowDataNullStruct, otherRowDataNullStruct); + + // Project lat only. + Schema latOnly = + new Schema( + Types.NestedField.optional( + 3, + "location", + Types.StructType.of(Types.NestedField.required(1, "lat", Types.FloatType.get())))); + Assertions.assertThat(latOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, latOnly); + testEqualsAndHashCode(schema, latOnly, rowData, copyRowData, otherRowData); + testEqualsAndHashCode( + schema, latOnly, rowDataNullStruct, copyRowDataNullStruct, otherRowDataNullStruct, true); + + // Project long only. + Schema longOnly = + new Schema( + Types.NestedField.optional( + 3, + "location", + Types.StructType.of(Types.NestedField.required(2, "long", Types.FloatType.get())))); + Assertions.assertThat(longOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, longOnly); + testEqualsAndHashCode(schema, longOnly, rowData, copyRowData, otherRowData); + testEqualsAndHashCode( + schema, longOnly, rowDataNullStruct, copyRowDataNullStruct, otherRowDataNullStruct, true); + + // Project location. + Schema locationOnly = schema.select("location"); + Assertions.assertThat(locationOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, locationOnly); + testEqualsAndHashCode(schema, locationOnly, rowData, copyRowData, otherRowData); + testEqualsAndHashCode( + schema, + locationOnly, + rowDataNullStruct, + copyRowDataNullStruct, + otherRowDataNullStruct, + true); + } + + @Test + public void testPrimitivesFullProjection() { + DataGenerator dataGenerator = new DataGenerators.Primitives(); + Schema schema = dataGenerator.icebergSchema(); + generateAndValidate(schema, schema); + + GenericRowData rowData = dataGenerator.generateFlinkRowData(); + GenericRowData copyRowData = dataGenerator.generateFlinkRowData(); + GenericRowData otherRowData = dataGenerator.generateFlinkRowData(); + // modify the string field value (position 6) + otherRowData.setField(6, StringData.fromString("foo_bar")); + testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); + + GenericRowData rowDataNullOptionalFields = dataGenerator.generateFlinkRowData(); + setOptionalFieldsNullForPrimitives(rowDataNullOptionalFields); + GenericRowData copyRowDataNullOptionalFields = dataGenerator.generateFlinkRowData(); + setOptionalFieldsNullForPrimitives(copyRowDataNullOptionalFields); + GenericRowData otherRowDataNullOptionalFields = dataGenerator.generateFlinkRowData(); + // modify the string field value (position 6) + otherRowDataNullOptionalFields.setField(6, StringData.fromString("foo_bar")); + setOptionalFieldsNullForPrimitives(otherRowData); + testEqualsAndHashCode( + schema, + schema, + rowDataNullOptionalFields, + copyRowDataNullOptionalFields, + otherRowDataNullOptionalFields); + } + + private void setOptionalFieldsNullForPrimitives(GenericRowData rowData) { + // fields from [1, 5] range are optional + for (int pos = 1; pos <= 5; ++pos) { + rowData.setField(pos, null); + } + } + + @Test + public void testMapOfPrimitivesProjection() { + DataGenerator dataGenerator = new DataGenerators.MapOfPrimitives(); + Schema schema = dataGenerator.icebergSchema(); + + // Project id only. + Schema idOnly = schema.select("row_id"); + Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, idOnly); + + // Project map only. + Schema mapOnly = schema.select("map_of_primitives"); + Assertions.assertThat(mapOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, mapOnly); + + // Project all. + generateAndValidate(schema, schema); + + GenericRowData rowData = dataGenerator.generateFlinkRowData(); + GenericRowData copyRowData = dataGenerator.generateFlinkRowData(); + // modify the map field value + GenericRowData otherRowData = + GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericMapData( + ImmutableMap.of(StringData.fromString("foo"), 1, StringData.fromString("bar"), 2))); + testEqualsAndHashCode(schema, idOnly, rowData, copyRowData, otherRowData, true); + testEqualsAndHashCode(schema, mapOnly, rowData, copyRowData, otherRowData); + testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); + + GenericRowData rowDataNullOptionalFields = + GenericRowData.of(StringData.fromString("row_id_value"), null); + GenericRowData copyRowDataNullOptionalFields = + GenericRowData.of(StringData.fromString("row_id_value"), null); + // modify the map field value + GenericRowData otherRowDataNullOptionalFields = + GenericRowData.of(StringData.fromString("other_row_id_value"), null); + testEqualsAndHashCode( + schema, + idOnly, + rowDataNullOptionalFields, + copyRowDataNullOptionalFields, + otherRowDataNullOptionalFields); + testEqualsAndHashCode( + schema, + mapOnly, + rowDataNullOptionalFields, + copyRowDataNullOptionalFields, + otherRowDataNullOptionalFields, + true); + testEqualsAndHashCode( + schema, + schema, + rowDataNullOptionalFields, + copyRowDataNullOptionalFields, + otherRowDataNullOptionalFields); + } + + @Test + public void testMapOfStructStructProjection() { + DataGenerator dataGenerator = new DataGenerators.MapOfStructStruct(); + Schema schema = dataGenerator.icebergSchema(); + + // Project id only. + Schema idOnly = schema.select("row_id"); + Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, idOnly); + + // Project map only. + Schema mapOnly = schema.select("map"); + Assertions.assertThat(mapOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, mapOnly); + + // Project all. + generateAndValidate(schema, schema); + + // Project partial map key. + Schema partialMapKey = + new Schema( + Types.NestedField.optional( + 2, + "map", + Types.MapType.ofOptional( + 101, + 102, + Types.StructType.of( + Types.NestedField.required(201, "key", Types.LongType.get())), + Types.StructType.of( + Types.NestedField.required(203, "value", Types.LongType.get()), + Types.NestedField.required(204, "valueData", Types.StringType.get()))))); + Assertions.assertThatThrownBy(() -> generateAndValidate(schema, partialMapKey)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot project a partial map key or value struct."); + + // Project partial map key. + Schema partialMapValue = + new Schema( + Types.NestedField.optional( + 2, + "map", + Types.MapType.ofOptional( + 101, + 102, + Types.StructType.of( + Types.NestedField.required(201, "key", Types.LongType.get()), + Types.NestedField.required(202, "keyData", Types.StringType.get())), + Types.StructType.of( + Types.NestedField.required(203, "value", Types.LongType.get()))))); + Assertions.assertThatThrownBy(() -> generateAndValidate(schema, partialMapValue)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot project a partial map key or value struct."); + + GenericRowData rowData = dataGenerator.generateFlinkRowData(); + GenericRowData copyRowData = dataGenerator.generateFlinkRowData(); + // modify the map field value + GenericRowData otherRowData = + GenericRowData.of( + StringData.fromString("other_row_id_value"), + new GenericMapData( + ImmutableMap.of( + GenericRowData.of(1L, StringData.fromString("other_key_data")), + GenericRowData.of(1L, StringData.fromString("other_value_data"))))); + testEqualsAndHashCode(schema, idOnly, rowData, copyRowData, otherRowData); + testEqualsAndHashCode(schema, mapOnly, rowData, copyRowData, otherRowData); + testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); + + GenericRowData rowDataNullOptionalFields = + GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericMapData( + ImmutableMap.of(GenericRowData.of(1L, null), GenericRowData.of(1L, null)))); + GenericRowData copyRowDataNullOptionalFields = + GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericMapData( + ImmutableMap.of(GenericRowData.of(1L, null), GenericRowData.of(1L, null)))); + // modify the map field value + GenericRowData otherRowDataNullOptionalFields = + GenericRowData.of( + StringData.fromString("other_row_id_value"), + new GenericMapData( + ImmutableMap.of(GenericRowData.of(2L, null), GenericRowData.of(2L, null)))); + testEqualsAndHashCode( + schema, + idOnly, + rowDataNullOptionalFields, + copyRowDataNullOptionalFields, + otherRowDataNullOptionalFields); + testEqualsAndHashCode( + schema, + mapOnly, + rowDataNullOptionalFields, + copyRowDataNullOptionalFields, + otherRowDataNullOptionalFields); + testEqualsAndHashCode( + schema, + schema, + rowDataNullOptionalFields, + copyRowDataNullOptionalFields, + otherRowDataNullOptionalFields); + } + + @Test + public void testArrayOfPrimitiveProjection() { + DataGenerator dataGenerator = new DataGenerators.ArrayOfPrimitive(); + Schema schema = dataGenerator.icebergSchema(); + + // Project id only. + Schema idOnly = schema.select("row_id"); + Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, idOnly); + + // Project list only. + Schema arrayOnly = schema.select("array_of_int"); + Assertions.assertThat(arrayOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, arrayOnly); + + // Project all. + generateAndValidate(schema, schema); + + GenericRowData rowData = dataGenerator.generateFlinkRowData(); + GenericRowData copyRowData = dataGenerator.generateFlinkRowData(); + // modify the map field value + GenericRowData otherRowData = + GenericRowData.of( + StringData.fromString("other_row_id_value"), + new GenericArrayData(new Integer[] {4, 5, 6})); + testEqualsAndHashCode(schema, idOnly, rowData, copyRowData, otherRowData); + testEqualsAndHashCode(schema, arrayOnly, rowData, copyRowData, otherRowData); + testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); + + GenericRowData rowDataNullOptionalFields = + GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericArrayData(new Integer[] {1, null, 3})); + GenericRowData copyRowDataNullOptionalFields = + GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericArrayData(new Integer[] {1, null, 3})); + // modify the map field value + GenericRowData otherRowDataNullOptionalFields = + GenericRowData.of( + StringData.fromString("other_row_id_value"), + new GenericArrayData(new Integer[] {4, null, 6})); + testEqualsAndHashCode( + schema, + idOnly, + rowDataNullOptionalFields, + copyRowDataNullOptionalFields, + otherRowDataNullOptionalFields); + testEqualsAndHashCode( + schema, + arrayOnly, + rowDataNullOptionalFields, + copyRowDataNullOptionalFields, + otherRowDataNullOptionalFields); + testEqualsAndHashCode( + schema, + schema, + rowDataNullOptionalFields, + copyRowDataNullOptionalFields, + otherRowDataNullOptionalFields); + } + + @Test + public void testArrayOfStructProjection() { + DataGenerator dataGenerator = new DataGenerators.ArrayOfStruct(); + Schema schema = dataGenerator.icebergSchema(); + + // Project id only. + Schema idOnly = schema.select("row_id"); + Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, idOnly); + + // Project list only. + Schema arrayOnly = schema.select("array_of_struct"); + Assertions.assertThat(arrayOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, arrayOnly); + + // Project all. + generateAndValidate(schema, schema); + + // Project partial list value. + Schema partialList = + new Schema( + Types.NestedField.optional( + 2, + "array_of_struct", + Types.ListType.ofOptional( + 101, + Types.StructType.of( + Types.NestedField.required(202, "name", Types.StringType.get()))))); + + Assertions.assertThatThrownBy(() -> generateAndValidate(schema, partialList)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot project a partial list element struct."); + + GenericRowData rowData = dataGenerator.generateFlinkRowData(); + GenericRowData copyRowData = dataGenerator.generateFlinkRowData(); + // modify the map field value + GenericRowData otherRowData = + GenericRowData.of( + StringData.fromString("row_id_value"), new GenericArrayData(new Integer[] {4, 5, 6})); + testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); + + GenericRowData rowDataNullOptionalFields = + GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericArrayData(new Integer[] {1, null, 3})); + GenericRowData copyRowDataNullOptionalFields = + GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericArrayData(new Integer[] {1, null, 3})); + // modify the map field value + GenericRowData otherRowDataNullOptionalFields = + GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericArrayData(new Integer[] {4, null, 6})); + testEqualsAndHashCode( + schema, + schema, + rowDataNullOptionalFields, + copyRowDataNullOptionalFields, + otherRowDataNullOptionalFields); + } + + private void generateAndValidate(Schema schema, Schema projectSchema) { + int numRecords = 100; + List recordList = RandomGenericData.generate(schema, numRecords, 102L); + List rowDataList = + Lists.newArrayList(RandomRowData.generate(schema, numRecords, 102L).iterator()); + Assertions.assertThat(rowDataList).hasSize(recordList.size()); + + StructProjection structProjection = StructProjection.create(schema, projectSchema); + RowDataProjection rowDataProjection = RowDataProjection.create(schema, projectSchema); + + for (int i = 0; i < numRecords; i++) { + StructLike expected = structProjection.wrap(recordList.get(i)); + RowData projected = rowDataProjection.wrap(rowDataList.get(i)); + TestHelpers.assertRowData(projectSchema, expected, projected); + + Assertions.assertThat(projected).isEqualTo(projected); + Assertions.assertThat(projected).hasSameHashCodeAs(projected); + // make sure toString doesn't throw NPE for null values + Assertions.assertThatNoException().isThrownBy(projected::toString); + } + } + + private void testEqualsAndHashCode( + Schema schema, + Schema projectionSchema, + RowData rowData, + RowData copyRowData, + RowData otherRowData) { + testEqualsAndHashCode(schema, projectionSchema, rowData, copyRowData, otherRowData, false); + } + + /** + * @param isOtherRowDataSameAsRowData sometimes projection on otherRowData can result in the same + * RowData, e.g. due to empty projection or null struct + */ + private void testEqualsAndHashCode( + Schema schema, + Schema projectionSchema, + RowData rowData, + RowData copyRowData, + RowData otherRowData, + boolean isOtherRowDataSameAsRowData) { + RowDataProjection projection = RowDataProjection.create(schema, projectionSchema); + RowDataProjection copyProjection = RowDataProjection.create(schema, projectionSchema); + RowDataProjection otherProjection = RowDataProjection.create(schema, projectionSchema); + + Assertions.assertThat(projection.wrap(rowData)).isEqualTo(copyProjection.wrap(copyRowData)); + Assertions.assertThat(projection.wrap(rowData)) + .hasSameHashCodeAs(copyProjection.wrap(copyRowData)); + + if (isOtherRowDataSameAsRowData) { + Assertions.assertThat(projection.wrap(rowData)).isEqualTo(otherProjection.wrap(otherRowData)); + Assertions.assertThat(projection.wrap(rowData)) + .hasSameHashCodeAs(otherProjection.wrap(otherRowData)); + } else { + Assertions.assertThat(projection.wrap(rowData)) + .isNotEqualTo(otherProjection.wrap(otherRowData)); + Assertions.assertThat(projection.wrap(rowData)) + .doesNotHaveSameHashCodeAs(otherProjection.wrap(otherRowData)); + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java new file mode 100644 index 000000000000..df2e6ae21c7e --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java @@ -0,0 +1,580 @@ +/* + * 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.iceberg.flink.data; + +import java.io.File; +import java.io.IOException; +import java.util.Map; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Comparators; +import org.apache.iceberg.types.Types; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestRowProjection { + + @Rule public TemporaryFolder temp = new TemporaryFolder(); + + private RowData writeAndRead(String desc, Schema writeSchema, Schema readSchema, RowData row) + throws IOException { + File file = temp.newFile(desc + ".avro"); + Assert.assertTrue(file.delete()); + + try (FileAppender appender = + Avro.write(Files.localOutput(file)) + .schema(writeSchema) + .createWriterFunc(ignore -> new FlinkAvroWriter(FlinkSchemaUtil.convert(writeSchema))) + .build()) { + appender.add(row); + } + + Iterable records = + Avro.read(Files.localInput(file)) + .project(readSchema) + .createReaderFunc(FlinkAvroReader::new) + .build(); + + return Iterables.getOnlyElement(records); + } + + @Test + public void testFullProjection() throws Exception { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + RowData row = GenericRowData.of(34L, StringData.fromString("test")); + + RowData projected = writeAndRead("full_projection", schema, schema, row); + + Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); + + int cmp = Comparators.charSequences().compare("test", projected.getString(1).toString()); + Assert.assertEquals("Should contain the correct data value", cmp, 0); + } + + @Test + public void testSpecialCharacterProjection() throws Exception { + Schema schema = + new Schema( + Types.NestedField.required(0, "user id", Types.LongType.get()), + Types.NestedField.optional(1, "data%0", Types.StringType.get())); + + RowData row = GenericRowData.of(34L, StringData.fromString("test")); + + RowData full = writeAndRead("special_chars", schema, schema, row); + + Assert.assertEquals("Should contain the correct id value", 34L, full.getLong(0)); + Assert.assertEquals( + "Should contain the correct data value", + 0, + Comparators.charSequences().compare("test", full.getString(1).toString())); + + RowData projected = writeAndRead("special_characters", schema, schema.select("data%0"), full); + + Assert.assertEquals("Should not contain id value", 1, projected.getArity()); + Assert.assertEquals( + "Should contain the correct data value", + 0, + Comparators.charSequences().compare("test", projected.getString(0).toString())); + } + + @Test + public void testReorderedFullProjection() throws Exception { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + RowData row = GenericRowData.of(34L, StringData.fromString("test")); + + Schema reordered = + new Schema( + Types.NestedField.optional(1, "data", Types.StringType.get()), + Types.NestedField.required(0, "id", Types.LongType.get())); + + RowData projected = writeAndRead("full_projection", schema, reordered, row); + + Assert.assertEquals( + "Should contain the correct 0 value", "test", projected.getString(0).toString()); + Assert.assertEquals("Should contain the correct 1 value", 34L, projected.getLong(1)); + } + + @Test + public void testReorderedProjection() throws Exception { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + RowData row = GenericRowData.of(34L, StringData.fromString("test")); + + Schema reordered = + new Schema( + Types.NestedField.optional(2, "missing_1", Types.StringType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get()), + Types.NestedField.optional(3, "missing_2", Types.LongType.get())); + + RowData projected = writeAndRead("full_projection", schema, reordered, row); + + Assert.assertTrue("Should contain the correct 0 value", projected.isNullAt(0)); + Assert.assertEquals( + "Should contain the correct 1 value", "test", projected.getString(1).toString()); + Assert.assertTrue("Should contain the correct 2 value", projected.isNullAt(2)); + } + + @Test + public void testRenamedAddedField() throws Exception { + Schema schema = + new Schema( + Types.NestedField.required(1, "a", Types.LongType.get()), + Types.NestedField.required(2, "b", Types.LongType.get()), + Types.NestedField.required(3, "d", Types.LongType.get())); + + RowData row = GenericRowData.of(100L, 200L, 300L); + + Schema renamedAdded = + new Schema( + Types.NestedField.optional(1, "a", Types.LongType.get()), + Types.NestedField.optional(2, "b", Types.LongType.get()), + Types.NestedField.optional(3, "c", Types.LongType.get()), + Types.NestedField.optional(4, "d", Types.LongType.get())); + + RowData projected = writeAndRead("rename_and_add_column_projection", schema, renamedAdded, row); + Assert.assertEquals("Should contain the correct value in column 1", projected.getLong(0), 100L); + Assert.assertEquals("Should contain the correct value in column 2", projected.getLong(1), 200L); + Assert.assertEquals("Should contain the correct value in column 3", projected.getLong(2), 300L); + Assert.assertTrue("Should contain empty value on new column 4", projected.isNullAt(3)); + } + + @Test + public void testEmptyProjection() throws Exception { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + RowData row = GenericRowData.of(34L, StringData.fromString("test")); + + RowData projected = writeAndRead("empty_projection", schema, schema.select(), row); + + Assert.assertNotNull("Should read a non-null record", projected); + Assert.assertEquals(0, projected.getArity()); + } + + @Test + public void testBasicProjection() throws Exception { + Schema writeSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + RowData row = GenericRowData.of(34L, StringData.fromString("test")); + + Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); + + RowData projected = writeAndRead("basic_projection_id", writeSchema, idOnly, row); + Assert.assertEquals("Should not project data", 1, projected.getArity()); + Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); + + Schema dataOnly = new Schema(Types.NestedField.optional(1, "data", Types.StringType.get())); + + projected = writeAndRead("basic_projection_data", writeSchema, dataOnly, row); + + Assert.assertEquals("Should not project id", 1, projected.getArity()); + int cmp = Comparators.charSequences().compare("test", projected.getString(0).toString()); + Assert.assertEquals("Should contain the correct data value", 0, cmp); + } + + @Test + public void testRename() throws Exception { + Schema writeSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + RowData row = GenericRowData.of(34L, StringData.fromString("test")); + + Schema readSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "renamed", Types.StringType.get())); + + RowData projected = writeAndRead("project_and_rename", writeSchema, readSchema, row); + + Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); + int cmp = Comparators.charSequences().compare("test", projected.getString(1).toString()); + Assert.assertEquals("Should contain the correct data/renamed value", 0, cmp); + } + + @Test + public void testNestedStructProjection() throws Exception { + Schema writeSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional( + 3, + "location", + Types.StructType.of( + Types.NestedField.required(1, "lat", Types.FloatType.get()), + Types.NestedField.required(2, "long", Types.FloatType.get())))); + + RowData location = GenericRowData.of(52.995143f, -1.539054f); + RowData record = GenericRowData.of(34L, location); + + Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); + + RowData projected = writeAndRead("id_only", writeSchema, idOnly, record); + Assert.assertEquals("Should not project location", 1, projected.getArity()); + Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); + + Schema latOnly = + new Schema( + Types.NestedField.optional( + 3, + "location", + Types.StructType.of(Types.NestedField.required(1, "lat", Types.FloatType.get())))); + + projected = writeAndRead("latitude_only", writeSchema, latOnly, record); + RowData projectedLocation = projected.getRow(0, 1); + Assert.assertEquals("Should not project id", 1, projected.getArity()); + Assert.assertFalse("Should project location", projected.isNullAt(0)); + Assert.assertEquals("Should not project longitude", 1, projectedLocation.getArity()); + Assert.assertEquals( + "Should project latitude", 52.995143f, projectedLocation.getFloat(0), 0.000001f); + + Schema longOnly = + new Schema( + Types.NestedField.optional( + 3, + "location", + Types.StructType.of(Types.NestedField.required(2, "long", Types.FloatType.get())))); + + projected = writeAndRead("longitude_only", writeSchema, longOnly, record); + projectedLocation = projected.getRow(0, 1); + Assert.assertEquals("Should not project id", 1, projected.getArity()); + Assert.assertFalse("Should project location", projected.isNullAt(0)); + Assert.assertEquals("Should not project latitutde", 1, projectedLocation.getArity()); + Assert.assertEquals( + "Should project longitude", -1.539054f, projectedLocation.getFloat(0), 0.000001f); + + Schema locationOnly = writeSchema.select("location"); + projected = writeAndRead("location_only", writeSchema, locationOnly, record); + projectedLocation = projected.getRow(0, 1); + Assert.assertEquals("Should not project id", 1, projected.getArity()); + Assert.assertFalse("Should project location", projected.isNullAt(0)); + Assert.assertEquals( + "Should project latitude", 52.995143f, projectedLocation.getFloat(0), 0.000001f); + Assert.assertEquals( + "Should project longitude", -1.539054f, projectedLocation.getFloat(1), 0.000001f); + } + + @Test + public void testMapProjection() throws IOException { + Schema writeSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional( + 5, + "properties", + Types.MapType.ofOptional(6, 7, Types.StringType.get(), Types.StringType.get()))); + + GenericMapData properties = + new GenericMapData( + ImmutableMap.of( + StringData.fromString("a"), + StringData.fromString("A"), + StringData.fromString("b"), + StringData.fromString("B"))); + + RowData row = GenericRowData.of(34L, properties); + + Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); + + RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); + Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); + Assert.assertEquals("Should not project properties map", 1, projected.getArity()); + + Schema keyOnly = writeSchema.select("properties.key"); + projected = writeAndRead("key_only", writeSchema, keyOnly, row); + Assert.assertEquals("Should not project id", 1, projected.getArity()); + Assert.assertEquals("Should project entire map", properties, projected.getMap(0)); + + Schema valueOnly = writeSchema.select("properties.value"); + projected = writeAndRead("value_only", writeSchema, valueOnly, row); + Assert.assertEquals("Should not project id", 1, projected.getArity()); + Assert.assertEquals("Should project entire map", properties, projected.getMap(0)); + + Schema mapOnly = writeSchema.select("properties"); + projected = writeAndRead("map_only", writeSchema, mapOnly, row); + Assert.assertEquals("Should not project id", 1, projected.getArity()); + Assert.assertEquals("Should project entire map", properties, projected.getMap(0)); + } + + private Map toStringMap(Map map) { + Map stringMap = Maps.newHashMap(); + for (Map.Entry entry : map.entrySet()) { + if (entry.getValue() instanceof CharSequence) { + stringMap.put(entry.getKey().toString(), entry.getValue().toString()); + } else { + stringMap.put(entry.getKey().toString(), entry.getValue()); + } + } + return stringMap; + } + + @Test + public void testMapOfStructsProjection() throws IOException { + Schema writeSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional( + 5, + "locations", + Types.MapType.ofOptional( + 6, + 7, + Types.StringType.get(), + Types.StructType.of( + Types.NestedField.required(1, "lat", Types.FloatType.get()), + Types.NestedField.required(2, "long", Types.FloatType.get()))))); + + RowData l1 = GenericRowData.of(53.992811f, -1.542616f); + RowData l2 = GenericRowData.of(52.995143f, -1.539054f); + GenericMapData map = + new GenericMapData( + ImmutableMap.of(StringData.fromString("L1"), l1, StringData.fromString("L2"), l2)); + RowData row = GenericRowData.of(34L, map); + + Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); + + RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); + Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); + Assert.assertEquals("Should not project locations map", 1, projected.getArity()); + + projected = writeAndRead("all_locations", writeSchema, writeSchema.select("locations"), row); + Assert.assertEquals("Should not project id", 1, projected.getArity()); + Assert.assertEquals("Should project locations map", row.getMap(1), projected.getMap(0)); + + projected = writeAndRead("lat_only", writeSchema, writeSchema.select("locations.lat"), row); + GenericMapData locations = (GenericMapData) projected.getMap(0); + Assert.assertNotNull("Should project locations map", locations); + GenericArrayData l1l2Array = + new GenericArrayData( + new Object[] {StringData.fromString("L2"), StringData.fromString("L1")}); + Assert.assertEquals("Should contain L1 and L2", l1l2Array, locations.keyArray()); + RowData projectedL1 = (RowData) locations.get(StringData.fromString("L1")); + Assert.assertNotNull("L1 should not be null", projectedL1); + Assert.assertEquals("L1 should contain lat", 53.992811f, projectedL1.getFloat(0), 0.000001); + Assert.assertEquals("L1 should not contain long", 1, projectedL1.getArity()); + RowData projectedL2 = (RowData) locations.get(StringData.fromString("L2")); + Assert.assertNotNull("L2 should not be null", projectedL2); + Assert.assertEquals("L2 should contain lat", 52.995143f, projectedL2.getFloat(0), 0.000001); + Assert.assertEquals("L2 should not contain long", 1, projectedL2.getArity()); + + projected = writeAndRead("long_only", writeSchema, writeSchema.select("locations.long"), row); + Assert.assertEquals("Should not project id", 1, projected.getArity()); + locations = (GenericMapData) projected.getMap(0); + Assert.assertNotNull("Should project locations map", locations); + Assert.assertEquals("Should contain L1 and L2", l1l2Array, locations.keyArray()); + projectedL1 = (RowData) locations.get(StringData.fromString("L1")); + Assert.assertNotNull("L1 should not be null", projectedL1); + Assert.assertEquals("L1 should not contain lat", 1, projectedL1.getArity()); + Assert.assertEquals("L1 should contain long", -1.542616f, projectedL1.getFloat(0), 0.000001); + projectedL2 = (RowData) locations.get(StringData.fromString("L2")); + Assert.assertNotNull("L2 should not be null", projectedL2); + Assert.assertEquals("L2 should not contain lat", 1, projectedL2.getArity()); + Assert.assertEquals("L2 should contain long", -1.539054f, projectedL2.getFloat(0), 0.000001); + + Schema latitiudeRenamed = + new Schema( + Types.NestedField.optional( + 5, + "locations", + Types.MapType.ofOptional( + 6, + 7, + Types.StringType.get(), + Types.StructType.of( + Types.NestedField.required(1, "latitude", Types.FloatType.get()))))); + + projected = writeAndRead("latitude_renamed", writeSchema, latitiudeRenamed, row); + Assert.assertEquals("Should not project id", 1, projected.getArity()); + locations = (GenericMapData) projected.getMap(0); + Assert.assertNotNull("Should project locations map", locations); + Assert.assertEquals("Should contain L1 and L2", l1l2Array, locations.keyArray()); + projectedL1 = (RowData) locations.get(StringData.fromString("L1")); + Assert.assertNotNull("L1 should not be null", projectedL1); + Assert.assertEquals( + "L1 should contain latitude", 53.992811f, projectedL1.getFloat(0), 0.000001); + projectedL2 = (RowData) locations.get(StringData.fromString("L2")); + Assert.assertNotNull("L2 should not be null", projectedL2); + Assert.assertEquals( + "L2 should contain latitude", 52.995143f, projectedL2.getFloat(0), 0.000001); + } + + @Test + public void testListProjection() throws IOException { + Schema writeSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional( + 10, "values", Types.ListType.ofOptional(11, Types.LongType.get()))); + + GenericArrayData values = new GenericArrayData(new Long[] {56L, 57L, 58L}); + + RowData row = GenericRowData.of(34L, values); + + Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); + + RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); + Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); + Assert.assertEquals("Should not project values list", 1, projected.getArity()); + + Schema elementOnly = writeSchema.select("values.element"); + projected = writeAndRead("element_only", writeSchema, elementOnly, row); + Assert.assertEquals("Should not project id", 1, projected.getArity()); + Assert.assertEquals("Should project entire list", values, projected.getArray(0)); + + Schema listOnly = writeSchema.select("values"); + projected = writeAndRead("list_only", writeSchema, listOnly, row); + Assert.assertEquals("Should not project id", 1, projected.getArity()); + Assert.assertEquals("Should project entire list", values, projected.getArray(0)); + } + + @Test + @SuppressWarnings("unchecked") + public void testListOfStructsProjection() throws IOException { + Schema writeSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional( + 22, + "points", + Types.ListType.ofOptional( + 21, + Types.StructType.of( + Types.NestedField.required(19, "x", Types.IntegerType.get()), + Types.NestedField.optional(18, "y", Types.IntegerType.get()))))); + + RowData p1 = GenericRowData.of(1, 2); + RowData p2 = GenericRowData.of(3, null); + GenericArrayData arrayData = new GenericArrayData(new RowData[] {p1, p2}); + RowData row = GenericRowData.of(34L, arrayData); + + Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); + + RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); + Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); + Assert.assertEquals("Should not project points list", 1, projected.getArity()); + + projected = writeAndRead("all_points", writeSchema, writeSchema.select("points"), row); + Assert.assertEquals("Should not project id", 1, projected.getArity()); + Assert.assertEquals("Should project points list", row.getArray(1), projected.getArray(0)); + + projected = writeAndRead("x_only", writeSchema, writeSchema.select("points.x"), row); + Assert.assertEquals("Should not project id", 1, projected.getArity()); + Assert.assertFalse("Should project points list", projected.isNullAt(0)); + ArrayData points = projected.getArray(0); + Assert.assertEquals("Should read 2 points", 2, points.size()); + RowData projectedP1 = points.getRow(0, 2); + Assert.assertEquals("Should project x", 1, projectedP1.getInt(0)); + Assert.assertEquals("Should not project y", 1, projectedP1.getArity()); + RowData projectedP2 = points.getRow(1, 2); + Assert.assertEquals("Should not project y", 1, projectedP2.getArity()); + Assert.assertEquals("Should project x", 3, projectedP2.getInt(0)); + + projected = writeAndRead("y_only", writeSchema, writeSchema.select("points.y"), row); + Assert.assertEquals("Should not project id", 1, projected.getArity()); + Assert.assertFalse("Should project points list", projected.isNullAt(0)); + points = projected.getArray(0); + Assert.assertEquals("Should read 2 points", 2, points.size()); + projectedP1 = points.getRow(0, 2); + Assert.assertEquals("Should not project x", 1, projectedP1.getArity()); + Assert.assertEquals("Should project y", 2, projectedP1.getInt(0)); + projectedP2 = points.getRow(1, 2); + Assert.assertEquals("Should not project x", 1, projectedP2.getArity()); + Assert.assertTrue("Should project null y", projectedP2.isNullAt(0)); + + Schema yRenamed = + new Schema( + Types.NestedField.optional( + 22, + "points", + Types.ListType.ofOptional( + 21, + Types.StructType.of( + Types.NestedField.optional(18, "z", Types.IntegerType.get()))))); + + projected = writeAndRead("y_renamed", writeSchema, yRenamed, row); + Assert.assertEquals("Should not project id", 1, projected.getArity()); + Assert.assertFalse("Should project points list", projected.isNullAt(0)); + points = projected.getArray(0); + Assert.assertEquals("Should read 2 points", 2, points.size()); + projectedP1 = points.getRow(0, 2); + Assert.assertEquals("Should not project x and y", 1, projectedP1.getArity()); + Assert.assertEquals("Should project z", 2, projectedP1.getInt(0)); + projectedP2 = points.getRow(1, 2); + Assert.assertEquals("Should not project x and y", 1, projectedP2.getArity()); + Assert.assertTrue("Should project null z", projectedP2.isNullAt(0)); + } + + @Test + public void testAddedFieldsWithRequiredChildren() throws Exception { + Schema schema = new Schema(Types.NestedField.required(1, "a", Types.LongType.get())); + + RowData row = GenericRowData.of(100L); + + Schema addedFields = + new Schema( + Types.NestedField.optional(1, "a", Types.LongType.get()), + Types.NestedField.optional( + 2, + "b", + Types.StructType.of(Types.NestedField.required(3, "c", Types.LongType.get()))), + Types.NestedField.optional(4, "d", Types.ListType.ofRequired(5, Types.LongType.get())), + Types.NestedField.optional( + 6, + "e", + Types.MapType.ofRequired(7, 8, Types.LongType.get(), Types.LongType.get()))); + + RowData projected = + writeAndRead("add_fields_with_required_children_projection", schema, addedFields, row); + Assert.assertEquals("Should contain the correct value in column 1", projected.getLong(0), 100L); + Assert.assertTrue("Should contain empty value in new column 2", projected.isNullAt(1)); + Assert.assertTrue("Should contain empty value in new column 4", projected.isNullAt(2)); + Assert.assertTrue("Should contain empty value in new column 6", projected.isNullAt(3)); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java new file mode 100644 index 000000000000..e0340e0743b0 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.data; + +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.flink.DataGenerator; +import org.apache.iceberg.flink.DataGenerators; +import org.apache.iceberg.flink.TestHelpers; +import org.junit.Test; + +public class TestStructRowData { + + protected void testConverter(DataGenerator dataGenerator) { + StructRowData converter = new StructRowData(dataGenerator.icebergSchema().asStruct()); + GenericRecord expected = dataGenerator.generateIcebergGenericRecord(); + StructRowData actual = converter.setStruct(expected); + TestHelpers.assertRowData(dataGenerator.icebergSchema(), expected, actual); + } + + @Test + public void testPrimitiveTypes() { + testConverter(new DataGenerators.Primitives()); + } + + @Test + public void testStructOfPrimitive() { + testConverter(new DataGenerators.StructOfPrimitive()); + } + + @Test + public void testStructOfArray() { + testConverter(new DataGenerators.StructOfArray()); + } + + @Test + public void testStructOfMap() { + testConverter(new DataGenerators.StructOfMap()); + } + + @Test + public void testStructOfStruct() { + testConverter(new DataGenerators.StructOfStruct()); + } + + @Test + public void testArrayOfPrimitive() { + testConverter(new DataGenerators.ArrayOfPrimitive()); + } + + @Test + public void testArrayOfArray() { + testConverter(new DataGenerators.ArrayOfArray()); + } + + @Test + public void testArrayOfMap() { + testConverter(new DataGenerators.ArrayOfMap()); + } + + @Test + public void testArrayOfStruct() { + testConverter(new DataGenerators.ArrayOfStruct()); + } + + @Test + public void testMapOfPrimitives() { + testConverter(new DataGenerators.MapOfPrimitives()); + } + + @Test + public void testMapOfArray() { + testConverter(new DataGenerators.MapOfArray()); + } + + @Test + public void testMapOfMap() { + testConverter(new DataGenerators.MapOfMap()); + } + + @Test + public void testMapOfStruct() { + testConverter(new DataGenerators.MapOfStruct()); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java new file mode 100644 index 000000000000..6a493692c20d --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.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.iceberg.flink.sink; + +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.flink.AvroGenericRecordConverterBase; +import org.apache.iceberg.flink.DataGenerator; +import org.junit.Assert; + +public class TestAvroGenericRecordToRowDataMapper extends AvroGenericRecordConverterBase { + @Override + protected void testConverter(DataGenerator dataGenerator) throws Exception { + // Need to use avroSchema from DataGenerator because some primitive types have special Avro + // type handling. Hence the Avro schema converted from Iceberg schema won't work. + AvroGenericRecordToRowDataMapper mapper = + AvroGenericRecordToRowDataMapper.forAvroSchema(dataGenerator.avroSchema()); + RowData expected = dataGenerator.generateFlinkRowData(); + RowData actual = mapper.map(dataGenerator.generateAvroGenericRecord()); + Assert.assertEquals(expected, actual); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java new file mode 100644 index 000000000000..5ebcc6361c7b --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java @@ -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. + */ +package org.apache.iceberg.flink.sink; + +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.sink.TestBucketPartitionerUtil.TableSchemaType; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; + +public class TestBucketPartitionKeySelector { + + @ParameterizedTest + @EnumSource( + value = TableSchemaType.class, + names = {"ONE_BUCKET", "IDENTITY_AND_BUCKET"}) + public void testCorrectKeySelection(TableSchemaType tableSchemaType) { + int numBuckets = 60; + + PartitionSpec partitionSpec = tableSchemaType.getPartitionSpec(numBuckets); + BucketPartitionKeySelector keySelector = + new BucketPartitionKeySelector( + partitionSpec, SimpleDataUtil.SCHEMA, SimpleDataUtil.ROW_TYPE); + + TestBucketPartitionerUtil.generateRowsForBucketIdRange(2, numBuckets) + .forEach( + rowData -> { + int expectedBucketId = + TestBucketPartitionerUtil.computeBucketId( + numBuckets, rowData.getString(1).toString()); + Integer key = keySelector.getKey(rowData); + Assertions.assertThat(key).isEqualTo(expectedBucketId); + }); + } + + @Test + public void testKeySelectorMultipleBucketsFail() { + PartitionSpec partitionSpec = TableSchemaType.TWO_BUCKETS.getPartitionSpec(1); + + Assertions.assertThatExceptionOfType(RuntimeException.class) + .isThrownBy( + () -> + new BucketPartitionKeySelector( + partitionSpec, SimpleDataUtil.SCHEMA, SimpleDataUtil.ROW_TYPE)) + .withMessage(BucketPartitionerUtil.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE, 2); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java new file mode 100644 index 000000000000..835713e6b417 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java @@ -0,0 +1,107 @@ +/* + * 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.iceberg.flink.sink; + +import static org.apache.iceberg.flink.sink.BucketPartitioner.BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE; +import static org.apache.iceberg.flink.sink.BucketPartitioner.BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE; +import static org.apache.iceberg.flink.sink.BucketPartitioner.BUCKET_NULL_MESSAGE; + +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.flink.sink.TestBucketPartitionerUtil.TableSchemaType; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; + +public class TestBucketPartitioner { + + static final int DEFAULT_NUM_BUCKETS = 60; + + @ParameterizedTest + @CsvSource({"ONE_BUCKET,50", "IDENTITY_AND_BUCKET,50", "ONE_BUCKET,60", "IDENTITY_AND_BUCKET,60"}) + public void testPartitioningParallelismGreaterThanBuckets( + String schemaTypeStr, String numBucketsStr) { + int numPartitions = 500; + TableSchemaType tableSchemaType = TableSchemaType.valueOf(schemaTypeStr); + int numBuckets = Integer.parseInt(numBucketsStr); + PartitionSpec partitionSpec = tableSchemaType.getPartitionSpec(numBuckets); + BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); + + int bucketId = 0; + for (int expectedIndex = 0; expectedIndex < numPartitions; expectedIndex++) { + int actualPartitionIndex = bucketPartitioner.partition(bucketId, numPartitions); + Assertions.assertThat(actualPartitionIndex).isEqualTo(expectedIndex); + bucketId++; + if (bucketId == numBuckets) { + bucketId = 0; + } + } + } + + @ParameterizedTest + @CsvSource({"ONE_BUCKET,50", "IDENTITY_AND_BUCKET,50", "ONE_BUCKET,60", "IDENTITY_AND_BUCKET,60"}) + public void testPartitioningParallelismEqualLessThanBuckets( + String schemaTypeStr, String numBucketsStr) { + int numPartitions = 30; + TableSchemaType tableSchemaType = TableSchemaType.valueOf(schemaTypeStr); + int numBuckets = Integer.parseInt(numBucketsStr); + PartitionSpec partitionSpec = tableSchemaType.getPartitionSpec(numBuckets); + BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); + + for (int bucketId = 0; bucketId < numBuckets; bucketId++) { + int actualPartitionIndex = bucketPartitioner.partition(bucketId, numPartitions); + Assertions.assertThat(actualPartitionIndex).isEqualTo(bucketId % numPartitions); + } + } + + @Test + public void testPartitionerBucketIdNullFail() { + PartitionSpec partitionSpec = TableSchemaType.ONE_BUCKET.getPartitionSpec(DEFAULT_NUM_BUCKETS); + BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); + + Assertions.assertThatExceptionOfType(RuntimeException.class) + .isThrownBy(() -> bucketPartitioner.partition(null, DEFAULT_NUM_BUCKETS)) + .withMessage(BUCKET_NULL_MESSAGE); + } + + @Test + public void testPartitionerMultipleBucketsFail() { + PartitionSpec partitionSpec = TableSchemaType.TWO_BUCKETS.getPartitionSpec(DEFAULT_NUM_BUCKETS); + + Assertions.assertThatExceptionOfType(RuntimeException.class) + .isThrownBy(() -> new BucketPartitioner(partitionSpec)) + .withMessage(BucketPartitionerUtil.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE, 2); + } + + @Test + public void testPartitionerBucketIdOutOfRangeFail() { + PartitionSpec partitionSpec = TableSchemaType.ONE_BUCKET.getPartitionSpec(DEFAULT_NUM_BUCKETS); + BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); + + int negativeBucketId = -1; + Assertions.assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> bucketPartitioner.partition(negativeBucketId, 1)) + .withMessage(BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE, negativeBucketId); + + int tooBigBucketId = DEFAULT_NUM_BUCKETS; + Assertions.assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> bucketPartitioner.partition(tooBigBucketId, 1)) + .withMessage(BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE, tooBigBucketId, DEFAULT_NUM_BUCKETS); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java new file mode 100644 index 000000000000..9dae43ce5e58 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java @@ -0,0 +1,227 @@ +/* + * 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.iceberg.flink.sink; + +import static org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.apache.iceberg.flink.TestFixtures.DATABASE; +import static org.apache.iceberg.flink.TestFixtures.TABLE_IDENTIFIER; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.util.DataFormatConverters; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.sink.TestBucketPartitionerUtil.TableSchemaType; +import org.apache.iceberg.flink.source.BoundedTestSource; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; + +public class TestBucketPartitionerFlinkIcebergSink { + + private static final int NUMBER_TASK_MANAGERS = 1; + private static final int SLOTS_PER_TASK_MANAGER = 8; + + @RegisterExtension + private static final MiniClusterExtension MINI_CLUSTER_RESOURCE = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(NUMBER_TASK_MANAGERS) + .setNumberSlotsPerTaskManager(SLOTS_PER_TASK_MANAGER) + .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) + .build()); + + @RegisterExtension + private static final HadoopCatalogExtension catalogExtension = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); + + private static final TypeInformation ROW_TYPE_INFO = + new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); + + // Parallelism = 8 (parallelism > numBuckets) throughout the test suite + private final int parallelism = NUMBER_TASK_MANAGERS * SLOTS_PER_TASK_MANAGER; + private final FileFormat format = FileFormat.PARQUET; + private final int numBuckets = 4; + + private Table table; + private StreamExecutionEnvironment env; + private TableLoader tableLoader; + + private void setupEnvironment(TableSchemaType tableSchemaType) { + PartitionSpec partitionSpec = tableSchemaType.getPartitionSpec(numBuckets); + table = + catalogExtension + .catalog() + .createTable( + TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitionSpec, + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); + env = + StreamExecutionEnvironment.getExecutionEnvironment(DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism * 2); + tableLoader = catalogExtension.tableLoader(); + } + + private void appendRowsToTable(List allRows) throws Exception { + DataFormatConverters.RowConverter converter = + new DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes()); + + DataStream dataStream = + env.addSource( + new BoundedTestSource<>( + allRows.stream().map(converter::toExternal).toArray(Row[]::new)), + ROW_TYPE_INFO) + .map(converter::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)) + .partitionCustom( + new BucketPartitioner(table.spec()), + new BucketPartitionKeySelector( + table.spec(), + table.schema(), + FlinkSink.toFlinkRowType(table.schema(), SimpleDataUtil.FLINK_SCHEMA))); + + FlinkSink.forRowData(dataStream) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .distributionMode(DistributionMode.NONE) + .append(); + + env.execute("Test Iceberg DataStream"); + + SimpleDataUtil.assertTableRows(table, allRows); + } + + @ParameterizedTest + @EnumSource( + value = TableSchemaType.class, + names = {"ONE_BUCKET", "IDENTITY_AND_BUCKET"}) + public void testSendRecordsToAllBucketsEvenly(TableSchemaType tableSchemaType) throws Exception { + setupEnvironment(tableSchemaType); + List rows = generateTestDataRows(); + + appendRowsToTable(rows); + TableTestStats stats = extractPartitionResults(tableSchemaType); + + Assertions.assertThat(stats.totalRowCount).isEqualTo(rows.size()); + // All 4 buckets should've been written to + Assertions.assertThat(stats.writersPerBucket.size()).isEqualTo(numBuckets); + Assertions.assertThat(stats.numFilesPerBucket.size()).isEqualTo(numBuckets); + // Writer expectation (2 writers per bucket): + // - Bucket0 -> Writers [0, 4] + // - Bucket1 -> Writers [1, 5] + // - Bucket2 -> Writers [2, 6] + // - Bucket3 -> Writers [3, 7] + for (int i = 0, j = numBuckets; i < numBuckets; i++, j++) { + Assertions.assertThat(stats.writersPerBucket.get(i)).hasSameElementsAs(Arrays.asList(i, j)); + // 2 files per bucket (one file is created by each writer) + Assertions.assertThat(stats.numFilesPerBucket.get(i)).isEqualTo(2); + // 2 rows per file (total of 16 rows across 8 files) + Assertions.assertThat(stats.rowsPerWriter.get(i)).isEqualTo(2); + } + } + + /** + * Generating 16 rows to be sent uniformly to all writers (round-robin across 8 writers -> 4 + * buckets) + */ + private List generateTestDataRows() { + int totalNumRows = parallelism * 2; + int numRowsPerBucket = totalNumRows / numBuckets; + return TestBucketPartitionerUtil.generateRowsForBucketIdRange(numRowsPerBucket, numBuckets); + } + + private TableTestStats extractPartitionResults(TableSchemaType tableSchemaType) + throws IOException { + int totalRecordCount = 0; + Map> writersPerBucket = Maps.newHashMap(); // > + Map filesPerBucket = Maps.newHashMap(); // + Map rowsPerWriter = Maps.newHashMap(); // + + try (CloseableIterable fileScanTasks = table.newScan().planFiles()) { + for (FileScanTask scanTask : fileScanTasks) { + long recordCountInFile = scanTask.file().recordCount(); + + String[] splitFilePath = scanTask.file().path().toString().split("/"); + // Filename example: 00007-0-a7d3a29a-33e9-4740-88f4-0f494397d60c-00001.parquet + // Writer ID: .......^^^^^ + String filename = splitFilePath[splitFilePath.length - 1]; + int writerId = Integer.parseInt(filename.split("-")[0]); + + totalRecordCount += recordCountInFile; + int bucketId = + scanTask + .file() + .partition() + .get(tableSchemaType.bucketPartitionColumnPosition(), Integer.class); + writersPerBucket.computeIfAbsent(bucketId, k -> Lists.newArrayList()); + writersPerBucket.get(bucketId).add(writerId); + filesPerBucket.put(bucketId, filesPerBucket.getOrDefault(bucketId, 0) + 1); + rowsPerWriter.put(writerId, rowsPerWriter.getOrDefault(writerId, 0L) + recordCountInFile); + } + } + + return new TableTestStats(totalRecordCount, writersPerBucket, filesPerBucket, rowsPerWriter); + } + + /** DTO to hold Test Stats */ + private static class TableTestStats { + final int totalRowCount; + final Map> writersPerBucket; + final Map numFilesPerBucket; + final Map rowsPerWriter; + + TableTestStats( + int totalRecordCount, + Map> writersPerBucket, + Map numFilesPerBucket, + Map rowsPerWriter) { + this.totalRowCount = totalRecordCount; + this.writersPerBucket = writersPerBucket; + this.numFilesPerBucket = numFilesPerBucket; + this.rowsPerWriter = rowsPerWriter; + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java new file mode 100644 index 000000000000..e1309bfac6d5 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.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.iceberg.flink.sink; + +import java.util.List; +import java.util.UUID; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.BucketUtil; + +final class TestBucketPartitionerUtil { + + enum TableSchemaType { + ONE_BUCKET { + @Override + public int bucketPartitionColumnPosition() { + return 0; + } + + @Override + public PartitionSpec getPartitionSpec(int numBuckets) { + return PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).bucket("data", numBuckets).build(); + } + }, + IDENTITY_AND_BUCKET { + @Override + public int bucketPartitionColumnPosition() { + return 1; + } + + @Override + public PartitionSpec getPartitionSpec(int numBuckets) { + return PartitionSpec.builderFor(SimpleDataUtil.SCHEMA) + .identity("id") + .bucket("data", numBuckets) + .build(); + } + }, + TWO_BUCKETS { + @Override + public int bucketPartitionColumnPosition() { + return 1; + } + + @Override + public PartitionSpec getPartitionSpec(int numBuckets) { + return PartitionSpec.builderFor(SimpleDataUtil.SCHEMA) + .bucket("id", numBuckets) + .bucket("data", numBuckets) + .build(); + } + }; + + public abstract int bucketPartitionColumnPosition(); + + public abstract PartitionSpec getPartitionSpec(int numBuckets); + } + + private TestBucketPartitionerUtil() {} + + /** + * Utility method to generate rows whose values will "hash" to a range of bucketIds (from 0 to + * numBuckets - 1) + * + * @param numRowsPerBucket how many different rows should be generated per bucket + * @param numBuckets max number of buckets to consider + * @return the list of rows whose data "hashes" to the desired bucketId + */ + static List generateRowsForBucketIdRange(int numRowsPerBucket, int numBuckets) { + List rows = Lists.newArrayListWithCapacity(numBuckets * numRowsPerBucket); + // For some of our tests, this order of the generated rows matters + for (int i = 0; i < numRowsPerBucket; i++) { + for (int bucketId = 0; bucketId < numBuckets; bucketId++) { + String value = generateValueForBucketId(bucketId, numBuckets); + rows.add(GenericRowData.of(1, StringData.fromString(value))); + } + } + return rows; + } + + /** + * Utility method to generate a UUID string that will "hash" to a desired bucketId + * + * @param bucketId the desired bucketId + * @return the string data that "hashes" to the desired bucketId + */ + private static String generateValueForBucketId(int bucketId, int numBuckets) { + while (true) { + String uuid = UUID.randomUUID().toString(); + if (computeBucketId(numBuckets, uuid) == bucketId) { + return uuid; + } + } + } + + /** + * Utility that performs the same hashing/bucketing mechanism used by Bucket.java + * + * @param numBuckets max number of buckets to consider + * @param value the string to compute the bucketId from + * @return the computed bucketId + */ + static int computeBucketId(int numBuckets, String value) { + return (BucketUtil.hash(value) & Integer.MAX_VALUE) % numBuckets; + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java new file mode 100644 index 000000000000..360db658cd2f --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java @@ -0,0 +1,81 @@ +/* + * 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.iceberg.flink.sink; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.time.Duration; +import java.util.concurrent.TimeUnit; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.Test; + +public class TestCachingTableSupplier { + + @Test + public void testCheckArguments() { + SerializableTable initialTable = mock(SerializableTable.class); + + Table loadedTable = mock(Table.class); + TableLoader tableLoader = mock(TableLoader.class); + when(tableLoader.loadTable()).thenReturn(loadedTable); + + new CachingTableSupplier(initialTable, tableLoader, Duration.ofMillis(100)); + + assertThatThrownBy(() -> new CachingTableSupplier(initialTable, tableLoader, null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("tableRefreshInterval cannot be null"); + assertThatThrownBy(() -> new CachingTableSupplier(null, tableLoader, Duration.ofMillis(100))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("initialTable cannot be null"); + assertThatThrownBy(() -> new CachingTableSupplier(initialTable, null, Duration.ofMillis(100))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("tableLoader cannot be null"); + } + + @Test + public void testTableReload() { + SerializableTable initialTable = mock(SerializableTable.class); + + Table loadedTable = mock(Table.class); + TableLoader tableLoader = mock(TableLoader.class); + when(tableLoader.loadTable()).thenReturn(loadedTable); + + CachingTableSupplier cachingTableSupplier = + new CachingTableSupplier(initialTable, tableLoader, Duration.ofMillis(100)); + + // refresh shouldn't do anything as the min reload interval hasn't passed + cachingTableSupplier.refreshTable(); + assertThat(cachingTableSupplier.get()).isEqualTo(initialTable); + + // refresh after waiting past the min reload interval + Awaitility.await() + .atLeast(100, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + cachingTableSupplier.refreshTable(); + assertThat(cachingTableSupplier.get()).isEqualTo(loadedTable); + }); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java new file mode 100644 index 000000000000..14f12422da96 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java @@ -0,0 +1,254 @@ +/* + * 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.iceberg.flink.sink; + +import java.io.File; +import java.io.IOException; +import java.util.Map; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.common.DynFields; +import org.apache.iceberg.flink.FlinkWriteConf; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.io.BaseTaskWriter; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestCompressionSettings { + @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); + + private Table table; + + private final Map initProperties; + + @Parameterized.Parameters(name = "tableProperties = {0}") + public static Object[] parameters() { + return new Object[] { + ImmutableMap.of(), + ImmutableMap.of( + TableProperties.AVRO_COMPRESSION, + "zstd", + TableProperties.AVRO_COMPRESSION_LEVEL, + "3", + TableProperties.PARQUET_COMPRESSION, + "zstd", + TableProperties.PARQUET_COMPRESSION_LEVEL, + "3", + TableProperties.ORC_COMPRESSION, + "zstd", + TableProperties.ORC_COMPRESSION_STRATEGY, + "compression") + }; + } + + public TestCompressionSettings(Map initProperties) { + this.initProperties = initProperties; + } + + @Before + public void before() throws IOException { + File folder = tempFolder.newFolder(); + table = SimpleDataUtil.createTable(folder.getAbsolutePath(), initProperties, false); + } + + @Test + public void testCompressionAvro() throws Exception { + // No override provided + Map resultProperties = + appenderProperties( + table, + SimpleDataUtil.FLINK_SCHEMA, + ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "AVRO")); + + if (initProperties.get(TableProperties.AVRO_COMPRESSION) == null) { + Assert.assertEquals( + TableProperties.AVRO_COMPRESSION_DEFAULT, + resultProperties.get(TableProperties.AVRO_COMPRESSION)); + Assert.assertEquals( + TableProperties.AVRO_COMPRESSION_LEVEL_DEFAULT, + resultProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); + } else { + Assert.assertEquals( + initProperties.get(TableProperties.AVRO_COMPRESSION), + resultProperties.get(TableProperties.AVRO_COMPRESSION)); + Assert.assertEquals( + initProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL), + resultProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); + } + + // Override compression to snappy and some random level + resultProperties = + appenderProperties( + table, + SimpleDataUtil.FLINK_SCHEMA, + ImmutableMap.of( + FlinkWriteOptions.WRITE_FORMAT.key(), + "AVRO", + FlinkWriteOptions.COMPRESSION_CODEC.key(), + "snappy", + FlinkWriteOptions.COMPRESSION_LEVEL.key(), + "6")); + + Assert.assertEquals("snappy", resultProperties.get(TableProperties.AVRO_COMPRESSION)); + Assert.assertEquals("6", resultProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); + } + + @Test + public void testCompressionParquet() throws Exception { + // No override provided + Map resultProperties = + appenderProperties( + table, + SimpleDataUtil.FLINK_SCHEMA, + ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "PARQUET")); + + if (initProperties.get(TableProperties.PARQUET_COMPRESSION) == null) { + Assert.assertEquals( + TableProperties.PARQUET_COMPRESSION_DEFAULT_SINCE_1_4_0, + resultProperties.get(TableProperties.PARQUET_COMPRESSION)); + Assert.assertEquals( + TableProperties.PARQUET_COMPRESSION_LEVEL_DEFAULT, + resultProperties.get(TableProperties.PARQUET_COMPRESSION_DEFAULT_SINCE_1_4_0)); + } else { + Assert.assertEquals( + initProperties.get(TableProperties.PARQUET_COMPRESSION), + resultProperties.get(TableProperties.PARQUET_COMPRESSION)); + Assert.assertEquals( + initProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL), + resultProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL)); + } + + // Override compression to snappy and some random level + resultProperties = + appenderProperties( + table, + SimpleDataUtil.FLINK_SCHEMA, + ImmutableMap.of( + FlinkWriteOptions.WRITE_FORMAT.key(), + "PARQUET", + FlinkWriteOptions.COMPRESSION_CODEC.key(), + "snappy", + FlinkWriteOptions.COMPRESSION_LEVEL.key(), + "6")); + + Assert.assertEquals("snappy", resultProperties.get(TableProperties.PARQUET_COMPRESSION)); + Assert.assertEquals("6", resultProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL)); + } + + @Test + public void testCompressionOrc() throws Exception { + // No override provided + Map resultProperties = + appenderProperties( + table, + SimpleDataUtil.FLINK_SCHEMA, + ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "ORC")); + + if (initProperties.get(TableProperties.ORC_COMPRESSION) == null) { + Assert.assertEquals( + TableProperties.ORC_COMPRESSION_DEFAULT, + resultProperties.get(TableProperties.ORC_COMPRESSION)); + Assert.assertEquals( + TableProperties.ORC_COMPRESSION_STRATEGY_DEFAULT, + resultProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); + } else { + Assert.assertEquals( + initProperties.get(TableProperties.ORC_COMPRESSION), + resultProperties.get(TableProperties.ORC_COMPRESSION)); + Assert.assertEquals( + initProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY), + resultProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); + } + + // Override compression to snappy and a different strategy + resultProperties = + appenderProperties( + table, + SimpleDataUtil.FLINK_SCHEMA, + ImmutableMap.of( + FlinkWriteOptions.WRITE_FORMAT.key(), + "ORC", + FlinkWriteOptions.COMPRESSION_CODEC.key(), + "snappy", + FlinkWriteOptions.COMPRESSION_STRATEGY.key(), + "speed")); + + Assert.assertEquals("snappy", resultProperties.get(TableProperties.ORC_COMPRESSION)); + Assert.assertEquals("speed", resultProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); + } + + private static OneInputStreamOperatorTestHarness createIcebergStreamWriter( + Table icebergTable, TableSchema flinkSchema, Map override) throws Exception { + RowType flinkRowType = FlinkSink.toFlinkRowType(icebergTable.schema(), flinkSchema); + FlinkWriteConf flinkWriteConfig = + new FlinkWriteConf( + icebergTable, override, new org.apache.flink.configuration.Configuration()); + + IcebergStreamWriter streamWriter = + FlinkSink.createStreamWriter(() -> icebergTable, flinkWriteConfig, flinkRowType, null); + OneInputStreamOperatorTestHarness harness = + new OneInputStreamOperatorTestHarness<>(streamWriter, 1, 1, 0); + + harness.setup(); + harness.open(); + + return harness; + } + + private static Map appenderProperties( + Table table, TableSchema schema, Map override) throws Exception { + try (OneInputStreamOperatorTestHarness testHarness = + createIcebergStreamWriter(table, schema, override)) { + testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); + + testHarness.prepareSnapshotPreBarrier(1L); + DynFields.BoundField operatorField = + DynFields.builder() + .hiddenImpl(testHarness.getOperatorFactory().getClass(), "operator") + .build(testHarness.getOperatorFactory()); + DynFields.BoundField writerField = + DynFields.builder() + .hiddenImpl(IcebergStreamWriter.class, "writer") + .build(operatorField.get()); + DynFields.BoundField appenderField = + DynFields.builder() + .hiddenImpl(BaseTaskWriter.class, "appenderFactory") + .build(writerField.get()); + DynFields.BoundField> propsField = + DynFields.builder() + .hiddenImpl(FlinkAppenderFactory.class, "props") + .build(appenderField.get()); + return propsField.get(); + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java new file mode 100644 index 000000000000..4ecbd1c12921 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java @@ -0,0 +1,440 @@ +/* + * 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.iceberg.flink.sink; + +import static org.apache.iceberg.flink.SimpleDataUtil.createDelete; +import static org.apache.iceberg.flink.SimpleDataUtil.createInsert; +import static org.apache.iceberg.flink.SimpleDataUtil.createRecord; +import static org.apache.iceberg.flink.SimpleDataUtil.createUpdateAfter; +import static org.apache.iceberg.flink.SimpleDataUtil.createUpdateBefore; +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.time.OffsetDateTime; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.LocalZonedTimestampType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.FileContent; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestTables; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.StructLikeSet; +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestDeltaTaskWriter extends TableTestBase { + private static final int FORMAT_V2 = 2; + + private final FileFormat format; + + @Parameterized.Parameters(name = "FileFormat = {0}") + public static Object[][] parameters() { + return new Object[][] {{"avro"}, {"orc"}, {"parquet"}}; + } + + public TestDeltaTaskWriter(String fileFormat) { + super(FORMAT_V2); + this.format = FileFormat.fromString(fileFormat); + } + + @Override + @Before + public void setupTable() throws IOException { + this.tableDir = temp.newFolder(); + Assert.assertTrue(tableDir.delete()); // created by table create + + this.metadataDir = new File(tableDir, "metadata"); + } + + private int idFieldId() { + return table.schema().findField("id").fieldId(); + } + + private int dataFieldId() { + return table.schema().findField("data").fieldId(); + } + + private void testCdcEvents(boolean partitioned) throws IOException { + List equalityFieldIds = Lists.newArrayList(idFieldId()); + TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); + taskWriterFactory.initialize(1, 1); + + // Start the 1th transaction. + TaskWriter writer = taskWriterFactory.create(); + + writer.write(createInsert(1, "aaa")); + writer.write(createInsert(2, "bbb")); + writer.write(createInsert(3, "ccc")); + + // Update <2, 'bbb'> to <2, 'ddd'> + writer.write(createUpdateBefore(2, "bbb")); // 1 pos-delete and 1 eq-delete. + writer.write(createUpdateAfter(2, "ddd")); + + // Update <1, 'aaa'> to <1, 'eee'> + writer.write(createUpdateBefore(1, "aaa")); // 1 pos-delete and 1 eq-delete. + writer.write(createUpdateAfter(1, "eee")); + + // Insert <4, 'fff'> + writer.write(createInsert(4, "fff")); + // Insert <5, 'ggg'> + writer.write(createInsert(5, "ggg")); + + // Delete <3, 'ccc'> + writer.write(createDelete(3, "ccc")); // 1 pos-delete and 1 eq-delete. + + WriteResult result = writer.complete(); + Assert.assertEquals(partitioned ? 7 : 1, result.dataFiles().length); + Assert.assertEquals(partitioned ? 3 : 1, result.deleteFiles().length); + commitTransaction(result); + + Assert.assertEquals( + "Should have expected records.", + expectedRowSet( + createRecord(1, "eee"), + createRecord(2, "ddd"), + createRecord(4, "fff"), + createRecord(5, "ggg")), + actualRowSet("*")); + + // Start the 2nd transaction. + writer = taskWriterFactory.create(); + + // Update <2, 'ddd'> to <6, 'hhh'> - (Update both key and value) + writer.write(createUpdateBefore(2, "ddd")); // 1 eq-delete + writer.write(createUpdateAfter(6, "hhh")); + + // Update <5, 'ggg'> to <5, 'iii'> + writer.write(createUpdateBefore(5, "ggg")); // 1 eq-delete + writer.write(createUpdateAfter(5, "iii")); + + // Delete <4, 'fff'> + writer.write(createDelete(4, "fff")); // 1 eq-delete. + + result = writer.complete(); + Assert.assertEquals(partitioned ? 2 : 1, result.dataFiles().length); + Assert.assertEquals(partitioned ? 3 : 1, result.deleteFiles().length); + commitTransaction(result); + + Assert.assertEquals( + "Should have expected records", + expectedRowSet(createRecord(1, "eee"), createRecord(5, "iii"), createRecord(6, "hhh")), + actualRowSet("*")); + } + + @Test + public void testUnpartitioned() throws IOException { + createAndInitTable(false); + testCdcEvents(false); + } + + @Test + public void testPartitioned() throws IOException { + createAndInitTable(true); + testCdcEvents(true); + } + + private void testWritePureEqDeletes(boolean partitioned) throws IOException { + createAndInitTable(partitioned); + List equalityFieldIds = Lists.newArrayList(idFieldId()); + TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); + taskWriterFactory.initialize(1, 1); + + TaskWriter writer = taskWriterFactory.create(); + writer.write(createDelete(1, "aaa")); + writer.write(createDelete(2, "bbb")); + writer.write(createDelete(3, "ccc")); + + WriteResult result = writer.complete(); + Assert.assertEquals(0, result.dataFiles().length); + Assert.assertEquals(partitioned ? 3 : 1, result.deleteFiles().length); + commitTransaction(result); + + Assert.assertEquals("Should have no record", expectedRowSet(), actualRowSet("*")); + } + + @Test + public void testUnpartitionedPureEqDeletes() throws IOException { + testWritePureEqDeletes(false); + } + + @Test + public void testPartitionedPureEqDeletes() throws IOException { + testWritePureEqDeletes(true); + } + + private void testAbort(boolean partitioned) throws IOException { + createAndInitTable(partitioned); + List equalityFieldIds = Lists.newArrayList(idFieldId()); + TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); + taskWriterFactory.initialize(1, 1); + + TaskWriter writer = taskWriterFactory.create(); + for (int i = 0; i < 8_000; i += 2) { + writer.write(createUpdateBefore(i + 1, "aaa")); + writer.write(createUpdateAfter(i + 1, "aaa")); + + writer.write(createUpdateBefore(i + 2, "bbb")); + writer.write(createUpdateAfter(i + 2, "bbb")); + } + + // Assert the current data/delete file count. + List files = + Files.walk(Paths.get(tableDir.getPath(), "data")) + .filter(p -> p.toFile().isFile()) + .filter(p -> !p.toString().endsWith(".crc")) + .collect(Collectors.toList()); + Assert.assertEquals( + "Should have expected file count, but files are: " + files, + partitioned ? 4 : 2, + files.size()); + + writer.abort(); + for (Path file : files) { + Assert.assertFalse(Files.exists(file)); + } + } + + @Test + public void testUnpartitionedAbort() throws IOException { + testAbort(false); + } + + @Test + public void testPartitionedAbort() throws IOException { + testAbort(true); + } + + @Test + public void testPartitionedTableWithDataAsKey() throws IOException { + createAndInitTable(true); + List equalityFieldIds = Lists.newArrayList(dataFieldId()); + TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); + taskWriterFactory.initialize(1, 1); + + // Start the 1th transaction. + TaskWriter writer = taskWriterFactory.create(); + writer.write(createInsert(1, "aaa")); + writer.write(createInsert(2, "aaa")); + writer.write(createInsert(3, "bbb")); + writer.write(createInsert(4, "ccc")); + + WriteResult result = writer.complete(); + Assert.assertEquals(3, result.dataFiles().length); + Assert.assertEquals(1, result.deleteFiles().length); + commitTransaction(result); + + Assert.assertEquals( + "Should have expected records", + expectedRowSet(createRecord(2, "aaa"), createRecord(3, "bbb"), createRecord(4, "ccc")), + actualRowSet("*")); + + // Start the 2nd transaction. + writer = taskWriterFactory.create(); + writer.write(createInsert(5, "aaa")); + writer.write(createInsert(6, "bbb")); + writer.write(createDelete(7, "ccc")); // 1 eq-delete. + + result = writer.complete(); + Assert.assertEquals(2, result.dataFiles().length); + Assert.assertEquals(1, result.deleteFiles().length); + commitTransaction(result); + + Assert.assertEquals( + "Should have expected records", + expectedRowSet( + createRecord(2, "aaa"), + createRecord(5, "aaa"), + createRecord(3, "bbb"), + createRecord(6, "bbb")), + actualRowSet("*")); + } + + @Test + public void testPartitionedTableWithDataAndIdAsKey() throws IOException { + createAndInitTable(true); + List equalityFieldIds = Lists.newArrayList(dataFieldId(), idFieldId()); + TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); + taskWriterFactory.initialize(1, 1); + + TaskWriter writer = taskWriterFactory.create(); + writer.write(createInsert(1, "aaa")); + writer.write(createInsert(2, "aaa")); + + writer.write(createDelete(2, "aaa")); // 1 pos-delete. + + WriteResult result = writer.complete(); + Assert.assertEquals(1, result.dataFiles().length); + Assert.assertEquals(1, result.deleteFiles().length); + Assert.assertEquals( + Sets.newHashSet(FileContent.POSITION_DELETES), + Sets.newHashSet(result.deleteFiles()[0].content())); + commitTransaction(result); + + Assert.assertEquals( + "Should have expected records", expectedRowSet(createRecord(1, "aaa")), actualRowSet("*")); + } + + @Test + public void testEqualityColumnOnCustomPrecisionTSColumn() throws IOException { + Schema tableSchema = + new Schema( + required(3, "id", Types.IntegerType.get()), + required(4, "ts", Types.TimestampType.withZone())); + RowType flinkType = + new RowType( + false, + ImmutableList.of( + new RowType.RowField("id", new IntType()), + new RowType.RowField("ts", new LocalZonedTimestampType(3)))); + + this.table = create(tableSchema, PartitionSpec.unpartitioned()); + initTable(table); + + List equalityIds = ImmutableList.of(table.schema().findField("ts").fieldId()); + TaskWriterFactory taskWriterFactory = createTaskWriterFactory(flinkType, equalityIds); + taskWriterFactory.initialize(1, 1); + + TaskWriter writer = taskWriterFactory.create(); + RowDataSerializer serializer = new RowDataSerializer(flinkType); + OffsetDateTime start = OffsetDateTime.now(); + writer.write( + serializer.toBinaryRow( + GenericRowData.ofKind( + RowKind.INSERT, 1, TimestampData.fromInstant(start.toInstant())))); + writer.write( + serializer.toBinaryRow( + GenericRowData.ofKind( + RowKind.INSERT, 2, TimestampData.fromInstant(start.plusSeconds(1).toInstant())))); + writer.write( + serializer.toBinaryRow( + GenericRowData.ofKind( + RowKind.DELETE, 2, TimestampData.fromInstant(start.plusSeconds(1).toInstant())))); + + WriteResult result = writer.complete(); + // One data file + Assertions.assertThat(result.dataFiles().length).isEqualTo(1); + // One eq delete file + one pos delete file + Assertions.assertThat(result.deleteFiles().length).isEqualTo(2); + Assertions.assertThat( + Arrays.stream(result.deleteFiles()) + .map(ContentFile::content) + .collect(Collectors.toSet())) + .isEqualTo(Sets.newHashSet(FileContent.POSITION_DELETES, FileContent.EQUALITY_DELETES)); + commitTransaction(result); + + Record expectedRecord = GenericRecord.create(tableSchema); + expectedRecord.setField("id", 1); + int cutPrecisionNano = start.getNano() / 1000000 * 1000000; + expectedRecord.setField("ts", start.withNano(cutPrecisionNano)); + + Assertions.assertThat(actualRowSet("*")).isEqualTo(expectedRowSet(expectedRecord)); + } + + private void commitTransaction(WriteResult result) { + RowDelta rowDelta = table.newRowDelta(); + Arrays.stream(result.dataFiles()).forEach(rowDelta::addRows); + Arrays.stream(result.deleteFiles()).forEach(rowDelta::addDeletes); + rowDelta + .validateDeletedFiles() + .validateDataFilesExist(Lists.newArrayList(result.referencedDataFiles())) + .commit(); + } + + private StructLikeSet expectedRowSet(Record... records) { + return SimpleDataUtil.expectedRowSet(table, records); + } + + private StructLikeSet actualRowSet(String... columns) throws IOException { + return SimpleDataUtil.actualRowSet(table, columns); + } + + private TaskWriterFactory createTaskWriterFactory(List equalityFieldIds) { + return new RowDataTaskWriterFactory( + SerializableTable.copyOf(table), + FlinkSchemaUtil.convert(table.schema()), + 128 * 1024 * 1024, + format, + table.properties(), + equalityFieldIds, + false); + } + + private TaskWriterFactory createTaskWriterFactory( + RowType flinkType, List equalityFieldIds) { + return new RowDataTaskWriterFactory( + SerializableTable.copyOf(table), + flinkType, + 128 * 1024 * 1024, + format, + table.properties(), + equalityFieldIds, + true); + } + + private void createAndInitTable(boolean partitioned) { + if (partitioned) { + this.table = create(SCHEMA, PartitionSpec.builderFor(SCHEMA).identity("data").build()); + } else { + this.table = create(SCHEMA, PartitionSpec.unpartitioned()); + } + + initTable(table); + } + + private void initTable(TestTables.TestTable testTable) { + testTable + .updateProperties() + .set(TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, String.valueOf(8 * 1024)) + .defaultFormat(format) + .commit(); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java new file mode 100644 index 000000000000..d25b2792ac65 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java @@ -0,0 +1,70 @@ +/* + * 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.iceberg.flink.sink; + +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.TestAppenderFactory; +import org.apache.iceberg.util.ArrayUtil; +import org.apache.iceberg.util.StructLikeSet; + +public class TestFlinkAppenderFactory extends TestAppenderFactory { + + private final RowType rowType; + + public TestFlinkAppenderFactory(String fileFormat, boolean partitioned) { + super(fileFormat, partitioned); + this.rowType = FlinkSchemaUtil.convert(SCHEMA); + } + + @Override + protected FileAppenderFactory createAppenderFactory( + List equalityFieldIds, Schema eqDeleteSchema, Schema posDeleteRowSchema) { + return new FlinkAppenderFactory( + table, + table.schema(), + rowType, + table.properties(), + table.spec(), + ArrayUtil.toIntArray(equalityFieldIds), + eqDeleteSchema, + posDeleteRowSchema); + } + + @Override + protected RowData createRow(Integer id, String data) { + return SimpleDataUtil.createRowData(id, data); + } + + @Override + protected StructLikeSet expectedRowSet(Iterable rows) { + StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); + for (RowData row : rows) { + RowDataWrapper wrapper = new RowDataWrapper(rowType, table.schema().asStruct()); + set.add(wrapper.wrap(row)); + } + return set; + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java new file mode 100644 index 000000000000..da45241256f5 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.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.iceberg.flink.sink; + +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.io.FileWriterFactory; +import org.apache.iceberg.io.TestFileWriterFactory; +import org.apache.iceberg.util.ArrayUtil; +import org.apache.iceberg.util.StructLikeSet; + +public class TestFlinkFileWriterFactory extends TestFileWriterFactory { + + public TestFlinkFileWriterFactory(FileFormat fileFormat, boolean partitioned) { + super(fileFormat, partitioned); + } + + @Override + protected FileWriterFactory newWriterFactory( + Schema dataSchema, + List equalityFieldIds, + Schema equalityDeleteRowSchema, + Schema positionDeleteRowSchema) { + return FlinkFileWriterFactory.builderFor(table) + .dataSchema(table.schema()) + .dataFileFormat(format()) + .deleteFileFormat(format()) + .equalityFieldIds(ArrayUtil.toIntArray(equalityFieldIds)) + .equalityDeleteRowSchema(equalityDeleteRowSchema) + .positionDeleteRowSchema(positionDeleteRowSchema) + .build(); + } + + @Override + protected RowData toRow(Integer id, String data) { + return SimpleDataUtil.createRowData(id, data); + } + + @Override + protected StructLikeSet toSet(Iterable rows) { + StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); + RowType flinkType = FlinkSchemaUtil.convert(table.schema()); + for (RowData row : rows) { + RowDataWrapper wrapper = new RowDataWrapper(flinkType, table.schema().asStruct()); + set.add(wrapper.wrap(row)); + } + return set; + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java new file mode 100644 index 000000000000..11a73d2cc144 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java @@ -0,0 +1,397 @@ +/* + * 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.iceberg.flink.sink; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.HadoopCatalogResource; +import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestFlinkIcebergSink extends TestFlinkIcebergSinkBase { + + @ClassRule + public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = + MiniClusterResource.createWithClassloaderCheckDisabled(); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final HadoopCatalogResource catalogResource = + new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); + + private TableLoader tableLoader; + + private final FileFormat format; + private final int parallelism; + private final boolean partitioned; + + @Parameterized.Parameters(name = "format={0}, parallelism = {1}, partitioned = {2}") + public static Object[][] parameters() { + return new Object[][] { + {"avro", 1, true}, + {"avro", 1, false}, + {"avro", 2, true}, + {"avro", 2, false}, + {"orc", 1, true}, + {"orc", 1, false}, + {"orc", 2, true}, + {"orc", 2, false}, + {"parquet", 1, true}, + {"parquet", 1, false}, + {"parquet", 2, true}, + {"parquet", 2, false} + }; + } + + public TestFlinkIcebergSink(String format, int parallelism, boolean partitioned) { + this.format = FileFormat.fromString(format); + this.parallelism = parallelism; + this.partitioned = partitioned; + } + + @Before + public void before() throws IOException { + table = + catalogResource + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + + tableLoader = catalogResource.tableLoader(); + } + + @Test + public void testWriteRowData() throws Exception { + List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) + .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); + + FlinkSink.forRowData(dataStream) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream"); + + // Assert the iceberg table's records. + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } + + private void testWriteRow(TableSchema tableSchema, DistributionMode distributionMode) + throws Exception { + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(tableSchema) + .writeParallelism(parallelism) + .distributionMode(distributionMode) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } + + private int partitionFiles(String partition) throws IOException { + return SimpleDataUtil.partitionDataFiles(table, ImmutableMap.of("data", partition)).size(); + } + + @Test + public void testWriteRow() throws Exception { + testWriteRow(null, DistributionMode.NONE); + } + + @Test + public void testWriteRowWithTableSchema() throws Exception { + testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); + } + + @Test + public void testJobNoneDistributeMode() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + testWriteRow(null, DistributionMode.NONE); + + if (parallelism > 1) { + if (partitioned) { + int files = partitionFiles("aaa") + partitionFiles("bbb") + partitionFiles("ccc"); + Assert.assertTrue("Should have more than 3 files in iceberg table.", files > 3); + } + } + } + + @Test + public void testJobHashDistributionMode() { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + Assertions.assertThatThrownBy(() -> testWriteRow(null, DistributionMode.RANGE)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Flink does not support 'range' write distribution mode now."); + } + + @Test + public void testJobNullDistributionMode() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + testWriteRow(null, null); + + if (partitioned) { + Assert.assertEquals( + "There should be only 1 data file in partition 'aaa'", 1, partitionFiles("aaa")); + Assert.assertEquals( + "There should be only 1 data file in partition 'bbb'", 1, partitionFiles("bbb")); + Assert.assertEquals( + "There should be only 1 data file in partition 'ccc'", 1, partitionFiles("ccc")); + } + } + + @Test + public void testPartitionWriteMode() throws Exception { + testWriteRow(null, DistributionMode.HASH); + if (partitioned) { + Assert.assertEquals( + "There should be only 1 data file in partition 'aaa'", 1, partitionFiles("aaa")); + Assert.assertEquals( + "There should be only 1 data file in partition 'bbb'", 1, partitionFiles("bbb")); + Assert.assertEquals( + "There should be only 1 data file in partition 'ccc'", 1, partitionFiles("ccc")); + } + } + + @Test + public void testShuffleByPartitionWithSchema() throws Exception { + testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.HASH); + if (partitioned) { + Assert.assertEquals( + "There should be only 1 data file in partition 'aaa'", 1, partitionFiles("aaa")); + Assert.assertEquals( + "There should be only 1 data file in partition 'bbb'", 1, partitionFiles("bbb")); + Assert.assertEquals( + "There should be only 1 data file in partition 'ccc'", 1, partitionFiles("ccc")); + } + } + + @Test + public void testTwoSinksInDisjointedDAG() throws Exception { + Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); + + Table leftTable = + catalogResource + .catalog() + .createTable( + TableIdentifier.of("left"), + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + props); + TableLoader leftTableLoader = + TableLoader.fromCatalog(catalogResource.catalogLoader(), TableIdentifier.of("left")); + + Table rightTable = + catalogResource + .catalog() + .createTable( + TableIdentifier.of("right"), + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + props); + TableLoader rightTableLoader = + TableLoader.fromCatalog(catalogResource.catalogLoader(), TableIdentifier.of("right")); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + env.getConfig().disableAutoGeneratedUIDs(); + + List leftRows = createRows("left-"); + DataStream leftStream = + env.fromCollection(leftRows, ROW_TYPE_INFO) + .name("leftCustomSource") + .uid("leftCustomSource"); + FlinkSink.forRow(leftStream, SimpleDataUtil.FLINK_SCHEMA) + .table(leftTable) + .tableLoader(leftTableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .distributionMode(DistributionMode.NONE) + .uidPrefix("leftIcebergSink") + .append(); + + List rightRows = createRows("right-"); + DataStream rightStream = + env.fromCollection(rightRows, ROW_TYPE_INFO) + .name("rightCustomSource") + .uid("rightCustomSource"); + FlinkSink.forRow(rightStream, SimpleDataUtil.FLINK_SCHEMA) + .table(rightTable) + .tableLoader(rightTableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .distributionMode(DistributionMode.HASH) + .uidPrefix("rightIcebergSink") + .setSnapshotProperty("flink.test", TestFlinkIcebergSink.class.getName()) + .setSnapshotProperties(Collections.singletonMap("direction", "rightTable")) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(leftTable, convertToRowData(leftRows)); + SimpleDataUtil.assertTableRows(rightTable, convertToRowData(rightRows)); + + leftTable.refresh(); + Assert.assertNull(leftTable.currentSnapshot().summary().get("flink.test")); + Assert.assertNull(leftTable.currentSnapshot().summary().get("direction")); + rightTable.refresh(); + Assert.assertEquals( + TestFlinkIcebergSink.class.getName(), + rightTable.currentSnapshot().summary().get("flink.test")); + Assert.assertEquals("rightTable", rightTable.currentSnapshot().summary().get("direction")); + } + + @Test + public void testOverrideWriteConfigWithUnknownDistributionMode() { + Map newProps = Maps.newHashMap(); + newProps.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), "UNRECOGNIZED"); + + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .setAll(newProps); + + Assertions.assertThatThrownBy(builder::append) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid distribution mode: UNRECOGNIZED"); + } + + @Test + public void testOverrideWriteConfigWithUnknownFileFormat() { + Map newProps = Maps.newHashMap(); + newProps.put(FlinkWriteOptions.WRITE_FORMAT.key(), "UNRECOGNIZED"); + + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .setAll(newProps); + + Assertions.assertThatThrownBy(builder::append) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid file format: UNRECOGNIZED"); + } + + @Test + public void testWriteRowWithTableRefreshInterval() throws Exception { + List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) + .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); + + Configuration flinkConf = new Configuration(); + flinkConf.setString(FlinkWriteOptions.TABLE_REFRESH_INTERVAL.key(), "100ms"); + + FlinkSink.forRowData(dataStream) + .table(table) + .tableLoader(tableLoader) + .flinkConf(flinkConf) + .writeParallelism(parallelism) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream"); + + // Assert the iceberg table's records. + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java new file mode 100644 index 000000000000..b38aa6b50ce6 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.util.DataFormatConverters; +import org.apache.flink.types.Row; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.source.BoundedTestSource; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +public class TestFlinkIcebergSinkBase { + + protected Table table; + protected StreamExecutionEnvironment env; + protected static final TypeInformation ROW_TYPE_INFO = + new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); + + protected static final DataFormatConverters.RowConverter CONVERTER = + new DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes()); + + protected BoundedTestSource createBoundedSource(List rows) { + return new BoundedTestSource<>(rows.toArray(new Row[0])); + } + + protected List createRows(String prefix) { + return Lists.newArrayList( + Row.of(1, prefix + "aaa"), + Row.of(1, prefix + "bbb"), + Row.of(1, prefix + "ccc"), + Row.of(2, prefix + "aaa"), + Row.of(2, prefix + "bbb"), + Row.of(2, prefix + "ccc"), + Row.of(3, prefix + "aaa"), + Row.of(3, prefix + "bbb"), + Row.of(3, prefix + "ccc")); + } + + protected List convertToRowData(List rows) { + return rows.stream().map(CONVERTER::toInternal).collect(Collectors.toList()); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java new file mode 100644 index 000000000000..16b4542b00d3 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java @@ -0,0 +1,137 @@ +/* + * 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.iceberg.flink.sink; + +import java.io.IOException; +import java.util.List; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.HadoopCatalogResource; +import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.Assert; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestFlinkIcebergSinkBranch extends TestFlinkIcebergSinkBase { + + @ClassRule + public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = + MiniClusterResource.createWithClassloaderCheckDisabled(); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final HadoopCatalogResource catalogResource = + new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); + + private final String branch; + private TableLoader tableLoader; + + @Parameterized.Parameters(name = "formatVersion = {0}, branch = {1}") + public static Object[] parameters() { + return new Object[] {"main", "testBranch"}; + } + + public TestFlinkIcebergSinkBranch(String branch) { + this.branch = branch; + } + + @Before + public void before() throws IOException { + table = + catalogResource + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, + FileFormat.AVRO.name(), + TableProperties.FORMAT_VERSION, + "1")); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100); + + tableLoader = catalogResource.tableLoader(); + } + + @Test + public void testWriteRowWithTableSchema() throws Exception { + testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); + verifyOtherBranchUnmodified(); + } + + private void testWriteRow(TableSchema tableSchema, DistributionMode distributionMode) + throws Exception { + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(tableSchema) + .toBranch(branch) + .distributionMode(distributionMode) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(table, convertToRowData(rows), branch); + SimpleDataUtil.assertTableRows( + table, + ImmutableList.of(), + branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH); + + verifyOtherBranchUnmodified(); + } + + private void verifyOtherBranchUnmodified() { + String otherBranch = + branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH; + if (otherBranch.equals(SnapshotRef.MAIN_BRANCH)) { + Assert.assertNull(table.currentSnapshot()); + } + + Assert.assertTrue(table.snapshot(otherBranch) == null); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java new file mode 100644 index 000000000000..b5c3bcf41734 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java @@ -0,0 +1,236 @@ +/* + * 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.iceberg.flink.sink; + +import java.util.List; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.types.Row; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopCatalogResource; +import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.BoundedTestSource; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestFlinkIcebergSinkV2 extends TestFlinkIcebergSinkV2Base { + + @ClassRule + public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = + MiniClusterResource.createWithClassloaderCheckDisabled(); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final HadoopCatalogResource catalogResource = + new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); + + @Parameterized.Parameters( + name = "FileFormat = {0}, Parallelism = {1}, Partitioned={2}, WriteDistributionMode ={3}") + public static Object[][] parameters() { + return new Object[][] { + new Object[] {"avro", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, + new Object[] {"avro", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, + new Object[] {"avro", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, + new Object[] {"avro", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, + new Object[] {"orc", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, + new Object[] {"orc", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, + new Object[] {"orc", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, + new Object[] {"orc", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, + new Object[] {"parquet", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, + new Object[] {"parquet", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, + new Object[] {"parquet", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, + new Object[] {"parquet", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE} + }; + } + + public TestFlinkIcebergSinkV2( + String format, int parallelism, boolean partitioned, String writeDistributionMode) { + this.format = FileFormat.fromString(format); + this.parallelism = parallelism; + this.partitioned = partitioned; + this.writeDistributionMode = writeDistributionMode; + } + + @Before + public void setupTable() { + table = + catalogResource + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, + format.name(), + TableProperties.FORMAT_VERSION, + String.valueOf(FORMAT_V2))); + + table + .updateProperties() + .set(TableProperties.DEFAULT_FILE_FORMAT, format.name()) + .set(TableProperties.WRITE_DISTRIBUTION_MODE, writeDistributionMode) + .commit(); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100L) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + + tableLoader = catalogResource.tableLoader(); + } + + @Test + public void testCheckAndGetEqualityFieldIds() { + table + .updateSchema() + .allowIncompatibleChanges() + .addRequiredColumn("type", Types.StringType.get()) + .setIdentifierFields("type") + .commit(); + + DataStream dataStream = + env.addSource(new BoundedTestSource<>(ImmutableList.of()), ROW_TYPE_INFO); + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA).table(table); + + // Use schema identifier field IDs as equality field id list by default + Assert.assertEquals( + table.schema().identifierFieldIds(), + Sets.newHashSet(builder.checkAndGetEqualityFieldIds())); + + // Use user-provided equality field column as equality field id list + builder.equalityFieldColumns(Lists.newArrayList("id")); + Assert.assertEquals( + Sets.newHashSet(table.schema().findField("id").fieldId()), + Sets.newHashSet(builder.checkAndGetEqualityFieldIds())); + + builder.equalityFieldColumns(Lists.newArrayList("type")); + Assert.assertEquals( + Sets.newHashSet(table.schema().findField("type").fieldId()), + Sets.newHashSet(builder.checkAndGetEqualityFieldIds())); + } + + @Test + public void testChangeLogOnIdKey() throws Exception { + testChangeLogOnIdKey(SnapshotRef.MAIN_BRANCH); + } + + @Test + public void testUpsertOnlyDeletesOnDataKey() throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of(row("+I", 1, "aaa")), + ImmutableList.of(row("-D", 1, "aaa"), row("-D", 2, "bbb"))); + + List> expectedRecords = + ImmutableList.of(ImmutableList.of(record(1, "aaa")), ImmutableList.of()); + + testChangeLogs( + ImmutableList.of("data"), + row -> row.getField(ROW_DATA_POS), + true, + elementsPerCheckpoint, + expectedRecords, + SnapshotRef.MAIN_BRANCH); + } + + @Test + public void testChangeLogOnDataKey() throws Exception { + testChangeLogOnDataKey(SnapshotRef.MAIN_BRANCH); + } + + @Test + public void testChangeLogOnIdDataKey() throws Exception { + testChangeLogOnIdDataKey(SnapshotRef.MAIN_BRANCH); + } + + @Test + public void testChangeLogOnSameKey() throws Exception { + testChangeLogOnSameKey(SnapshotRef.MAIN_BRANCH); + } + + @Test + public void testUpsertModeCheck() throws Exception { + DataStream dataStream = + env.addSource(new BoundedTestSource<>(ImmutableList.of()), ROW_TYPE_INFO); + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .tableLoader(tableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .upsert(true); + + Assertions.assertThatThrownBy( + () -> + builder + .equalityFieldColumns(ImmutableList.of("id", "data")) + .overwrite(true) + .append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); + + Assertions.assertThatThrownBy( + () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + } + + @Test + public void testUpsertOnIdKey() throws Exception { + testUpsertOnIdKey(SnapshotRef.MAIN_BRANCH); + } + + @Test + public void testUpsertOnDataKey() throws Exception { + testUpsertOnDataKey(SnapshotRef.MAIN_BRANCH); + } + + @Test + public void testUpsertOnIdDataKey() throws Exception { + testUpsertOnIdDataKey(SnapshotRef.MAIN_BRANCH); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java new file mode 100644 index 000000000000..0b403756cefe --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java @@ -0,0 +1,362 @@ +/* + * 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.iceberg.flink.sink; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.types.Row; +import org.apache.flink.types.RowKind; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.IcebergGenerics; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.source.BoundedTestSource; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.StructLikeSet; +import org.assertj.core.api.Assertions; +import org.junit.Assert; + +public class TestFlinkIcebergSinkV2Base { + + protected static final int FORMAT_V2 = 2; + protected static final TypeInformation ROW_TYPE_INFO = + new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); + + protected static final int ROW_ID_POS = 0; + protected static final int ROW_DATA_POS = 1; + + protected int parallelism = 1; + protected TableLoader tableLoader; + protected Table table; + protected StreamExecutionEnvironment env; + protected FileFormat format; + protected boolean partitioned; + protected String writeDistributionMode; + + protected static final Map ROW_KIND_MAP = + ImmutableMap.of( + "+I", RowKind.INSERT, + "-D", RowKind.DELETE, + "-U", RowKind.UPDATE_BEFORE, + "+U", RowKind.UPDATE_AFTER); + + protected Row row(String rowKind, int id, String data) { + RowKind kind = ROW_KIND_MAP.get(rowKind); + if (kind == null) { + throw new IllegalArgumentException("Unknown row kind: " + rowKind); + } + + return Row.ofKind(kind, id, data); + } + + protected void testUpsertOnIdDataKey(String branch) throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of(row("+I", 1, "aaa"), row("+U", 1, "aaa"), row("+I", 2, "bbb")), + ImmutableList.of(row("+I", 1, "aaa"), row("-D", 2, "bbb"), row("+I", 2, "ccc")), + ImmutableList.of(row("+U", 1, "bbb"), row("-U", 1, "ccc"), row("-D", 1, "aaa"))); + + List> expectedRecords = + ImmutableList.of( + ImmutableList.of(record(1, "aaa"), record(2, "bbb")), + ImmutableList.of(record(1, "aaa"), record(2, "ccc")), + ImmutableList.of(record(1, "bbb"), record(2, "ccc"))); + testChangeLogs( + ImmutableList.of("id", "data"), + row -> Row.of(row.getField(ROW_ID_POS), row.getField(ROW_DATA_POS)), + true, + elementsPerCheckpoint, + expectedRecords, + branch); + } + + protected void testChangeLogOnIdDataKey(String branch) throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of( + row("+I", 1, "aaa"), + row("-D", 1, "aaa"), + row("+I", 2, "bbb"), + row("+I", 1, "bbb"), + row("+I", 2, "aaa")), + ImmutableList.of(row("-U", 2, "aaa"), row("+U", 1, "ccc"), row("+I", 1, "aaa")), + ImmutableList.of(row("-D", 1, "bbb"), row("+I", 2, "aaa"))); + + List> expectedRecords = + ImmutableList.of( + ImmutableList.of(record(1, "bbb"), record(2, "aaa"), record(2, "bbb")), + ImmutableList.of( + record(1, "aaa"), record(1, "bbb"), record(1, "ccc"), record(2, "bbb")), + ImmutableList.of( + record(1, "aaa"), record(1, "ccc"), record(2, "aaa"), record(2, "bbb"))); + + testChangeLogs( + ImmutableList.of("data", "id"), + row -> Row.of(row.getField(ROW_ID_POS), row.getField(ROW_DATA_POS)), + false, + elementsPerCheckpoint, + expectedRecords, + branch); + } + + protected void testChangeLogOnSameKey(String branch) throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + // Checkpoint #1 + ImmutableList.of(row("+I", 1, "aaa"), row("-D", 1, "aaa"), row("+I", 1, "aaa")), + // Checkpoint #2 + ImmutableList.of(row("-U", 1, "aaa"), row("+U", 1, "aaa")), + // Checkpoint #3 + ImmutableList.of(row("-D", 1, "aaa"), row("+I", 1, "aaa")), + // Checkpoint #4 + ImmutableList.of(row("-U", 1, "aaa"), row("+U", 1, "aaa"), row("+I", 1, "aaa"))); + + List> expectedRecords = + ImmutableList.of( + ImmutableList.of(record(1, "aaa")), + ImmutableList.of(record(1, "aaa")), + ImmutableList.of(record(1, "aaa")), + ImmutableList.of(record(1, "aaa"), record(1, "aaa"))); + + testChangeLogs( + ImmutableList.of("id", "data"), + row -> Row.of(row.getField(ROW_ID_POS), row.getField(ROW_DATA_POS)), + false, + elementsPerCheckpoint, + expectedRecords, + branch); + } + + protected void testChangeLogOnDataKey(String branch) throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of( + row("+I", 1, "aaa"), + row("-D", 1, "aaa"), + row("+I", 2, "bbb"), + row("+I", 1, "bbb"), + row("+I", 2, "aaa")), + ImmutableList.of(row("-U", 2, "aaa"), row("+U", 1, "ccc"), row("+I", 1, "aaa")), + ImmutableList.of(row("-D", 1, "bbb"), row("+I", 2, "aaa"), row("+I", 2, "ccc"))); + + List> expectedRecords = + ImmutableList.of( + ImmutableList.of(record(1, "bbb"), record(2, "aaa")), + ImmutableList.of(record(1, "aaa"), record(1, "bbb"), record(1, "ccc")), + ImmutableList.of( + record(1, "aaa"), record(1, "ccc"), record(2, "aaa"), record(2, "ccc"))); + + testChangeLogs( + ImmutableList.of("data"), + row -> row.getField(ROW_DATA_POS), + false, + elementsPerCheckpoint, + expectedRecords, + branch); + } + + protected void testUpsertOnDataKey(String branch) throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of(row("+I", 1, "aaa"), row("+I", 2, "aaa"), row("+I", 3, "bbb")), + ImmutableList.of(row("+U", 4, "aaa"), row("-U", 3, "bbb"), row("+U", 5, "bbb")), + ImmutableList.of(row("+I", 6, "aaa"), row("+U", 7, "bbb"))); + + List> expectedRecords = + ImmutableList.of( + ImmutableList.of(record(2, "aaa"), record(3, "bbb")), + ImmutableList.of(record(4, "aaa"), record(5, "bbb")), + ImmutableList.of(record(6, "aaa"), record(7, "bbb"))); + + testChangeLogs( + ImmutableList.of("data"), + row -> row.getField(ROW_DATA_POS), + true, + elementsPerCheckpoint, + expectedRecords, + branch); + } + + protected void testChangeLogOnIdKey(String branch) throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of( + row("+I", 1, "aaa"), + row("-D", 1, "aaa"), + row("+I", 1, "bbb"), + row("+I", 2, "aaa"), + row("-D", 2, "aaa"), + row("+I", 2, "bbb")), + ImmutableList.of( + row("-U", 2, "bbb"), row("+U", 2, "ccc"), row("-D", 2, "ccc"), row("+I", 2, "ddd")), + ImmutableList.of( + row("-D", 1, "bbb"), + row("+I", 1, "ccc"), + row("-D", 1, "ccc"), + row("+I", 1, "ddd"))); + + List> expectedRecords = + ImmutableList.of( + ImmutableList.of(record(1, "bbb"), record(2, "bbb")), + ImmutableList.of(record(1, "bbb"), record(2, "ddd")), + ImmutableList.of(record(1, "ddd"), record(2, "ddd"))); + + if (partitioned && writeDistributionMode.equals(TableProperties.WRITE_DISTRIBUTION_MODE_HASH)) { + Assertions.assertThatThrownBy( + () -> + testChangeLogs( + ImmutableList.of("id"), + row -> row.getField(ROW_ID_POS), + false, + elementsPerCheckpoint, + expectedRecords, + branch)) + .isInstanceOf(IllegalStateException.class) + .hasMessageStartingWith( + "In 'hash' distribution mode with equality fields set, partition field") + .hasMessageContaining("should be included in equality fields:"); + + } else { + testChangeLogs( + ImmutableList.of("id"), + row -> row.getField(ROW_ID_POS), + false, + elementsPerCheckpoint, + expectedRecords, + branch); + } + } + + protected void testUpsertOnIdKey(String branch) throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of(row("+I", 1, "aaa"), row("+U", 1, "bbb")), + ImmutableList.of(row("+I", 1, "ccc")), + ImmutableList.of(row("+U", 1, "ddd"), row("+I", 1, "eee"))); + + List> expectedRecords = + ImmutableList.of( + ImmutableList.of(record(1, "bbb")), + ImmutableList.of(record(1, "ccc")), + ImmutableList.of(record(1, "eee"))); + + if (!partitioned) { + testChangeLogs( + ImmutableList.of("id"), + row -> row.getField(ROW_ID_POS), + true, + elementsPerCheckpoint, + expectedRecords, + branch); + } else { + Assertions.assertThatThrownBy( + () -> + testChangeLogs( + ImmutableList.of("id"), + row -> row.getField(ROW_ID_POS), + true, + elementsPerCheckpoint, + expectedRecords, + branch)) + .isInstanceOf(IllegalStateException.class) + .hasMessageContaining("should be included in equality fields:"); + } + } + + protected void testChangeLogs( + List equalityFieldColumns, + KeySelector keySelector, + boolean insertAsUpsert, + List> elementsPerCheckpoint, + List> expectedRecordsPerCheckpoint, + String branch) + throws Exception { + DataStream dataStream = + env.addSource(new BoundedTestSource<>(elementsPerCheckpoint), ROW_TYPE_INFO); + + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .tableLoader(tableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .equalityFieldColumns(equalityFieldColumns) + .upsert(insertAsUpsert) + .toBranch(branch) + .append(); + + // Execute the program. + env.execute("Test Iceberg Change-Log DataStream."); + + table.refresh(); + List snapshots = findValidSnapshots(); + int expectedSnapshotNum = expectedRecordsPerCheckpoint.size(); + Assert.assertEquals( + "Should have the expected snapshot number", expectedSnapshotNum, snapshots.size()); + + for (int i = 0; i < expectedSnapshotNum; i++) { + long snapshotId = snapshots.get(i).snapshotId(); + List expectedRecords = expectedRecordsPerCheckpoint.get(i); + Assert.assertEquals( + "Should have the expected records for the checkpoint#" + i, + expectedRowSet(expectedRecords.toArray(new Record[0])), + actualRowSet(snapshotId, "*")); + } + } + + protected Record record(int id, String data) { + return SimpleDataUtil.createRecord(id, data); + } + + private List findValidSnapshots() { + List validSnapshots = Lists.newArrayList(); + for (Snapshot snapshot : table.snapshots()) { + if (snapshot.allManifests(table.io()).stream() + .anyMatch(m -> snapshot.snapshotId() == m.snapshotId())) { + validSnapshots.add(snapshot); + } + } + return validSnapshots; + } + + private StructLikeSet expectedRowSet(Record... records) { + return SimpleDataUtil.expectedRowSet(table, records); + } + + private StructLikeSet actualRowSet(long snapshotId, String... columns) throws IOException { + table.refresh(); + StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); + try (CloseableIterable reader = + IcebergGenerics.read(table).useSnapshot(snapshotId).select(columns).build()) { + reader.forEach(set::add); + } + return set; + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java new file mode 100644 index 000000000000..fed333848279 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java @@ -0,0 +1,134 @@ +/* + * 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.iceberg.flink.sink; + +import java.io.IOException; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.HadoopCatalogResource; +import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.Assert; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestFlinkIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Base { + + @ClassRule + public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = + MiniClusterResource.createWithClassloaderCheckDisabled(); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final HadoopCatalogResource catalogResource = + new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); + + private final String branch; + + @Parameterized.Parameters(name = "branch = {0}") + public static Object[] parameters() { + return new Object[] {"main", "testBranch"}; + } + + public TestFlinkIcebergSinkV2Branch(String branch) { + this.branch = branch; + } + + @Before + public void before() throws IOException { + table = + catalogResource + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, + FileFormat.AVRO.name(), + TableProperties.FORMAT_VERSION, + "2")); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100); + + tableLoader = catalogResource.tableLoader(); + } + + @Test + public void testChangeLogOnIdKey() throws Exception { + testChangeLogOnIdKey(branch); + verifyOtherBranchUnmodified(); + } + + @Test + public void testChangeLogOnDataKey() throws Exception { + testChangeLogOnDataKey(branch); + verifyOtherBranchUnmodified(); + } + + @Test + public void testChangeLogOnIdDataKey() throws Exception { + testChangeLogOnIdDataKey(branch); + verifyOtherBranchUnmodified(); + } + + @Test + public void testUpsertOnIdKey() throws Exception { + testUpsertOnIdKey(branch); + verifyOtherBranchUnmodified(); + } + + @Test + public void testUpsertOnDataKey() throws Exception { + testUpsertOnDataKey(branch); + verifyOtherBranchUnmodified(); + } + + @Test + public void testUpsertOnIdDataKey() throws Exception { + testUpsertOnIdDataKey(branch); + verifyOtherBranchUnmodified(); + } + + private void verifyOtherBranchUnmodified() { + String otherBranch = + branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH; + if (otherBranch.equals(SnapshotRef.MAIN_BRANCH)) { + Assert.assertNull(table.currentSnapshot()); + } + + Assert.assertTrue(table.snapshot(otherBranch) == null); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java new file mode 100644 index 000000000000..ce1f208a4b07 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java @@ -0,0 +1,315 @@ +/* + * 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.iceberg.flink.sink; + +import static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Paths; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.flink.core.io.SimpleVersionedSerialization; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.table.data.RowData; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.Pair; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestFlinkManifest { + private static final Configuration CONF = new Configuration(); + + @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); + + private Table table; + private FileAppenderFactory appenderFactory; + private final AtomicInteger fileCount = new AtomicInteger(0); + + @Before + public void before() throws IOException { + File folder = tempFolder.newFolder(); + String warehouse = folder.getAbsolutePath(); + + String tablePath = warehouse.concat("/test"); + Assert.assertTrue("Should create the table directory correctly.", new File(tablePath).mkdir()); + + // Construct the iceberg table. + table = SimpleDataUtil.createTable(tablePath, ImmutableMap.of(), false); + + int[] equalityFieldIds = + new int[] { + table.schema().findField("id").fieldId(), table.schema().findField("data").fieldId() + }; + this.appenderFactory = + new FlinkAppenderFactory( + table, + table.schema(), + FlinkSchemaUtil.convert(table.schema()), + table.properties(), + table.spec(), + equalityFieldIds, + table.schema(), + null); + } + + @Test + public void testIO() throws IOException { + String flinkJobId = newFlinkJobId(); + String operatorId = newOperatorUniqueId(); + for (long checkpointId = 1; checkpointId <= 3; checkpointId++) { + ManifestOutputFileFactory factory = + FlinkManifestUtil.createOutputFileFactory( + () -> table, table.properties(), flinkJobId, operatorId, 1, 1); + final long curCkpId = checkpointId; + + List dataFiles = generateDataFiles(10); + List eqDeleteFiles = generateEqDeleteFiles(5); + List posDeleteFiles = generatePosDeleteFiles(5); + DeltaManifests deltaManifests = + FlinkManifestUtil.writeCompletedFiles( + WriteResult.builder() + .addDataFiles(dataFiles) + .addDeleteFiles(eqDeleteFiles) + .addDeleteFiles(posDeleteFiles) + .build(), + () -> factory.create(curCkpId), + table.spec()); + + WriteResult result = + FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs()); + Assert.assertEquals("Size of data file list are not equal.", 10, result.deleteFiles().length); + for (int i = 0; i < dataFiles.size(); i++) { + TestHelpers.assertEquals(dataFiles.get(i), result.dataFiles()[i]); + } + Assert.assertEquals("Size of delete file list are not equal.", 10, result.dataFiles().length); + for (int i = 0; i < 5; i++) { + TestHelpers.assertEquals(eqDeleteFiles.get(i), result.deleteFiles()[i]); + } + for (int i = 0; i < 5; i++) { + TestHelpers.assertEquals(posDeleteFiles.get(i), result.deleteFiles()[5 + i]); + } + } + } + + @Test + public void testUserProvidedManifestLocation() throws IOException { + long checkpointId = 1; + String flinkJobId = newFlinkJobId(); + String operatorId = newOperatorUniqueId(); + File userProvidedFolder = tempFolder.newFolder(); + Map props = + ImmutableMap.of(FLINK_MANIFEST_LOCATION, userProvidedFolder.getAbsolutePath() + "///"); + ManifestOutputFileFactory factory = + new ManifestOutputFileFactory(() -> table, props, flinkJobId, operatorId, 1, 1); + + List dataFiles = generateDataFiles(5); + DeltaManifests deltaManifests = + FlinkManifestUtil.writeCompletedFiles( + WriteResult.builder().addDataFiles(dataFiles).build(), + () -> factory.create(checkpointId), + table.spec()); + + Assert.assertNotNull("Data manifest shouldn't be null", deltaManifests.dataManifest()); + Assert.assertNull("Delete manifest should be null", deltaManifests.deleteManifest()); + Assert.assertEquals( + "The newly created manifest file should be located under the user provided directory", + userProvidedFolder.toPath(), + Paths.get(deltaManifests.dataManifest().path()).getParent()); + + WriteResult result = + FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs()); + + Assert.assertEquals(0, result.deleteFiles().length); + Assert.assertEquals(5, result.dataFiles().length); + + Assert.assertEquals( + "Size of data file list are not equal.", dataFiles.size(), result.dataFiles().length); + for (int i = 0; i < dataFiles.size(); i++) { + TestHelpers.assertEquals(dataFiles.get(i), result.dataFiles()[i]); + } + } + + @Test + public void testVersionedSerializer() throws IOException { + long checkpointId = 1; + String flinkJobId = newFlinkJobId(); + String operatorId = newOperatorUniqueId(); + ManifestOutputFileFactory factory = + FlinkManifestUtil.createOutputFileFactory( + () -> table, table.properties(), flinkJobId, operatorId, 1, 1); + + List dataFiles = generateDataFiles(10); + List eqDeleteFiles = generateEqDeleteFiles(10); + List posDeleteFiles = generatePosDeleteFiles(10); + DeltaManifests expected = + FlinkManifestUtil.writeCompletedFiles( + WriteResult.builder() + .addDataFiles(dataFiles) + .addDeleteFiles(eqDeleteFiles) + .addDeleteFiles(posDeleteFiles) + .build(), + () -> factory.create(checkpointId), + table.spec()); + + byte[] versionedSerializeData = + SimpleVersionedSerialization.writeVersionAndSerialize( + DeltaManifestsSerializer.INSTANCE, expected); + DeltaManifests actual = + SimpleVersionedSerialization.readVersionAndDeSerialize( + DeltaManifestsSerializer.INSTANCE, versionedSerializeData); + TestHelpers.assertEquals(expected.dataManifest(), actual.dataManifest()); + TestHelpers.assertEquals(expected.deleteManifest(), actual.deleteManifest()); + + byte[] versionedSerializeData2 = + SimpleVersionedSerialization.writeVersionAndSerialize( + DeltaManifestsSerializer.INSTANCE, actual); + Assert.assertArrayEquals(versionedSerializeData, versionedSerializeData2); + } + + @Test + public void testCompatibility() throws IOException { + // The v2 deserializer should be able to deserialize the v1 binary. + long checkpointId = 1; + String flinkJobId = newFlinkJobId(); + String operatorId = newOperatorUniqueId(); + ManifestOutputFileFactory factory = + FlinkManifestUtil.createOutputFileFactory( + () -> table, table.properties(), flinkJobId, operatorId, 1, 1); + + List dataFiles = generateDataFiles(10); + ManifestFile manifest = + FlinkManifestUtil.writeDataFiles(factory.create(checkpointId), table.spec(), dataFiles); + byte[] dataV1 = + SimpleVersionedSerialization.writeVersionAndSerialize(new V1Serializer(), manifest); + + DeltaManifests delta = + SimpleVersionedSerialization.readVersionAndDeSerialize( + DeltaManifestsSerializer.INSTANCE, dataV1); + Assert.assertNull("Serialization v1 don't include delete files.", delta.deleteManifest()); + Assert.assertNotNull( + "Serialization v1 should not have null data manifest.", delta.dataManifest()); + TestHelpers.assertEquals(manifest, delta.dataManifest()); + + List actualFiles = + FlinkManifestUtil.readDataFiles(delta.dataManifest(), table.io(), table.specs()); + Assert.assertEquals(10, actualFiles.size()); + for (int i = 0; i < 10; i++) { + TestHelpers.assertEquals(dataFiles.get(i), actualFiles.get(i)); + } + } + + private static class V1Serializer implements SimpleVersionedSerializer { + + @Override + public int getVersion() { + return 1; + } + + @Override + public byte[] serialize(ManifestFile m) throws IOException { + return ManifestFiles.encode(m); + } + + @Override + public ManifestFile deserialize(int version, byte[] serialized) throws IOException { + return ManifestFiles.decode(serialized); + } + } + + private DataFile writeDataFile(String filename, List rows) throws IOException { + return SimpleDataUtil.writeFile( + table, + table.schema(), + table.spec(), + CONF, + table.location(), + FileFormat.PARQUET.addExtension(filename), + rows); + } + + private DeleteFile writeEqDeleteFile(String filename, List deletes) throws IOException { + return SimpleDataUtil.writeEqDeleteFile( + table, FileFormat.PARQUET, filename, appenderFactory, deletes); + } + + private DeleteFile writePosDeleteFile(String filename, List> positions) + throws IOException { + return SimpleDataUtil.writePosDeleteFile( + table, FileFormat.PARQUET, filename, appenderFactory, positions); + } + + private List generateDataFiles(int fileNum) throws IOException { + List rowDataList = Lists.newArrayList(); + List dataFiles = Lists.newArrayList(); + for (int i = 0; i < fileNum; i++) { + rowDataList.add(SimpleDataUtil.createRowData(i, "a" + i)); + dataFiles.add(writeDataFile("data-file-" + fileCount.incrementAndGet(), rowDataList)); + } + return dataFiles; + } + + private List generateEqDeleteFiles(int fileNum) throws IOException { + List rowDataList = Lists.newArrayList(); + List deleteFiles = Lists.newArrayList(); + for (int i = 0; i < fileNum; i++) { + rowDataList.add(SimpleDataUtil.createDelete(i, "a" + i)); + deleteFiles.add( + writeEqDeleteFile("eq-delete-file-" + fileCount.incrementAndGet(), rowDataList)); + } + return deleteFiles; + } + + private List generatePosDeleteFiles(int fileNum) throws IOException { + List> positions = Lists.newArrayList(); + List deleteFiles = Lists.newArrayList(); + for (int i = 0; i < fileNum; i++) { + positions.add(Pair.of("data-file-1", (long) i)); + deleteFiles.add( + writePosDeleteFile("pos-delete-file-" + fileCount.incrementAndGet(), positions)); + } + return deleteFiles; + } + + private static String newFlinkJobId() { + return UUID.randomUUID().toString(); + } + + private static String newOperatorUniqueId() { + return UUID.randomUUID().toString(); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java new file mode 100644 index 000000000000..3951c2e70f65 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.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.iceberg.flink.sink; + +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.io.FileWriterFactory; +import org.apache.iceberg.io.TestPartitioningWriters; +import org.apache.iceberg.util.ArrayUtil; +import org.apache.iceberg.util.StructLikeSet; + +public class TestFlinkPartitioningWriters extends TestPartitioningWriters { + + public TestFlinkPartitioningWriters(FileFormat fileFormat) { + super(fileFormat); + } + + @Override + protected FileWriterFactory newWriterFactory( + Schema dataSchema, + List equalityFieldIds, + Schema equalityDeleteRowSchema, + Schema positionDeleteRowSchema) { + return FlinkFileWriterFactory.builderFor(table) + .dataSchema(table.schema()) + .dataFileFormat(format()) + .deleteFileFormat(format()) + .equalityFieldIds(ArrayUtil.toIntArray(equalityFieldIds)) + .equalityDeleteRowSchema(equalityDeleteRowSchema) + .positionDeleteRowSchema(positionDeleteRowSchema) + .build(); + } + + @Override + protected RowData toRow(Integer id, String data) { + return SimpleDataUtil.createRowData(id, data); + } + + @Override + protected StructLikeSet toSet(Iterable rows) { + StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); + RowType flinkType = FlinkSchemaUtil.convert(table.schema()); + for (RowData row : rows) { + RowDataWrapper wrapper = new RowDataWrapper(flinkType, table.schema().asStruct()); + set.add(wrapper.wrap(row)); + } + return set; + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java new file mode 100644 index 000000000000..9e846efe6fc9 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.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.iceberg.flink.sink; + +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.io.FileWriterFactory; +import org.apache.iceberg.io.TestPositionDeltaWriters; +import org.apache.iceberg.util.ArrayUtil; +import org.apache.iceberg.util.StructLikeSet; + +public class TestFlinkPositionDeltaWriters extends TestPositionDeltaWriters { + + public TestFlinkPositionDeltaWriters(FileFormat fileFormat) { + super(fileFormat); + } + + @Override + protected FileWriterFactory newWriterFactory( + Schema dataSchema, + List equalityFieldIds, + Schema equalityDeleteRowSchema, + Schema positionDeleteRowSchema) { + return FlinkFileWriterFactory.builderFor(table) + .dataSchema(table.schema()) + .dataFileFormat(format()) + .deleteFileFormat(format()) + .equalityFieldIds(ArrayUtil.toIntArray(equalityFieldIds)) + .equalityDeleteRowSchema(equalityDeleteRowSchema) + .positionDeleteRowSchema(positionDeleteRowSchema) + .build(); + } + + @Override + protected RowData toRow(Integer id, String data) { + return SimpleDataUtil.createRowData(id, data); + } + + @Override + protected StructLikeSet toSet(Iterable rows) { + StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); + RowType flinkType = FlinkSchemaUtil.convert(table.schema()); + for (RowData row : rows) { + RowDataWrapper wrapper = new RowDataWrapper(flinkType, table.schema().asStruct()); + set.add(wrapper.wrap(row)); + } + return set; + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java new file mode 100644 index 000000000000..07716b9c3e60 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.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.iceberg.flink.sink; + +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.io.FileWriterFactory; +import org.apache.iceberg.io.TestRollingFileWriters; +import org.apache.iceberg.util.ArrayUtil; + +public class TestFlinkRollingFileWriters extends TestRollingFileWriters { + + public TestFlinkRollingFileWriters(FileFormat fileFormat, boolean partitioned) { + super(fileFormat, partitioned); + } + + @Override + protected FileWriterFactory newWriterFactory( + Schema dataSchema, + List equalityFieldIds, + Schema equalityDeleteRowSchema, + Schema positionDeleteRowSchema) { + return FlinkFileWriterFactory.builderFor(table) + .dataSchema(table.schema()) + .dataFileFormat(format()) + .deleteFileFormat(format()) + .equalityFieldIds(ArrayUtil.toIntArray(equalityFieldIds)) + .equalityDeleteRowSchema(equalityDeleteRowSchema) + .positionDeleteRowSchema(positionDeleteRowSchema) + .build(); + } + + @Override + protected RowData toRow(Integer id, String data) { + return SimpleDataUtil.createRowData(id, data); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java new file mode 100644 index 000000000000..e6d64ef2c720 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java @@ -0,0 +1,60 @@ +/* + * 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.iceberg.flink.sink; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.io.FileWriterFactory; +import org.apache.iceberg.io.TestWriterMetrics; + +public class TestFlinkWriterMetrics extends TestWriterMetrics { + + public TestFlinkWriterMetrics(FileFormat fileFormat) { + super(fileFormat); + } + + @Override + protected FileWriterFactory newWriterFactory(Table sourceTable) { + return FlinkFileWriterFactory.builderFor(sourceTable) + .dataSchema(sourceTable.schema()) + .dataFileFormat(fileFormat) + .deleteFileFormat(fileFormat) + .positionDeleteRowSchema(sourceTable.schema()) + .build(); + } + + @Override + protected RowData toRow(Integer id, String data, boolean boolValue, Long longValue) { + GenericRowData nested = GenericRowData.of(boolValue, longValue); + GenericRowData row = GenericRowData.of(id, StringData.fromString(data), nested); + return row; + } + + @Override + public RowData toGenericRow(int value, int repeated) { + GenericRowData row = new GenericRowData(repeated); + for (int i = 0; i < repeated; i++) { + row.setField(i, value); + } + return row; + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java new file mode 100644 index 000000000000..a2fe092b7cf6 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java @@ -0,0 +1,1152 @@ +/* + * 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.iceberg.flink.sink; + +import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; +import static org.apache.iceberg.flink.sink.IcebergFilesCommitter.MAX_CONTINUOUS_EMPTY_COMMITS; +import static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Collections; +import java.util.List; +import java.util.NavigableMap; +import java.util.SortedMap; +import java.util.stream.Collectors; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.OperatorStateStore; +import org.apache.flink.core.io.SimpleVersionedSerialization; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.testutils.MockEnvironment; +import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; +import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.BoundedOneInput; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.table.data.RowData; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.GenericManifestFile; +import org.apache.iceberg.ManifestContent; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.PartitionData; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.flink.TestTableLoader; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.Pair; +import org.apache.iceberg.util.ThreadPools; +import org.junit.Assert; +import org.junit.Assume; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestIcebergFilesCommitter extends TableTestBase { + private static final Configuration CONF = new Configuration(); + + private File flinkManifestFolder; + + private final FileFormat format; + private final String branch; + + @Parameterized.Parameters(name = "FileFormat = {0}, FormatVersion = {1}, branch = {2}") + public static Object[][] parameters() { + return new Object[][] { + new Object[] {"avro", 1, "main"}, + new Object[] {"avro", 2, "test-branch"}, + new Object[] {"parquet", 1, "main"}, + new Object[] {"parquet", 2, "test-branch"}, + new Object[] {"orc", 1, "main"}, + new Object[] {"orc", 2, "test-branch"} + }; + } + + public TestIcebergFilesCommitter(String format, int formatVersion, String branch) { + super(formatVersion); + this.format = FileFormat.fromString(format); + this.branch = branch; + } + + @Override + @Before + public void setupTable() throws IOException { + flinkManifestFolder = temp.newFolder(); + + this.tableDir = temp.newFolder(); + this.metadataDir = new File(tableDir, "metadata"); + Assert.assertTrue(tableDir.delete()); + + // Construct the iceberg table. + table = create(SimpleDataUtil.SCHEMA, PartitionSpec.unpartitioned()); + + table + .updateProperties() + .set(DEFAULT_FILE_FORMAT, format.name()) + .set(FLINK_MANIFEST_LOCATION, flinkManifestFolder.getAbsolutePath()) + .set(MAX_CONTINUOUS_EMPTY_COMMITS, "1") + .commit(); + } + + @Test + public void testCommitTxnWithoutDataFiles() throws Exception { + long checkpointId = 0; + long timestamp = 0; + JobID jobId = new JobID(); + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + SimpleDataUtil.assertTableRows(table, Lists.newArrayList(), branch); + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + // It's better to advance the max-committed-checkpoint-id in iceberg snapshot, so that the + // future flink job + // failover won't fail. + for (int i = 1; i <= 3; i++) { + harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(0); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + assertSnapshotSize(i); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); + } + } + } + + @Test + public void testMaxContinuousEmptyCommits() throws Exception { + table.updateProperties().set(MAX_CONTINUOUS_EMPTY_COMMITS, "3").commit(); + + JobID jobId = new JobID(); + long checkpointId = 0; + long timestamp = 0; + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.setup(); + harness.open(); + + assertSnapshotSize(0); + + for (int i = 1; i <= 9; i++) { + harness.snapshot(++checkpointId, ++timestamp); + harness.notifyOfCompletedCheckpoint(checkpointId); + + assertSnapshotSize(i / 3); + } + } + } + + private WriteResult of(DataFile dataFile) { + return WriteResult.builder().addDataFiles(dataFile).build(); + } + + @Test + public void testCommitTxn() throws Exception { + // Test with 3 continues checkpoints: + // 1. snapshotState for checkpoint#1 + // 2. notifyCheckpointComplete for checkpoint#1 + // 3. snapshotState for checkpoint#2 + // 4. notifyCheckpointComplete for checkpoint#2 + // 5. snapshotState for checkpoint#3 + // 6. notifyCheckpointComplete for checkpoint#3 + long timestamp = 0; + + JobID jobID = new JobID(); + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobID)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(0); + + List rows = Lists.newArrayListWithExpectedSize(3); + for (int i = 1; i <= 3; i++) { + RowData rowData = SimpleDataUtil.createRowData(i, "hello" + i); + DataFile dataFile = writeDataFile("data-" + i, ImmutableList.of(rowData)); + harness.processElement(of(dataFile), ++timestamp); + rows.add(rowData); + + harness.snapshot(i, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(i); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, ImmutableList.copyOf(rows), branch); + assertSnapshotSize(i); + assertMaxCommittedCheckpointId(jobID, operatorId, i); + Assert.assertEquals( + TestIcebergFilesCommitter.class.getName(), + SimpleDataUtil.latestSnapshot(table, branch).summary().get("flink.test")); + } + } + } + + @Test + public void testOrderedEventsBetweenCheckpoints() throws Exception { + // It's possible that two checkpoints happen in the following orders: + // 1. snapshotState for checkpoint#1; + // 2. snapshotState for checkpoint#2; + // 3. notifyCheckpointComplete for checkpoint#1; + // 4. notifyCheckpointComplete for checkpoint#2; + long timestamp = 0; + + JobID jobId = new JobID(); + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello"); + DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); + + harness.processElement(of(dataFile1), ++timestamp); + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + // 1. snapshotState for checkpoint#1 + long firstCheckpointId = 1; + harness.snapshot(firstCheckpointId, ++timestamp); + assertFlinkManifests(1); + + RowData row2 = SimpleDataUtil.createRowData(2, "world"); + DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); + harness.processElement(of(dataFile2), ++timestamp); + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + // 2. snapshotState for checkpoint#2 + long secondCheckpointId = 2; + harness.snapshot(secondCheckpointId, ++timestamp); + assertFlinkManifests(2); + + // 3. notifyCheckpointComplete for checkpoint#1 + harness.notifyOfCompletedCheckpoint(firstCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, firstCheckpointId); + assertFlinkManifests(1); + + // 4. notifyCheckpointComplete for checkpoint#2 + harness.notifyOfCompletedCheckpoint(secondCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, secondCheckpointId); + assertFlinkManifests(0); + } + } + + @Test + public void testDisorderedEventsBetweenCheckpoints() throws Exception { + // It's possible that the two checkpoints happen in the following orders: + // 1. snapshotState for checkpoint#1; + // 2. snapshotState for checkpoint#2; + // 3. notifyCheckpointComplete for checkpoint#2; + // 4. notifyCheckpointComplete for checkpoint#1; + long timestamp = 0; + + JobID jobId = new JobID(); + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello"); + DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); + + harness.processElement(of(dataFile1), ++timestamp); + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + // 1. snapshotState for checkpoint#1 + long firstCheckpointId = 1; + harness.snapshot(firstCheckpointId, ++timestamp); + assertFlinkManifests(1); + + RowData row2 = SimpleDataUtil.createRowData(2, "world"); + DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); + harness.processElement(of(dataFile2), ++timestamp); + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + // 2. snapshotState for checkpoint#2 + long secondCheckpointId = 2; + harness.snapshot(secondCheckpointId, ++timestamp); + assertFlinkManifests(2); + + // 3. notifyCheckpointComplete for checkpoint#2 + harness.notifyOfCompletedCheckpoint(secondCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, secondCheckpointId); + assertFlinkManifests(0); + + // 4. notifyCheckpointComplete for checkpoint#1 + harness.notifyOfCompletedCheckpoint(firstCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, secondCheckpointId); + assertFlinkManifests(0); + } + } + + @Test + public void testRecoveryFromValidSnapshot() throws Exception { + long checkpointId = 0; + long timestamp = 0; + List expectedRows = Lists.newArrayList(); + OperatorSubtaskState snapshot; + + JobID jobId = new JobID(); + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + RowData row = SimpleDataUtil.createRowData(1, "hello"); + expectedRows.add(row); + DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row)); + + harness.processElement(of(dataFile1), ++timestamp); + snapshot = harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row), branch); + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); + } + + // Restore from the given snapshot + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.getStreamConfig().setOperatorID(operatorId); + harness.setup(); + harness.initializeState(snapshot); + harness.open(); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); + + RowData row = SimpleDataUtil.createRowData(2, "world"); + expectedRows.add(row); + DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); + harness.processElement(of(dataFile), ++timestamp); + + harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); + } + } + + @Test + public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Exception { + // We've two steps in checkpoint: 1. snapshotState(ckp); 2. notifyCheckpointComplete(ckp). It's + // possible that we + // flink job will restore from a checkpoint with only step#1 finished. + long checkpointId = 0; + long timestamp = 0; + OperatorSubtaskState snapshot; + List expectedRows = Lists.newArrayList(); + JobID jobId = new JobID(); + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + RowData row = SimpleDataUtil.createRowData(1, "hello"); + expectedRows.add(row); + DataFile dataFile = writeDataFile("data-1", ImmutableList.of(row)); + harness.processElement(of(dataFile), ++timestamp); + + snapshot = harness.snapshot(++checkpointId, ++timestamp); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + assertFlinkManifests(1); + } + + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.getStreamConfig().setOperatorID(operatorId); + harness.setup(); + harness.initializeState(snapshot); + harness.open(); + + // All flink manifests should be cleaned because it has committed the unfinished iceberg + // transaction. + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); + + harness.snapshot(++checkpointId, ++timestamp); + // Did not write any new record, so it won't generate new manifest. + assertFlinkManifests(0); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); + + RowData row = SimpleDataUtil.createRowData(2, "world"); + expectedRows.add(row); + DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); + harness.processElement(of(dataFile), ++timestamp); + + snapshot = harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(1); + } + + // Redeploying flink job from external checkpoint. + JobID newJobId = new JobID(); + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(newJobId)) { + harness.setup(); + harness.initializeState(snapshot); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + // All flink manifests should be cleaned because it has committed the unfinished iceberg + // transaction. + assertFlinkManifests(0); + + assertMaxCommittedCheckpointId(newJobId, operatorId, -1); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(3); + + RowData row = SimpleDataUtil.createRowData(3, "foo"); + expectedRows.add(row); + DataFile dataFile = writeDataFile("data-3", ImmutableList.of(row)); + harness.processElement(of(dataFile), ++timestamp); + + harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(4); + assertMaxCommittedCheckpointId(newJobId, operatorId, checkpointId); + } + } + + @Test + public void testStartAnotherJobToWriteSameTable() throws Exception { + long checkpointId = 0; + long timestamp = 0; + List rows = Lists.newArrayList(); + List tableRows = Lists.newArrayList(); + + JobID oldJobId = new JobID(); + OperatorID oldOperatorId; + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(oldJobId)) { + harness.setup(); + harness.open(); + oldOperatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(oldJobId, oldOperatorId, -1L); + + for (int i = 1; i <= 3; i++) { + rows.add(SimpleDataUtil.createRowData(i, "hello" + i)); + tableRows.addAll(rows); + + DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); + harness.processElement(of(dataFile), ++timestamp); + harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, tableRows, branch); + assertSnapshotSize(i); + assertMaxCommittedCheckpointId(oldJobId, oldOperatorId, checkpointId); + } + } + + // The new started job will start with checkpoint = 1 again. + checkpointId = 0; + timestamp = 0; + JobID newJobId = new JobID(); + OperatorID newOperatorId; + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(newJobId)) { + harness.setup(); + harness.open(); + newOperatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(3); + assertMaxCommittedCheckpointId(oldJobId, oldOperatorId, 3); + assertMaxCommittedCheckpointId(newJobId, newOperatorId, -1); + + rows.add(SimpleDataUtil.createRowData(2, "world")); + tableRows.addAll(rows); + + DataFile dataFile = writeDataFile("data-new-1", rows); + harness.processElement(of(dataFile), ++timestamp); + harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + SimpleDataUtil.assertTableRows(table, tableRows, branch); + assertSnapshotSize(4); + assertMaxCommittedCheckpointId(newJobId, newOperatorId, checkpointId); + } + } + + @Test + public void testMultipleJobsWriteSameTable() throws Exception { + long timestamp = 0; + List tableRows = Lists.newArrayList(); + + JobID[] jobs = new JobID[] {new JobID(), new JobID(), new JobID()}; + OperatorID[] operatorIds = + new OperatorID[] {new OperatorID(), new OperatorID(), new OperatorID()}; + for (int i = 0; i < 20; i++) { + int jobIndex = i % 3; + int checkpointId = i / 3; + JobID jobId = jobs[jobIndex]; + OperatorID operatorId = operatorIds[jobIndex]; + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.getStreamConfig().setOperatorID(operatorId); + harness.setup(); + harness.open(); + + assertSnapshotSize(i); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId == 0 ? -1 : checkpointId); + + List rows = Lists.newArrayList(SimpleDataUtil.createRowData(i, "word-" + i)); + tableRows.addAll(rows); + + DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); + harness.processElement(of(dataFile), ++timestamp); + harness.snapshot(checkpointId + 1, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId + 1); + assertFlinkManifests(0); + SimpleDataUtil.assertTableRows(table, tableRows, branch); + assertSnapshotSize(i + 1); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId + 1); + } + } + } + + @Test + public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { + long checkpointId = 0; + long timestamp = 0; + List expectedRows = Lists.newArrayList(); + OperatorSubtaskState snapshot1; + OperatorSubtaskState snapshot2; + + JobID jobId = new JobID(); + OperatorID operatorId1 = new OperatorID(); + OperatorID operatorId2 = new OperatorID(); + try (OneInputStreamOperatorTestHarness harness1 = createStreamSink(jobId); + OneInputStreamOperatorTestHarness harness2 = createStreamSink(jobId)) { + harness1.getStreamConfig().setOperatorID(operatorId1); + harness1.setup(); + harness1.open(); + harness2.getStreamConfig().setOperatorID(operatorId2); + harness2.setup(); + harness2.open(); + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId, operatorId1, -1L); + assertMaxCommittedCheckpointId(jobId, operatorId2, -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello1"); + expectedRows.add(row1); + DataFile dataFile1 = writeDataFile("data-1-1", ImmutableList.of(row1)); + + harness1.processElement(of(dataFile1), ++timestamp); + snapshot1 = harness1.snapshot(++checkpointId, ++timestamp); + + RowData row2 = SimpleDataUtil.createRowData(1, "hello2"); + expectedRows.add(row2); + DataFile dataFile2 = writeDataFile("data-1-2", ImmutableList.of(row2)); + + harness2.processElement(of(dataFile2), ++timestamp); + snapshot2 = harness2.snapshot(checkpointId, ++timestamp); + assertFlinkManifests(2); + + // Only notify one of the committers + harness1.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(1); + + // Only the first row is committed at this point + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobId, operatorId1, checkpointId); + assertMaxCommittedCheckpointId(jobId, operatorId2, -1); + } + + // Restore from the given snapshot + try (OneInputStreamOperatorTestHarness harness1 = createStreamSink(jobId); + OneInputStreamOperatorTestHarness harness2 = createStreamSink(jobId)) { + harness1.getStreamConfig().setOperatorID(operatorId1); + harness1.setup(); + harness1.initializeState(snapshot1); + harness1.open(); + + harness2.getStreamConfig().setOperatorID(operatorId2); + harness2.setup(); + harness2.initializeState(snapshot2); + harness2.open(); + + // All flink manifests should be cleaned because it has committed the unfinished iceberg + // transaction. + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId, operatorId1, checkpointId); + assertMaxCommittedCheckpointId(jobId, operatorId2, checkpointId); + + RowData row1 = SimpleDataUtil.createRowData(2, "world1"); + expectedRows.add(row1); + DataFile dataFile1 = writeDataFile("data-2-1", ImmutableList.of(row1)); + + harness1.processElement(of(dataFile1), ++timestamp); + harness1.snapshot(++checkpointId, ++timestamp); + + RowData row2 = SimpleDataUtil.createRowData(2, "world2"); + expectedRows.add(row2); + DataFile dataFile2 = writeDataFile("data-2-2", ImmutableList.of(row2)); + harness2.processElement(of(dataFile2), ++timestamp); + harness2.snapshot(checkpointId, ++timestamp); + + assertFlinkManifests(2); + + harness1.notifyOfCompletedCheckpoint(checkpointId); + harness2.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(4); + assertMaxCommittedCheckpointId(jobId, operatorId1, checkpointId); + assertMaxCommittedCheckpointId(jobId, operatorId2, checkpointId); + } + } + + @Test + public void testBoundedStream() throws Exception { + JobID jobId = new JobID(); + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertFlinkManifests(0); + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + List tableRows = Lists.newArrayList(SimpleDataUtil.createRowData(1, "word-1")); + + DataFile dataFile = writeDataFile("data-1", tableRows); + harness.processElement(of(dataFile), 1); + ((BoundedOneInput) harness.getOneInputOperator()).endInput(); + + assertFlinkManifests(0); + SimpleDataUtil.assertTableRows(table, tableRows, branch); + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobId, operatorId, Long.MAX_VALUE); + Assert.assertEquals( + TestIcebergFilesCommitter.class.getName(), + SimpleDataUtil.latestSnapshot(table, branch).summary().get("flink.test")); + } + } + + @Test + public void testFlinkManifests() throws Exception { + long timestamp = 0; + final long checkpoint = 10; + + JobID jobId = new JobID(); + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello"); + DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); + + harness.processElement(of(dataFile1), ++timestamp); + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + // 1. snapshotState for checkpoint#1 + harness.snapshot(checkpoint, ++timestamp); + List manifestPaths = assertFlinkManifests(1); + Path manifestPath = manifestPaths.get(0); + Assert.assertEquals( + "File name should have the expected pattern.", + String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1), + manifestPath.getFileName().toString()); + + // 2. Read the data files from manifests and assert. + List dataFiles = + FlinkManifestUtil.readDataFiles( + createTestingManifestFile(manifestPath), table.io(), table.specs()); + Assert.assertEquals(1, dataFiles.size()); + TestHelpers.assertEquals(dataFile1, dataFiles.get(0)); + + // 3. notifyCheckpointComplete for checkpoint#1 + harness.notifyOfCompletedCheckpoint(checkpoint); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); + assertFlinkManifests(0); + } + } + + @Test + public void testDeleteFiles() throws Exception { + Assume.assumeFalse("Only support equality-delete in format v2.", formatVersion < 2); + + long timestamp = 0; + long checkpoint = 10; + + JobID jobId = new JobID(); + OperatorID operatorId; + FileAppenderFactory appenderFactory = createDeletableAppenderFactory(); + + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + RowData row1 = SimpleDataUtil.createInsert(1, "aaa"); + DataFile dataFile1 = writeDataFile("data-file-1", ImmutableList.of(row1)); + harness.processElement(of(dataFile1), ++timestamp); + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + // 1. snapshotState for checkpoint#1 + harness.snapshot(checkpoint, ++timestamp); + List manifestPaths = assertFlinkManifests(1); + Path manifestPath = manifestPaths.get(0); + Assert.assertEquals( + "File name should have the expected pattern.", + String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1), + manifestPath.getFileName().toString()); + + // 2. Read the data files from manifests and assert. + List dataFiles = + FlinkManifestUtil.readDataFiles( + createTestingManifestFile(manifestPath), table.io(), table.specs()); + Assert.assertEquals(1, dataFiles.size()); + TestHelpers.assertEquals(dataFile1, dataFiles.get(0)); + + // 3. notifyCheckpointComplete for checkpoint#1 + harness.notifyOfCompletedCheckpoint(checkpoint); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); + assertFlinkManifests(0); + + // 4. process both data files and delete files. + RowData row2 = SimpleDataUtil.createInsert(2, "bbb"); + DataFile dataFile2 = writeDataFile("data-file-2", ImmutableList.of(row2)); + + RowData delete1 = SimpleDataUtil.createDelete(1, "aaa"); + DeleteFile deleteFile1 = + writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete1)); + harness.processElement( + WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile1).build(), + ++timestamp); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); + + // 5. snapshotState for checkpoint#2 + harness.snapshot(++checkpoint, ++timestamp); + assertFlinkManifests(2); + + // 6. notifyCheckpointComplete for checkpoint#2 + harness.notifyOfCompletedCheckpoint(checkpoint); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row2), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); + assertFlinkManifests(0); + } + } + + @Test + public void testCommitTwoCheckpointsInSingleTxn() throws Exception { + Assume.assumeFalse("Only support equality-delete in format v2.", formatVersion < 2); + + long timestamp = 0; + long checkpoint = 10; + + JobID jobId = new JobID(); + OperatorID operatorId; + FileAppenderFactory appenderFactory = createDeletableAppenderFactory(); + + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + RowData insert1 = SimpleDataUtil.createInsert(1, "aaa"); + RowData insert2 = SimpleDataUtil.createInsert(2, "bbb"); + RowData delete3 = SimpleDataUtil.createDelete(3, "ccc"); + DataFile dataFile1 = writeDataFile("data-file-1", ImmutableList.of(insert1, insert2)); + DeleteFile deleteFile1 = + writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete3)); + harness.processElement( + WriteResult.builder().addDataFiles(dataFile1).addDeleteFiles(deleteFile1).build(), + ++timestamp); + + // The 1th snapshotState. + harness.snapshot(checkpoint, ++timestamp); + + RowData insert4 = SimpleDataUtil.createInsert(4, "ddd"); + RowData delete2 = SimpleDataUtil.createDelete(2, "bbb"); + DataFile dataFile2 = writeDataFile("data-file-2", ImmutableList.of(insert4)); + DeleteFile deleteFile2 = + writeEqDeleteFile(appenderFactory, "delete-file-2", ImmutableList.of(delete2)); + harness.processElement( + WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile2).build(), + ++timestamp); + + // The 2nd snapshotState. + harness.snapshot(++checkpoint, ++timestamp); + + // Notify the 2nd snapshot to complete. + harness.notifyOfCompletedCheckpoint(checkpoint); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(insert1, insert4), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); + assertFlinkManifests(0); + Assert.assertEquals( + "Should have committed 2 txn.", 2, ImmutableList.copyOf(table.snapshots()).size()); + } + } + + @Test + public void testSpecEvolution() throws Exception { + long timestamp = 0; + int checkpointId = 0; + List rows = Lists.newArrayList(); + JobID jobId = new JobID(); + + OperatorID operatorId; + OperatorSubtaskState snapshot; + DataFile dataFile; + int specId; + + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(0); + + checkpointId++; + RowData rowData = SimpleDataUtil.createRowData(checkpointId, "hello" + checkpointId); + // table unpartitioned + dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(rowData)); + harness.processElement(of(dataFile), ++timestamp); + rows.add(rowData); + harness.snapshot(checkpointId, ++timestamp); + + specId = + getStagingManifestSpecId(harness.getOperator().getOperatorStateBackend(), checkpointId); + assertThat(specId).isEqualTo(table.spec().specId()); + + harness.notifyOfCompletedCheckpoint(checkpointId); + + // Change partition spec + table.refresh(); + PartitionSpec oldSpec = table.spec(); + table.updateSpec().addField("id").commit(); + + checkpointId++; + rowData = SimpleDataUtil.createRowData(checkpointId, "hello" + checkpointId); + // write data with old partition spec + dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(rowData), oldSpec, null); + harness.processElement(of(dataFile), ++timestamp); + rows.add(rowData); + snapshot = harness.snapshot(checkpointId, ++timestamp); + + specId = + getStagingManifestSpecId(harness.getOperator().getOperatorStateBackend(), checkpointId); + assertThat(specId).isEqualTo(oldSpec.specId()); + + harness.notifyOfCompletedCheckpoint(checkpointId); + + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, ImmutableList.copyOf(rows), branch); + assertSnapshotSize(checkpointId); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); + } + + // Restore from the given snapshot + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.getStreamConfig().setOperatorID(operatorId); + harness.setup(); + harness.initializeState(snapshot); + harness.open(); + + SimpleDataUtil.assertTableRows(table, rows, branch); + assertSnapshotSize(checkpointId); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); + + checkpointId++; + RowData row = SimpleDataUtil.createRowData(checkpointId, "world" + checkpointId); + StructLike partition = new PartitionData(table.spec().partitionType()); + partition.set(0, checkpointId); + dataFile = + writeDataFile("data-" + checkpointId, ImmutableList.of(row), table.spec(), partition); + harness.processElement(of(dataFile), ++timestamp); + rows.add(row); + harness.snapshot(checkpointId, ++timestamp); + assertFlinkManifests(1); + + specId = + getStagingManifestSpecId(harness.getOperator().getOperatorStateBackend(), checkpointId); + assertThat(specId).isEqualTo(table.spec().specId()); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, rows, branch); + assertSnapshotSize(checkpointId); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); + } + } + + private int getStagingManifestSpecId(OperatorStateStore operatorStateStore, long checkPointId) + throws Exception { + ListState> checkpointsState = + operatorStateStore.getListState(IcebergFilesCommitter.buildStateDescriptor()); + NavigableMap statedDataFiles = + Maps.newTreeMap(checkpointsState.get().iterator().next()); + DeltaManifests deltaManifests = + SimpleVersionedSerialization.readVersionAndDeSerialize( + DeltaManifestsSerializer.INSTANCE, statedDataFiles.get(checkPointId)); + return deltaManifests.dataManifest().partitionSpecId(); + } + + private DeleteFile writeEqDeleteFile( + FileAppenderFactory appenderFactory, String filename, List deletes) + throws IOException { + return SimpleDataUtil.writeEqDeleteFile(table, format, filename, appenderFactory, deletes); + } + + private DeleteFile writePosDeleteFile( + FileAppenderFactory appenderFactory, + String filename, + List> positions) + throws IOException { + return SimpleDataUtil.writePosDeleteFile(table, format, filename, appenderFactory, positions); + } + + private FileAppenderFactory createDeletableAppenderFactory() { + int[] equalityFieldIds = + new int[] { + table.schema().findField("id").fieldId(), table.schema().findField("data").fieldId() + }; + return new FlinkAppenderFactory( + table, + table.schema(), + FlinkSchemaUtil.convert(table.schema()), + table.properties(), + table.spec(), + equalityFieldIds, + table.schema(), + null); + } + + private ManifestFile createTestingManifestFile(Path manifestPath) { + return new GenericManifestFile( + manifestPath.toAbsolutePath().toString(), + manifestPath.toFile().length(), + 0, + ManifestContent.DATA, + 0, + 0, + 0L, + 0, + 0, + 0, + 0, + 0, + 0, + null, + null); + } + + private List assertFlinkManifests(int expectedCount) throws IOException { + List manifests = + Files.list(flinkManifestFolder.toPath()) + .filter(p -> !p.toString().endsWith(".crc")) + .collect(Collectors.toList()); + Assert.assertEquals( + String.format("Expected %s flink manifests, but the list is: %s", expectedCount, manifests), + expectedCount, + manifests.size()); + return manifests; + } + + private DataFile writeDataFile(String filename, List rows) throws IOException { + return SimpleDataUtil.writeFile( + table, + table.schema(), + table.spec(), + CONF, + table.location(), + format.addExtension(filename), + rows); + } + + private DataFile writeDataFile( + String filename, List rows, PartitionSpec spec, StructLike partition) + throws IOException { + return SimpleDataUtil.writeFile( + table, + table.schema(), + spec, + CONF, + table.location(), + format.addExtension(filename), + rows, + partition); + } + + private void assertMaxCommittedCheckpointId(JobID jobID, OperatorID operatorID, long expectedId) { + table.refresh(); + long actualId = + IcebergFilesCommitter.getMaxCommittedCheckpointId( + table, jobID.toString(), operatorID.toHexString(), branch); + Assert.assertEquals(expectedId, actualId); + } + + private void assertSnapshotSize(int expectedSnapshotSize) { + table.refresh(); + Assert.assertEquals(expectedSnapshotSize, Lists.newArrayList(table.snapshots()).size()); + } + + private OneInputStreamOperatorTestHarness createStreamSink(JobID jobID) + throws Exception { + TestOperatorFactory factory = TestOperatorFactory.of(table.location(), branch, table.spec()); + return new OneInputStreamOperatorTestHarness<>(factory, createEnvironment(jobID)); + } + + private static MockEnvironment createEnvironment(JobID jobID) { + return new MockEnvironmentBuilder() + .setTaskName("test task") + .setManagedMemorySize(32 * 1024) + .setInputSplitProvider(new MockInputSplitProvider()) + .setBufferSize(256) + .setTaskConfiguration(new org.apache.flink.configuration.Configuration()) + .setExecutionConfig(new ExecutionConfig()) + .setMaxParallelism(16) + .setJobID(jobID) + .build(); + } + + private static class TestOperatorFactory extends AbstractStreamOperatorFactory + implements OneInputStreamOperatorFactory { + private final String tablePath; + private final String branch; + private final PartitionSpec spec; + + private TestOperatorFactory(String tablePath, String branch, PartitionSpec spec) { + this.tablePath = tablePath; + this.branch = branch; + this.spec = spec; + } + + private static TestOperatorFactory of(String tablePath, String branch, PartitionSpec spec) { + return new TestOperatorFactory(tablePath, branch, spec); + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters param) { + IcebergFilesCommitter committer = + new IcebergFilesCommitter( + new TestTableLoader(tablePath), + false, + Collections.singletonMap("flink.test", TestIcebergFilesCommitter.class.getName()), + ThreadPools.WORKER_THREAD_POOL_SIZE, + branch, + spec); + committer.setup(param.getContainingTask(), param.getStreamConfig(), param.getOutput()); + return (T) committer; + } + + @Override + public Class getStreamOperatorClass(ClassLoader classLoader) { + return IcebergFilesCommitter.class; + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java new file mode 100644 index 000000000000..0968f89f55e0 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java @@ -0,0 +1,388 @@ +/* + * 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.iceberg.flink.sink; + +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.flink.streaming.api.operators.BoundedOneInput; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +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.flink.FlinkWriteConf; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestIcebergStreamWriter { + @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); + + private Table table; + + private final FileFormat format; + private final boolean partitioned; + + @Parameterized.Parameters(name = "format = {0}, partitioned = {1}") + public static Object[][] parameters() { + return new Object[][] { + {"avro", true}, + {"avro", false}, + {"orc", true}, + {"orc", false}, + {"parquet", true}, + {"parquet", false} + }; + } + + public TestIcebergStreamWriter(String format, boolean partitioned) { + this.format = FileFormat.fromString(format); + this.partitioned = partitioned; + } + + @Before + public void before() throws IOException { + File folder = tempFolder.newFolder(); + // Construct the iceberg table. + Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); + table = SimpleDataUtil.createTable(folder.getAbsolutePath(), props, partitioned); + } + + @Test + public void testWritingTable() throws Exception { + long checkpointId = 1L; + try (OneInputStreamOperatorTestHarness testHarness = + createIcebergStreamWriter()) { + // The first checkpoint + testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); + testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 1); + testHarness.processElement(SimpleDataUtil.createRowData(3, "hello"), 1); + + testHarness.prepareSnapshotPreBarrier(checkpointId); + long expectedDataFiles = partitioned ? 2 : 1; + WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + Assert.assertEquals(0, result.deleteFiles().length); + Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + + checkpointId = checkpointId + 1; + + // The second checkpoint + testHarness.processElement(SimpleDataUtil.createRowData(4, "foo"), 1); + testHarness.processElement(SimpleDataUtil.createRowData(5, "bar"), 2); + + testHarness.prepareSnapshotPreBarrier(checkpointId); + expectedDataFiles = partitioned ? 4 : 2; + result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + Assert.assertEquals(0, result.deleteFiles().length); + Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + + // Commit the iceberg transaction. + AppendFiles appendFiles = table.newAppend(); + Arrays.stream(result.dataFiles()).forEach(appendFiles::appendFile); + appendFiles.commit(); + + // Assert the table records. + SimpleDataUtil.assertTableRecords( + table, + Lists.newArrayList( + SimpleDataUtil.createRecord(1, "hello"), + SimpleDataUtil.createRecord(2, "world"), + SimpleDataUtil.createRecord(3, "hello"), + SimpleDataUtil.createRecord(4, "foo"), + SimpleDataUtil.createRecord(5, "bar"))); + } + } + + @Test + public void testSnapshotTwice() throws Exception { + long checkpointId = 1; + long timestamp = 1; + try (OneInputStreamOperatorTestHarness testHarness = + createIcebergStreamWriter()) { + testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), timestamp++); + testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), timestamp); + + testHarness.prepareSnapshotPreBarrier(checkpointId++); + long expectedDataFiles = partitioned ? 2 : 1; + WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + Assert.assertEquals(0, result.deleteFiles().length); + Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + + // snapshot again immediately. + for (int i = 0; i < 5; i++) { + testHarness.prepareSnapshotPreBarrier(checkpointId++); + + result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + Assert.assertEquals(0, result.deleteFiles().length); + Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + } + } + } + + @Test + public void testTableWithoutSnapshot() throws Exception { + try (OneInputStreamOperatorTestHarness testHarness = + createIcebergStreamWriter()) { + Assert.assertEquals(0, testHarness.extractOutputValues().size()); + } + // Even if we closed the iceberg stream writer, there's no orphan data file. + Assert.assertEquals(0, scanDataFiles().size()); + + try (OneInputStreamOperatorTestHarness testHarness = + createIcebergStreamWriter()) { + testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); + // Still not emit the data file yet, because there is no checkpoint. + Assert.assertEquals(0, testHarness.extractOutputValues().size()); + } + // Once we closed the iceberg stream writer, there will left an orphan data file. + Assert.assertEquals(1, scanDataFiles().size()); + } + + private Set scanDataFiles() throws IOException { + Path dataDir = new Path(table.location(), "data"); + FileSystem fs = FileSystem.get(new Configuration()); + if (!fs.exists(dataDir)) { + return ImmutableSet.of(); + } else { + Set paths = Sets.newHashSet(); + RemoteIterator iterators = fs.listFiles(dataDir, true); + while (iterators.hasNext()) { + LocatedFileStatus status = iterators.next(); + if (status.isFile()) { + Path path = status.getPath(); + if (path.getName().endsWith("." + format.toString().toLowerCase())) { + paths.add(path.toString()); + } + } + } + return paths; + } + } + + @Test + public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { + try (OneInputStreamOperatorTestHarness testHarness = + createIcebergStreamWriter()) { + testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); + testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 2); + + Assertions.assertThat(testHarness.getOneInputOperator()).isInstanceOf(BoundedOneInput.class); + ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); + + long expectedDataFiles = partitioned ? 2 : 1; + WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + Assert.assertEquals(0, result.deleteFiles().length); + Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + + ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); + + result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + Assert.assertEquals(0, result.deleteFiles().length); + // Datafiles should not be sent again + Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + } + } + + @Test + public void testBoundedStreamTriggeredEndInputBeforeTriggeringCheckpoint() throws Exception { + try (OneInputStreamOperatorTestHarness testHarness = + createIcebergStreamWriter()) { + testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); + testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 2); + + testHarness.endInput(); + + long expectedDataFiles = partitioned ? 2 : 1; + WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + Assert.assertEquals(0, result.deleteFiles().length); + Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + + testHarness.prepareSnapshotPreBarrier(1L); + + result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + Assert.assertEquals(0, result.deleteFiles().length); + // It should be ensured that after endInput is triggered, when prepareSnapshotPreBarrier + // is triggered, write should only send WriteResult once + Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + } + } + + @Test + public void testTableWithTargetFileSize() throws Exception { + // Adjust the target-file-size in table properties. + table + .updateProperties() + .set(TableProperties.WRITE_TARGET_FILE_SIZE_BYTES, "4") // ~4 bytes; low enough to trigger + .commit(); + + List rows = Lists.newArrayListWithCapacity(8000); + List records = Lists.newArrayListWithCapacity(8000); + for (int i = 0; i < 2000; i++) { + for (String data : new String[] {"a", "b", "c", "d"}) { + rows.add(SimpleDataUtil.createRowData(i, data)); + records.add(SimpleDataUtil.createRecord(i, data)); + } + } + + try (OneInputStreamOperatorTestHarness testHarness = + createIcebergStreamWriter()) { + for (RowData row : rows) { + testHarness.processElement(row, 1); + } + + // snapshot the operator. + testHarness.prepareSnapshotPreBarrier(1); + WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + Assert.assertEquals(0, result.deleteFiles().length); + Assert.assertEquals(8, result.dataFiles().length); + + // Assert that the data file have the expected records. + for (DataFile dataFile : result.dataFiles()) { + Assert.assertEquals(1000, dataFile.recordCount()); + } + + // Commit the iceberg transaction. + AppendFiles appendFiles = table.newAppend(); + Arrays.stream(result.dataFiles()).forEach(appendFiles::appendFile); + appendFiles.commit(); + } + + // Assert the table records. + SimpleDataUtil.assertTableRecords(table, records); + } + + @Test + public void testPromotedFlinkDataType() throws Exception { + Schema iSchema = + new Schema( + Types.NestedField.required(1, "tinyint", Types.IntegerType.get()), + Types.NestedField.required(2, "smallint", Types.IntegerType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get())); + TableSchema flinkSchema = + TableSchema.builder() + .field("tinyint", DataTypes.TINYINT().notNull()) + .field("smallint", DataTypes.SMALLINT().notNull()) + .field("int", DataTypes.INT().nullable()) + .build(); + + PartitionSpec spec; + if (partitioned) { + spec = + PartitionSpec.builderFor(iSchema) + .identity("smallint") + .identity("tinyint") + .identity("int") + .build(); + } else { + spec = PartitionSpec.unpartitioned(); + } + + String location = tempFolder.newFolder().getAbsolutePath(); + Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); + Table icebergTable = new HadoopTables().create(iSchema, spec, props, location); + + List rows = + Lists.newArrayList( + GenericRowData.of((byte) 0x01, (short) -32768, 101), + GenericRowData.of((byte) 0x02, (short) 0, 102), + GenericRowData.of((byte) 0x03, (short) 32767, 103)); + + Record record = GenericRecord.create(iSchema); + List expected = + Lists.newArrayList( + record.copy(ImmutableMap.of("tinyint", 1, "smallint", -32768, "int", 101)), + record.copy(ImmutableMap.of("tinyint", 2, "smallint", 0, "int", 102)), + record.copy(ImmutableMap.of("tinyint", 3, "smallint", 32767, "int", 103))); + + try (OneInputStreamOperatorTestHarness testHarness = + createIcebergStreamWriter(icebergTable, flinkSchema)) { + for (RowData row : rows) { + testHarness.processElement(row, 1); + } + testHarness.prepareSnapshotPreBarrier(1); + WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + Assert.assertEquals(0, result.deleteFiles().length); + Assert.assertEquals(partitioned ? 3 : 1, result.dataFiles().length); + + // Commit the iceberg transaction. + AppendFiles appendFiles = icebergTable.newAppend(); + Arrays.stream(result.dataFiles()).forEach(appendFiles::appendFile); + appendFiles.commit(); + } + + SimpleDataUtil.assertTableRecords(location, expected); + } + + private OneInputStreamOperatorTestHarness createIcebergStreamWriter() + throws Exception { + return createIcebergStreamWriter(table, SimpleDataUtil.FLINK_SCHEMA); + } + + private OneInputStreamOperatorTestHarness createIcebergStreamWriter( + Table icebergTable, TableSchema flinkSchema) throws Exception { + RowType flinkRowType = FlinkSink.toFlinkRowType(icebergTable.schema(), flinkSchema); + FlinkWriteConf flinkWriteConfig = + new FlinkWriteConf( + icebergTable, Maps.newHashMap(), new org.apache.flink.configuration.Configuration()); + + IcebergStreamWriter streamWriter = + FlinkSink.createStreamWriter(() -> icebergTable, flinkWriteConfig, flinkRowType, null); + OneInputStreamOperatorTestHarness harness = + new OneInputStreamOperatorTestHarness<>(streamWriter, 1, 1, 0); + + harness.setup(); + harness.open(); + + return harness; + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java new file mode 100644 index 000000000000..79be96d20902 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java @@ -0,0 +1,252 @@ +/* + * 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.iceberg.flink.sink; + +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +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.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.data.RandomRowData; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.junit.Assert; +import org.junit.Test; + +public class TestRowDataPartitionKey { + private static final Schema SCHEMA = + new Schema( + Types.NestedField.required(0, "boolType", Types.BooleanType.get()), + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "longType", Types.LongType.get()), + Types.NestedField.required(3, "dateType", Types.DateType.get()), + Types.NestedField.required(4, "timeType", Types.TimeType.get()), + Types.NestedField.required(5, "stringType", Types.StringType.get()), + Types.NestedField.required(6, "timestampWithoutZone", Types.TimestampType.withoutZone()), + Types.NestedField.required(7, "timestampWithZone", Types.TimestampType.withZone()), + Types.NestedField.required(8, "fixedType", Types.FixedType.ofLength(5)), + Types.NestedField.required(9, "uuidType", Types.UUIDType.get()), + Types.NestedField.required(10, "binaryType", Types.BinaryType.get()), + Types.NestedField.required(11, "decimalType1", Types.DecimalType.of(18, 3)), + Types.NestedField.required(12, "decimalType2", Types.DecimalType.of(10, 5)), + Types.NestedField.required(13, "decimalType3", Types.DecimalType.of(38, 19)), + Types.NestedField.required(14, "floatType", Types.FloatType.get()), + Types.NestedField.required(15, "doubleType", Types.DoubleType.get())); + + private static final List SUPPORTED_PRIMITIVES = + SCHEMA.asStruct().fields().stream().map(Types.NestedField::name).collect(Collectors.toList()); + + private static final Schema NESTED_SCHEMA = + new Schema( + Types.NestedField.required( + 1, + "structType", + Types.StructType.of( + Types.NestedField.optional(2, "innerStringType", Types.StringType.get()), + Types.NestedField.optional(3, "innerIntegerType", Types.IntegerType.get())))); + + @Test + public void testNullPartitionValue() { + Schema schema = + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get())); + + PartitionSpec spec = PartitionSpec.builderFor(schema).identity("data").build(); + + List rows = + Lists.newArrayList( + GenericRowData.of(1, StringData.fromString("a")), + GenericRowData.of(2, StringData.fromString("b")), + GenericRowData.of(3, null)); + + RowDataWrapper rowWrapper = + new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); + + for (RowData row : rows) { + PartitionKey partitionKey = new PartitionKey(spec, schema); + partitionKey.partition(rowWrapper.wrap(row)); + Assert.assertEquals(partitionKey.size(), 1); + + String expectedStr = row.isNullAt(1) ? null : row.getString(1).toString(); + Assert.assertEquals(expectedStr, partitionKey.get(0, String.class)); + } + } + + @Test + public void testPartitionWithOneNestedField() { + RowDataWrapper rowWrapper = + new RowDataWrapper(FlinkSchemaUtil.convert(NESTED_SCHEMA), NESTED_SCHEMA.asStruct()); + List records = RandomGenericData.generate(NESTED_SCHEMA, 10, 1991); + List rows = Lists.newArrayList(RandomRowData.convert(NESTED_SCHEMA, records)); + + PartitionSpec spec1 = + PartitionSpec.builderFor(NESTED_SCHEMA).identity("structType.innerStringType").build(); + PartitionSpec spec2 = + PartitionSpec.builderFor(NESTED_SCHEMA).identity("structType.innerIntegerType").build(); + + for (int i = 0; i < rows.size(); i++) { + RowData row = rows.get(i); + Record record = (Record) records.get(i).get(0); + + PartitionKey partitionKey1 = new PartitionKey(spec1, NESTED_SCHEMA); + partitionKey1.partition(rowWrapper.wrap(row)); + Assert.assertEquals(partitionKey1.size(), 1); + + Assert.assertEquals(record.get(0), partitionKey1.get(0, String.class)); + + PartitionKey partitionKey2 = new PartitionKey(spec2, NESTED_SCHEMA); + partitionKey2.partition(rowWrapper.wrap(row)); + Assert.assertEquals(partitionKey2.size(), 1); + + Assert.assertEquals(record.get(1), partitionKey2.get(0, Integer.class)); + } + } + + @Test + public void testPartitionMultipleNestedField() { + RowDataWrapper rowWrapper = + new RowDataWrapper(FlinkSchemaUtil.convert(NESTED_SCHEMA), NESTED_SCHEMA.asStruct()); + List records = RandomGenericData.generate(NESTED_SCHEMA, 10, 1992); + List rows = Lists.newArrayList(RandomRowData.convert(NESTED_SCHEMA, records)); + + PartitionSpec spec1 = + PartitionSpec.builderFor(NESTED_SCHEMA) + .identity("structType.innerIntegerType") + .identity("structType.innerStringType") + .build(); + PartitionSpec spec2 = + PartitionSpec.builderFor(NESTED_SCHEMA) + .identity("structType.innerStringType") + .identity("structType.innerIntegerType") + .build(); + + PartitionKey pk1 = new PartitionKey(spec1, NESTED_SCHEMA); + PartitionKey pk2 = new PartitionKey(spec2, NESTED_SCHEMA); + + for (int i = 0; i < rows.size(); i++) { + RowData row = rows.get(i); + Record record = (Record) records.get(i).get(0); + + pk1.partition(rowWrapper.wrap(row)); + Assert.assertEquals(2, pk1.size()); + + Assert.assertEquals(record.get(1), pk1.get(0, Integer.class)); + Assert.assertEquals(record.get(0), pk1.get(1, String.class)); + + pk2.partition(rowWrapper.wrap(row)); + Assert.assertEquals(2, pk2.size()); + + Assert.assertEquals(record.get(0), pk2.get(0, String.class)); + Assert.assertEquals(record.get(1), pk2.get(1, Integer.class)); + } + } + + @Test + public void testPartitionValueTypes() { + RowType rowType = FlinkSchemaUtil.convert(SCHEMA); + RowDataWrapper rowWrapper = new RowDataWrapper(rowType, SCHEMA.asStruct()); + InternalRecordWrapper recordWrapper = new InternalRecordWrapper(SCHEMA.asStruct()); + + List records = RandomGenericData.generate(SCHEMA, 10, 1993); + List rows = Lists.newArrayList(RandomRowData.convert(SCHEMA, records)); + + for (String column : SUPPORTED_PRIMITIVES) { + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity(column).build(); + Class[] javaClasses = spec.javaClasses(); + + PartitionKey pk = new PartitionKey(spec, SCHEMA); + PartitionKey expectedPK = new PartitionKey(spec, SCHEMA); + + for (int j = 0; j < rows.size(); j++) { + RowData row = rows.get(j); + Record record = records.get(j); + + pk.partition(rowWrapper.wrap(row)); + expectedPK.partition(recordWrapper.wrap(record)); + + Assert.assertEquals( + "Partition with column " + column + " should have one field.", 1, pk.size()); + + if (column.equals("timeType")) { + Assert.assertEquals( + "Partition with column " + column + " should have the expected values", + expectedPK.get(0, Long.class) / 1000, + pk.get(0, Long.class) / 1000); + } else { + Assert.assertEquals( + "Partition with column " + column + " should have the expected values", + expectedPK.get(0, javaClasses[0]), + pk.get(0, javaClasses[0])); + } + } + } + } + + @Test + public void testNestedPartitionValues() { + Schema nestedSchema = new Schema(Types.NestedField.optional(1001, "nested", SCHEMA.asStruct())); + RowType rowType = FlinkSchemaUtil.convert(nestedSchema); + + RowDataWrapper rowWrapper = new RowDataWrapper(rowType, nestedSchema.asStruct()); + InternalRecordWrapper recordWrapper = new InternalRecordWrapper(nestedSchema.asStruct()); + + List records = RandomGenericData.generate(nestedSchema, 10, 1994); + List rows = Lists.newArrayList(RandomRowData.convert(nestedSchema, records)); + + for (String supportedPrimitive : SUPPORTED_PRIMITIVES) { + String column = String.format("nested.%s", supportedPrimitive); + + PartitionSpec spec = PartitionSpec.builderFor(nestedSchema).identity(column).build(); + Class[] javaClasses = spec.javaClasses(); + + PartitionKey pk = new PartitionKey(spec, nestedSchema); + PartitionKey expectedPK = new PartitionKey(spec, nestedSchema); + + for (int j = 0; j < rows.size(); j++) { + pk.partition(rowWrapper.wrap(rows.get(j))); + expectedPK.partition(recordWrapper.wrap(records.get(j))); + + Assert.assertEquals( + "Partition with nested column " + column + " should have one field.", 1, pk.size()); + + if (column.equals("nested.timeType")) { + Assert.assertEquals( + "Partition with nested column " + column + " should have the expected values.", + expectedPK.get(0, Long.class) / 1000, + pk.get(0, Long.class) / 1000); + } else { + Assert.assertEquals( + "Partition with nested column " + column + " should have the expected values.", + expectedPK.get(0, javaClasses[0]), + pk.get(0, javaClasses[0])); + } + } + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java new file mode 100644 index 000000000000..112dbb511310 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java @@ -0,0 +1,243 @@ +/* + * 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.iceberg.flink.sink; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.data.RandomRowData; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestTaskWriters { + private static final Configuration CONF = new Configuration(); + private static final long TARGET_FILE_SIZE = 128 * 1024 * 1024; + + @Rule public final TemporaryFolder tempFolder = new TemporaryFolder(); + + @Parameterized.Parameters(name = "format = {0}, partitioned = {1}") + public static Object[][] parameters() { + return new Object[][] { + {"avro", true}, + {"avro", false}, + {"orc", true}, + {"orc", false}, + {"parquet", true}, + {"parquet", false} + }; + } + + private final FileFormat format; + private final boolean partitioned; + + private Table table; + + public TestTaskWriters(String format, boolean partitioned) { + this.format = FileFormat.fromString(format); + this.partitioned = partitioned; + } + + @Before + public void before() throws IOException { + File folder = tempFolder.newFolder(); + // Construct the iceberg table with the specified file format. + Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); + table = SimpleDataUtil.createTable(folder.getAbsolutePath(), props, partitioned); + } + + @Test + public void testWriteZeroRecord() throws IOException { + try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { + taskWriter.close(); + + DataFile[] dataFiles = taskWriter.dataFiles(); + Assert.assertNotNull(dataFiles); + Assert.assertEquals(0, dataFiles.length); + + // Close again. + taskWriter.close(); + dataFiles = taskWriter.dataFiles(); + Assert.assertNotNull(dataFiles); + Assert.assertEquals(0, dataFiles.length); + } + } + + @Test + public void testCloseTwice() throws IOException { + try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { + taskWriter.write(SimpleDataUtil.createRowData(1, "hello")); + taskWriter.write(SimpleDataUtil.createRowData(2, "world")); + taskWriter.close(); // The first close + taskWriter.close(); // The second close + + int expectedFiles = partitioned ? 2 : 1; + DataFile[] dataFiles = taskWriter.dataFiles(); + Assert.assertEquals(expectedFiles, dataFiles.length); + + FileSystem fs = FileSystem.get(CONF); + for (DataFile dataFile : dataFiles) { + Assert.assertTrue(fs.exists(new Path(dataFile.path().toString()))); + } + } + } + + @Test + public void testAbort() throws IOException { + try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { + taskWriter.write(SimpleDataUtil.createRowData(1, "hello")); + taskWriter.write(SimpleDataUtil.createRowData(2, "world")); + + taskWriter.abort(); + DataFile[] dataFiles = taskWriter.dataFiles(); + + int expectedFiles = partitioned ? 2 : 1; + Assert.assertEquals(expectedFiles, dataFiles.length); + + FileSystem fs = FileSystem.get(CONF); + for (DataFile dataFile : dataFiles) { + Assert.assertFalse(fs.exists(new Path(dataFile.path().toString()))); + } + } + } + + @Test + public void testCompleteFiles() throws IOException { + try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { + taskWriter.write(SimpleDataUtil.createRowData(1, "a")); + taskWriter.write(SimpleDataUtil.createRowData(2, "b")); + taskWriter.write(SimpleDataUtil.createRowData(3, "c")); + taskWriter.write(SimpleDataUtil.createRowData(4, "d")); + + DataFile[] dataFiles = taskWriter.dataFiles(); + int expectedFiles = partitioned ? 4 : 1; + Assert.assertEquals(expectedFiles, dataFiles.length); + + dataFiles = taskWriter.dataFiles(); + Assert.assertEquals(expectedFiles, dataFiles.length); + + FileSystem fs = FileSystem.get(CONF); + for (DataFile dataFile : dataFiles) { + Assert.assertTrue(fs.exists(new Path(dataFile.path().toString()))); + } + + AppendFiles appendFiles = table.newAppend(); + for (DataFile dataFile : dataFiles) { + appendFiles.appendFile(dataFile); + } + appendFiles.commit(); + + // Assert the data rows. + SimpleDataUtil.assertTableRecords( + table, + Lists.newArrayList( + SimpleDataUtil.createRecord(1, "a"), + SimpleDataUtil.createRecord(2, "b"), + SimpleDataUtil.createRecord(3, "c"), + SimpleDataUtil.createRecord(4, "d"))); + } + } + + @Test + public void testRollingWithTargetFileSize() throws IOException { + try (TaskWriter taskWriter = createTaskWriter(4)) { + List rows = Lists.newArrayListWithCapacity(8000); + List records = Lists.newArrayListWithCapacity(8000); + for (int i = 0; i < 2000; i++) { + for (String data : new String[] {"a", "b", "c", "d"}) { + rows.add(SimpleDataUtil.createRowData(i, data)); + records.add(SimpleDataUtil.createRecord(i, data)); + } + } + + for (RowData row : rows) { + taskWriter.write(row); + } + + DataFile[] dataFiles = taskWriter.dataFiles(); + Assert.assertEquals(8, dataFiles.length); + + AppendFiles appendFiles = table.newAppend(); + for (DataFile dataFile : dataFiles) { + appendFiles.appendFile(dataFile); + } + appendFiles.commit(); + + // Assert the data rows. + SimpleDataUtil.assertTableRecords(table, records); + } + } + + @Test + public void testRandomData() throws IOException { + try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { + Iterable rows = RandomRowData.generate(SimpleDataUtil.SCHEMA, 100, 1996); + for (RowData row : rows) { + taskWriter.write(row); + } + + taskWriter.close(); + DataFile[] dataFiles = taskWriter.dataFiles(); + AppendFiles appendFiles = table.newAppend(); + for (DataFile dataFile : dataFiles) { + appendFiles.appendFile(dataFile); + } + appendFiles.commit(); + + // Assert the data rows. + SimpleDataUtil.assertTableRows(table, Lists.newArrayList(rows)); + } + } + + private TaskWriter createTaskWriter(long targetFileSize) { + TaskWriterFactory taskWriterFactory = + new RowDataTaskWriterFactory( + SerializableTable.copyOf(table), + (RowType) SimpleDataUtil.FLINK_SCHEMA.toRowDataType().getLogicalType(), + targetFileSize, + format, + table.properties(), + null, + false); + taskWriterFactory.initialize(1, 1); + return taskWriterFactory.create(); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java new file mode 100644 index 000000000000..dd7fcafe5336 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java @@ -0,0 +1,61 @@ +/* + * 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.iceberg.flink.sink.shuffle; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Map; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.VarCharType; +import org.junit.Test; + +public class TestAggregatedStatistics { + + @Test + public void mergeDataStatisticTest() { + GenericRowData rowDataA = GenericRowData.of(StringData.fromString("a")); + GenericRowData rowDataB = GenericRowData.of(StringData.fromString("b")); + + AggregatedStatistics> aggregatedStatistics = + new AggregatedStatistics<>( + 1, + MapDataStatisticsSerializer.fromKeySerializer( + new RowDataSerializer(RowType.of(new VarCharType())))); + MapDataStatistics mapDataStatistics1 = new MapDataStatistics(); + mapDataStatistics1.add(rowDataA); + mapDataStatistics1.add(rowDataA); + mapDataStatistics1.add(rowDataB); + aggregatedStatistics.mergeDataStatistic("testOperator", 1, mapDataStatistics1); + MapDataStatistics mapDataStatistics2 = new MapDataStatistics(); + mapDataStatistics2.add(rowDataA); + aggregatedStatistics.mergeDataStatistic("testOperator", 1, mapDataStatistics2); + assertThat(aggregatedStatistics.dataStatistics().statistics().get(rowDataA)) + .isEqualTo( + mapDataStatistics1.statistics().get(rowDataA) + + mapDataStatistics2.statistics().get(rowDataA)); + assertThat(aggregatedStatistics.dataStatistics().statistics().get(rowDataB)) + .isEqualTo( + mapDataStatistics1.statistics().get(rowDataB) + + mapDataStatistics2.statistics().getOrDefault(rowDataB, 0L)); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java new file mode 100644 index 000000000000..48e4e4d8f966 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java @@ -0,0 +1,177 @@ +/* + * 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.iceberg.flink.sink.shuffle; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Map; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.binary.BinaryRowData; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.VarCharType; +import org.junit.Before; +import org.junit.Test; + +public class TestAggregatedStatisticsTracker { + private static final int NUM_SUBTASKS = 2; + private final RowType rowType = RowType.of(new VarCharType()); + // When coordinator handles events from operator, DataStatisticsUtil#deserializeDataStatistics + // deserializes bytes into BinaryRowData + private final BinaryRowData binaryRowDataA = + new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("a"))); + private final BinaryRowData binaryRowDataB = + new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("b"))); + private final TypeSerializer rowSerializer = new RowDataSerializer(rowType); + private final TypeSerializer>> + statisticsSerializer = MapDataStatisticsSerializer.fromKeySerializer(rowSerializer); + private AggregatedStatisticsTracker> + aggregatedStatisticsTracker; + + @Before + public void before() throws Exception { + aggregatedStatisticsTracker = + new AggregatedStatisticsTracker<>("testOperator", statisticsSerializer, NUM_SUBTASKS); + } + + @Test + public void receiveNewerDataStatisticEvent() { + MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); + checkpoint1Subtask0DataStatistic.add(binaryRowDataA); + DataStatisticsEvent> + checkpoint1Subtask0DataStatisticEvent = + DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); + assertThat( + aggregatedStatisticsTracker.updateAndCheckCompletion( + 0, checkpoint1Subtask0DataStatisticEvent)) + .isNull(); + assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()).isEqualTo(1); + + MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); + checkpoint2Subtask0DataStatistic.add(binaryRowDataA); + DataStatisticsEvent> + checkpoint2Subtask0DataStatisticEvent = + DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); + assertThat( + aggregatedStatisticsTracker.updateAndCheckCompletion( + 0, checkpoint2Subtask0DataStatisticEvent)) + .isNull(); + // Checkpoint 2 is newer than checkpoint1, thus dropping in progress statistics for checkpoint1 + assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()).isEqualTo(2); + } + + @Test + public void receiveOlderDataStatisticEventTest() { + MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); + checkpoint2Subtask0DataStatistic.add(binaryRowDataA); + checkpoint2Subtask0DataStatistic.add(binaryRowDataB); + checkpoint2Subtask0DataStatistic.add(binaryRowDataB); + DataStatisticsEvent> + checkpoint3Subtask0DataStatisticEvent = + DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); + assertThat( + aggregatedStatisticsTracker.updateAndCheckCompletion( + 0, checkpoint3Subtask0DataStatisticEvent)) + .isNull(); + + MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); + checkpoint1Subtask1DataStatistic.add(binaryRowDataB); + DataStatisticsEvent> + checkpoint1Subtask1DataStatisticEvent = + DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); + // Receive event from old checkpoint, aggregatedStatisticsAggregatorTracker won't return + // completed statistics and in progress statistics won't be updated + assertThat( + aggregatedStatisticsTracker.updateAndCheckCompletion( + 1, checkpoint1Subtask1DataStatisticEvent)) + .isNull(); + assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()).isEqualTo(2); + } + + @Test + public void receiveCompletedDataStatisticEvent() { + MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); + checkpoint1Subtask0DataStatistic.add(binaryRowDataA); + checkpoint1Subtask0DataStatistic.add(binaryRowDataB); + checkpoint1Subtask0DataStatistic.add(binaryRowDataB); + DataStatisticsEvent> + checkpoint1Subtask0DataStatisticEvent = + DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); + assertThat( + aggregatedStatisticsTracker.updateAndCheckCompletion( + 0, checkpoint1Subtask0DataStatisticEvent)) + .isNull(); + + MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); + checkpoint1Subtask1DataStatistic.add(binaryRowDataA); + checkpoint1Subtask1DataStatistic.add(binaryRowDataA); + checkpoint1Subtask1DataStatistic.add(binaryRowDataB); + DataStatisticsEvent> + checkpoint1Subtask1DataStatisticEvent = + DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); + // Receive data statistics from all subtasks at checkpoint 1 + AggregatedStatistics> completedStatistics = + aggregatedStatisticsTracker.updateAndCheckCompletion( + 1, checkpoint1Subtask1DataStatisticEvent); + + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.checkpointId()).isEqualTo(1); + MapDataStatistics globalDataStatistics = + (MapDataStatistics) completedStatistics.dataStatistics(); + assertThat((long) globalDataStatistics.statistics().get(binaryRowDataA)) + .isEqualTo( + checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataA) + + checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataA)); + assertThat((long) globalDataStatistics.statistics().get(binaryRowDataB)) + .isEqualTo( + checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataB) + + checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataB)); + assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()) + .isEqualTo(completedStatistics.checkpointId() + 1); + + MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); + checkpoint2Subtask0DataStatistic.add(binaryRowDataA); + DataStatisticsEvent> + checkpoint2Subtask0DataStatisticEvent = + DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); + assertThat( + aggregatedStatisticsTracker.updateAndCheckCompletion( + 0, checkpoint2Subtask0DataStatisticEvent)) + .isNull(); + assertThat(completedStatistics.checkpointId()).isEqualTo(1); + + MapDataStatistics checkpoint2Subtask1DataStatistic = new MapDataStatistics(); + checkpoint2Subtask1DataStatistic.add(binaryRowDataB); + DataStatisticsEvent> + checkpoint2Subtask1DataStatisticEvent = + DataStatisticsEvent.create(2, checkpoint2Subtask1DataStatistic, statisticsSerializer); + // Receive data statistics from all subtasks at checkpoint 2 + completedStatistics = + aggregatedStatisticsTracker.updateAndCheckCompletion( + 1, checkpoint2Subtask1DataStatisticEvent); + + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.checkpointId()).isEqualTo(2); + assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()) + .isEqualTo(completedStatistics.checkpointId() + 1); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java new file mode 100644 index 000000000000..9ec2606e101d --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java @@ -0,0 +1,174 @@ +/* + * 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.iceberg.flink.sink.shuffle; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; +import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.binary.BinaryRowData; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.flink.util.ExceptionUtils; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.Before; +import org.junit.Test; + +public class TestDataStatisticsCoordinator { + private static final String OPERATOR_NAME = "TestCoordinator"; + private static final OperatorID TEST_OPERATOR_ID = new OperatorID(1234L, 5678L); + private static final int NUM_SUBTASKS = 2; + private TypeSerializer>> + statisticsSerializer; + + private EventReceivingTasks receivingTasks; + private DataStatisticsCoordinator> + dataStatisticsCoordinator; + + @Before + public void before() throws Exception { + receivingTasks = EventReceivingTasks.createForRunningTasks(); + statisticsSerializer = + MapDataStatisticsSerializer.fromKeySerializer( + new RowDataSerializer(RowType.of(new VarCharType()))); + + dataStatisticsCoordinator = + new DataStatisticsCoordinator<>( + OPERATOR_NAME, + new MockOperatorCoordinatorContext(TEST_OPERATOR_ID, NUM_SUBTASKS), + statisticsSerializer); + } + + private void tasksReady() throws Exception { + dataStatisticsCoordinator.start(); + setAllTasksReady(NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); + } + + @Test + public void testThrowExceptionWhenNotStarted() { + String failureMessage = "The coordinator of TestCoordinator has not started yet."; + + assertThatThrownBy( + () -> + dataStatisticsCoordinator.handleEventFromOperator( + 0, + 0, + DataStatisticsEvent.create(0, new MapDataStatistics(), statisticsSerializer))) + .isInstanceOf(IllegalStateException.class) + .hasMessage(failureMessage); + assertThatThrownBy(() -> dataStatisticsCoordinator.executionAttemptFailed(0, 0, null)) + .isInstanceOf(IllegalStateException.class) + .hasMessage(failureMessage); + assertThatThrownBy(() -> dataStatisticsCoordinator.checkpointCoordinator(0, null)) + .isInstanceOf(IllegalStateException.class) + .hasMessage(failureMessage); + } + + @Test + public void testDataStatisticsEventHandling() throws Exception { + tasksReady(); + // When coordinator handles events from operator, DataStatisticsUtil#deserializeDataStatistics + // deserializes bytes into BinaryRowData + RowType rowType = RowType.of(new VarCharType()); + BinaryRowData binaryRowDataA = + new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("a"))); + BinaryRowData binaryRowDataB = + new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("b"))); + BinaryRowData binaryRowDataC = + new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("c"))); + + MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); + checkpoint1Subtask0DataStatistic.add(binaryRowDataA); + checkpoint1Subtask0DataStatistic.add(binaryRowDataB); + checkpoint1Subtask0DataStatistic.add(binaryRowDataB); + checkpoint1Subtask0DataStatistic.add(binaryRowDataC); + checkpoint1Subtask0DataStatistic.add(binaryRowDataC); + checkpoint1Subtask0DataStatistic.add(binaryRowDataC); + DataStatisticsEvent> + checkpoint1Subtask0DataStatisticEvent = + DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); + MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); + checkpoint1Subtask1DataStatistic.add(binaryRowDataA); + checkpoint1Subtask1DataStatistic.add(binaryRowDataB); + checkpoint1Subtask1DataStatistic.add(binaryRowDataC); + checkpoint1Subtask1DataStatistic.add(binaryRowDataC); + DataStatisticsEvent> + checkpoint1Subtask1DataStatisticEvent = + DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); + // Handle events from operators for checkpoint 1 + dataStatisticsCoordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0DataStatisticEvent); + dataStatisticsCoordinator.handleEventFromOperator(1, 0, checkpoint1Subtask1DataStatisticEvent); + + waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + // Verify global data statistics is the aggregation of all subtasks data statistics + MapDataStatistics globalDataStatistics = + (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); + assertThat(globalDataStatistics.statistics()) + .containsExactlyInAnyOrderEntriesOf( + ImmutableMap.of( + binaryRowDataA, + checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataA) + + (long) checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataA), + binaryRowDataB, + checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataB) + + (long) checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataB), + binaryRowDataC, + checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataC) + + (long) checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataC))); + } + + static void setAllTasksReady( + int subtasks, + DataStatisticsCoordinator> dataStatisticsCoordinator, + EventReceivingTasks receivingTasks) { + for (int i = 0; i < subtasks; i++) { + dataStatisticsCoordinator.executionAttemptReady( + i, 0, receivingTasks.createGatewayForSubtask(i, 0)); + } + } + + static void waitForCoordinatorToProcessActions( + DataStatisticsCoordinator> coordinator) { + CompletableFuture future = new CompletableFuture<>(); + coordinator.callInCoordinatorThread( + () -> { + future.complete(null); + return null; + }, + "Coordinator fails to process action"); + + try { + future.get(); + } catch (InterruptedException e) { + throw new AssertionError("test interrupted"); + } catch (ExecutionException e) { + ExceptionUtils.rethrow(ExceptionUtils.stripExecutionException(e)); + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java new file mode 100644 index 000000000000..cb9d3f48ffeb --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java @@ -0,0 +1,147 @@ +/* + * 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.iceberg.flink.sink.shuffle; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; +import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; +import org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.binary.BinaryRowData; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.VarCharType; +import org.junit.Before; +import org.junit.Test; + +public class TestDataStatisticsCoordinatorProvider { + private static final OperatorID OPERATOR_ID = new OperatorID(); + private static final int NUM_SUBTASKS = 1; + + private DataStatisticsCoordinatorProvider> provider; + private EventReceivingTasks receivingTasks; + private TypeSerializer>> + statisticsSerializer; + + @Before + public void before() { + statisticsSerializer = + MapDataStatisticsSerializer.fromKeySerializer( + new RowDataSerializer(RowType.of(new VarCharType()))); + provider = + new DataStatisticsCoordinatorProvider<>( + "DataStatisticsCoordinatorProvider", OPERATOR_ID, statisticsSerializer); + receivingTasks = EventReceivingTasks.createForRunningTasks(); + } + + @Test + @SuppressWarnings("unchecked") + public void testCheckpointAndReset() throws Exception { + RowType rowType = RowType.of(new VarCharType()); + // When coordinator handles events from operator, DataStatisticsUtil#deserializeDataStatistics + // deserializes bytes into BinaryRowData + BinaryRowData binaryRowDataA = + new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("a"))); + BinaryRowData binaryRowDataB = + new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("b"))); + BinaryRowData binaryRowDataC = + new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("c"))); + BinaryRowData binaryRowDataD = + new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("d"))); + BinaryRowData binaryRowDataE = + new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("e"))); + + RecreateOnResetOperatorCoordinator coordinator = + (RecreateOnResetOperatorCoordinator) + provider.create(new MockOperatorCoordinatorContext(OPERATOR_ID, NUM_SUBTASKS)); + DataStatisticsCoordinator> dataStatisticsCoordinator = + (DataStatisticsCoordinator>) + coordinator.getInternalCoordinator(); + + // Start the coordinator + coordinator.start(); + TestDataStatisticsCoordinator.setAllTasksReady( + NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); + MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); + checkpoint1Subtask0DataStatistic.add(binaryRowDataA); + checkpoint1Subtask0DataStatistic.add(binaryRowDataB); + checkpoint1Subtask0DataStatistic.add(binaryRowDataC); + DataStatisticsEvent> + checkpoint1Subtask0DataStatisticEvent = + DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); + + // Handle events from operators for checkpoint 1 + coordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0DataStatisticEvent); + TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + // Verify checkpoint 1 global data statistics + MapDataStatistics checkpoint1GlobalDataStatistics = + (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); + assertThat(checkpoint1GlobalDataStatistics.statistics()) + .isEqualTo(checkpoint1Subtask0DataStatistic.statistics()); + byte[] checkpoint1Bytes = waitForCheckpoint(1L, dataStatisticsCoordinator); + + MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); + checkpoint2Subtask0DataStatistic.add(binaryRowDataD); + checkpoint2Subtask0DataStatistic.add(binaryRowDataE); + checkpoint2Subtask0DataStatistic.add(binaryRowDataE); + DataStatisticsEvent> + checkpoint2Subtask0DataStatisticEvent = + DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); + // Handle events from operators for checkpoint 2 + coordinator.handleEventFromOperator(0, 0, checkpoint2Subtask0DataStatisticEvent); + TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + // Verify checkpoint 2 global data statistics + MapDataStatistics checkpoint2GlobalDataStatistics = + (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); + assertThat(checkpoint2GlobalDataStatistics.statistics()) + .isEqualTo(checkpoint2Subtask0DataStatistic.statistics()); + waitForCheckpoint(2L, dataStatisticsCoordinator); + + // Reset coordinator to checkpoint 1 + coordinator.resetToCheckpoint(1L, checkpoint1Bytes); + DataStatisticsCoordinator> + restoredDataStatisticsCoordinator = + (DataStatisticsCoordinator>) + coordinator.getInternalCoordinator(); + assertThat(dataStatisticsCoordinator).isNotEqualTo(restoredDataStatisticsCoordinator); + // Verify restored data statistics + MapDataStatistics restoredAggregateDataStatistics = + (MapDataStatistics) + restoredDataStatisticsCoordinator.completedStatistics().dataStatistics(); + assertThat(restoredAggregateDataStatistics.statistics()) + .isEqualTo(checkpoint1GlobalDataStatistics.statistics()); + } + + private byte[] waitForCheckpoint( + long checkpointId, + DataStatisticsCoordinator> coordinator) + throws InterruptedException, ExecutionException { + CompletableFuture future = new CompletableFuture<>(); + coordinator.checkpointCoordinator(checkpointId, future); + return future.get(); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java new file mode 100644 index 000000000000..880cb3d5516f --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java @@ -0,0 +1,227 @@ +/* + * 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.iceberg.flink.sink.shuffle; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.state.OperatorStateStore; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.fs.CloseableRegistry; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.operators.coordination.MockOperatorEventGateway; +import org.apache.flink.runtime.operators.testutils.MockEnvironment; +import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; +import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; +import org.apache.flink.runtime.state.AbstractStateBackend; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateInitializationContextImpl; +import org.apache.flink.runtime.state.TestTaskStateManager; +import org.apache.flink.runtime.state.hashmap.HashMapStateBackend; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask; +import org.apache.flink.streaming.runtime.tasks.StreamMockEnvironment; +import org.apache.flink.streaming.util.MockOutput; +import org.apache.flink.streaming.util.MockStreamConfig; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.binary.BinaryRowData; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +public class TestDataStatisticsOperator { + private final RowType rowType = RowType.of(new VarCharType()); + private final TypeSerializer rowSerializer = new RowDataSerializer(rowType); + private final GenericRowData genericRowDataA = GenericRowData.of(StringData.fromString("a")); + private final GenericRowData genericRowDataB = GenericRowData.of(StringData.fromString("b")); + // When operator hands events from coordinator, DataStatisticsUtil#deserializeDataStatistics + // deserializes bytes into BinaryRowData + private final BinaryRowData binaryRowDataA = + new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("a"))); + private final BinaryRowData binaryRowDataB = + new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("b"))); + private final BinaryRowData binaryRowDataC = + new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("c"))); + private final TypeSerializer>> + statisticsSerializer = MapDataStatisticsSerializer.fromKeySerializer(rowSerializer); + private DataStatisticsOperator> operator; + + private Environment getTestingEnvironment() { + return new StreamMockEnvironment( + new Configuration(), + new Configuration(), + new ExecutionConfig(), + 1L, + new MockInputSplitProvider(), + 1, + new TestTaskStateManager()); + } + + @Before + public void before() throws Exception { + this.operator = createOperator(); + Environment env = getTestingEnvironment(); + this.operator.setup( + new OneInputStreamTask(env), + new MockStreamConfig(new Configuration(), 1), + new MockOutput<>(Lists.newArrayList())); + } + + private DataStatisticsOperator> createOperator() { + MockOperatorEventGateway mockGateway = new MockOperatorEventGateway(); + KeySelector keySelector = + new KeySelector() { + private static final long serialVersionUID = 7662520075515707428L; + + @Override + public RowData getKey(RowData value) { + return value; + } + }; + + return new DataStatisticsOperator<>( + "testOperator", keySelector, mockGateway, statisticsSerializer); + } + + @After + public void clean() throws Exception { + operator.close(); + } + + @Test + public void testProcessElement() throws Exception { + try (OneInputStreamOperatorTestHarness< + RowData, DataStatisticsOrRecord>> + testHarness = createHarness(this.operator)) { + StateInitializationContext stateContext = getStateContext(); + operator.initializeState(stateContext); + operator.processElement(new StreamRecord<>(genericRowDataA)); + operator.processElement(new StreamRecord<>(genericRowDataA)); + operator.processElement(new StreamRecord<>(genericRowDataB)); + assertThat(operator.localDataStatistics()).isInstanceOf(MapDataStatistics.class); + MapDataStatistics mapDataStatistics = (MapDataStatistics) operator.localDataStatistics(); + Map statsMap = mapDataStatistics.statistics(); + assertThat(statsMap).hasSize(2); + assertThat(statsMap) + .containsExactlyInAnyOrderEntriesOf( + ImmutableMap.of(genericRowDataA, 2L, genericRowDataB, 1L)); + testHarness.endInput(); + } + } + + @Test + public void testOperatorOutput() throws Exception { + try (OneInputStreamOperatorTestHarness< + RowData, DataStatisticsOrRecord>> + testHarness = createHarness(this.operator)) { + testHarness.processElement(new StreamRecord<>(genericRowDataA)); + testHarness.processElement(new StreamRecord<>(genericRowDataB)); + testHarness.processElement(new StreamRecord<>(genericRowDataB)); + + List recordsOutput = + testHarness.extractOutputValues().stream() + .filter(DataStatisticsOrRecord::hasRecord) + .map(DataStatisticsOrRecord::record) + .collect(Collectors.toList()); + assertThat(recordsOutput) + .containsExactlyInAnyOrderElementsOf( + ImmutableList.of(genericRowDataA, genericRowDataB, genericRowDataB)); + } + } + + @Test + public void testRestoreState() throws Exception { + OperatorSubtaskState snapshot; + try (OneInputStreamOperatorTestHarness< + RowData, DataStatisticsOrRecord>> + testHarness1 = createHarness(this.operator)) { + DataStatistics> mapDataStatistics = + new MapDataStatistics(); + mapDataStatistics.add(binaryRowDataA); + mapDataStatistics.add(binaryRowDataA); + mapDataStatistics.add(binaryRowDataB); + mapDataStatistics.add(binaryRowDataC); + operator.handleOperatorEvent( + DataStatisticsEvent.create(0, mapDataStatistics, statisticsSerializer)); + assertThat(operator.globalDataStatistics()).isInstanceOf(MapDataStatistics.class); + assertThat(operator.globalDataStatistics().statistics()) + .containsExactlyInAnyOrderEntriesOf( + ImmutableMap.of(binaryRowDataA, 2L, binaryRowDataB, 1L, binaryRowDataC, 1L)); + snapshot = testHarness1.snapshot(1L, 0); + } + + // Use the snapshot to initialize state for another new operator and then verify that the global + // statistics for the new operator is same as before + DataStatisticsOperator> restoredOperator = + createOperator(); + try (OneInputStreamOperatorTestHarness< + RowData, DataStatisticsOrRecord>> + testHarness2 = new OneInputStreamOperatorTestHarness<>(restoredOperator, 2, 2, 1)) { + testHarness2.setup(); + testHarness2.initializeState(snapshot); + assertThat(restoredOperator.globalDataStatistics()).isInstanceOf(MapDataStatistics.class); + assertThat(restoredOperator.globalDataStatistics().statistics()) + .containsExactlyInAnyOrderEntriesOf( + ImmutableMap.of(binaryRowDataA, 2L, binaryRowDataB, 1L, binaryRowDataC, 1L)); + } + } + + private StateInitializationContext getStateContext() throws Exception { + MockEnvironment env = new MockEnvironmentBuilder().build(); + AbstractStateBackend abstractStateBackend = new HashMapStateBackend(); + CloseableRegistry cancelStreamRegistry = new CloseableRegistry(); + OperatorStateStore operatorStateStore = + abstractStateBackend.createOperatorStateBackend( + env, "test-operator", Collections.emptyList(), cancelStreamRegistry); + return new StateInitializationContextImpl(null, operatorStateStore, null, null, null); + } + + private OneInputStreamOperatorTestHarness< + RowData, DataStatisticsOrRecord>> + createHarness( + final DataStatisticsOperator> + dataStatisticsOperator) + throws Exception { + + OneInputStreamOperatorTestHarness< + RowData, DataStatisticsOrRecord>> + harness = new OneInputStreamOperatorTestHarness<>(dataStatisticsOperator, 1, 1, 0); + harness.setup( + new DataStatisticsOrRecordSerializer<>( + MapDataStatisticsSerializer.fromKeySerializer(rowSerializer), rowSerializer)); + harness.open(); + return harness; + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java new file mode 100644 index 000000000000..a08578a4c106 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java @@ -0,0 +1,170 @@ +/* + * 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.iceberg.flink.source; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Supplier; +import java.util.stream.Stream; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.ProviderContext; +import org.apache.flink.table.connector.source.DataStreamScanProvider; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.connector.source.ScanTableSource; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.util.DataFormatConverters; +import org.apache.flink.table.factories.DynamicTableSourceFactory; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.utils.TableSchemaUtils; +import org.apache.flink.types.Row; +import org.apache.flink.types.RowKind; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +public class BoundedTableFactory implements DynamicTableSourceFactory { + private static final AtomicInteger DATA_SET_ID = new AtomicInteger(0); + private static final Map>> DATA_SETS = Maps.newHashMap(); + + private static final ConfigOption DATA_ID = + ConfigOptions.key("data-id").stringType().noDefaultValue(); + + public static String registerDataSet(List> dataSet) { + String dataSetId = String.valueOf(DATA_SET_ID.incrementAndGet()); + DATA_SETS.put(dataSetId, dataSet); + return dataSetId; + } + + public static void clearDataSets() { + DATA_SETS.clear(); + } + + @Override + public DynamicTableSource createDynamicTableSource(Context context) { + TableSchema tableSchema = + TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema()); + + Configuration configuration = Configuration.fromMap(context.getCatalogTable().getOptions()); + String dataId = configuration.getString(DATA_ID); + Preconditions.checkArgument( + DATA_SETS.containsKey(dataId), "data-id %s does not found in registered data set.", dataId); + + return new BoundedTableSource(DATA_SETS.get(dataId), tableSchema); + } + + @Override + public String factoryIdentifier() { + return "BoundedSource"; + } + + @Override + public Set> requiredOptions() { + return ImmutableSet.of(); + } + + @Override + public Set> optionalOptions() { + return ImmutableSet.of(DATA_ID); + } + + private static class BoundedTableSource implements ScanTableSource { + + private final List> elementsPerCheckpoint; + private final TableSchema tableSchema; + + private BoundedTableSource(List> elementsPerCheckpoint, TableSchema tableSchema) { + this.elementsPerCheckpoint = elementsPerCheckpoint; + this.tableSchema = tableSchema; + } + + private BoundedTableSource(BoundedTableSource toCopy) { + this.elementsPerCheckpoint = toCopy.elementsPerCheckpoint; + this.tableSchema = toCopy.tableSchema; + } + + @Override + public ChangelogMode getChangelogMode() { + Supplier> supplier = () -> elementsPerCheckpoint.stream().flatMap(List::stream); + + // Add the INSERT row kind by default. + ChangelogMode.Builder builder = ChangelogMode.newBuilder().addContainedKind(RowKind.INSERT); + + if (supplier.get().anyMatch(r -> r.getKind() == RowKind.DELETE)) { + builder.addContainedKind(RowKind.DELETE); + } + + if (supplier.get().anyMatch(r -> r.getKind() == RowKind.UPDATE_BEFORE)) { + builder.addContainedKind(RowKind.UPDATE_BEFORE); + } + + if (supplier.get().anyMatch(r -> r.getKind() == RowKind.UPDATE_AFTER)) { + builder.addContainedKind(RowKind.UPDATE_AFTER); + } + + return builder.build(); + } + + @Override + public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) { + return new DataStreamScanProvider() { + @Override + public DataStream produceDataStream( + ProviderContext providerContext, StreamExecutionEnvironment env) { + boolean checkpointEnabled = env.getCheckpointConfig().isCheckpointingEnabled(); + SourceFunction source = + new BoundedTestSource<>(elementsPerCheckpoint, checkpointEnabled); + + RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType(); + // Converter to convert the Row to RowData. + DataFormatConverters.RowConverter rowConverter = + new DataFormatConverters.RowConverter(tableSchema.getFieldDataTypes()); + + return env.addSource(source, new RowTypeInfo(tableSchema.getFieldTypes())) + .map(rowConverter::toInternal, FlinkCompatibilityUtil.toTypeInfo(rowType)); + } + + @Override + public boolean isBounded() { + return true; + } + }; + } + + @Override + public DynamicTableSource copy() { + return new BoundedTableSource(this); + } + + @Override + public String asSummaryString() { + return "Bounded test table source"; + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java new file mode 100644 index 000000000000..7b435d059845 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java @@ -0,0 +1,108 @@ +/* + * 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.iceberg.flink.source; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.flink.api.common.state.CheckpointListener; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * A stream source that: 1) emits the elements from elementsPerCheckpoint.get(0) without allowing + * checkpoints. 2) then waits for the checkpoint to complete. 3) emits the elements from + * elementsPerCheckpoint.get(1) without allowing checkpoints. 4) then waits for the checkpoint to + * complete. 5) ... + * + *

    Util all the list from elementsPerCheckpoint are exhausted. + */ +public final class BoundedTestSource implements SourceFunction, CheckpointListener { + + private final List> elementsPerCheckpoint; + private final boolean checkpointEnabled; + private volatile boolean running = true; + + private final AtomicInteger numCheckpointsComplete = new AtomicInteger(0); + + /** Emits all those elements in several checkpoints. */ + public BoundedTestSource(List> elementsPerCheckpoint, boolean checkpointEnabled) { + this.elementsPerCheckpoint = elementsPerCheckpoint; + this.checkpointEnabled = checkpointEnabled; + } + + public BoundedTestSource(List> elementsPerCheckpoint) { + this(elementsPerCheckpoint, true); + } + + /** Emits all those elements in a single checkpoint. */ + public BoundedTestSource(T... elements) { + this(Collections.singletonList(Arrays.asList(elements))); + } + + @Override + public void run(SourceContext ctx) throws Exception { + if (!checkpointEnabled) { + Preconditions.checkArgument( + elementsPerCheckpoint.size() <= 1, + "There should be at most one list in the elementsPerCheckpoint when checkpoint is disabled."); + elementsPerCheckpoint.stream().flatMap(List::stream).forEach(ctx::collect); + return; + } + + for (List elements : elementsPerCheckpoint) { + + final int checkpointToAwait; + synchronized (ctx.getCheckpointLock()) { + // Let's say checkpointToAwait = numCheckpointsComplete.get() + delta, in fact the value of + // delta should not + // affect the final table records because we only need to make sure that there will be + // exactly + // elementsPerCheckpoint.size() checkpoints to emit each records buffer from the original + // elementsPerCheckpoint. + // Even if the checkpoints that emitted results are not continuous, the correctness of the + // data should not be + // affected in the end. Setting the delta to be 2 is introducing the variable that produce + // un-continuous + // checkpoints that emit the records buffer from elementsPerCheckpoints. + checkpointToAwait = numCheckpointsComplete.get() + 2; + for (T element : elements) { + ctx.collect(element); + } + } + + synchronized (ctx.getCheckpointLock()) { + while (running && numCheckpointsComplete.get() < checkpointToAwait) { + ctx.getCheckpointLock().wait(1); + } + } + } + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + numCheckpointsComplete.incrementAndGet(); + } + + @Override + public void cancel() { + running = false; + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java new file mode 100644 index 000000000000..7aa2b8034bc5 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.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.iceberg.flink.source; + +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.types.Row; +import org.apache.flink.types.RowKind; +import org.apache.iceberg.flink.FlinkTestBase; +import org.apache.iceberg.flink.MiniClusterResource; +import org.junit.After; +import org.junit.Rule; +import org.junit.rules.TestName; + +public class ChangeLogTableTestBase extends FlinkTestBase { + private volatile TableEnvironment tEnv = null; + + @Rule public TestName name = new TestName(); + + @After + public void clean() { + sql("DROP TABLE IF EXISTS %s", name.getMethodName()); + BoundedTableFactory.clearDataSets(); + } + + @Override + protected TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + if (tEnv == null) { + EnvironmentSettings settings = + EnvironmentSettings.newInstance().inStreamingMode().build(); + + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(400) + .setMaxParallelism(1) + .setParallelism(1); + + tEnv = StreamTableEnvironment.create(env, settings); + } + } + } + return tEnv; + } + + protected static Row insertRow(Object... values) { + return Row.ofKind(RowKind.INSERT, values); + } + + protected static Row deleteRow(Object... values) { + return Row.ofKind(RowKind.DELETE, values); + } + + protected static Row updateBeforeRow(Object... values) { + return Row.ofKind(RowKind.UPDATE_BEFORE, values); + } + + protected static Row updateAfterRow(Object... values) { + return Row.ofKind(RowKind.UPDATE_AFTER, values); + } + + protected static List listJoin(List> lists) { + return lists.stream().flatMap(List::stream).collect(Collectors.toList()); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java new file mode 100644 index 000000000000..3a8071523b7c --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java @@ -0,0 +1,132 @@ +/* + * 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.iceberg.flink.source; + +import java.io.File; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.BaseCombinedScanTask; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.ThreadPools; +import org.junit.Assert; +import org.junit.rules.TemporaryFolder; + +public class SplitHelpers { + + private SplitHelpers() {} + + /** + * This create a list of IcebergSourceSplit from real files + *

  • Create a new Hadoop table under the {@code temporaryFolder} + *
  • write {@code fileCount} number of files to the new Iceberg table + *
  • Discover the splits from the table and partition the splits by the {@code filePerSplit} + * limit + *
  • Delete the Hadoop table + * + *

    Since the table and data files are deleted before this method return, caller shouldn't + * attempt to read the data files. + * + *

    By default, v1 Iceberg table is created. For v2 table use {@link + * SplitHelpers#createSplitsFromTransientHadoopTable(TemporaryFolder, int, int, String)} + * + * @param temporaryFolder Folder to place the data to + * @param fileCount The number of files to create and add to the table + * @param filesPerSplit The number of files used for a split + */ + public static List createSplitsFromTransientHadoopTable( + TemporaryFolder temporaryFolder, int fileCount, int filesPerSplit) throws Exception { + return createSplitsFromTransientHadoopTable(temporaryFolder, fileCount, filesPerSplit, "1"); + } + + /** + * This create a list of IcebergSourceSplit from real files + *

  • Create a new Hadoop table under the {@code temporaryFolder} + *
  • write {@code fileCount} number of files to the new Iceberg table + *
  • Discover the splits from the table and partition the splits by the {@code filePerSplit} + * limit + *
  • Delete the Hadoop table + * + *

    Since the table and data files are deleted before this method return, caller shouldn't + * attempt to read the data files. + * + * @param temporaryFolder Folder to place the data to + * @param fileCount The number of files to create and add to the table + * @param filesPerSplit The number of files used for a split + * @param version The table version to create + */ + public static List createSplitsFromTransientHadoopTable( + TemporaryFolder temporaryFolder, int fileCount, int filesPerSplit, String version) + throws Exception { + final File warehouseFile = temporaryFolder.newFolder(); + Assert.assertTrue(warehouseFile.delete()); + final String warehouse = "file:" + warehouseFile; + Configuration hadoopConf = new Configuration(); + final HadoopCatalog catalog = new HadoopCatalog(hadoopConf, warehouse); + ImmutableMap properties = + ImmutableMap.of(TableProperties.FORMAT_VERSION, version); + try { + final Table table = + catalog.createTable( + TestFixtures.TABLE_IDENTIFIER, + TestFixtures.SCHEMA, + PartitionSpec.unpartitioned(), + null, + properties); + final GenericAppenderHelper dataAppender = + new GenericAppenderHelper(table, FileFormat.PARQUET, temporaryFolder); + for (int i = 0; i < fileCount; ++i) { + List records = RandomGenericData.generate(TestFixtures.SCHEMA, 2, i); + dataAppender.appendToTable(records); + } + + final ScanContext scanContext = ScanContext.builder().build(); + final List splits = + FlinkSplitPlanner.planIcebergSourceSplits( + table, scanContext, ThreadPools.getWorkerPool()); + return splits.stream() + .flatMap( + split -> { + List> filesList = + Lists.partition(Lists.newArrayList(split.task().files()), filesPerSplit); + return filesList.stream() + .map(files -> new BaseCombinedScanTask(files)) + .map( + combinedScanTask -> + IcebergSourceSplit.fromCombinedScanTask(combinedScanTask)); + }) + .collect(Collectors.toList()); + } finally { + catalog.dropTable(TestFixtures.TABLE_IDENTIFIER); + catalog.close(); + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java new file mode 100644 index 000000000000..e4e48ca67f66 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java @@ -0,0 +1,60 @@ +/* + * 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.iceberg.flink.source; + +import java.util.List; +import java.util.Map; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +public class SqlHelpers { + private SqlHelpers() {} + + public static List sql(TableEnvironment tableEnv, String query, Object... args) { + TableResult tableResult = tableEnv.executeSql(String.format(query, args)); + try (CloseableIterator iter = tableResult.collect()) { + List results = Lists.newArrayList(iter); + return results; + } catch (Exception e) { + throw new RuntimeException("Failed to collect table result", e); + } + } + + public static String sqlOptionsToString(Map sqlOptions) { + StringBuilder builder = new StringBuilder(); + sqlOptions.forEach((key, value) -> builder.append(optionToKv(key, value)).append(",")); + String optionStr = builder.toString(); + if (optionStr.endsWith(",")) { + optionStr = optionStr.substring(0, optionStr.length() - 1); + } + + if (!optionStr.isEmpty()) { + optionStr = String.format("/*+ OPTIONS(%s)*/", optionStr); + } + + return optionStr; + } + + private static String optionToKv(String key, Object value) { + return "'" + key + "'='" + value + "'"; + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java new file mode 100644 index 000000000000..7b5f9328694c --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.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.iceberg.flink.source; + +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; +import org.apache.flink.types.Row; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Streams; +import org.junit.Assert; +import org.junit.Test; + +public class TestBoundedTableFactory extends ChangeLogTableTestBase { + + @Test + public void testEmptyDataSet() { + String table = name.getMethodName(); + List> emptyDataSet = ImmutableList.of(); + + String dataId = BoundedTableFactory.registerDataSet(emptyDataSet); + sql( + "CREATE TABLE %s(id INT, data STRING) WITH ('connector'='BoundedSource', 'data-id'='%s')", + table, dataId); + + Assert.assertEquals( + "Should have caught empty change log set.", + ImmutableList.of(), + sql("SELECT * FROM %s", table)); + } + + @Test + public void testBoundedTableFactory() { + String table = name.getMethodName(); + List> dataSet = + ImmutableList.of( + ImmutableList.of( + insertRow(1, "aaa"), + deleteRow(1, "aaa"), + insertRow(1, "bbb"), + insertRow(2, "aaa"), + deleteRow(2, "aaa"), + insertRow(2, "bbb")), + ImmutableList.of( + updateBeforeRow(2, "bbb"), + updateAfterRow(2, "ccc"), + deleteRow(2, "ccc"), + insertRow(2, "ddd")), + ImmutableList.of( + deleteRow(1, "bbb"), + insertRow(1, "ccc"), + deleteRow(1, "ccc"), + insertRow(1, "ddd"))); + + String dataId = BoundedTableFactory.registerDataSet(dataSet); + sql( + "CREATE TABLE %s(id INT, data STRING) WITH ('connector'='BoundedSource', 'data-id'='%s')", + table, dataId); + + List rowSet = dataSet.stream().flatMap(Streams::stream).collect(Collectors.toList()); + Assert.assertEquals( + "Should have the expected change log events.", rowSet, sql("SELECT * FROM %s", table)); + + Assert.assertEquals( + "Should have the expected change log events", + rowSet.stream() + .filter(r -> Objects.equals(r.getField(1), "aaa")) + .collect(Collectors.toList()), + sql("SELECT * FROM %s WHERE data='aaa'", table)); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java new file mode 100644 index 000000000000..73d03710d32c --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java @@ -0,0 +1,193 @@ +/* + * 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.iceberg.flink.source; + +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.Row; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.junit.Assume; +import org.junit.Test; + +/** Test {@link FlinkInputFormat}. */ +public class TestFlinkInputFormat extends TestFlinkSource { + + public TestFlinkInputFormat(String fileFormat) { + super(fileFormat); + } + + @Override + protected List run( + FlinkSource.Builder formatBuilder, + Map sqlOptions, + String sqlFilter, + String... sqlSelectedFields) + throws Exception { + return runFormat(formatBuilder.tableLoader(tableLoader()).buildFormat()); + } + + @Test + public void testNestedProjection() throws Exception { + Schema schema = + new Schema( + required(1, "data", Types.StringType.get()), + required( + 2, + "nested", + Types.StructType.of( + Types.NestedField.required(3, "f1", Types.StringType.get()), + Types.NestedField.required(4, "f2", Types.StringType.get()), + Types.NestedField.required(5, "f3", Types.LongType.get()))), + required(6, "id", Types.LongType.get())); + + Table table = catalogResource.catalog().createTable(TableIdentifier.of("default", "t"), schema); + + List writeRecords = RandomGenericData.generate(schema, 2, 0L); + new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(writeRecords); + + // Schema: [data, nested[f1, f2, f3], id] + // Projection: [nested.f2, data] + // The Flink SQL output: [f2, data] + // The FlinkInputFormat output: [nested[f2], data] + + TableSchema projectedSchema = + TableSchema.builder() + .field("nested", DataTypes.ROW(DataTypes.FIELD("f2", DataTypes.STRING()))) + .field("data", DataTypes.STRING()) + .build(); + List result = + runFormat( + FlinkSource.forRowData() + .tableLoader(tableLoader()) + .project(projectedSchema) + .buildFormat()); + + List expected = Lists.newArrayList(); + for (Record record : writeRecords) { + Row nested = Row.of(((Record) record.get(1)).get(1)); + expected.add(Row.of(nested, record.get(0))); + } + + TestHelpers.assertRows(result, expected); + } + + @Test + public void testBasicProjection() throws IOException { + Schema writeSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get()), + Types.NestedField.optional(2, "time", Types.TimestampType.withZone())); + + Table table = + catalogResource.catalog().createTable(TableIdentifier.of("default", "t"), writeSchema); + + List writeRecords = RandomGenericData.generate(writeSchema, 2, 0L); + new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(writeRecords); + + TableSchema projectedSchema = + TableSchema.builder() + .field("id", DataTypes.BIGINT()) + .field("data", DataTypes.STRING()) + .build(); + List result = + runFormat( + FlinkSource.forRowData() + .tableLoader(tableLoader()) + .project(projectedSchema) + .buildFormat()); + + List expected = Lists.newArrayList(); + for (Record record : writeRecords) { + expected.add(Row.of(record.get(0), record.get(1))); + } + + TestHelpers.assertRows(result, expected); + } + + @Test + public void testReadPartitionColumn() throws Exception { + Assume.assumeTrue("Temporary skip ORC", FileFormat.ORC != fileFormat); + + Schema nestedSchema = + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional( + 2, + "struct", + Types.StructType.of( + Types.NestedField.optional(3, "innerId", Types.LongType.get()), + Types.NestedField.optional(4, "innerName", Types.StringType.get())))); + PartitionSpec spec = + PartitionSpec.builderFor(nestedSchema).identity("struct.innerName").build(); + + Table table = + catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, nestedSchema, spec); + List records = RandomGenericData.generate(nestedSchema, 10, 0L); + GenericAppenderHelper appender = new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER); + for (Record record : records) { + org.apache.iceberg.TestHelpers.Row partition = + org.apache.iceberg.TestHelpers.Row.of(record.get(1, Record.class).get(1)); + appender.appendToTable(partition, Collections.singletonList(record)); + } + + TableSchema projectedSchema = + TableSchema.builder() + .field("struct", DataTypes.ROW(DataTypes.FIELD("innerName", DataTypes.STRING()))) + .build(); + List result = + runFormat( + FlinkSource.forRowData() + .tableLoader(tableLoader()) + .project(projectedSchema) + .buildFormat()); + + List expected = Lists.newArrayList(); + for (Record record : records) { + Row nested = Row.of(((Record) record.get(1)).get(1)); + expected.add(Row.of(nested)); + } + + TestHelpers.assertRows(result, expected); + } + + private List runFormat(FlinkInputFormat inputFormat) throws IOException { + RowType rowType = FlinkSchemaUtil.convert(inputFormat.projectedSchema()); + return TestHelpers.readRows(inputFormat, rowType); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java new file mode 100644 index 000000000000..b2f914e51299 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source; + +import java.io.IOException; +import java.util.Map; +import org.apache.flink.table.types.logical.RowType; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.CatalogLoader; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.StructLikeSet; + +public class TestFlinkInputFormatReaderDeletes extends TestFlinkReaderDeletesBase { + + public TestFlinkInputFormatReaderDeletes(FileFormat inputFormat) { + super(inputFormat); + } + + @Override + protected StructLikeSet rowSet(String tableName, Table testTable, String... columns) + throws IOException { + Schema projected = testTable.schema().select(columns); + RowType rowType = FlinkSchemaUtil.convert(projected); + Map properties = Maps.newHashMap(); + properties.put( + CatalogProperties.WAREHOUSE_LOCATION, + hiveConf.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname)); + properties.put(CatalogProperties.URI, hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname)); + properties.put( + CatalogProperties.CLIENT_POOL_SIZE, + Integer.toString(hiveConf.getInt("iceberg.hive.client-pool-size", 5))); + CatalogLoader hiveCatalogLoader = CatalogLoader.hive(catalog.name(), hiveConf, properties); + FlinkInputFormat inputFormat = + FlinkSource.forRowData() + .tableLoader( + TableLoader.fromCatalog( + hiveCatalogLoader, TableIdentifier.of("default", tableName))) + .project(FlinkSchemaUtil.toSchema(rowType)) + .buildFormat(); + + StructLikeSet set = StructLikeSet.create(projected.asStruct()); + TestHelpers.readRowData(inputFormat, rowType) + .forEach( + rowData -> { + RowDataWrapper wrapper = new RowDataWrapper(rowType, projected.asStruct()); + set.add(wrapper.wrap(rowData)); + }); + + return set; + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java new file mode 100644 index 000000000000..1d52acb2fe7b --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.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.iceberg.flink.source; + +import java.io.IOException; +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.TestMergingMetrics; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.RowDataConverter; +import org.apache.iceberg.flink.sink.FlinkAppenderFactory; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.rules.TemporaryFolder; + +public class TestFlinkMergingMetrics extends TestMergingMetrics { + + @ClassRule public static final TemporaryFolder TEMP_FOLDER = new TemporaryFolder(); + + @Rule + public final HadoopTableResource tableResource = + new HadoopTableResource(TEMP_FOLDER, "test_db", "test_table", SCHEMA); + + public TestFlinkMergingMetrics(FileFormat fileFormat) { + super(fileFormat); + } + + @Override + protected FileAppender writeAndGetAppender(List records) throws IOException { + RowType flinkSchema = FlinkSchemaUtil.convert(SCHEMA); + + FileAppender appender = + new FlinkAppenderFactory( + tableResource.table(), + SCHEMA, + flinkSchema, + ImmutableMap.of(), + PartitionSpec.unpartitioned(), + null, + null, + null) + .newAppender(org.apache.iceberg.Files.localOutput(temp.newFile()), fileFormat); + try (FileAppender fileAppender = appender) { + records.stream().map(r -> RowDataConverter.convert(SCHEMA, r)).forEach(fileAppender::add); + } + return appender; + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java new file mode 100644 index 000000000000..5ecf4f4536bb --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java @@ -0,0 +1,829 @@ +/* + * 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.iceberg.flink.source; + +import java.io.IOException; +import java.time.Instant; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.avro.generic.GenericData; +import org.apache.commons.collections.ListUtils; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.types.Row; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileContent; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Files; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.MetadataTableUtils; +import org.apache.iceberg.MetricsUtil; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.FileHelpers; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.flink.FlinkCatalogTestBase; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.SnapshotUtil; +import org.junit.After; +import org.junit.Assert; +import org.junit.Assume; +import org.junit.Before; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestFlinkMetaDataTable extends FlinkCatalogTestBase { + private static final String TABLE_NAME = "test_table"; + private final FileFormat format = FileFormat.AVRO; + private static final TemporaryFolder TEMP = new TemporaryFolder(); + private final boolean isPartition; + + public TestFlinkMetaDataTable(String catalogName, Namespace baseNamespace, Boolean isPartition) { + super(catalogName, baseNamespace); + this.isPartition = isPartition; + } + + @Parameterized.Parameters(name = "catalogName={0}, baseNamespace={1}, isPartition={2}") + public static Iterable parameters() { + List parameters = Lists.newArrayList(); + + for (Boolean isPartition : new Boolean[] {true, false}) { + String catalogName = "testhadoop"; + Namespace baseNamespace = Namespace.of("default"); + parameters.add(new Object[] {catalogName, baseNamespace, isPartition}); + } + return parameters; + } + + @Override + protected TableEnvironment getTableEnv() { + Configuration configuration = super.getTableEnv().getConfig().getConfiguration(); + configuration.set(CoreOptions.DEFAULT_PARALLELISM, 1); + return super.getTableEnv(); + } + + @Before + public void before() { + super.before(); + sql("USE CATALOG %s", catalogName); + sql("CREATE DATABASE %s", flinkDatabase); + sql("USE %s", DATABASE); + if (isPartition) { + sql( + "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) PARTITIONED BY (data) WITH ('format-version'='2', 'write.format.default'='%s')", + TABLE_NAME, format.name()); + sql("INSERT INTO %s VALUES (1,'a',10),(2,'a',20)", TABLE_NAME); + sql("INSERT INTO %s VALUES (1,'b',10),(2,'b',20)", TABLE_NAME); + } else { + sql( + "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('format-version'='2', 'write.format.default'='%s')", + TABLE_NAME, format.name()); + sql( + "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)", + TABLE_NAME); + sql("INSERT INTO %s VALUES (4,'iceberg',10)", TABLE_NAME); + } + } + + @Override + @After + public void clean() { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME); + sql("DROP DATABASE IF EXISTS %s", flinkDatabase); + super.clean(); + } + + @Test + public void testSnapshots() { + String sql = String.format("SELECT * FROM %s$snapshots ", TABLE_NAME); + List result = sql(sql); + + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); + Iterator snapshots = table.snapshots().iterator(); + for (Row row : result) { + Snapshot next = snapshots.next(); + Assert.assertEquals( + "Should have expected timestamp", + ((Instant) row.getField(0)).toEpochMilli(), + next.timestampMillis()); + Assert.assertEquals("Should have expected snapshot id", next.snapshotId(), row.getField(1)); + Assert.assertEquals("Should have expected parent id", next.parentId(), row.getField(2)); + Assert.assertEquals("Should have expected operation", next.operation(), row.getField(3)); + Assert.assertEquals( + "Should have expected manifest list location", + row.getField(4), + next.manifestListLocation()); + Assert.assertEquals("Should have expected summary", next.summary(), row.getField(5)); + } + } + + @Test + public void testHistory() { + String sql = String.format("SELECT * FROM %s$history ", TABLE_NAME); + List result = sql(sql); + + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); + Iterator snapshots = table.snapshots().iterator(); + for (Row row : result) { + Snapshot next = snapshots.next(); + Assert.assertEquals( + "Should have expected made_current_at", + ((Instant) row.getField(0)).toEpochMilli(), + next.timestampMillis()); + Assert.assertEquals("Should have expected snapshot id", next.snapshotId(), row.getField(1)); + Assert.assertEquals("Should have expected parent id", next.parentId(), row.getField(2)); + + Assert.assertEquals( + "Should have expected is current ancestor", + SnapshotUtil.isAncestorOf(table, table.currentSnapshot().snapshotId(), next.snapshotId()), + row.getField(3)); + } + } + + @Test + public void testManifests() { + String sql = String.format("SELECT * FROM %s$manifests ", TABLE_NAME); + List result = sql(sql); + + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); + List expectedDataManifests = dataManifests(table); + + for (int i = 0; i < result.size(); i++) { + Row row = result.get(i); + ManifestFile manifestFile = expectedDataManifests.get(i); + Assert.assertEquals( + "Should have expected content", manifestFile.content().id(), row.getField(0)); + Assert.assertEquals("Should have expected path", manifestFile.path(), row.getField(1)); + Assert.assertEquals("Should have expected length", manifestFile.length(), row.getField(2)); + Assert.assertEquals( + "Should have expected partition_spec_id", + manifestFile.partitionSpecId(), + row.getField(3)); + Assert.assertEquals( + "Should have expected added_snapshot_id", manifestFile.snapshotId(), row.getField(4)); + Assert.assertEquals( + "Should have expected added_data_files_count", + manifestFile.addedFilesCount(), + row.getField(5)); + Assert.assertEquals( + "Should have expected existing_data_files_count", + manifestFile.existingFilesCount(), + row.getField(6)); + Assert.assertEquals( + "Should have expected deleted_data_files_count", + manifestFile.deletedFilesCount(), + row.getField(7)); + } + } + + @Test + public void testAllManifests() { + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); + + String sql = String.format("SELECT * FROM %s$all_manifests ", TABLE_NAME); + List result = sql(sql); + + List expectedDataManifests = allDataManifests(table); + + Assert.assertEquals(expectedDataManifests.size(), result.size()); + for (int i = 0; i < result.size(); i++) { + Row row = result.get(i); + ManifestFile manifestFile = expectedDataManifests.get(i); + Assert.assertEquals( + "Should have expected content", manifestFile.content().id(), row.getField(0)); + Assert.assertEquals("Should have expected path", manifestFile.path(), row.getField(1)); + Assert.assertEquals("Should have expected length", manifestFile.length(), row.getField(2)); + Assert.assertEquals( + "Should have expected partition_spec_id", + manifestFile.partitionSpecId(), + row.getField(3)); + Assert.assertEquals( + "Should have expected added_snapshot_id", manifestFile.snapshotId(), row.getField(4)); + Assert.assertEquals( + "Should have expected added_data_files_count", + manifestFile.addedFilesCount(), + row.getField(5)); + Assert.assertEquals( + "Should have expected existing_data_files_count", + manifestFile.existingFilesCount(), + row.getField(6)); + Assert.assertEquals( + "Should have expected deleted_data_files_count", + manifestFile.deletedFilesCount(), + row.getField(7)); + } + } + + @Test + public void testUnPartitionedTable() throws IOException { + Assume.assumeFalse(isPartition); + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); + + Schema deleteRowSchema = table.schema().select("id"); + Record dataDelete = GenericRecord.create(deleteRowSchema); + List dataDeletes = Lists.newArrayList(dataDelete.copy("id", 1)); + + TEMP.create(); + DeleteFile eqDeletes = + FileHelpers.writeDeleteFile( + table, Files.localOutput(TEMP.newFile()), dataDeletes, deleteRowSchema); + table.newRowDelta().addDeletes(eqDeletes).commit(); + + List expectedDataManifests = dataManifests(table); + List expectedDeleteManifests = deleteManifests(table); + + Assert.assertEquals("Should have 2 data manifest", 2, expectedDataManifests.size()); + Assert.assertEquals("Should have 1 delete manifest", 1, expectedDeleteManifests.size()); + + Schema entriesTableSchema = + MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.from("entries")) + .schema(); + + // check delete files table + Schema deleteFilesTableSchema = + MetadataTableUtils.createMetadataTableInstance( + table, MetadataTableType.from("delete_files")) + .schema(); + + List deleteColumns = + deleteFilesTableSchema.columns().stream() + .map(Types.NestedField::name) + .filter(c -> !c.equals(MetricsUtil.READABLE_METRICS)) + .collect(Collectors.toList()); + String deleteNames = + deleteColumns.stream().map(n -> "`" + n + "`").collect(Collectors.joining(",")); + + deleteFilesTableSchema = deleteFilesTableSchema.select(deleteColumns); + + List actualDeleteFiles = sql("SELECT %s FROM %s$delete_files", deleteNames, TABLE_NAME); + Assert.assertEquals("Metadata table should return 1 delete file", 1, actualDeleteFiles.size()); + + List expectedDeleteFiles = + expectedEntries( + table, FileContent.EQUALITY_DELETES, entriesTableSchema, expectedDeleteManifests, null); + Assert.assertEquals("Should be 1 delete file manifest entry", 1, expectedDeleteFiles.size()); + TestHelpers.assertEquals( + deleteFilesTableSchema, expectedDeleteFiles.get(0), actualDeleteFiles.get(0)); + + // Check data files table + Schema filesTableSchema = + MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.from("files")) + .schema(); + + List columns = + filesTableSchema.columns().stream() + .map(Types.NestedField::name) + .filter(c -> !c.equals(MetricsUtil.READABLE_METRICS)) + .collect(Collectors.toList()); + String names = columns.stream().map(n -> "`" + n + "`").collect(Collectors.joining(",")); + + filesTableSchema = filesTableSchema.select(columns); + + List actualDataFiles = sql("SELECT %s FROM %s$data_files", names, TABLE_NAME); + Assert.assertEquals("Metadata table should return 2 data file", 2, actualDataFiles.size()); + + List expectedDataFiles = + expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, null); + Assert.assertEquals("Should be 2 data file manifest entry", 2, expectedDataFiles.size()); + TestHelpers.assertEquals(filesTableSchema, expectedDataFiles.get(0), actualDataFiles.get(0)); + + // check all files table + List actualFiles = sql("SELECT %s FROM %s$files ORDER BY content", names, TABLE_NAME); + Assert.assertEquals("Metadata table should return 3 files", 3, actualFiles.size()); + + List expectedFiles = + Stream.concat(expectedDataFiles.stream(), expectedDeleteFiles.stream()) + .collect(Collectors.toList()); + Assert.assertEquals("Should have 3 files manifest entries", 3, expectedFiles.size()); + TestHelpers.assertEquals(filesTableSchema, expectedFiles.get(0), actualFiles.get(0)); + TestHelpers.assertEquals(filesTableSchema, expectedFiles.get(1), actualFiles.get(1)); + } + + @Test + public void testPartitionedTable() throws Exception { + Assume.assumeFalse(!isPartition); + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); + + Schema deleteRowSchema = table.schema().select("id", "data"); + Record dataDelete = GenericRecord.create(deleteRowSchema); + TEMP.create(); + + Map deleteRow = Maps.newHashMap(); + deleteRow.put("id", 1); + deleteRow.put("data", "a"); + DeleteFile eqDeletes = + FileHelpers.writeDeleteFile( + table, + Files.localOutput(TEMP.newFile()), + org.apache.iceberg.TestHelpers.Row.of("a"), + Lists.newArrayList(dataDelete.copy(deleteRow)), + deleteRowSchema); + table.newRowDelta().addDeletes(eqDeletes).commit(); + + deleteRow.put("data", "b"); + DeleteFile eqDeletes2 = + FileHelpers.writeDeleteFile( + table, + Files.localOutput(TEMP.newFile()), + org.apache.iceberg.TestHelpers.Row.of("b"), + Lists.newArrayList(dataDelete.copy(deleteRow)), + deleteRowSchema); + table.newRowDelta().addDeletes(eqDeletes2).commit(); + + Schema entriesTableSchema = + MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.from("entries")) + .schema(); + + List expectedDataManifests = dataManifests(table); + List expectedDeleteManifests = deleteManifests(table); + + Assert.assertEquals("Should have 2 data manifests", 2, expectedDataManifests.size()); + Assert.assertEquals("Should have 2 delete manifests", 2, expectedDeleteManifests.size()); + + Table deleteFilesTable = + MetadataTableUtils.createMetadataTableInstance( + table, MetadataTableType.from("delete_files")); + Schema filesTableSchema = deleteFilesTable.schema(); + + List columns = + filesTableSchema.columns().stream() + .map(Types.NestedField::name) + .filter(c -> !c.equals(MetricsUtil.READABLE_METRICS)) + .collect(Collectors.toList()); + String names = columns.stream().map(n -> "`" + n + "`").collect(Collectors.joining(",")); + + filesTableSchema = filesTableSchema.select(columns); + + // Check delete files table + List expectedDeleteFiles = + expectedEntries( + table, FileContent.EQUALITY_DELETES, entriesTableSchema, expectedDeleteManifests, "a"); + Assert.assertEquals( + "Should have one delete file manifest entry", 1, expectedDeleteFiles.size()); + + List actualDeleteFiles = + sql("SELECT %s FROM %s$delete_files WHERE `partition`.`data`='a'", names, TABLE_NAME); + + Assert.assertEquals( + "Metadata table should return one delete file", 1, actualDeleteFiles.size()); + TestHelpers.assertEquals( + filesTableSchema, expectedDeleteFiles.get(0), actualDeleteFiles.get(0)); + + // Check data files table + List expectedDataFiles = + expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, "a"); + Assert.assertEquals("Should have one data file manifest entry", 1, expectedDataFiles.size()); + + List actualDataFiles = + sql("SELECT %s FROM %s$data_files WHERE `partition`.`data`='a'", names, TABLE_NAME); + Assert.assertEquals("Metadata table should return one data file", 1, actualDataFiles.size()); + TestHelpers.assertEquals(filesTableSchema, expectedDataFiles.get(0), actualDataFiles.get(0)); + + List actualPartitionsWithProjection = + sql("SELECT file_count FROM %s$partitions ", TABLE_NAME); + Assert.assertEquals( + "Metadata table should return two partitions record", + 2, + actualPartitionsWithProjection.size()); + for (int i = 0; i < 2; ++i) { + Assert.assertEquals(1, actualPartitionsWithProjection.get(i).getField(0)); + } + + // Check files table + List expectedFiles = + Stream.concat(expectedDataFiles.stream(), expectedDeleteFiles.stream()) + .collect(Collectors.toList()); + Assert.assertEquals("Should have two file manifest entries", 2, expectedFiles.size()); + + List actualFiles = + sql( + "SELECT %s FROM %s$files WHERE `partition`.`data`='a' ORDER BY content", + names, TABLE_NAME); + Assert.assertEquals("Metadata table should return two files", 2, actualFiles.size()); + TestHelpers.assertEquals(filesTableSchema, expectedFiles.get(0), actualFiles.get(0)); + TestHelpers.assertEquals(filesTableSchema, expectedFiles.get(1), actualFiles.get(1)); + } + + @Test + public void testAllFilesUnpartitioned() throws Exception { + Assume.assumeFalse(isPartition); + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); + + Schema deleteRowSchema = table.schema().select("id", "data"); + Record dataDelete = GenericRecord.create(deleteRowSchema); + TEMP.create(); + + Map deleteRow = Maps.newHashMap(); + deleteRow.put("id", 1); + DeleteFile eqDeletes = + FileHelpers.writeDeleteFile( + table, + Files.localOutput(TEMP.newFile()), + Lists.newArrayList(dataDelete.copy(deleteRow)), + deleteRowSchema); + table.newRowDelta().addDeletes(eqDeletes).commit(); + + List expectedDataManifests = dataManifests(table); + Assert.assertEquals("Should have 2 data manifest", 2, expectedDataManifests.size()); + List expectedDeleteManifests = deleteManifests(table); + Assert.assertEquals("Should have 1 delete manifest", 1, expectedDeleteManifests.size()); + + // Clear table to test whether 'all_files' can read past files + table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit(); + + Schema entriesTableSchema = + MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.from("entries")) + .schema(); + Schema filesTableSchema = + MetadataTableUtils.createMetadataTableInstance( + table, MetadataTableType.from("all_data_files")) + .schema(); + + List columns = + filesTableSchema.columns().stream() + .map(Types.NestedField::name) + .filter(c -> !c.equals(MetricsUtil.READABLE_METRICS)) + .collect(Collectors.toList()); + String names = columns.stream().map(n -> "`" + n + "`").collect(Collectors.joining(",")); + + filesTableSchema = filesTableSchema.select(columns); + + // Check all data files table + List actualDataFiles = + sql("SELECT %s FROM %s$all_data_files order by record_count ", names, TABLE_NAME); + + List expectedDataFiles = + expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, null); + Assert.assertEquals("Should be 2 data file manifest entry", 2, expectedDataFiles.size()); + Assert.assertEquals("Metadata table should return 2 data file", 2, actualDataFiles.size()); + TestHelpers.assertEquals(filesTableSchema, expectedDataFiles, actualDataFiles); + + // Check all delete files table + List actualDeleteFiles = sql("SELECT %s FROM %s$all_delete_files", names, TABLE_NAME); + List expectedDeleteFiles = + expectedEntries( + table, FileContent.EQUALITY_DELETES, entriesTableSchema, expectedDeleteManifests, null); + Assert.assertEquals("Should be one delete file manifest entry", 1, expectedDeleteFiles.size()); + Assert.assertEquals( + "Metadata table should return one delete file", 1, actualDeleteFiles.size()); + TestHelpers.assertEquals( + filesTableSchema, expectedDeleteFiles.get(0), actualDeleteFiles.get(0)); + + // Check all files table + List actualFiles = + sql("SELECT %s FROM %s$all_files ORDER BY content, record_count asc", names, TABLE_NAME); + List expectedFiles = + ListUtils.union(expectedDataFiles, expectedDeleteFiles); + expectedFiles.sort(Comparator.comparing(r -> ((Integer) r.get("content")))); + Assert.assertEquals("Metadata table should return 3 files", 3, actualFiles.size()); + TestHelpers.assertEquals(filesTableSchema, expectedFiles, actualFiles); + } + + @Test + public void testAllFilesPartitioned() throws Exception { + Assume.assumeFalse(!isPartition); + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); + + // Create delete file + Schema deleteRowSchema = table.schema().select("id"); + Record dataDelete = GenericRecord.create(deleteRowSchema); + TEMP.create(); + + Map deleteRow = Maps.newHashMap(); + deleteRow.put("id", 1); + DeleteFile eqDeletes = + FileHelpers.writeDeleteFile( + table, + Files.localOutput(TEMP.newFile()), + org.apache.iceberg.TestHelpers.Row.of("a"), + Lists.newArrayList(dataDelete.copy(deleteRow)), + deleteRowSchema); + DeleteFile eqDeletes2 = + FileHelpers.writeDeleteFile( + table, + Files.localOutput(TEMP.newFile()), + org.apache.iceberg.TestHelpers.Row.of("b"), + Lists.newArrayList(dataDelete.copy(deleteRow)), + deleteRowSchema); + table.newRowDelta().addDeletes(eqDeletes).addDeletes(eqDeletes2).commit(); + + List expectedDataManifests = dataManifests(table); + Assert.assertEquals("Should have 2 data manifests", 2, expectedDataManifests.size()); + List expectedDeleteManifests = deleteManifests(table); + Assert.assertEquals("Should have 1 delete manifest", 1, expectedDeleteManifests.size()); + + // Clear table to test whether 'all_files' can read past files + table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit(); + + Schema entriesTableSchema = + MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.from("entries")) + .schema(); + Schema filesTableSchema = + MetadataTableUtils.createMetadataTableInstance( + table, MetadataTableType.from("all_data_files")) + .schema(); + + List columns = + filesTableSchema.columns().stream() + .map(Types.NestedField::name) + .filter(c -> !c.equals(MetricsUtil.READABLE_METRICS)) + .collect(Collectors.toList()); + String names = columns.stream().map(n -> "`" + n + "`").collect(Collectors.joining(",")); + + filesTableSchema = filesTableSchema.select(columns); + + // Check all data files table + List actualDataFiles = + sql("SELECT %s FROM %s$all_data_files WHERE `partition`.`data`='a'", names, TABLE_NAME); + List expectedDataFiles = + expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, "a"); + Assert.assertEquals("Should be one data file manifest entry", 1, expectedDataFiles.size()); + Assert.assertEquals("Metadata table should return one data file", 1, actualDataFiles.size()); + TestHelpers.assertEquals(filesTableSchema, expectedDataFiles.get(0), actualDataFiles.get(0)); + + // Check all delete files table + List actualDeleteFiles = + sql("SELECT %s FROM %s$all_delete_files WHERE `partition`.`data`='a'", names, TABLE_NAME); + List expectedDeleteFiles = + expectedEntries( + table, FileContent.EQUALITY_DELETES, entriesTableSchema, expectedDeleteManifests, "a"); + Assert.assertEquals("Should be one data file manifest entry", 1, expectedDeleteFiles.size()); + Assert.assertEquals("Metadata table should return one data file", 1, actualDeleteFiles.size()); + + TestHelpers.assertEquals( + filesTableSchema, expectedDeleteFiles.get(0), actualDeleteFiles.get(0)); + + // Check all files table + List actualFiles = + sql( + "SELECT %s FROM %s$all_files WHERE `partition`.`data`='a' ORDER BY content", + names, TABLE_NAME); + List expectedFiles = + ListUtils.union(expectedDataFiles, expectedDeleteFiles); + expectedFiles.sort(Comparator.comparing(r -> ((Integer) r.get("content")))); + Assert.assertEquals("Metadata table should return two files", 2, actualFiles.size()); + TestHelpers.assertEquals(filesTableSchema, expectedFiles, actualFiles); + } + + @Test + public void testMetadataLogEntries() { + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); + + Long currentSnapshotId = table.currentSnapshot().snapshotId(); + TableMetadata tableMetadata = ((HasTableOperations) table).operations().current(); + Snapshot currentSnapshot = tableMetadata.currentSnapshot(); + Snapshot parentSnapshot = table.snapshot(currentSnapshot.parentId()); + List metadataLogEntries = + Lists.newArrayList(tableMetadata.previousFiles()); + + // Check metadataLog table + List metadataLogs = sql("SELECT * FROM %s$metadata_log_entries", TABLE_NAME); + + Assert.assertEquals("metadataLogEntries table should return 3 row", 3, metadataLogs.size()); + Row metadataLog = metadataLogs.get(0); + Assert.assertEquals( + Instant.ofEpochMilli(metadataLogEntries.get(0).timestampMillis()), + metadataLog.getField("timestamp")); + Assert.assertEquals(metadataLogEntries.get(0).file(), metadataLog.getField("file")); + Assert.assertNull(metadataLog.getField("latest_snapshot_id")); + Assert.assertNull(metadataLog.getField("latest_schema_id")); + Assert.assertNull(metadataLog.getField("latest_sequence_number")); + + metadataLog = metadataLogs.get(1); + Assert.assertEquals( + Instant.ofEpochMilli(metadataLogEntries.get(1).timestampMillis()), + metadataLog.getField("timestamp")); + Assert.assertEquals(metadataLogEntries.get(1).file(), metadataLog.getField("file")); + Assert.assertEquals(parentSnapshot.snapshotId(), metadataLog.getField("latest_snapshot_id")); + Assert.assertEquals(parentSnapshot.schemaId(), metadataLog.getField("latest_schema_id")); + Assert.assertEquals( + parentSnapshot.sequenceNumber(), metadataLog.getField("latest_sequence_number")); + + metadataLog = metadataLogs.get(2); + Assert.assertEquals( + Instant.ofEpochMilli(currentSnapshot.timestampMillis()), metadataLog.getField("timestamp")); + Assert.assertEquals(tableMetadata.metadataFileLocation(), metadataLog.getField("file")); + Assert.assertEquals(currentSnapshot.snapshotId(), metadataLog.getField("latest_snapshot_id")); + Assert.assertEquals(currentSnapshot.schemaId(), metadataLog.getField("latest_schema_id")); + Assert.assertEquals( + currentSnapshot.sequenceNumber(), metadataLog.getField("latest_sequence_number")); + + // test filtering + List metadataLogWithFilters = + sql( + "SELECT * FROM %s$metadata_log_entries WHERE latest_snapshot_id = %s", + TABLE_NAME, currentSnapshotId); + Assert.assertEquals( + "metadataLogEntries table should return 1 row", 1, metadataLogWithFilters.size()); + + metadataLog = metadataLogWithFilters.get(0); + Assert.assertEquals( + Instant.ofEpochMilli(tableMetadata.currentSnapshot().timestampMillis()), + metadataLog.getField("timestamp")); + Assert.assertEquals(tableMetadata.metadataFileLocation(), metadataLog.getField("file")); + Assert.assertEquals( + tableMetadata.currentSnapshot().snapshotId(), metadataLog.getField("latest_snapshot_id")); + Assert.assertEquals( + tableMetadata.currentSnapshot().schemaId(), metadataLog.getField("latest_schema_id")); + Assert.assertEquals( + tableMetadata.currentSnapshot().sequenceNumber(), + metadataLog.getField("latest_sequence_number")); + + // test projection + List metadataFiles = + metadataLogEntries.stream() + .map(TableMetadata.MetadataLogEntry::file) + .collect(Collectors.toList()); + metadataFiles.add(tableMetadata.metadataFileLocation()); + List metadataLogWithProjection = + sql("SELECT file FROM %s$metadata_log_entries", TABLE_NAME); + Assert.assertEquals( + "metadataLogEntries table should return 3 rows", 3, metadataLogWithProjection.size()); + for (int i = 0; i < metadataFiles.size(); i++) { + Assert.assertEquals(metadataFiles.get(i), metadataLogWithProjection.get(i).getField("file")); + } + } + + @Test + public void testSnapshotReferencesMetatable() { + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); + + Long currentSnapshotId = table.currentSnapshot().snapshotId(); + + // Create branch + table + .manageSnapshots() + .createBranch("testBranch", currentSnapshotId) + .setMaxRefAgeMs("testBranch", 10) + .setMinSnapshotsToKeep("testBranch", 20) + .setMaxSnapshotAgeMs("testBranch", 30) + .commit(); + // Create Tag + table + .manageSnapshots() + .createTag("testTag", currentSnapshotId) + .setMaxRefAgeMs("testTag", 50) + .commit(); + // Check refs table + List references = sql("SELECT * FROM %s$refs", TABLE_NAME); + Assert.assertEquals("Refs table should return 3 rows", 3, references.size()); + List branches = sql("SELECT * FROM %s$refs WHERE type='BRANCH'", TABLE_NAME); + Assert.assertEquals("Refs table should return 2 branches", 2, branches.size()); + List tags = sql("SELECT * FROM %s$refs WHERE type='TAG'", TABLE_NAME); + Assert.assertEquals("Refs table should return 1 tag", 1, tags.size()); + + // Check branch entries in refs table + List mainBranch = + sql("SELECT * FROM %s$refs WHERE name='main' AND type='BRANCH'", TABLE_NAME); + Assert.assertEquals("main", mainBranch.get(0).getFieldAs("name")); + Assert.assertEquals("BRANCH", mainBranch.get(0).getFieldAs("type")); + Assert.assertEquals(currentSnapshotId, mainBranch.get(0).getFieldAs("snapshot_id")); + + List testBranch = + sql("SELECT * FROM %s$refs WHERE name='testBranch' AND type='BRANCH'", TABLE_NAME); + Assert.assertEquals("testBranch", testBranch.get(0).getFieldAs("name")); + Assert.assertEquals("BRANCH", testBranch.get(0).getFieldAs("type")); + Assert.assertEquals(currentSnapshotId, testBranch.get(0).getFieldAs("snapshot_id")); + Assert.assertEquals(Long.valueOf(10), testBranch.get(0).getFieldAs("max_reference_age_in_ms")); + Assert.assertEquals(Integer.valueOf(20), testBranch.get(0).getFieldAs("min_snapshots_to_keep")); + Assert.assertEquals(Long.valueOf(30), testBranch.get(0).getFieldAs("max_snapshot_age_in_ms")); + + // Check tag entries in refs table + List testTag = + sql("SELECT * FROM %s$refs WHERE name='testTag' AND type='TAG'", TABLE_NAME); + Assert.assertEquals("testTag", testTag.get(0).getFieldAs("name")); + Assert.assertEquals("TAG", testTag.get(0).getFieldAs("type")); + Assert.assertEquals(currentSnapshotId, testTag.get(0).getFieldAs("snapshot_id")); + Assert.assertEquals(Long.valueOf(50), testTag.get(0).getFieldAs("max_reference_age_in_ms")); + + // Check projection in refs table + List testTagProjection = + sql( + "SELECT name,type,snapshot_id,max_reference_age_in_ms,min_snapshots_to_keep FROM %s$refs where type='TAG'", + TABLE_NAME); + Assert.assertEquals("testTag", testTagProjection.get(0).getFieldAs("name")); + Assert.assertEquals("TAG", testTagProjection.get(0).getFieldAs("type")); + Assert.assertEquals(currentSnapshotId, testTagProjection.get(0).getFieldAs("snapshot_id")); + Assert.assertEquals( + Long.valueOf(50), testTagProjection.get(0).getFieldAs("max_reference_age_in_ms")); + Assert.assertNull(testTagProjection.get(0).getFieldAs("min_snapshots_to_keep")); + + List mainBranchProjection = + sql("SELECT name, type FROM %s$refs WHERE name='main' AND type = 'BRANCH'", TABLE_NAME); + Assert.assertEquals("main", mainBranchProjection.get(0).getFieldAs("name")); + Assert.assertEquals("BRANCH", mainBranchProjection.get(0).getFieldAs("type")); + + List testBranchProjection = + sql( + "SELECT type, name, max_reference_age_in_ms, snapshot_id FROM %s$refs WHERE name='testBranch' AND type = 'BRANCH'", + TABLE_NAME); + Assert.assertEquals("testBranch", testBranchProjection.get(0).getFieldAs("name")); + Assert.assertEquals("BRANCH", testBranchProjection.get(0).getFieldAs("type")); + Assert.assertEquals(currentSnapshotId, testBranchProjection.get(0).getFieldAs("snapshot_id")); + Assert.assertEquals( + Long.valueOf(10), testBranchProjection.get(0).getFieldAs("max_reference_age_in_ms")); + } + + /** + * Find matching manifest entries of an Iceberg table + * + * @param table iceberg table + * @param expectedContent file content to populate on entries + * @param entriesTableSchema schema of Manifest entries + * @param manifestsToExplore manifests to explore of the table + * @param partValue partition value that manifest entries must match, or null to skip filtering + */ + private List expectedEntries( + Table table, + FileContent expectedContent, + Schema entriesTableSchema, + List manifestsToExplore, + String partValue) + throws IOException { + List expected = Lists.newArrayList(); + for (ManifestFile manifest : manifestsToExplore) { + InputFile in = table.io().newInputFile(manifest.path()); + try (CloseableIterable rows = + Avro.read(in).project(entriesTableSchema).build()) { + for (GenericData.Record record : rows) { + if ((Integer) record.get("status") < 2 /* added or existing */) { + GenericData.Record file = (GenericData.Record) record.get("data_file"); + if (partitionMatch(file, partValue)) { + asMetadataRecord(file, expectedContent); + expected.add(file); + } + } + } + } + } + return expected; + } + + // Populate certain fields derived in the metadata tables + private void asMetadataRecord(GenericData.Record file, FileContent content) { + file.put(0, content.id()); + file.put(3, 0); // specId + } + + private boolean partitionMatch(GenericData.Record file, String partValue) { + if (partValue == null) { + return true; + } + GenericData.Record partition = (GenericData.Record) file.get(4); + return partValue.equals(partition.get(0).toString()); + } + + private List dataManifests(Table table) { + return table.currentSnapshot().dataManifests(table.io()); + } + + private List allDataManifests(Table table) { + List manifests = Lists.newArrayList(); + for (Snapshot snapshot : table.snapshots()) { + manifests.addAll(snapshot.dataManifests(table.io())); + } + return manifests; + } + + private List deleteManifests(Table table) { + return table.currentSnapshot().deleteManifests(table.io()); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java new file mode 100644 index 000000000000..987d79fed3c3 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java @@ -0,0 +1,110 @@ +/* + * 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.iceberg.flink.source; + +import java.util.Map; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.iceberg.BaseTable; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.DeleteReadTests; +import org.apache.iceberg.hive.HiveCatalog; +import org.apache.iceberg.hive.TestHiveMetastore; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public abstract class TestFlinkReaderDeletesBase extends DeleteReadTests { + + @ClassRule public static final TemporaryFolder TEMP_FOLDER = new TemporaryFolder(); + + protected static String databaseName = "default"; + + protected static HiveConf hiveConf = null; + protected static HiveCatalog catalog = null; + private static TestHiveMetastore metastore = null; + + protected final FileFormat format; + + @Parameterized.Parameters(name = "fileFormat={0}") + public static Object[][] parameters() { + return new Object[][] { + new Object[] {FileFormat.PARQUET}, + new Object[] {FileFormat.AVRO}, + new Object[] {FileFormat.ORC} + }; + } + + TestFlinkReaderDeletesBase(FileFormat fileFormat) { + this.format = fileFormat; + } + + @BeforeClass + public static void startMetastore() { + metastore = new TestHiveMetastore(); + metastore.start(); + hiveConf = metastore.hiveConf(); + catalog = + (HiveCatalog) + CatalogUtil.loadCatalog( + HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf); + } + + @AfterClass + public static void stopMetastore() throws Exception { + metastore.stop(); + catalog = null; + } + + @Override + protected Table createTable(String name, Schema schema, PartitionSpec spec) { + Map props = Maps.newHashMap(); + props.put(TableProperties.DEFAULT_FILE_FORMAT, format.name()); + + Table table = catalog.createTable(TableIdentifier.of(databaseName, name), schema, spec, props); + TableOperations ops = ((BaseTable) table).operations(); + TableMetadata meta = ops.current(); + ops.commit(meta, meta.upgradeToFormatVersion(2)); + + return table; + } + + @Override + protected void dropTable(String name) { + catalog.dropTable(TableIdentifier.of(databaseName, name)); + } + + @Override + protected boolean expectPruned() { + return false; + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java new file mode 100644 index 000000000000..b537efa7272b --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java @@ -0,0 +1,542 @@ +/* + * 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.iceberg.flink.source; + +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.types.Row; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.flink.HadoopCatalogResource; +import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.DateTimeUtil; +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public abstract class TestFlinkScan { + + @ClassRule + public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = + MiniClusterResource.createWithClassloaderCheckDisabled(); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final HadoopCatalogResource catalogResource = + new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); + + // parametrized variables + protected final FileFormat fileFormat; + + @Parameterized.Parameters(name = "format={0}") + public static Object[] parameters() { + return new Object[] {"avro", "parquet", "orc"}; + } + + TestFlinkScan(String fileFormat) { + this.fileFormat = FileFormat.fromString(fileFormat); + } + + protected TableLoader tableLoader() { + return catalogResource.tableLoader(); + } + + protected abstract List runWithProjection(String... projected) throws Exception; + + protected abstract List runWithFilter( + Expression filter, String sqlFilter, boolean caseSensitive) throws Exception; + + protected List runWithFilter(Expression filter, String sqlFilter) throws Exception { + return runWithFilter(filter, sqlFilter, true); + } + + protected abstract List runWithOptions(Map options) throws Exception; + + protected abstract List run() throws Exception; + + @Test + public void testUnpartitionedTable() throws Exception { + Table table = + catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); + new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(expectedRecords); + TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); + } + + @Test + public void testPartitionedTable() throws Exception { + Table table = + catalogResource + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); + List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); + expectedRecords.get(0).set(2, "2020-03-20"); + new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER) + .appendToTable(org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), expectedRecords); + TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); + } + + @Test + public void testProjection() throws Exception { + Table table = + catalogResource + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); + List inputRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); + new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER) + .appendToTable(org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), inputRecords); + assertRows(runWithProjection("data"), Row.of(inputRecords.get(0).get(0))); + } + + @Test + public void testIdentityPartitionProjections() throws Exception { + Schema logSchema = + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get()), + Types.NestedField.optional(3, "level", Types.StringType.get()), + Types.NestedField.optional(4, "message", Types.StringType.get())); + PartitionSpec spec = + PartitionSpec.builderFor(logSchema).identity("dt").identity("level").build(); + + Table table = + catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, logSchema, spec); + List inputRecords = RandomGenericData.generate(logSchema, 10, 0L); + + int idx = 0; + AppendFiles append = table.newAppend(); + for (Record record : inputRecords) { + record.set(1, "2020-03-2" + idx); + record.set(2, Integer.toString(idx)); + append.appendFile( + new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER) + .writeFile( + org.apache.iceberg.TestHelpers.Row.of("2020-03-2" + idx, Integer.toString(idx)), + ImmutableList.of(record))); + idx += 1; + } + append.commit(); + + // individual fields + validateIdentityPartitionProjections(table, Collections.singletonList("dt"), inputRecords); + validateIdentityPartitionProjections(table, Collections.singletonList("level"), inputRecords); + validateIdentityPartitionProjections(table, Collections.singletonList("message"), inputRecords); + validateIdentityPartitionProjections(table, Collections.singletonList("id"), inputRecords); + // field pairs + validateIdentityPartitionProjections(table, Arrays.asList("dt", "message"), inputRecords); + validateIdentityPartitionProjections(table, Arrays.asList("level", "message"), inputRecords); + validateIdentityPartitionProjections(table, Arrays.asList("dt", "level"), inputRecords); + // out-of-order pairs + validateIdentityPartitionProjections(table, Arrays.asList("message", "dt"), inputRecords); + validateIdentityPartitionProjections(table, Arrays.asList("message", "level"), inputRecords); + validateIdentityPartitionProjections(table, Arrays.asList("level", "dt"), inputRecords); + // out-of-order triplets + validateIdentityPartitionProjections( + table, Arrays.asList("dt", "level", "message"), inputRecords); + validateIdentityPartitionProjections( + table, Arrays.asList("level", "dt", "message"), inputRecords); + validateIdentityPartitionProjections( + table, Arrays.asList("dt", "message", "level"), inputRecords); + validateIdentityPartitionProjections( + table, Arrays.asList("level", "message", "dt"), inputRecords); + validateIdentityPartitionProjections( + table, Arrays.asList("message", "dt", "level"), inputRecords); + validateIdentityPartitionProjections( + table, Arrays.asList("message", "level", "dt"), inputRecords); + } + + private void validateIdentityPartitionProjections( + Table table, List projectedFields, List inputRecords) throws Exception { + List rows = runWithProjection(projectedFields.toArray(new String[0])); + + for (int pos = 0; pos < inputRecords.size(); pos++) { + Record inputRecord = inputRecords.get(pos); + Row actualRecord = rows.get(pos); + + for (int i = 0; i < projectedFields.size(); i++) { + String name = projectedFields.get(i); + Assert.assertEquals( + "Projected field " + name + " should match", + inputRecord.getField(name), + actualRecord.getField(i)); + } + } + } + + @Test + public void testSnapshotReads() throws Exception { + Table table = + catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + + GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER); + + List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); + helper.appendToTable(expectedRecords); + long snapshotId = table.currentSnapshot().snapshotId(); + + long timestampMillis = table.currentSnapshot().timestampMillis(); + + // produce another timestamp + waitUntilAfter(timestampMillis); + helper.appendToTable(RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L)); + + TestHelpers.assertRecords( + runWithOptions(ImmutableMap.of("snapshot-id", Long.toString(snapshotId))), + expectedRecords, + TestFixtures.SCHEMA); + TestHelpers.assertRecords( + runWithOptions(ImmutableMap.of("as-of-timestamp", Long.toString(timestampMillis))), + expectedRecords, + TestFixtures.SCHEMA); + } + + @Test + public void testTagReads() throws Exception { + Table table = + catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + + GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER); + + List expectedRecords1 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); + helper.appendToTable(expectedRecords1); + long snapshotId = table.currentSnapshot().snapshotId(); + + table.manageSnapshots().createTag("t1", snapshotId).commit(); + + TestHelpers.assertRecords( + runWithOptions(ImmutableMap.of("tag", "t1")), expectedRecords1, TestFixtures.SCHEMA); + + List expectedRecords2 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); + helper.appendToTable(expectedRecords2); + snapshotId = table.currentSnapshot().snapshotId(); + + table.manageSnapshots().replaceTag("t1", snapshotId).commit(); + + List expectedRecords = Lists.newArrayList(); + expectedRecords.addAll(expectedRecords1); + expectedRecords.addAll(expectedRecords2); + TestHelpers.assertRecords( + runWithOptions(ImmutableMap.of("tag", "t1")), expectedRecords, TestFixtures.SCHEMA); + } + + @Test + public void testBranchReads() throws Exception { + Table table = + catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + + GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER); + + List expectedRecordsBase = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); + helper.appendToTable(expectedRecordsBase); + long snapshotId = table.currentSnapshot().snapshotId(); + + String branchName = "b1"; + table.manageSnapshots().createBranch(branchName, snapshotId).commit(); + + List expectedRecordsForBranch = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); + helper.appendToTable(branchName, expectedRecordsForBranch); + + List expectedRecordsForMain = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); + helper.appendToTable(expectedRecordsForMain); + + List branchExpectedRecords = Lists.newArrayList(); + branchExpectedRecords.addAll(expectedRecordsBase); + branchExpectedRecords.addAll(expectedRecordsForBranch); + + TestHelpers.assertRecords( + runWithOptions(ImmutableMap.of("branch", branchName)), + branchExpectedRecords, + TestFixtures.SCHEMA); + + List mainExpectedRecords = Lists.newArrayList(); + mainExpectedRecords.addAll(expectedRecordsBase); + mainExpectedRecords.addAll(expectedRecordsForMain); + + TestHelpers.assertRecords(run(), mainExpectedRecords, TestFixtures.SCHEMA); + } + + @Test + public void testIncrementalReadViaTag() throws Exception { + Table table = + catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + + GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER); + + List records1 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); + helper.appendToTable(records1); + long snapshotId1 = table.currentSnapshot().snapshotId(); + String startTag = "t1"; + table.manageSnapshots().createTag(startTag, snapshotId1).commit(); + + List records2 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 1L); + helper.appendToTable(records2); + + List records3 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 2L); + helper.appendToTable(records3); + long snapshotId3 = table.currentSnapshot().snapshotId(); + String endTag = "t2"; + table.manageSnapshots().createTag(endTag, snapshotId3).commit(); + + helper.appendToTable(RandomGenericData.generate(TestFixtures.SCHEMA, 1, 3L)); + + List expected = Lists.newArrayList(); + expected.addAll(records2); + expected.addAll(records3); + + TestHelpers.assertRecords( + runWithOptions( + ImmutableMap.builder() + .put("start-tag", startTag) + .put("end-tag", endTag) + .buildOrThrow()), + expected, + TestFixtures.SCHEMA); + + TestHelpers.assertRecords( + runWithOptions( + ImmutableMap.builder() + .put("start-snapshot-id", Long.toString(snapshotId1)) + .put("end-tag", endTag) + .buildOrThrow()), + expected, + TestFixtures.SCHEMA); + + TestHelpers.assertRecords( + runWithOptions( + ImmutableMap.builder() + .put("start-tag", startTag) + .put("end-snapshot-id", Long.toString(snapshotId3)) + .buildOrThrow()), + expected, + TestFixtures.SCHEMA); + + Assertions.assertThatThrownBy( + () -> + runWithOptions( + ImmutableMap.builder() + .put("start-tag", startTag) + .put("end-tag", endTag) + .put("start-snapshot-id", Long.toString(snapshotId1)) + .buildOrThrow())) + .isInstanceOf(Exception.class) + .hasMessage("START_SNAPSHOT_ID and START_TAG cannot both be set."); + + Assertions.assertThatThrownBy( + () -> + runWithOptions( + ImmutableMap.builder() + .put("start-tag", startTag) + .put("end-tag", endTag) + .put("end-snapshot-id", Long.toString(snapshotId3)) + .buildOrThrow())) + .isInstanceOf(Exception.class) + .hasMessage("END_SNAPSHOT_ID and END_TAG cannot both be set."); + } + + @Test + public void testIncrementalRead() throws Exception { + Table table = + catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + + GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER); + + List records1 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); + helper.appendToTable(records1); + long snapshotId1 = table.currentSnapshot().snapshotId(); + + // snapshot 2 + List records2 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 1L); + helper.appendToTable(records2); + + List records3 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 2L); + helper.appendToTable(records3); + long snapshotId3 = table.currentSnapshot().snapshotId(); + + // snapshot 4 + helper.appendToTable(RandomGenericData.generate(TestFixtures.SCHEMA, 1, 3L)); + + List expected2 = Lists.newArrayList(); + expected2.addAll(records2); + expected2.addAll(records3); + TestHelpers.assertRecords( + runWithOptions( + ImmutableMap.builder() + .put("start-snapshot-id", Long.toString(snapshotId1)) + .put("end-snapshot-id", Long.toString(snapshotId3)) + .buildOrThrow()), + expected2, + TestFixtures.SCHEMA); + } + + @Test + public void testFilterExpPartition() throws Exception { + Table table = + catalogResource + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); + + List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); + expectedRecords.get(0).set(2, "2020-03-20"); + expectedRecords.get(1).set(2, "2020-03-20"); + + GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER); + DataFile dataFile1 = + helper.writeFile(org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), expectedRecords); + DataFile dataFile2 = + helper.writeFile( + org.apache.iceberg.TestHelpers.Row.of("2020-03-21", 0), + RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L)); + helper.appendToTable(dataFile1, dataFile2); + TestHelpers.assertRecords( + runWithFilter(Expressions.equal("dt", "2020-03-20"), "where dt='2020-03-20'", true), + expectedRecords, + TestFixtures.SCHEMA); + } + + private void testFilterExp(Expression filter, String sqlFilter, boolean caseSensitive) + throws Exception { + Table table = + catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + + List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 3, 0L); + expectedRecords.get(0).set(0, "a"); + expectedRecords.get(1).set(0, "b"); + expectedRecords.get(2).set(0, "c"); + + GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER); + DataFile dataFile = helper.writeFile(expectedRecords); + helper.appendToTable(dataFile); + + List actual = + runWithFilter(Expressions.greaterThanOrEqual("data", "b"), "where data>='b'", true); + + TestHelpers.assertRecords(actual, expectedRecords.subList(1, 3), TestFixtures.SCHEMA); + } + + @Test + public void testFilterExp() throws Exception { + testFilterExp(Expressions.greaterThanOrEqual("data", "b"), "where data>='b'", true); + } + + @Test + public void testFilterExpCaseInsensitive() throws Exception { + // sqlFilter does not support case-insensitive filtering: + // https://issues.apache.org/jira/browse/FLINK-16175 + testFilterExp(Expressions.greaterThanOrEqual("DATA", "b"), "where data>='b'", false); + } + + @Test + public void testPartitionTypes() throws Exception { + Schema typesSchema = + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "decimal", Types.DecimalType.of(38, 18)), + Types.NestedField.optional(3, "str", Types.StringType.get()), + Types.NestedField.optional(4, "binary", Types.BinaryType.get()), + Types.NestedField.optional(5, "date", Types.DateType.get()), + Types.NestedField.optional(6, "time", Types.TimeType.get()), + Types.NestedField.optional(7, "timestamp", Types.TimestampType.withoutZone())); + PartitionSpec spec = + PartitionSpec.builderFor(typesSchema) + .identity("decimal") + .identity("str") + .identity("binary") + .identity("date") + .identity("time") + .identity("timestamp") + .build(); + + Table table = + catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, typesSchema, spec); + List records = RandomGenericData.generate(typesSchema, 10, 0L); + GenericAppenderHelper appender = new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER); + for (Record record : records) { + org.apache.iceberg.TestHelpers.Row partition = + org.apache.iceberg.TestHelpers.Row.of( + record.get(1), + record.get(2), + record.get(3), + record.get(4) == null ? null : DateTimeUtil.daysFromDate((LocalDate) record.get(4)), + record.get(5) == null ? null : DateTimeUtil.microsFromTime((LocalTime) record.get(5)), + record.get(6) == null + ? null + : DateTimeUtil.microsFromTimestamp((LocalDateTime) record.get(6))); + appender.appendToTable(partition, Collections.singletonList(record)); + } + + TestHelpers.assertRecords(run(), records, typesSchema); + } + + @Test + public void testCustomizedFlinkDataTypes() throws Exception { + Schema schema = + new Schema( + Types.NestedField.required( + 1, + "map", + Types.MapType.ofRequired(2, 3, Types.StringType.get(), Types.StringType.get())), + Types.NestedField.required( + 4, "arr", Types.ListType.ofRequired(5, Types.StringType.get()))); + Table table = catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, schema); + List records = RandomGenericData.generate(schema, 10, 0L); + GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER); + helper.appendToTable(records); + TestHelpers.assertRecords(run(), records, schema); + } + + private static void assertRows(List results, Row... expected) { + TestHelpers.assertRows(results, Arrays.asList(expected)); + } + + private static void waitUntilAfter(long timestampMillis) { + long current = System.currentTimeMillis(); + while (current <= timestampMillis) { + current = System.currentTimeMillis(); + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java new file mode 100644 index 000000000000..023166801b19 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.flink.types.Row; +import org.junit.Before; + +/** Test Flink SELECT SQLs. */ +public class TestFlinkScanSql extends TestFlinkSource { + + private volatile TableEnvironment tEnv; + + public TestFlinkScanSql(String fileFormat) { + super(fileFormat); + } + + @Before + public void before() throws IOException { + SqlHelpers.sql( + getTableEnv(), + "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", + catalogResource.warehouse()); + SqlHelpers.sql(getTableEnv(), "use catalog iceberg_catalog"); + getTableEnv() + .getConfig() + .getConfiguration() + .set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); + } + + private TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + if (tEnv == null) { + this.tEnv = + TableEnvironment.create(EnvironmentSettings.newInstance().inBatchMode().build()); + } + } + } + return tEnv; + } + + @Override + protected List run( + FlinkSource.Builder formatBuilder, + Map sqlOptions, + String sqlFilter, + String... sqlSelectedFields) { + String select = String.join(",", sqlSelectedFields); + String optionStr = SqlHelpers.sqlOptionsToString(sqlOptions); + return SqlHelpers.sql(getTableEnv(), "select %s from t %s %s", select, optionStr, sqlFilter); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java new file mode 100644 index 000000000000..2b55bee6e54c --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java @@ -0,0 +1,94 @@ +/* + * 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.iceberg.flink.source; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import org.apache.flink.table.api.TableColumn; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.types.Row; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +public abstract class TestFlinkSource extends TestFlinkScan { + + TestFlinkSource(String fileFormat) { + super(fileFormat); + } + + @Override + protected List runWithProjection(String... projected) throws Exception { + TableSchema.Builder builder = TableSchema.builder(); + TableSchema schema = + FlinkSchemaUtil.toSchema( + FlinkSchemaUtil.convert( + catalogResource.catalog().loadTable(TestFixtures.TABLE_IDENTIFIER).schema())); + for (String field : projected) { + TableColumn column = schema.getTableColumn(field).get(); + builder.field(column.getName(), column.getType()); + } + return run(FlinkSource.forRowData().project(builder.build()), Maps.newHashMap(), "", projected); + } + + @Override + protected List runWithFilter(Expression filter, String sqlFilter, boolean caseSensitive) + throws Exception { + FlinkSource.Builder builder = + FlinkSource.forRowData().filters(Collections.singletonList(filter)); + Map options = Maps.newHashMap(); + options.put("case-sensitive", Boolean.toString(caseSensitive)); + return run(builder, options, sqlFilter, "*"); + } + + @Override + protected List runWithOptions(Map options) throws Exception { + FlinkSource.Builder builder = FlinkSource.forRowData(); + Optional.ofNullable(options.get("case-sensitive")) + .ifPresent(value -> builder.caseSensitive(Boolean.getBoolean(value))); + Optional.ofNullable(options.get("snapshot-id")) + .ifPresent(value -> builder.snapshotId(Long.parseLong(value))); + Optional.ofNullable(options.get("tag")).ifPresent(value -> builder.tag(value)); + Optional.ofNullable(options.get("branch")).ifPresent(value -> builder.branch(value)); + Optional.ofNullable(options.get("start-tag")).ifPresent(value -> builder.startTag(value)); + Optional.ofNullable(options.get("end-tag")).ifPresent(value -> builder.endTag(value)); + Optional.ofNullable(options.get("start-snapshot-id")) + .ifPresent(value -> builder.startSnapshotId(Long.parseLong(value))); + Optional.ofNullable(options.get("end-snapshot-id")) + .ifPresent(value -> builder.endSnapshotId(Long.parseLong(value))); + Optional.ofNullable(options.get("as-of-timestamp")) + .ifPresent(value -> builder.asOfTimestamp(Long.parseLong(value))); + return run(builder, options, "", "*"); + } + + @Override + protected List run() throws Exception { + return run(FlinkSource.forRowData(), Maps.newHashMap(), "", "*"); + } + + protected abstract List run( + FlinkSource.Builder formatBuilder, + Map sqlOptions, + String sqlFilter, + String... sqlSelectedFields) + throws Exception; +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java new file mode 100644 index 000000000000..1814ff8f8542 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java @@ -0,0 +1,58 @@ +/* + * 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.iceberg.flink.source; + +import java.util.List; +import org.apache.flink.types.Row; +import org.apache.iceberg.flink.FlinkReadOptions; +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Test; + +public class TestFlinkSourceConfig extends TestFlinkTableSource { + private static final String TABLE = "test_table"; + + @Test + public void testFlinkSessionConfig() { + getTableEnv().getConfig().set(FlinkReadOptions.STREAMING_OPTION, true); + Assertions.assertThatThrownBy( + () -> sql("SELECT * FROM %s /*+ OPTIONS('as-of-timestamp'='1')*/", TABLE)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot set as-of-timestamp option for streaming reader"); + } + + @Test + public void testFlinkHintConfig() { + List result = + sql( + "SELECT * FROM %s /*+ OPTIONS('as-of-timestamp'='%d','streaming'='false')*/", + TABLE, System.currentTimeMillis()); + Assert.assertEquals(3, result.size()); + } + + @Test + public void testReadOptionHierarchy() { + getTableEnv().getConfig().set(FlinkReadOptions.LIMIT_OPTION, 1L); + List result = sql("SELECT * FROM %s", TABLE); + Assert.assertEquals(1, result.size()); + + result = sql("SELECT * FROM %s /*+ OPTIONS('limit'='3')*/", TABLE); + Assert.assertEquals(3, result.size()); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java new file mode 100644 index 000000000000..affd90c347dd --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java @@ -0,0 +1,85 @@ +/* + * 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.iceberg.flink.source; + +import java.io.IOException; +import java.util.List; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.PipelineOptions; +import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.flink.types.Row; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.Test; + +/** Use the FlinkSource */ +public class TestFlinkSourceSql extends TestSqlBase { + @Override + public void before() throws IOException { + SqlHelpers.sql( + getTableEnv(), + "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", + catalogResource.warehouse()); + SqlHelpers.sql(getTableEnv(), "use catalog iceberg_catalog"); + getTableEnv() + .getConfig() + .getConfiguration() + .set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); + } + + @Test + public void testInferParallelismWithGlobalSetting() throws IOException { + Configuration cfg = getTableEnv().getConfig().getConfiguration(); + cfg.set(PipelineOptions.MAX_PARALLELISM, 1); + + Table table = + catalogResource + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, null); + + GenericAppenderHelper helper = + new GenericAppenderHelper(table, FileFormat.PARQUET, TEMPORARY_FOLDER); + List expectedRecords = Lists.newArrayList(); + long maxFileLen = 0; + for (int i = 0; i < 5; i++) { + List records = RandomGenericData.generate(TestFixtures.SCHEMA, 2, i); + DataFile dataFile = helper.writeFile(null, records); + helper.appendToTable(dataFile); + expectedRecords.addAll(records); + maxFileLen = Math.max(dataFile.fileSizeInBytes(), maxFileLen); + } + + // Make sure to generate multiple CombinedScanTasks + SqlHelpers.sql( + getTableEnv(), + "ALTER TABLE t SET ('read.split.open-file-cost'='1', 'read.split.target-size'='%s')", + maxFileLen); + + List results = run(Maps.newHashMap(), "", "*"); + org.apache.iceberg.flink.TestHelpers.assertRecords( + results, expectedRecords, TestFixtures.SCHEMA); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java new file mode 100644 index 000000000000..ff14bc406276 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java @@ -0,0 +1,614 @@ +/* + * 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.iceberg.flink.source; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.table.api.SqlParserException; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.types.Row; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.events.Listeners; +import org.apache.iceberg.events.ScanEvent; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.flink.FlinkTestBase; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.assertj.core.api.Assertions; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +public class TestFlinkTableSource extends FlinkTestBase { + + private static final String CATALOG_NAME = "test_catalog"; + private static final String DATABASE_NAME = "test_db"; + private static final String TABLE_NAME = "test_table"; + private final FileFormat format = FileFormat.AVRO; + private static String warehouse; + + private int scanEventCount = 0; + private ScanEvent lastScanEvent = null; + + public TestFlinkTableSource() { + // register a scan event listener to validate pushdown + Listeners.register( + event -> { + scanEventCount += 1; + lastScanEvent = event; + }, + ScanEvent.class); + } + + @Override + protected TableEnvironment getTableEnv() { + super.getTableEnv().getConfig().getConfiguration().set(CoreOptions.DEFAULT_PARALLELISM, 1); + return super.getTableEnv(); + } + + @BeforeClass + public static void createWarehouse() throws IOException { + File warehouseFile = TEMPORARY_FOLDER.newFolder(); + Assert.assertTrue("The warehouse should be deleted", warehouseFile.delete()); + // before variables + warehouse = "file:" + warehouseFile; + } + + @Before + public void before() { + sql( + "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", + CATALOG_NAME, warehouse); + sql("USE CATALOG %s", CATALOG_NAME); + sql("CREATE DATABASE %s", DATABASE_NAME); + sql("USE %s", DATABASE_NAME); + sql( + "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('write.format.default'='%s')", + TABLE_NAME, format.name()); + sql( + "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)", + TABLE_NAME); + + this.scanEventCount = 0; + this.lastScanEvent = null; + } + + @After + public void clean() { + sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, TABLE_NAME); + sql("DROP DATABASE IF EXISTS %s", DATABASE_NAME); + dropCatalog(CATALOG_NAME, true); + } + + @Test + public void testLimitPushDown() { + + Assertions.assertThatThrownBy(() -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME)) + .isInstanceOf(SqlParserException.class) + .hasMessageStartingWith("SQL parse failed."); + + Assert.assertEquals( + "Should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size()); + + String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME); + List resultExceed = sql(sqlLimitExceed); + Assert.assertEquals("Should have 3 records", 3, resultExceed.size()); + List expectedList = + Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); + assertSameElements(expectedList, resultExceed); + + String querySql = String.format("SELECT * FROM %s LIMIT 1", TABLE_NAME); + String explain = getTableEnv().explainSql(querySql); + String expectedExplain = "limit=[1]"; + Assert.assertTrue("Explain should contain LimitPushDown", explain.contains(expectedExplain)); + List result = sql(querySql); + Assert.assertEquals("Should have 1 record", 1, result.size()); + Assertions.assertThat(result).containsAnyElementsOf(expectedList); + + String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME); + List mixedResult = sql(sqlMixed); + Assert.assertEquals("Should have 1 record", 1, mixedResult.size()); + Assert.assertEquals( + "Should produce the expected records", Row.of(1, "iceberg", 10.0), mixedResult.get(0)); + } + + @Test + public void testNoFilterPushDown() { + String sql = String.format("SELECT * FROM %s ", TABLE_NAME); + List result = sql(sql); + List expectedRecords = + Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); + assertSameElements(expectedRecords, result); + Assert.assertEquals( + "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + } + + @Test + public void testFilterPushDownEqual() { + String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME); + String expectedFilter = "ref(name=\"id\") == 1"; + + List result = sql(sqlLiteralRight); + Assert.assertEquals("Should have 1 record", 1, result.size()); + Assert.assertEquals( + "Should produce the expected record", Row.of(1, "iceberg", 10.0), result.get(0)); + + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownEqualNull() { + String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME); + + List result = sql(sqlEqualNull); + Assert.assertEquals("Should have 0 record", 0, result.size()); + Assert.assertNull("Should not push down a filter", lastScanEvent); + } + + @Test + public void testFilterPushDownEqualLiteralOnLeft() { + String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME); + String expectedFilter = "ref(name=\"id\") == 1"; + + List resultLeft = sql(sqlLiteralLeft); + Assert.assertEquals("Should have 1 record", 1, resultLeft.size()); + Assert.assertEquals( + "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultLeft.get(0)); + + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownNoEqual() { + String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME); + String expectedFilter = "ref(name=\"id\") != 1"; + + List resultNE = sql(sqlNE); + Assert.assertEquals("Should have 2 records", 2, resultNE.size()); + + List expectedNE = Lists.newArrayList(Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); + assertSameElements(expectedNE, resultNE); + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownNoEqualNull() { + String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME); + + List resultNE = sql(sqlNotEqualNull); + Assert.assertEquals("Should have 0 records", 0, resultNE.size()); + Assert.assertNull("Should not push down a filter", lastScanEvent); + } + + @Test + public void testFilterPushDownAnd() { + String sqlAnd = + String.format("SELECT * FROM %s WHERE id = 1 AND data = 'iceberg' ", TABLE_NAME); + + List resultAnd = sql(sqlAnd); + Assert.assertEquals("Should have 1 record", 1, resultAnd.size()); + Assert.assertEquals( + "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultAnd.get(0)); + + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + String expected = "(ref(name=\"id\") == 1 and ref(name=\"data\") == \"iceberg\")"; + Assert.assertEquals( + "Should contain the push down filter", expected, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownOr() { + String sqlOr = String.format("SELECT * FROM %s WHERE id = 1 OR data = 'b' ", TABLE_NAME); + String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"data\") == \"b\")"; + + List resultOr = sql(sqlOr); + Assert.assertEquals("Should have 2 record", 2, resultOr.size()); + + List expectedOR = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); + assertSameElements(expectedOR, resultOr); + + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownGreaterThan() { + String sqlGT = String.format("SELECT * FROM %s WHERE id > 1 ", TABLE_NAME); + String expectedFilter = "ref(name=\"id\") > 1"; + + List resultGT = sql(sqlGT); + Assert.assertEquals("Should have 2 record", 2, resultGT.size()); + + List expectedGT = Lists.newArrayList(Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); + assertSameElements(expectedGT, resultGT); + + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownGreaterThanNull() { + String sqlGT = String.format("SELECT * FROM %s WHERE data > null ", TABLE_NAME); + + List resultGT = sql(sqlGT); + Assert.assertEquals("Should have 0 record", 0, resultGT.size()); + Assert.assertNull("Should not push down a filter", lastScanEvent); + } + + @Test + public void testFilterPushDownGreaterThanLiteralOnLeft() { + String sqlGT = String.format("SELECT * FROM %s WHERE 3 > id ", TABLE_NAME); + String expectedFilter = "ref(name=\"id\") < 3"; + + List resultGT = sql(sqlGT); + Assert.assertEquals("Should have 2 records", 2, resultGT.size()); + + List expectedGT = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); + assertSameElements(expectedGT, resultGT); + + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownGreaterThanEqual() { + String sqlGTE = String.format("SELECT * FROM %s WHERE id >= 2 ", TABLE_NAME); + String expectedFilter = "ref(name=\"id\") >= 2"; + + List resultGTE = sql(sqlGTE); + Assert.assertEquals("Should have 2 records", 2, resultGTE.size()); + + List expectedGTE = Lists.newArrayList(Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); + assertSameElements(expectedGTE, resultGTE); + + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownGreaterThanEqualNull() { + String sqlGTE = String.format("SELECT * FROM %s WHERE data >= null ", TABLE_NAME); + + List resultGT = sql(sqlGTE); + Assert.assertEquals("Should have 0 record", 0, resultGT.size()); + Assert.assertNull("Should not push down a filter", lastScanEvent); + } + + @Test + public void testFilterPushDownGreaterThanEqualLiteralOnLeft() { + String sqlGTE = String.format("SELECT * FROM %s WHERE 2 >= id ", TABLE_NAME); + String expectedFilter = "ref(name=\"id\") <= 2"; + + List resultGTE = sql(sqlGTE); + Assert.assertEquals("Should have 2 records", 2, resultGTE.size()); + + List expectedGTE = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); + assertSameElements(expectedGTE, resultGTE); + + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownLessThan() { + String sqlLT = String.format("SELECT * FROM %s WHERE id < 2 ", TABLE_NAME); + String expectedFilter = "ref(name=\"id\") < 2"; + + List resultLT = sql(sqlLT); + Assert.assertEquals("Should have 1 record", 1, resultLT.size()); + Assert.assertEquals( + "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultLT.get(0)); + + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownLessThanNull() { + String sqlLT = String.format("SELECT * FROM %s WHERE data < null ", TABLE_NAME); + + List resultGT = sql(sqlLT); + Assert.assertEquals("Should have 0 record", 0, resultGT.size()); + Assert.assertNull("Should not push down a filter", lastScanEvent); + } + + @Test + public void testFilterPushDownLessThanLiteralOnLeft() { + String sqlLT = String.format("SELECT * FROM %s WHERE 2 < id ", TABLE_NAME); + String expectedFilter = "ref(name=\"id\") > 2"; + + List resultLT = sql(sqlLT); + Assert.assertEquals("Should have 1 record", 1, resultLT.size()); + Assert.assertEquals( + "Should produce the expected record", Row.of(3, null, 30.0), resultLT.get(0)); + + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownLessThanEqual() { + String sqlLTE = String.format("SELECT * FROM %s WHERE id <= 1 ", TABLE_NAME); + String expectedFilter = "ref(name=\"id\") <= 1"; + + List resultLTE = sql(sqlLTE); + Assert.assertEquals("Should have 1 record", 1, resultLTE.size()); + Assert.assertEquals( + "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultLTE.get(0)); + + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownLessThanEqualNull() { + String sqlLTE = String.format("SELECT * FROM %s WHERE data <= null ", TABLE_NAME); + + List resultGT = sql(sqlLTE); + Assert.assertEquals("Should have 0 record", 0, resultGT.size()); + Assert.assertNull("Should not push down a filter", lastScanEvent); + } + + @Test + public void testFilterPushDownLessThanEqualLiteralOnLeft() { + String sqlLTE = String.format("SELECT * FROM %s WHERE 3 <= id ", TABLE_NAME); + String expectedFilter = "ref(name=\"id\") >= 3"; + + List resultLTE = sql(sqlLTE); + Assert.assertEquals("Should have 1 record", 1, resultLTE.size()); + Assert.assertEquals( + "Should produce the expected record", Row.of(3, null, 30.0), resultLTE.get(0)); + + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownIn() { + String sqlIN = String.format("SELECT * FROM %s WHERE id IN (1,2) ", TABLE_NAME); + String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"id\") == 2)"; + List resultIN = sql(sqlIN); + Assert.assertEquals("Should have 2 records", 2, resultIN.size()); + + List expectedIN = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); + assertSameElements(expectedIN, resultIN); + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownInNull() { + String sqlInNull = + String.format("SELECT * FROM %s WHERE data IN ('iceberg',NULL) ", TABLE_NAME); + + List result = sql(sqlInNull); + Assert.assertEquals("Should have 1 record", 1, result.size()); + Assert.assertEquals( + "Should produce the expected record", Row.of(1, "iceberg", 10.0), result.get(0)); + + // In SQL, null check can only be done as IS NULL or IS NOT NULL, so it's correct to ignore it + // and push the rest down. + String expectedScan = "ref(name=\"data\") == \"iceberg\""; + Assert.assertEquals( + "Should contain the push down filter", expectedScan, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownNotIn() { + String sqlNotIn = String.format("SELECT * FROM %s WHERE id NOT IN (3,2) ", TABLE_NAME); + + List resultNotIn = sql(sqlNotIn); + Assert.assertEquals("Should have 1 record", 1, resultNotIn.size()); + Assert.assertEquals( + "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultNotIn.get(0)); + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + String expectedScan = "(ref(name=\"id\") != 2 and ref(name=\"id\") != 3)"; + Assert.assertEquals( + "Should contain the push down filter", expectedScan, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownNotInNull() { + String sqlNotInNull = String.format("SELECT * FROM %s WHERE id NOT IN (1,2,NULL) ", TABLE_NAME); + List resultGT = sql(sqlNotInNull); + Assert.assertEquals("Should have 0 record", 0, resultGT.size()); + Assert.assertNull( + "As the predicate pushdown filter out all rows, Flink did not create scan plan, so it doesn't publish any ScanEvent.", + lastScanEvent); + } + + @Test + public void testFilterPushDownIsNotNull() { + String sqlNotNull = String.format("SELECT * FROM %s WHERE data IS NOT NULL", TABLE_NAME); + String expectedFilter = "not_null(ref(name=\"data\"))"; + + List resultNotNull = sql(sqlNotNull); + Assert.assertEquals("Should have 2 record", 2, resultNotNull.size()); + + List expected = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); + assertSameElements(expected, resultNotNull); + + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownIsNull() { + String sqlNull = String.format("SELECT * FROM %s WHERE data IS NULL", TABLE_NAME); + String expectedFilter = "is_null(ref(name=\"data\"))"; + + List resultNull = sql(sqlNull); + Assert.assertEquals("Should have 1 record", 1, resultNull.size()); + Assert.assertEquals( + "Should produce the expected record", Row.of(3, null, 30.0), resultNull.get(0)); + + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownNot() { + String sqlNot = String.format("SELECT * FROM %s WHERE NOT (id = 1 OR id = 2 ) ", TABLE_NAME); + + List resultNot = sql(sqlNot); + Assert.assertEquals("Should have 1 record", 1, resultNot.size()); + Assert.assertEquals( + "Should produce the expected record", Row.of(3, null, 30.0), resultNot.get(0)); + + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + String expectedFilter = "(ref(name=\"id\") != 1 and ref(name=\"id\") != 2)"; + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownBetween() { + String sqlBetween = String.format("SELECT * FROM %s WHERE id BETWEEN 1 AND 2 ", TABLE_NAME); + + List resultBetween = sql(sqlBetween); + Assert.assertEquals("Should have 2 record", 2, resultBetween.size()); + + List expectedBetween = + Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); + assertSameElements(expectedBetween, resultBetween); + + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + String expected = "(ref(name=\"id\") >= 1 and ref(name=\"id\") <= 2)"; + Assert.assertEquals( + "Should contain the push down filter", expected, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownNotBetween() { + String sqlNotBetween = + String.format("SELECT * FROM %s WHERE id NOT BETWEEN 2 AND 3 ", TABLE_NAME); + String expectedFilter = "(ref(name=\"id\") < 2 or ref(name=\"id\") > 3)"; + + List resultNotBetween = sql(sqlNotBetween); + Assert.assertEquals("Should have 1 record", 1, resultNotBetween.size()); + Assert.assertEquals( + "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultNotBetween.get(0)); + + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownLike() { + String expectedFilter = "ref(name=\"data\") startsWith \"\"ice\"\""; + + String sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'ice%%' "; + List resultLike = sql(sqlLike); + Assert.assertEquals("Should have 1 record", 1, resultLike.size()); + Assert.assertEquals( + "The like result should produce the expected record", + Row.of(1, "iceberg", 10.0), + resultLike.get(0)); + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + + // %% won't match the row with null value + sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%' "; + resultLike = sql(sqlLike); + Assert.assertEquals("Should have 2 records", 2, resultLike.size()); + List expectedRecords = + Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); + assertSameElements(expectedRecords, resultLike); + String expectedScan = "not_null(ref(name=\"data\"))"; + Assert.assertEquals( + "Should contain the push down filter", expectedScan, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterNotPushDownLike() { + Row expectRecord = Row.of(1, "iceberg", 10.0); + String sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i' "; + List resultLike = sql(sqlNoPushDown); + Assert.assertEquals("Should have 0 record", 0, resultLike.size()); + Assert.assertEquals( + "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + + sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i%%' "; + resultLike = sql(sqlNoPushDown); + Assert.assertEquals("Should have 1 record", 1, resultLike.size()); + Assert.assertEquals("Should produce the expected record", expectRecord, resultLike.get(0)); + Assert.assertEquals( + "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + + sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%ice%%g' "; + resultLike = sql(sqlNoPushDown); + Assert.assertEquals("Should have 1 record", 1, resultLike.size()); + Assert.assertEquals("Should produce the expected record", expectRecord, resultLike.get(0)); + Assert.assertEquals( + "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + + sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'iceber_' "; + resultLike = sql(sqlNoPushDown); + Assert.assertEquals("Should have 1 record", 1, resultLike.size()); + Assert.assertEquals("Should produce the expected record", expectRecord, resultLike.get(0)); + Assert.assertEquals( + "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + + sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'i%%g' "; + resultLike = sql(sqlNoPushDown); + Assert.assertEquals("Should have 1 record", 1, resultLike.size()); + Assert.assertEquals("Should produce the expected record", expectRecord, resultLike.get(0)); + Assert.assertEquals( + "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + } + + @Test + public void testFilterPushDown2Literal() { + String sql2Literal = String.format("SELECT * FROM %s WHERE 1 > 0 ", TABLE_NAME); + List result = sql(sql2Literal); + List expectedRecords = + Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); + assertSameElements(expectedRecords, result); + Assert.assertEquals( + "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + } + + @Test + public void testSqlParseNaN() { + // todo add some test case to test NaN + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java new file mode 100644 index 000000000000..a80f87d648d5 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java @@ -0,0 +1,134 @@ +/* + * 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.iceberg.flink.source; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableColumn; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.data.RowDataToRowMapper; +import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestIcebergSourceBounded extends TestFlinkScan { + + public TestIcebergSourceBounded(String fileFormat) { + super(fileFormat); + } + + @Override + protected List runWithProjection(String... projected) throws Exception { + Schema icebergTableSchema = + catalogResource.catalog().loadTable(TestFixtures.TABLE_IDENTIFIER).schema(); + TableSchema.Builder builder = TableSchema.builder(); + TableSchema schema = FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(icebergTableSchema)); + for (String field : projected) { + TableColumn column = schema.getTableColumn(field).get(); + builder.field(column.getName(), column.getType()); + } + TableSchema flinkSchema = builder.build(); + Schema projectedSchema = FlinkSchemaUtil.convert(icebergTableSchema, flinkSchema); + return run(projectedSchema, Lists.newArrayList(), Maps.newHashMap(), "", projected); + } + + @Override + protected List runWithFilter(Expression filter, String sqlFilter, boolean caseSensitive) + throws Exception { + Map options = Maps.newHashMap(); + options.put("case-sensitive", Boolean.toString(caseSensitive)); + return run(null, Collections.singletonList(filter), options, sqlFilter, "*"); + } + + @Override + protected List runWithOptions(Map options) throws Exception { + return run(null, Lists.newArrayList(), options, "", "*"); + } + + @Override + protected List run() throws Exception { + return run(null, Lists.newArrayList(), Maps.newHashMap(), "", "*"); + } + + protected List run( + Schema projectedSchema, + List filters, + Map options, + String sqlFilter, + String... sqlSelectedFields) + throws Exception { + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + Configuration config = new Configuration(); + config.setInteger(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 128); + Table table; + try (TableLoader tableLoader = tableLoader()) { + tableLoader.open(); + table = tableLoader.loadTable(); + } + + IcebergSource.Builder sourceBuilder = + IcebergSource.forRowData() + .tableLoader(tableLoader()) + .table(table) + .assignerFactory(new SimpleSplitAssignerFactory()) + .flinkConfig(config); + if (projectedSchema != null) { + sourceBuilder.project(projectedSchema); + } + + sourceBuilder.filters(filters); + sourceBuilder.properties(options); + + DataStream stream = + env.fromSource( + sourceBuilder.build(), + WatermarkStrategy.noWatermarks(), + "testBasicRead", + TypeInformation.of(RowData.class)) + .map( + new RowDataToRowMapper( + FlinkSchemaUtil.convert( + projectedSchema == null ? table.schema() : projectedSchema))); + + try (CloseableIterator iter = stream.executeAndCollect()) { + return Lists.newArrayList(iter); + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java new file mode 100644 index 000000000000..0337f3597053 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java @@ -0,0 +1,203 @@ +/* + * 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.iceberg.flink.source; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import org.apache.avro.generic.GenericRecord; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.formats.avro.typeutils.GenericRecordAvroTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.HadoopCatalogResource; +import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.flink.data.RowDataToRowMapper; +import org.apache.iceberg.flink.sink.AvroGenericRecordToRowDataMapper; +import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; +import org.apache.iceberg.flink.source.reader.AvroGenericRecordReaderFunction; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.TypeUtil; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestIcebergSourceBoundedGenericRecord { + @ClassRule + public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = + MiniClusterResource.createWithClassloaderCheckDisabled(); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final HadoopCatalogResource catalogResource = + new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); + + @Parameterized.Parameters(name = "format={0}, parallelism = {1}") + public static Object[][] parameters() { + return new Object[][] { + {"avro", 2}, + {"parquet", 2}, + {"orc", 2} + }; + } + + private final FileFormat fileFormat; + private final int parallelism; + + public TestIcebergSourceBoundedGenericRecord(String format, int parallelism) { + this.fileFormat = FileFormat.valueOf(format.toUpperCase(Locale.ENGLISH)); + this.parallelism = parallelism; + } + + @Test + public void testUnpartitionedTable() throws Exception { + Table table = + catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); + new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(expectedRecords); + TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); + } + + @Test + public void testPartitionedTable() throws Exception { + String dateStr = "2020-03-20"; + Table table = + catalogResource + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); + List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); + for (int i = 0; i < expectedRecords.size(); ++i) { + expectedRecords.get(i).setField("dt", dateStr); + } + + new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER) + .appendToTable(org.apache.iceberg.TestHelpers.Row.of(dateStr, 0), expectedRecords); + TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); + } + + @Test + public void testProjection() throws Exception { + Table table = + catalogResource + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); + List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); + new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER) + .appendToTable(org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), expectedRecords); + // select the "data" field (fieldId == 1) + Schema projectedSchema = TypeUtil.select(TestFixtures.SCHEMA, Sets.newHashSet(1)); + List expectedRows = + Arrays.asList(Row.of(expectedRecords.get(0).get(0)), Row.of(expectedRecords.get(1).get(0))); + TestHelpers.assertRows( + run(projectedSchema, Collections.emptyList(), Collections.emptyMap()), expectedRows); + } + + private List run() throws Exception { + return run(null, Collections.emptyList(), Collections.emptyMap()); + } + + private List run( + Schema projectedSchema, List filters, Map options) + throws Exception { + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(parallelism); + env.getConfig().enableObjectReuse(); + + Configuration config = new Configuration(); + config.setInteger(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 128); + Table table; + try (TableLoader tableLoader = catalogResource.tableLoader()) { + tableLoader.open(); + table = tableLoader.loadTable(); + } + + AvroGenericRecordReaderFunction readerFunction = + new AvroGenericRecordReaderFunction( + TestFixtures.TABLE_IDENTIFIER.name(), + new Configuration(), + table.schema(), + null, + null, + false, + table.io(), + table.encryption(), + filters); + + IcebergSource.Builder sourceBuilder = + IcebergSource.builder() + .tableLoader(catalogResource.tableLoader()) + .readerFunction(readerFunction) + .assignerFactory(new SimpleSplitAssignerFactory()) + .flinkConfig(config); + if (projectedSchema != null) { + sourceBuilder.project(projectedSchema); + } + + sourceBuilder.filters(filters); + sourceBuilder.setAll(options); + + Schema readSchema = projectedSchema != null ? projectedSchema : table.schema(); + RowType rowType = FlinkSchemaUtil.convert(readSchema); + org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(readSchema, TestFixtures.TABLE_IDENTIFIER.name()); + + DataStream stream = + env.fromSource( + sourceBuilder.build(), + WatermarkStrategy.noWatermarks(), + "testBasicRead", + new GenericRecordAvroTypeInfo(avroSchema)) + // There are two reasons for converting GenericRecord back to Row. + // 1. Avro GenericRecord/Schema is not serializable. + // 2. leverage the TestHelpers.assertRecords for validation. + .map(AvroGenericRecordToRowDataMapper.forAvroSchema(avroSchema)) + .map(new RowDataToRowMapper(rowType)); + + try (CloseableIterator iter = stream.executeAndCollect()) { + return Lists.newArrayList(iter); + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java new file mode 100644 index 000000000000..3652e0bb56df --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java @@ -0,0 +1,80 @@ +/* + * 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.iceberg.flink.source; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.flink.types.Row; +import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.junit.Before; + +public class TestIcebergSourceBoundedSql extends TestIcebergSourceBounded { + private volatile TableEnvironment tEnv; + + public TestIcebergSourceBoundedSql(String fileFormat) { + super(fileFormat); + } + + @Before + public void before() throws IOException { + Configuration tableConf = getTableEnv().getConfig().getConfiguration(); + tableConf.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), true); + SqlHelpers.sql( + getTableEnv(), + "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", + catalogResource.warehouse()); + SqlHelpers.sql(getTableEnv(), "use catalog iceberg_catalog"); + getTableEnv() + .getConfig() + .getConfiguration() + .set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); + } + + private TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + if (tEnv == null) { + this.tEnv = + TableEnvironment.create(EnvironmentSettings.newInstance().inBatchMode().build()); + } + } + } + return tEnv; + } + + @Override + protected List run( + Schema projectedSchema, + List filters, + Map options, + String sqlFilter, + String... sqlSelectedFields) + throws Exception { + String select = String.join(",", sqlSelectedFields); + String optionStr = SqlHelpers.sqlOptionsToString(options); + return SqlHelpers.sql(getTableEnv(), "select %s from t %s %s", select, optionStr, sqlFilter); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java new file mode 100644 index 000000000000..31e9733fcd60 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java @@ -0,0 +1,421 @@ +/* + * 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.iceberg.flink.source; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.time.Duration; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.runtime.client.JobStatusMessage; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.RowData; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.flink.data.RowDataToRowMapper; +import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.awaitility.Awaitility; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestIcebergSourceContinuous { + + @ClassRule + public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = + MiniClusterResource.createWithClassloaderCheckDisabled(); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final HadoopTableResource tableResource = + new HadoopTableResource( + TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); + + private final AtomicLong randomSeed = new AtomicLong(0L); + + @Test + public void testTableScanThenIncremental() throws Exception { + GenericAppenderHelper dataAppender = + new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + + // snapshot1 + List batch1 = + RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch1); + + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10L)) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + + try (CloseableIterator iter = + createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { + List result1 = waitForResult(iter, 2); + TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); + + // snapshot2 + List batch2 = + RandomGenericData.generate( + tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch2); + tableResource.table().currentSnapshot().snapshotId(); + + List result2 = waitForResult(iter, 2); + TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + + // snapshot3 + List batch3 = + RandomGenericData.generate( + tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch3); + tableResource.table().currentSnapshot().snapshotId(); + + List result3 = waitForResult(iter, 2); + TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + } + } + + @Test + public void testTableScanThenIncrementalAfterExpiration() throws Exception { + GenericAppenderHelper dataAppender = + new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + + // snapshot1 + List batch1 = + RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch1); + long snapshotId = tableResource.table().currentSnapshot().snapshotId(); + + // snapshot2 + List batch2 = + RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch2); + + tableResource.table().expireSnapshots().expireSnapshotId(snapshotId).commit(); + + Assert.assertEquals(1, tableResource.table().history().size()); + + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10L)) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + + Assert.assertEquals( + FlinkSplitPlanner.ScanMode.BATCH, FlinkSplitPlanner.checkScanMode(scanContext)); + + try (CloseableIterator iter = + createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { + List result1 = waitForResult(iter, 4); + List initialRecords = Lists.newArrayList(); + initialRecords.addAll(batch1); + initialRecords.addAll(batch2); + TestHelpers.assertRecords(result1, initialRecords, tableResource.table().schema()); + + // snapshot3 + List batch3 = + RandomGenericData.generate( + tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch3); + tableResource.table().currentSnapshot().snapshotId(); + + List result3 = waitForResult(iter, 2); + TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + } + } + + @Test + public void testEarliestSnapshot() throws Exception { + GenericAppenderHelper dataAppender = + new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + + // snapshot0 + List batch0 = + RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch0); + + // snapshot1 + List batch1 = + RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch1); + + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10L)) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) + .build(); + + try (CloseableIterator iter = + createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { + List result1 = waitForResult(iter, 4); + List combinedBatch0AndBatch1 = Lists.newArrayList(batch0); + combinedBatch0AndBatch1.addAll(batch1); + TestHelpers.assertRecords(result1, combinedBatch0AndBatch1, tableResource.table().schema()); + + // snapshot2 + List batch2 = + RandomGenericData.generate( + tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch2); + + List result2 = waitForResult(iter, 2); + TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + + // snapshot3 + List batch3 = + RandomGenericData.generate( + tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch3); + + List result3 = waitForResult(iter, 2); + TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + } + } + + @Test + public void testLatestSnapshot() throws Exception { + GenericAppenderHelper dataAppender = + new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + + // snapshot0 + List batch0 = + RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch0); + + // snapshot1 + List batch1 = + RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch1); + + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10L)) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) + .build(); + + try (CloseableIterator iter = + createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { + // we want to make sure job is running first so that enumerator can + // start from the latest snapshot before inserting the next batch2 below. + waitUntilJobIsRunning(MINI_CLUSTER_RESOURCE.getClusterClient()); + + // inclusive behavior for starting snapshot + List result1 = waitForResult(iter, 2); + TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); + + // snapshot2 + List batch2 = + RandomGenericData.generate( + tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch2); + + List result2 = waitForResult(iter, 2); + TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + + // snapshot3 + List batch3 = + RandomGenericData.generate( + tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch3); + + List result3 = waitForResult(iter, 2); + TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + } + } + + @Test + public void testSpecificSnapshotId() throws Exception { + GenericAppenderHelper dataAppender = + new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + + // snapshot0 + List batch0 = + RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch0); + long snapshot0 = tableResource.table().currentSnapshot().snapshotId(); + + // snapshot1 + List batch1 = + RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch1); + long snapshot1 = tableResource.table().currentSnapshot().snapshotId(); + + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10L)) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) + .startSnapshotId(snapshot1) + .build(); + + try (CloseableIterator iter = + createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { + List result1 = waitForResult(iter, 2); + TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); + + // snapshot2 + List batch2 = + RandomGenericData.generate( + tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch2); + + List result2 = waitForResult(iter, 2); + TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + + // snapshot3 + List batch3 = + RandomGenericData.generate( + tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch3); + + List result3 = waitForResult(iter, 2); + TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + } + } + + @Test + public void testSpecificSnapshotTimestamp() throws Exception { + GenericAppenderHelper dataAppender = + new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + + // snapshot0 + List batch0 = + RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch0); + long snapshot0Timestamp = tableResource.table().currentSnapshot().timestampMillis(); + + // sleep for 2 ms to make sure snapshot1 has a higher timestamp value + Thread.sleep(2); + + // snapshot1 + List batch1 = + RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch1); + long snapshot1Timestamp = tableResource.table().currentSnapshot().timestampMillis(); + + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10L)) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) + .startSnapshotTimestamp(snapshot1Timestamp) + .build(); + + try (CloseableIterator iter = + createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { + // consume data from snapshot1 + List result1 = waitForResult(iter, 2); + TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); + + // snapshot2 + List batch2 = + RandomGenericData.generate( + tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch2); + + List result2 = waitForResult(iter, 2); + TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + + // snapshot3 + List batch3 = + RandomGenericData.generate( + tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch3); + + List result3 = waitForResult(iter, 2); + TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + } + } + + private DataStream createStream(ScanContext scanContext) throws Exception { + // start the source and collect output + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + DataStream stream = + env.fromSource( + IcebergSource.forRowData() + .tableLoader(tableResource.tableLoader()) + .assignerFactory(new SimpleSplitAssignerFactory()) + .streaming(scanContext.isStreaming()) + .streamingStartingStrategy(scanContext.streamingStartingStrategy()) + .startSnapshotTimestamp(scanContext.startSnapshotTimestamp()) + .startSnapshotId(scanContext.startSnapshotId()) + .monitorInterval(Duration.ofMillis(10L)) + .build(), + WatermarkStrategy.noWatermarks(), + "icebergSource", + TypeInformation.of(RowData.class)) + .map(new RowDataToRowMapper(FlinkSchemaUtil.convert(tableResource.table().schema()))); + return stream; + } + + public static List waitForResult(CloseableIterator iter, int limit) { + List results = Lists.newArrayListWithCapacity(limit); + while (results.size() < limit) { + if (iter.hasNext()) { + results.add(iter.next()); + } else { + break; + } + } + return results; + } + + public static void waitUntilJobIsRunning(ClusterClient client) { + Awaitility.await("job should be running") + .atMost(Duration.ofSeconds(30)) + .pollInterval(Duration.ofMillis(10)) + .untilAsserted(() -> assertThat(getRunningJobs(client)).isNotEmpty()); + } + + public static List getRunningJobs(ClusterClient client) throws Exception { + Collection statusMessages = client.listJobs().get(); + return statusMessages.stream() + .filter(status -> status.getJobState() == JobStatus.RUNNING) + .map(JobStatusMessage::getJobId) + .collect(Collectors.toList()); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java new file mode 100644 index 000000000000..7d991ee603c9 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java @@ -0,0 +1,297 @@ +/* + * 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.iceberg.flink.source; + +import java.time.Duration; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.highavailability.nonha.embedded.HaLeadershipControl; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.minicluster.RpcServiceSharing; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.RowData; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.sink.FlinkSink; +import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestIcebergSourceFailover { + + private static final int PARALLELISM = 4; + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final MiniClusterWithClientResource miniClusterResource = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(PARALLELISM) + .setRpcServiceSharing(RpcServiceSharing.DEDICATED) + .withHaLeadershipControl() + .build()); + + @Rule + public final HadoopTableResource sourceTableResource = + new HadoopTableResource( + TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, schema()); + + @Rule + public final HadoopTableResource sinkTableResource = + new HadoopTableResource( + TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.SINK_TABLE, schema()); + + protected IcebergSource.Builder sourceBuilder() { + Configuration config = new Configuration(); + config.setInteger(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 128); + return IcebergSource.forRowData() + .tableLoader(sourceTableResource.tableLoader()) + .assignerFactory(new SimpleSplitAssignerFactory()) + .flinkConfig(config); + } + + protected Schema schema() { + return TestFixtures.SCHEMA; + } + + protected List generateRecords(int numRecords, long seed) { + return RandomGenericData.generate(schema(), numRecords, seed); + } + + protected void assertRecords(Table table, List expectedRecords, Duration timeout) + throws Exception { + SimpleDataUtil.assertTableRecords(table, expectedRecords, timeout); + } + + @Test + public void testBoundedWithTaskManagerFailover() throws Exception { + testBoundedIcebergSource(FailoverType.TM); + } + + @Test + public void testBoundedWithJobManagerFailover() throws Exception { + testBoundedIcebergSource(FailoverType.JM); + } + + private void testBoundedIcebergSource(FailoverType failoverType) throws Exception { + List expectedRecords = Lists.newArrayList(); + GenericAppenderHelper dataAppender = + new GenericAppenderHelper( + sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + for (int i = 0; i < 4; ++i) { + List records = generateRecords(2, i); + expectedRecords.addAll(records); + dataAppender.appendToTable(records); + } + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(PARALLELISM); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); + + DataStream stream = + env.fromSource( + sourceBuilder().build(), + WatermarkStrategy.noWatermarks(), + "IcebergSource", + TypeInformation.of(RowData.class)); + + DataStream streamFailingInTheMiddleOfReading = + RecordCounterToFail.wrapWithFailureAfter(stream, expectedRecords.size() / 2); + + // CollectStreamSink from DataStream#executeAndCollect() doesn't guarantee + // exactly-once behavior. When Iceberg sink, we can verify end-to-end + // exactly-once. Here we mainly about source exactly-once behavior. + FlinkSink.forRowData(streamFailingInTheMiddleOfReading) + .table(sinkTableResource.table()) + .tableLoader(sinkTableResource.tableLoader()) + .append(); + + JobClient jobClient = env.executeAsync("Bounded Iceberg Source Failover Test"); + JobID jobId = jobClient.getJobID(); + + RecordCounterToFail.waitToFail(); + triggerFailover( + failoverType, + jobId, + RecordCounterToFail::continueProcessing, + miniClusterResource.getMiniCluster()); + + assertRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); + } + + @Test + public void testContinuousWithTaskManagerFailover() throws Exception { + testContinuousIcebergSource(FailoverType.TM); + } + + @Test + public void testContinuousWithJobManagerFailover() throws Exception { + testContinuousIcebergSource(FailoverType.JM); + } + + private void testContinuousIcebergSource(FailoverType failoverType) throws Exception { + GenericAppenderHelper dataAppender = + new GenericAppenderHelper( + sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + List expectedRecords = Lists.newArrayList(); + + List batch = generateRecords(2, 0); + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(PARALLELISM); + env.enableCheckpointing(10L); + Configuration config = new Configuration(); + config.setInteger(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 128); + + DataStream stream = + env.fromSource( + sourceBuilder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10)) + .streamingStartingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(), + WatermarkStrategy.noWatermarks(), + "IcebergSource", + TypeInformation.of(RowData.class)); + + // CollectStreamSink from DataStream#executeAndCollect() doesn't guarantee + // exactly-once behavior. When Iceberg sink, we can verify end-to-end + // exactly-once. Here we mainly about source exactly-once behavior. + FlinkSink.forRowData(stream) + .table(sinkTableResource.table()) + .tableLoader(sinkTableResource.tableLoader()) + .append(); + + JobClient jobClient = env.executeAsync("Continuous Iceberg Source Failover Test"); + JobID jobId = jobClient.getJobID(); + + for (int i = 1; i < 5; i++) { + Thread.sleep(10); + List records = generateRecords(2, i); + expectedRecords.addAll(records); + dataAppender.appendToTable(records); + if (i == 2) { + triggerFailover(failoverType, jobId, () -> {}, miniClusterResource.getMiniCluster()); + } + } + + // wait longer for continuous source to reduce flakiness + // because CI servers tend to be overloaded. + assertRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); + } + + // ------------------------------------------------------------------------ + // test utilities copied from Flink's FileSourceTextLinesITCase + // ------------------------------------------------------------------------ + + private enum FailoverType { + NONE, + TM, + JM + } + + private static void triggerFailover( + FailoverType type, JobID jobId, Runnable afterFailAction, MiniCluster miniCluster) + throws Exception { + switch (type) { + case NONE: + afterFailAction.run(); + break; + case TM: + restartTaskManager(afterFailAction, miniCluster); + break; + case JM: + triggerJobManagerFailover(jobId, afterFailAction, miniCluster); + break; + } + } + + private static void triggerJobManagerFailover( + JobID jobId, Runnable afterFailAction, MiniCluster miniCluster) throws Exception { + HaLeadershipControl haLeadershipControl = miniCluster.getHaLeadershipControl().get(); + haLeadershipControl.revokeJobMasterLeadership(jobId).get(); + afterFailAction.run(); + haLeadershipControl.grantJobMasterLeadership(jobId).get(); + } + + private static void restartTaskManager(Runnable afterFailAction, MiniCluster miniCluster) + throws Exception { + miniCluster.terminateTaskManager(0).get(); + afterFailAction.run(); + miniCluster.startTaskManager(); + } + + private static class RecordCounterToFail { + + private static AtomicInteger records; + private static CompletableFuture fail; + private static CompletableFuture continueProcessing; + + private static DataStream wrapWithFailureAfter(DataStream stream, int failAfter) { + + records = new AtomicInteger(); + fail = new CompletableFuture<>(); + continueProcessing = new CompletableFuture<>(); + return stream.map( + record -> { + boolean reachedFailPoint = records.incrementAndGet() > failAfter; + boolean notFailedYet = !fail.isDone(); + if (notFailedYet && reachedFailPoint) { + fail.complete(null); + continueProcessing.get(); + } + return record; + }); + } + + private static void waitToFail() throws ExecutionException, InterruptedException { + fail.get(); + } + + private static void continueProcessing() { + continueProcessing.complete(null); + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java new file mode 100644 index 000000000000..f7dc931c506c --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.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.iceberg.flink.source; + +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.types.Comparators; +import org.apache.iceberg.util.StructLikeWrapper; +import org.awaitility.Awaitility; + +public class TestIcebergSourceFailoverWithWatermarkExtractor extends TestIcebergSourceFailover { + // Increment ts by 15 minutes for each generateRecords batch + private static final long RECORD_BATCH_TS_INCREMENT_MILLI = TimeUnit.MINUTES.toMillis(15); + // Within a batch, increment ts by 1 second + private static final long RECORD_TS_INCREMENT_MILLI = TimeUnit.SECONDS.toMillis(1); + + private final AtomicLong tsMilli = new AtomicLong(System.currentTimeMillis()); + + @Override + protected IcebergSource.Builder sourceBuilder() { + return IcebergSource.builder() + .tableLoader(sourceTableResource.tableLoader()) + .watermarkColumn("ts") + .project(TestFixtures.TS_SCHEMA); + } + + @Override + protected Schema schema() { + return TestFixtures.TS_SCHEMA; + } + + @Override + protected List generateRecords(int numRecords, long seed) { + // Override the ts field to create a more realistic situation for event time alignment + tsMilli.addAndGet(RECORD_BATCH_TS_INCREMENT_MILLI); + return RandomGenericData.generate(schema(), numRecords, seed).stream() + .peek( + record -> { + LocalDateTime ts = + LocalDateTime.ofInstant( + Instant.ofEpochMilli(tsMilli.addAndGet(RECORD_TS_INCREMENT_MILLI)), + ZoneId.of("Z")); + record.setField("ts", ts); + }) + .collect(Collectors.toList()); + } + + /** + * This override is needed because {@link Comparators} used by {@link StructLikeWrapper} retrieves + * Timestamp type using Long type as inner class, while the {@link RandomGenericData} generates + * {@link LocalDateTime} for {@code TimestampType.withoutZone()}. This method normalizes the + * {@link LocalDateTime} to a Long type so that Comparators can continue to work. + */ + @Override + protected void assertRecords(Table table, List expectedRecords, Duration timeout) + throws Exception { + List expectedNormalized = convertLocalDateTimeToMilli(expectedRecords); + Awaitility.await("expected list of records should be produced") + .atMost(timeout) + .untilAsserted( + () -> { + SimpleDataUtil.equalsRecords( + expectedNormalized, + convertLocalDateTimeToMilli(SimpleDataUtil.tableRecords(table)), + table.schema()); + SimpleDataUtil.assertRecordsEqual( + expectedNormalized, + convertLocalDateTimeToMilli(SimpleDataUtil.tableRecords(table)), + table.schema()); + }); + } + + private List convertLocalDateTimeToMilli(List records) { + return records.stream() + .peek( + r -> { + LocalDateTime localDateTime = ((LocalDateTime) r.getField("ts")); + r.setField("ts", localDateTime.atZone(ZoneOffset.UTC).toInstant().toEpochMilli()); + }) + .collect(Collectors.toList()); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java new file mode 100644 index 000000000000..2974f4bc94a2 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java @@ -0,0 +1,114 @@ +/* + * 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.iceberg.flink.source; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.CloseableIterator; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.CatalogLoader; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.StructLikeSet; +import org.junit.ClassRule; +import org.junit.rules.TemporaryFolder; + +public class TestIcebergSourceReaderDeletes extends TestFlinkReaderDeletesBase { + + private static final int PARALLELISM = 4; + + @ClassRule public static final TemporaryFolder TMP_FOLDER = new TemporaryFolder(); + + @ClassRule + public static final MiniClusterWithClientResource MINI_CLUSTER = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(PARALLELISM) + .build()); + + public TestIcebergSourceReaderDeletes(FileFormat inputFormat) { + super(inputFormat); + } + + @Override + protected StructLikeSet rowSet(String tableName, Table testTable, String... columns) + throws IOException { + Schema projected = testTable.schema().select(columns); + RowType rowType = FlinkSchemaUtil.convert(projected); + + Map properties = Maps.newHashMap(); + properties.put( + CatalogProperties.WAREHOUSE_LOCATION, + hiveConf.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname)); + properties.put(CatalogProperties.URI, hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname)); + properties.put( + CatalogProperties.CLIENT_POOL_SIZE, + Integer.toString(hiveConf.getInt("iceberg.hive.client-pool-size", 5))); + CatalogLoader hiveCatalogLoader = CatalogLoader.hive(catalog.name(), hiveConf, properties); + TableLoader hiveTableLoader = + TableLoader.fromCatalog(hiveCatalogLoader, TableIdentifier.of("default", tableName)); + hiveTableLoader.open(); + try (TableLoader tableLoader = hiveTableLoader) { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + DataStream stream = + env.fromSource( + IcebergSource.builder() + .tableLoader(tableLoader) + .assignerFactory(new SimpleSplitAssignerFactory()) + .project(projected) + .build(), + WatermarkStrategy.noWatermarks(), + "testBasicRead", + TypeInformation.of(RowData.class)); + + try (CloseableIterator iter = stream.executeAndCollect()) { + List rowDataList = Lists.newArrayList(iter); + StructLikeSet set = StructLikeSet.create(projected.asStruct()); + rowDataList.forEach( + rowData -> { + RowDataWrapper wrapper = new RowDataWrapper(rowType, projected.asStruct()); + set.add(wrapper.wrap(rowData)); + }); + return set; + } catch (Exception e) { + throw new IOException("Failed to collect result", e); + } + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java new file mode 100644 index 000000000000..e66ae79c28f8 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java @@ -0,0 +1,42 @@ +/* + * 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.iceberg.flink.source; + +import java.io.IOException; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.iceberg.flink.FlinkConfigOptions; + +/** Use the IcebergSource (FLIP-27) */ +public class TestIcebergSourceSql extends TestSqlBase { + @Override + public void before() throws IOException { + Configuration tableConf = getTableEnv().getConfig().getConfiguration(); + tableConf.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), true); + SqlHelpers.sql( + getTableEnv(), + "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", + catalogResource.warehouse()); + SqlHelpers.sql(getTableEnv(), "use catalog iceberg_catalog"); + getTableEnv() + .getConfig() + .getConfiguration() + .set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java new file mode 100644 index 000000000000..0bb2eb7766e9 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -0,0 +1,451 @@ +/* + * 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.iceberg.flink.source; + +import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +import java.io.Serializable; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.util.Collection; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.eventtime.SerializableTimestampAssigner; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.runtime.metrics.MetricNames; +import org.apache.flink.runtime.minicluster.RpcServiceSharing; +import org.apache.flink.runtime.testutils.InMemoryReporter; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.CloseableIterator; +import org.apache.flink.util.Collector; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.RowDataConverter; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.awaitility.Awaitility; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestIcebergSourceWithWatermarkExtractor implements Serializable { + private static final InMemoryReporter reporter = InMemoryReporter.createWithRetainedMetrics(); + private static final int PARALLELISM = 4; + private static final String SOURCE_NAME = "IcebergSource"; + private static final int RECORD_NUM_FOR_2_SPLITS = 200; + private static final ConcurrentMap windows = Maps.newConcurrentMap(); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final MiniClusterWithClientResource miniClusterResource = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(PARALLELISM) + .setRpcServiceSharing(RpcServiceSharing.DEDICATED) + .setConfiguration(reporter.addToConfiguration(new Configuration())) + .withHaLeadershipControl() + .build()); + + @Rule + public final HadoopTableResource sourceTableResource = + new HadoopTableResource( + TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.TS_SCHEMA); + + /** + * This is an integration test for watermark handling and windowing. Integration testing the + * following features: + * + *

      + *
    • - Ordering of the splits + *
    • - Emitting of watermarks + *
    • - Firing windows based on watermarks + *
    + * + *

    The test generates 4 splits + * + *

      + *
    • - Split 1 - Watermark 100 min + *
    • - Split 2, 3 - Watermark 0 min + *
    • - Split 4 - Watermark 6 min + *
    + * + *

    Creates a source with 5 minutes tumbling window with parallelism 1 (to prevent concurrency + * issues). + * + *

    Checks that windows are handled correctly based on the emitted watermarks, and splits are + * read in the following order: + * + *

      + *
    • - Split 2, 3 + *
    • - Split 4 + *
    • - Split 1 + *
    + * + *

    As a result the window aggregator emits the records based on in Split 2-3, and Split 4 data. + * + *

    Add 2 more splits, so the task manager close the windows for the original 4 splits and emit + * the appropriate aggregated records. + */ + @Test + public void testWindowing() throws Exception { + GenericAppenderHelper dataAppender = appender(); + List expectedRecords = Lists.newArrayList(); + + // Generate records with the following pattern: + // - File 1 - Later records (Watermark 6000000) + // - Split 1 - 2 records (100, "file_1-recordTs_100"), (103, "file_1-recordTs_103") + // - File 2 - First records (Watermark 0) + // - Split 1 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - Split 2 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - File 3 - Parallel write for the first records (Watermark 360000) + // - Split 1 - 2 records (6, "file_3-recordTs_6"), (7, "file_3-recordTs_7") + List batch = + ImmutableList.of( + generateRecord(100, "file_1-recordTs_100"), + generateRecord(101, "file_1-recordTs_101"), + generateRecord(103, "file_1-recordTs_103")); + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + batch = Lists.newArrayListWithCapacity(100); + for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { + // Generate records where the timestamps are out of order, but still between 0-5 minutes + batch.add(generateRecord(4 - i % 5, "file_2-recordTs_" + i)); + } + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + batch = + ImmutableList.of( + generateRecord(6, "file_3-recordTs_6"), generateRecord(7, "file_3-recordTs_7")); + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + DataStream stream = + env.fromSource( + source(), + WatermarkStrategy.noWatermarks() + .withTimestampAssigner(new RowDataTimestampAssigner()), + SOURCE_NAME, + TypeInformation.of(RowData.class)); + + stream + .windowAll(TumblingEventTimeWindows.of(Time.minutes(5))) + .apply( + new AllWindowFunction() { + @Override + public void apply( + TimeWindow window, Iterable values, Collector out) { + // Emit RowData which contains the window start time, and the record count in + // that window + AtomicInteger count = new AtomicInteger(0); + values.forEach(a -> count.incrementAndGet()); + out.collect(row(window.getStart(), count.get())); + windows.put(window.getStart(), count.get()); + } + }); + + // Use static variable to collect the windows, since other solutions were flaky + windows.clear(); + env.executeAsync("Iceberg Source Windowing Test"); + + // Wait for the 2 first windows from File 2 and File 3 + Awaitility.await() + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) + .until( + () -> + windows.equals( + ImmutableMap.of(0L, RECORD_NUM_FOR_2_SPLITS, TimeUnit.MINUTES.toMillis(5), 2))); + + // Write data so the windows containing test data are closed + dataAppender.appendToTable( + dataAppender.writeFile(ImmutableList.of(generateRecord(1500, "last-record")))); + + // Wait for last test record window from File 1 + Awaitility.await() + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) + .until( + () -> + windows.equals( + ImmutableMap.of( + 0L, + RECORD_NUM_FOR_2_SPLITS, + TimeUnit.MINUTES.toMillis(5), + 2, + TimeUnit.MINUTES.toMillis(100), + 3))); + } + + /** + * This is an integration test for watermark handling and throttling. Integration testing the + * following: + * + *

      + *
    • - Emitting of watermarks + *
    • - Watermark alignment + *
    + * + *

    The test generates 3 splits + * + *

      + *
    • - Split 1 - Watermark 100 min + *
    • - Split 2, 3 - Watermark 0 min + *
    + * + * The splits are read in the following order: + * + *
      + *
    • - Split 2, 3 (Task Manager 1, Task Manager 2) + *
    • - Split 1 (Task Manager 1 or ask Manager 2 depending on scheduling) + *
    + * + * Reading split 1 will cause the watermark alignment to pause reading for the given task manager. + * + *

    The status of the watermark alignment is checked by the alignment related metrics. + * + *

    Adding new records with old timestamps to the table will enable the running reader to + * continue reading the files, but the watermark alignment will still prevent the paused reader to + * continue. + * + *

    After adding some records with new timestamps the blocked reader is un-paused, and both ot + * the readers continue reading. + */ + @Test + public void testThrottling() throws Exception { + GenericAppenderHelper dataAppender = appender(); + + // Generate records with the following pattern: + // - File 1 - Later records (Watermark 6000000) + // - Split 1 - 2 records (100, "file_1-recordTs_100"), (103, "file_1-recordTs_103") + // - File 2 - First records (Watermark 0) + // - Split 1 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - Split 2 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + List batch = + ImmutableList.of( + generateRecord(100, "file_1-recordTs_100"), generateRecord(103, "file_1-recordTs_103")); + dataAppender.appendToTable(batch); + + batch = Lists.newArrayListWithCapacity(100); + for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { + batch.add(generateRecord(4 - i % 5, "file_2-recordTs_" + i)); + } + + dataAppender.appendToTable(batch); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(2); + + DataStream stream = + env.fromSource( + source(), + WatermarkStrategy.noWatermarks() + .withWatermarkAlignment("iceberg", Duration.ofMinutes(20), Duration.ofMillis(10)), + SOURCE_NAME, + TypeInformation.of(RowData.class)); + + try (CloseableIterator resultIterator = stream.collectAsync()) { + JobClient jobClient = env.executeAsync("Iceberg Source Throttling Test"); + + // Check that the read the non-blocked data + // The first RECORD_NUM_FOR_2_SPLITS should be read + // 1 or more from the runaway reader should be arrived depending on thread scheduling + waitForRecords(resultIterator, RECORD_NUM_FOR_2_SPLITS + 1); + + // Get the drift metric, wait for it to be created and reach the expected state + // (100 min - 20 min - 0 min) + // Also this validates that the WatermarkAlignment is working + Awaitility.await() + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) + .until( + () -> + findAlignmentDriftMetric(jobClient.getJobID(), TimeUnit.MINUTES.toMillis(80)) + .isPresent()); + Gauge drift = + findAlignmentDriftMetric(jobClient.getJobID(), TimeUnit.MINUTES.toMillis(80)).get(); + + // Add some old records with 2 splits, so even if the blocked gets one split, the other reader + // one gets one as well + List newBatch1 = + ImmutableList.of( + generateRecord(15, "file_3-recordTs_15"), + generateRecord(16, "file_3-recordTs_16"), + generateRecord(17, "file_3-recordTs_17")); + List newBatch2 = + ImmutableList.of( + generateRecord(15, "file_4-recordTs_15"), + generateRecord(16, "file_4-recordTs_16"), + generateRecord(17, "file_4-recordTs_17")); + dataAppender.appendToTable( + dataAppender.writeFile(newBatch1), dataAppender.writeFile(newBatch2)); + // The records received will highly depend on scheduling + // We minimally get 3 records from the non-blocked reader + // We might get 1 record from the blocked reader (as part of the previous batch - + // file_1-recordTs_100) + // We might get 3 records form the non-blocked reader if it gets both new splits + waitForRecords(resultIterator, 3); + + // Get the drift metric, wait for it to be created and reach the expected state (100 min - 20 + // min - 15 min) + Awaitility.await() + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) + .until(() -> drift.getValue() == TimeUnit.MINUTES.toMillis(65)); + + // Add some new records which should unblock the throttled reader + batch = + ImmutableList.of( + generateRecord(90, "file_5-recordTs_90"), generateRecord(91, "file_5-recordTs_91")); + dataAppender.appendToTable(batch); + // We should get all the records at this point + waitForRecords(resultIterator, 6); + + // Wait for the new drift to decrease below the allowed drift to signal the normal state + Awaitility.await() + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) + .until(() -> drift.getValue() < TimeUnit.MINUTES.toMillis(20)); + } + } + + protected IcebergSource source() { + return IcebergSource.builder() + .tableLoader(sourceTableResource.tableLoader()) + .watermarkColumn("ts") + .project(TestFixtures.TS_SCHEMA) + .splitSize(100L) + .streaming(true) + .monitorInterval(Duration.ofMillis(2)) + .streamingStartingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + } + + protected Record generateRecord(int minutes, String str) { + // Override the ts field to create a more realistic situation for event time alignment + Record record = GenericRecord.create(TestFixtures.TS_SCHEMA); + LocalDateTime ts = + LocalDateTime.ofInstant( + Instant.ofEpochMilli(Time.of(minutes, TimeUnit.MINUTES).toMilliseconds()), + ZoneId.of("Z")); + record.setField("ts", ts); + record.setField("str", str); + return record; + } + + protected void assertRecords( + Collection expectedRecords, CloseableIterator iterator) throws Exception { + Set expected = + expectedRecords.stream() + .map(e -> RowDataConverter.convert(TestFixtures.TS_SCHEMA, e)) + .collect(Collectors.toSet()); + Assert.assertEquals(expected, waitForRecords(iterator, expectedRecords.size())); + } + + protected Set waitForRecords(CloseableIterator iterator, int num) { + Set received = Sets.newHashSetWithExpectedSize(num); + assertThat( + CompletableFuture.supplyAsync( + () -> { + int count = 0; + while (count < num && iterator.hasNext()) { + received.add(iterator.next()); + count++; + } + + if (count < num) { + throw new IllegalStateException(String.format("Fail to get %d records.", num)); + } + + return true; + })) + .succeedsWithin(DEFAULT_COLLECT_DATA_TIMEOUT); + + return received; + } + + private Optional> findAlignmentDriftMetric(JobID jobID, long withValue) { + String metricsName = SOURCE_NAME + ".*" + MetricNames.WATERMARK_ALIGNMENT_DRIFT; + return reporter.findMetrics(jobID, metricsName).values().stream() + .map(m -> (Gauge) m) + .filter(m -> m.getValue() == withValue) + .findFirst(); + } + + private GenericAppenderHelper appender() { + // We need to create multiple splits, so we need to generate parquet files with multiple offsets + org.apache.hadoop.conf.Configuration hadoopConf = new org.apache.hadoop.conf.Configuration(); + hadoopConf.set("write.parquet.page-size-bytes", "64"); + hadoopConf.set("write.parquet.row-group-size-bytes", "64"); + return new GenericAppenderHelper( + sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER, hadoopConf); + } + + private static RowData row(long time, long count) { + GenericRowData result = new GenericRowData(2); + result.setField(0, time); + result.setField(1, String.valueOf(count)); + return result; + } + + private static class RowDataTimestampAssigner implements SerializableTimestampAssigner { + @Override + public long extractTimestamp(RowData element, long recordTimestamp) { + return element.getTimestamp(0, 0).getMillisecond(); + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java new file mode 100644 index 000000000000..cb6fda18a1ee --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java @@ -0,0 +1,301 @@ +/* + * 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.iceberg.flink.source; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.io.IOException; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.util.Base64; +import java.util.List; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.types.Row; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.Files; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.FileHelpers; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkCatalogTestBase; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runners.Parameterized; + +public class TestMetadataTableReadableMetrics extends FlinkCatalogTestBase { + private static final String TABLE_NAME = "test_table"; + + public TestMetadataTableReadableMetrics(String catalogName, Namespace baseNamespace) { + super(catalogName, baseNamespace); + } + + @Parameterized.Parameters(name = "catalogName={0}, baseNamespace={1}") + public static Iterable parameters() { + List parameters = Lists.newArrayList(); + String catalogName = "testhive"; + Namespace baseNamespace = Namespace.empty(); + parameters.add(new Object[] {catalogName, baseNamespace}); + return parameters; + } + + @Override + protected TableEnvironment getTableEnv() { + Configuration configuration = super.getTableEnv().getConfig().getConfiguration(); + configuration.set(CoreOptions.DEFAULT_PARALLELISM, 1); + return super.getTableEnv(); + } + + @Rule public TemporaryFolder temp = new TemporaryFolder(); + + private static final Types.StructType LEAF_STRUCT_TYPE = + Types.StructType.of( + optional(1, "leafLongCol", Types.LongType.get()), + optional(2, "leafDoubleCol", Types.DoubleType.get())); + + private static final Types.StructType NESTED_STRUCT_TYPE = + Types.StructType.of(required(3, "leafStructCol", LEAF_STRUCT_TYPE)); + + private static final Schema NESTED_SCHEMA = + new Schema(required(4, "nestedStructCol", NESTED_STRUCT_TYPE)); + + private static final Schema PRIMITIVE_SCHEMA = + new Schema( + required(1, "booleanCol", Types.BooleanType.get()), + required(2, "intCol", Types.IntegerType.get()), + required(3, "longCol", Types.LongType.get()), + required(4, "floatCol", Types.FloatType.get()), + required(5, "doubleCol", Types.DoubleType.get()), + optional(6, "decimalCol", Types.DecimalType.of(10, 2)), + optional(7, "stringCol", Types.StringType.get()), + optional(8, "fixedCol", Types.FixedType.ofLength(3)), + optional(9, "binaryCol", Types.BinaryType.get())); + + private Table createPrimitiveTable() throws IOException { + Table table = + catalog.createTable( + TableIdentifier.of(DATABASE, TABLE_NAME), + PRIMITIVE_SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of()); + List records = + Lists.newArrayList( + createPrimitiveRecord( + false, + 1, + 1L, + 0, + 1.0D, + new BigDecimal("1.00"), + "1", + Base64.getDecoder().decode("1111"), + ByteBuffer.wrap(Base64.getDecoder().decode("1111"))), + createPrimitiveRecord( + true, + 2, + 2L, + 0, + 2.0D, + new BigDecimal("2.00"), + "2", + Base64.getDecoder().decode("2222"), + ByteBuffer.wrap(Base64.getDecoder().decode("2222"))), + createPrimitiveRecord(false, 1, 1, Float.NaN, Double.NaN, null, "1", null, null), + createPrimitiveRecord( + false, 2, 2L, Float.NaN, 2.0D, new BigDecimal("2.00"), "2", null, null)); + + DataFile dataFile = + FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), records); + table.newAppend().appendFile(dataFile).commit(); + return table; + } + + private void createNestedTable() throws IOException { + Table table = + validationCatalog.createTable( + TableIdentifier.of(DATABASE, TABLE_NAME), + NESTED_SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of()); + + List records = + Lists.newArrayList( + createNestedRecord(0L, 0.0), + createNestedRecord(1L, Double.NaN), + createNestedRecord(null, null)); + DataFile dataFile = + FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), records); + table.newAppend().appendFile(dataFile).commit(); + } + + @Before + public void before() { + super.before(); + sql("USE CATALOG %s", catalogName); + sql("CREATE DATABASE %s", DATABASE); + sql("USE %s", DATABASE); + } + + @Override + @After + public void clean() { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME); + sql("DROP DATABASE IF EXISTS %s", flinkDatabase); + super.clean(); + } + + protected GenericRecord createPrimitiveRecord( + boolean booleanCol, + int intCol, + long longCol, + float floatCol, + double doubleCol, + BigDecimal decimalCol, + String stringCol, + byte[] fixedCol, + ByteBuffer binaryCol) { + GenericRecord record = GenericRecord.create(PRIMITIVE_SCHEMA); + record.set(0, booleanCol); + record.set(1, intCol); + record.set(2, longCol); + record.set(3, floatCol); + record.set(4, doubleCol); + record.set(5, decimalCol); + record.set(6, stringCol); + record.set(7, fixedCol); + record.set(8, binaryCol); + return record; + } + + private GenericRecord createNestedRecord(Long longCol, Double doubleCol) { + GenericRecord record = GenericRecord.create(NESTED_SCHEMA); + GenericRecord nested = GenericRecord.create(NESTED_STRUCT_TYPE); + GenericRecord leaf = GenericRecord.create(LEAF_STRUCT_TYPE); + leaf.set(0, longCol); + leaf.set(1, doubleCol); + nested.set(0, leaf); + record.set(0, nested); + return record; + } + + protected Object[] row(Object... values) { + return values; + } + + @Test + public void testPrimitiveColumns() throws Exception { + createPrimitiveTable(); + List result = sql("SELECT readable_metrics FROM %s$files", TABLE_NAME); + + Row binaryCol = + Row.of( + 59L, + 4L, + 2L, + null, + Base64.getDecoder().decode("1111"), + Base64.getDecoder().decode("2222")); + Row booleanCol = Row.of(44L, 4L, 0L, null, false, true); + Row decimalCol = Row.of(97L, 4L, 1L, null, new BigDecimal("1.00"), new BigDecimal("2.00")); + Row doubleCol = Row.of(99L, 4L, 0L, 1L, 1.0D, 2.0D); + Row fixedCol = + Row.of( + 55L, + 4L, + 2L, + null, + Base64.getDecoder().decode("1111"), + Base64.getDecoder().decode("2222")); + Row floatCol = Row.of(90L, 4L, 0L, 2L, 0f, 0f); + Row intCol = Row.of(91L, 4L, 0L, null, 1, 2); + Row longCol = Row.of(91L, 4L, 0L, null, 1L, 2L); + Row stringCol = Row.of(99L, 4L, 0L, null, "1", "2"); + + List expected = + Lists.newArrayList( + Row.of( + Row.of( + binaryCol, + booleanCol, + decimalCol, + doubleCol, + fixedCol, + floatCol, + intCol, + longCol, + stringCol))); + TestHelpers.assertRows(result, expected); + } + + @Test + public void testSelectPrimitiveValues() throws Exception { + createPrimitiveTable(); + + TestHelpers.assertRows( + sql( + "SELECT readable_metrics.intCol.lower_bound, readable_metrics.booleanCol.upper_bound FROM %s$files", + TABLE_NAME), + ImmutableList.of(Row.of(1, true))); + + TestHelpers.assertRows( + sql("SELECT content, readable_metrics.longCol.value_count FROM %s$files", TABLE_NAME), + ImmutableList.of(Row.of(0, 4L))); + + TestHelpers.assertRows( + sql("SELECT readable_metrics.longCol.value_count, content FROM %s$files", TABLE_NAME), + ImmutableList.of(Row.of(4L, 0))); + } + + @Test + public void testSelectNestedValues() throws Exception { + createNestedTable(); + TestHelpers.assertRows( + sql( + "SELECT readable_metrics.`nestedStructCol.leafStructCol.leafLongCol`.lower_bound, " + + "readable_metrics.`nestedStructCol.leafStructCol.leafDoubleCol`.value_count FROM %s$files", + TABLE_NAME), + ImmutableList.of(Row.of(0L, 3L))); + } + + @Test + public void testNestedValues() throws Exception { + createNestedTable(); + + Row leafDoubleCol = Row.of(53L, 3L, 1L, 1L, 0.0D, 0.0D); + Row leafLongCol = Row.of(54L, 3L, 1L, null, 0L, 1L); + Row metrics = Row.of(Row.of(leafDoubleCol, leafLongCol)); + + TestHelpers.assertRows( + sql("SELECT readable_metrics FROM %s$files", TABLE_NAME), ImmutableList.of(metrics)); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java new file mode 100644 index 000000000000..25ecec23d216 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java @@ -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. + */ +package org.apache.iceberg.flink.source; + +import java.io.IOException; +import java.util.List; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.flink.data.RowDataProjection; +import org.apache.iceberg.flink.sink.RowDataTaskWriterFactory; +import org.apache.iceberg.flink.sink.TaskWriterFactory; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestProjectMetaColumn { + + @Rule public final TemporaryFolder folder = new TemporaryFolder(); + private final FileFormat format; + + @Parameterized.Parameters(name = "fileFormat={0}") + public static Iterable parameters() { + return Lists.newArrayList( + new Object[] {FileFormat.PARQUET}, + new Object[] {FileFormat.ORC}, + new Object[] {FileFormat.AVRO}); + } + + public TestProjectMetaColumn(FileFormat format) { + this.format = format; + } + + private void testSkipToRemoveMetaColumn(int formatVersion) throws IOException { + // Create the table with given format version. + String location = folder.getRoot().getAbsolutePath(); + Table table = + SimpleDataUtil.createTable( + location, + ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)), + false); + + List rows = + Lists.newArrayList( + SimpleDataUtil.createInsert(1, "AAA"), + SimpleDataUtil.createInsert(2, "BBB"), + SimpleDataUtil.createInsert(3, "CCC")); + writeAndCommit(table, ImmutableList.of(), false, rows); + + FlinkInputFormat input = + FlinkSource.forRowData().tableLoader(TableLoader.fromHadoopTable(location)).buildFormat(); + + List results = Lists.newArrayList(); + TestHelpers.readRowData( + input, + rowData -> { + // If project to remove the meta columns, it will get a RowDataProjection. + Assert.assertTrue(rowData instanceof GenericRowData); + results.add(TestHelpers.copyRowData(rowData, SimpleDataUtil.ROW_TYPE)); + }); + + // Assert the results. + TestHelpers.assertRows(rows, results, SimpleDataUtil.ROW_TYPE); + } + + @Test + public void testV1SkipToRemoveMetaColumn() throws IOException { + testSkipToRemoveMetaColumn(1); + } + + @Test + public void testV2SkipToRemoveMetaColumn() throws IOException { + testSkipToRemoveMetaColumn(2); + } + + @Test + public void testV2RemoveMetaColumn() throws Exception { + // Create the v2 table. + String location = folder.getRoot().getAbsolutePath(); + Table table = + SimpleDataUtil.createTable( + location, ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"), false); + + List rows = + Lists.newArrayList( + SimpleDataUtil.createInsert(1, "AAA"), + SimpleDataUtil.createDelete(1, "AAA"), + SimpleDataUtil.createInsert(2, "AAA"), + SimpleDataUtil.createInsert(2, "BBB")); + int eqFieldId = table.schema().findField("data").fieldId(); + writeAndCommit(table, ImmutableList.of(eqFieldId), true, rows); + + FlinkInputFormat input = + FlinkSource.forRowData().tableLoader(TableLoader.fromHadoopTable(location)).buildFormat(); + + List results = Lists.newArrayList(); + TestHelpers.readRowData( + input, + rowData -> { + // If project to remove the meta columns, it will get a RowDataProjection. + Assert.assertTrue(rowData instanceof RowDataProjection); + results.add(TestHelpers.copyRowData(rowData, SimpleDataUtil.ROW_TYPE)); + }); + + // Assert the results. + TestHelpers.assertRows( + ImmutableList.of( + SimpleDataUtil.createInsert(2, "AAA"), SimpleDataUtil.createInsert(2, "BBB")), + results, + SimpleDataUtil.ROW_TYPE); + } + + private void writeAndCommit( + Table table, List eqFieldIds, boolean upsert, List rows) + throws IOException { + TaskWriter writer = createTaskWriter(table, eqFieldIds, upsert); + try (TaskWriter io = writer) { + for (RowData row : rows) { + io.write(row); + } + } + + RowDelta delta = table.newRowDelta(); + WriteResult result = writer.complete(); + + for (DataFile dataFile : result.dataFiles()) { + delta.addRows(dataFile); + } + + for (DeleteFile deleteFile : result.deleteFiles()) { + delta.addDeletes(deleteFile); + } + + delta.commit(); + } + + private TaskWriter createTaskWriter( + Table table, List equalityFieldIds, boolean upsert) { + TaskWriterFactory taskWriterFactory = + new RowDataTaskWriterFactory( + SerializableTable.copyOf(table), + SimpleDataUtil.ROW_TYPE, + TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT, + format, + table.properties(), + equalityFieldIds, + upsert); + + taskWriterFactory.initialize(1, 1); + return taskWriterFactory.create(); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java new file mode 100644 index 000000000000..485035787d6d --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java @@ -0,0 +1,35 @@ +/* + * 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.iceberg.flink.source; + +import org.apache.avro.generic.GenericRecord; +import org.apache.iceberg.flink.AvroGenericRecordConverterBase; +import org.apache.iceberg.flink.DataGenerator; +import org.junit.Assert; + +public class TestRowDataToAvroGenericRecordConverter extends AvroGenericRecordConverterBase { + @Override + protected void testConverter(DataGenerator dataGenerator) { + RowDataToAvroGenericRecordConverter converter = + RowDataToAvroGenericRecordConverter.fromAvroSchema(dataGenerator.avroSchema()); + GenericRecord expected = dataGenerator.generateAvroGenericRecord(); + GenericRecord actual = converter.apply(dataGenerator.generateFlinkRowData()); + Assert.assertEquals(expected, actual); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java new file mode 100644 index 000000000000..317301260f66 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java @@ -0,0 +1,60 @@ +/* + * 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.iceberg.flink.source; + +import java.io.IOException; +import org.apache.flink.configuration.Configuration; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.junit.Assert; +import org.junit.Test; + +public class TestSourceUtil { + @Test + public void testInferedParallelism() throws IOException { + Configuration configuration = new Configuration(); + // Empty table, infer parallelism should be at least 1 + int parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 0); + Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + + // 2 splits (max infer is the default value 100 , max > splits num), the parallelism is splits + // num : 2 + parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 2); + Assert.assertEquals("Should produce the expected parallelism.", 2, parallelism); + + // 2 splits and limit is 1 , max infer parallelism is default 100, + // which is greater than splits num and limit, the parallelism is the limit value : 1 + parallelism = SourceUtil.inferParallelism(configuration, 1, () -> 2); + Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + + // 2 splits and max infer parallelism is 1 (max < splits num), the parallelism is 1 + configuration.setInteger(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX, 1); + parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 2); + Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + + // 2 splits, max infer parallelism is 1, limit is 3, the parallelism is max infer parallelism : + // 1 + parallelism = SourceUtil.inferParallelism(configuration, 3, () -> 2); + Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + + // 2 splits, infer parallelism is disabled, the parallelism is flink default parallelism 1 + configuration.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); + parallelism = SourceUtil.inferParallelism(configuration, 3, () -> 2); + Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java new file mode 100644 index 000000000000..dda46033143e --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java @@ -0,0 +1,158 @@ +/* + * 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.iceberg.flink.source; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.types.Row; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.HadoopCatalogResource; +import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.Assert; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +/** Test other more advanced usage of SQL. They don't need to run for every file format. */ +public abstract class TestSqlBase { + @ClassRule + public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = + MiniClusterResource.createWithClassloaderCheckDisabled(); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final HadoopCatalogResource catalogResource = + new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); + + private volatile TableEnvironment tEnv; + + protected TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + if (tEnv == null) { + this.tEnv = + TableEnvironment.create(EnvironmentSettings.newInstance().inBatchMode().build()); + } + } + } + return tEnv; + } + + @Before + public abstract void before() throws IOException; + + @Test + public void testResiduals() throws Exception { + Table table = + catalogResource + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); + + List writeRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); + writeRecords.get(0).set(1, 123L); + writeRecords.get(0).set(2, "2020-03-20"); + writeRecords.get(1).set(1, 456L); + writeRecords.get(1).set(2, "2020-03-20"); + + GenericAppenderHelper helper = + new GenericAppenderHelper(table, FileFormat.PARQUET, TEMPORARY_FOLDER); + + List expectedRecords = Lists.newArrayList(); + expectedRecords.add(writeRecords.get(0)); + + DataFile dataFile1 = helper.writeFile(TestHelpers.Row.of("2020-03-20", 0), writeRecords); + DataFile dataFile2 = + helper.writeFile( + TestHelpers.Row.of("2020-03-21", 0), + RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L)); + helper.appendToTable(dataFile1, dataFile2); + + org.apache.iceberg.flink.TestHelpers.assertRecords( + run(Maps.newHashMap(), "where dt='2020-03-20' and id=123", "*"), + expectedRecords, + TestFixtures.SCHEMA); + } + + @Test + public void testExposeLocality() throws Exception { + Table table = + catalogResource + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); + + TableLoader tableLoader = TableLoader.fromHadoopTable(table.location()); + List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 10, 0L); + expectedRecords.forEach(expectedRecord -> expectedRecord.set(2, "2020-03-20")); + + GenericAppenderHelper helper = + new GenericAppenderHelper(table, FileFormat.PARQUET, TEMPORARY_FOLDER); + DataFile dataFile = helper.writeFile(TestHelpers.Row.of("2020-03-20", 0), expectedRecords); + helper.appendToTable(dataFile); + + // test sql api + Configuration tableConf = getTableEnv().getConfig().getConfiguration(); + tableConf.setBoolean( + FlinkConfigOptions.TABLE_EXEC_ICEBERG_EXPOSE_SPLIT_LOCALITY_INFO.key(), false); + + List results = SqlHelpers.sql(getTableEnv(), "select * from t"); + org.apache.iceberg.flink.TestHelpers.assertRecords( + results, expectedRecords, TestFixtures.SCHEMA); + + // test table api + tableConf.setBoolean( + FlinkConfigOptions.TABLE_EXEC_ICEBERG_EXPOSE_SPLIT_LOCALITY_INFO.key(), true); + FlinkSource.Builder builder = FlinkSource.forRowData().tableLoader(tableLoader).table(table); + + // When running with CI or local, `localityEnabled` will be false even if this configuration is + // enabled + Assert.assertFalse( + "Expose split locality info should be false.", + SourceUtil.isLocalityEnabled(table, tableConf, true)); + + results = run(Maps.newHashMap(), "where dt='2020-03-20'", "*"); + org.apache.iceberg.flink.TestHelpers.assertRecords( + results, expectedRecords, TestFixtures.SCHEMA); + } + + protected List run( + Map options, String sqlFilter, String... sqlSelectedFields) { + String select = String.join(",", sqlSelectedFields); + String optionStr = SqlHelpers.sqlOptionsToString(options); + return SqlHelpers.sql(getTableEnv(), "select %s from t %s %s", select, optionStr, sqlFilter); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java new file mode 100644 index 000000000000..633e11718b9b --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java @@ -0,0 +1,318 @@ +/* + * 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.iceberg.flink.source; + +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkCatalogTestBase; +import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.assertj.core.api.Assertions; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class TestStreamScanSql extends FlinkCatalogTestBase { + private static final String TABLE = "test_table"; + private static final FileFormat FORMAT = FileFormat.PARQUET; + + private TableEnvironment tEnv; + + public TestStreamScanSql(String catalogName, Namespace baseNamespace) { + super(catalogName, baseNamespace); + } + + @Override + protected TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + if (tEnv == null) { + EnvironmentSettings.Builder settingsBuilder = + EnvironmentSettings.newInstance().inStreamingMode(); + + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + env.enableCheckpointing(400); + + StreamTableEnvironment streamTableEnv = + StreamTableEnvironment.create(env, settingsBuilder.build()); + streamTableEnv + .getConfig() + .getConfiguration() + .set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); + tEnv = streamTableEnv; + } + } + } + return tEnv; + } + + @Override + @Before + public void before() { + super.before(); + sql("CREATE DATABASE %s", flinkDatabase); + sql("USE CATALOG %s", catalogName); + sql("USE %s", DATABASE); + } + + @Override + @After + public void clean() { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE); + sql("DROP DATABASE IF EXISTS %s", flinkDatabase); + super.clean(); + } + + private void insertRows(String partition, Table table, Row... rows) throws IOException { + GenericAppenderHelper appender = new GenericAppenderHelper(table, FORMAT, TEMPORARY_FOLDER); + + GenericRecord gRecord = GenericRecord.create(table.schema()); + List records = Lists.newArrayList(); + for (Row row : rows) { + records.add( + gRecord.copy( + "id", row.getField(0), + "data", row.getField(1), + "dt", row.getField(2))); + } + + if (partition != null) { + appender.appendToTable(TestHelpers.Row.of(partition, 0), records); + } else { + appender.appendToTable(records); + } + } + + private void insertRows(Table table, Row... rows) throws IOException { + insertRows(null, table, rows); + } + + private void assertRows(List expectedRows, Iterator iterator) { + for (Row expectedRow : expectedRows) { + Assert.assertTrue("Should have more records", iterator.hasNext()); + + Row actualRow = iterator.next(); + Assert.assertEquals("Should have expected fields", 3, actualRow.getArity()); + Assert.assertEquals( + "Should have expected id", expectedRow.getField(0), actualRow.getField(0)); + Assert.assertEquals( + "Should have expected data", expectedRow.getField(1), actualRow.getField(1)); + Assert.assertEquals( + "Should have expected dt", expectedRow.getField(2), actualRow.getField(2)); + } + } + + @Test + public void testUnPartitionedTable() throws Exception { + sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); + + TableResult result = + exec("SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s')*/", TABLE); + try (CloseableIterator iterator = result.collect()) { + + Row row1 = Row.of(1, "aaa", "2021-01-01"); + insertRows(table, row1); + assertRows(ImmutableList.of(row1), iterator); + + Row row2 = Row.of(2, "bbb", "2021-01-01"); + insertRows(table, row2); + assertRows(ImmutableList.of(row2), iterator); + } + result.getJobClient().ifPresent(JobClient::cancel); + } + + @Test + public void testPartitionedTable() throws Exception { + sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR) PARTITIONED BY (dt)", TABLE); + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); + + TableResult result = + exec("SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s')*/", TABLE); + try (CloseableIterator iterator = result.collect()) { + Row row1 = Row.of(1, "aaa", "2021-01-01"); + insertRows("2021-01-01", table, row1); + assertRows(ImmutableList.of(row1), iterator); + + Row row2 = Row.of(2, "bbb", "2021-01-02"); + insertRows("2021-01-02", table, row2); + assertRows(ImmutableList.of(row2), iterator); + + Row row3 = Row.of(1, "aaa", "2021-01-02"); + insertRows("2021-01-02", table, row3); + assertRows(ImmutableList.of(row3), iterator); + + Row row4 = Row.of(2, "bbb", "2021-01-01"); + insertRows("2021-01-01", table, row4); + assertRows(ImmutableList.of(row4), iterator); + } + result.getJobClient().ifPresent(JobClient::cancel); + } + + @Test + public void testConsumeFromBeginning() throws Exception { + sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); + + Row row1 = Row.of(1, "aaa", "2021-01-01"); + Row row2 = Row.of(2, "bbb", "2021-01-01"); + insertRows(table, row1, row2); + + TableResult result = + exec("SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s')*/", TABLE); + try (CloseableIterator iterator = result.collect()) { + assertRows(ImmutableList.of(row1, row2), iterator); + + Row row3 = Row.of(3, "ccc", "2021-01-01"); + insertRows(table, row3); + assertRows(ImmutableList.of(row3), iterator); + + Row row4 = Row.of(4, "ddd", "2021-01-01"); + insertRows(table, row4); + assertRows(ImmutableList.of(row4), iterator); + } + result.getJobClient().ifPresent(JobClient::cancel); + } + + @Test + public void testConsumeFilesWithBranch() throws Exception { + sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); + Row row1 = Row.of(1, "aaa", "2021-01-01"); + Row row2 = Row.of(2, "bbb", "2021-01-01"); + insertRows(table, row1, row2); + + Assertions.assertThatThrownBy( + () -> + exec( + "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', 'branch'='b1')*/", + TABLE)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot scan table using ref b1 configured for streaming reader yet"); + } + + @Test + public void testConsumeFromStartSnapshotId() throws Exception { + sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); + + // Produce two snapshots. + Row row1 = Row.of(1, "aaa", "2021-01-01"); + Row row2 = Row.of(2, "bbb", "2021-01-01"); + insertRows(table, row1); + insertRows(table, row2); + + long startSnapshotId = table.currentSnapshot().snapshotId(); + + Row row3 = Row.of(3, "ccc", "2021-01-01"); + Row row4 = Row.of(4, "ddd", "2021-01-01"); + insertRows(table, row3, row4); + + TableResult result = + exec( + "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', " + + "'start-snapshot-id'='%d')*/", + TABLE, startSnapshotId); + try (CloseableIterator iterator = result.collect()) { + // the start snapshot(row2) is exclusive. + assertRows(ImmutableList.of(row3, row4), iterator); + + Row row5 = Row.of(5, "eee", "2021-01-01"); + Row row6 = Row.of(6, "fff", "2021-01-01"); + insertRows(table, row5, row6); + assertRows(ImmutableList.of(row5, row6), iterator); + + Row row7 = Row.of(7, "ggg", "2021-01-01"); + insertRows(table, row7); + assertRows(ImmutableList.of(row7), iterator); + } + result.getJobClient().ifPresent(JobClient::cancel); + } + + @Test + public void testConsumeFromStartTag() throws Exception { + sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); + + // Produce two snapshots. + Row row1 = Row.of(1, "aaa", "2021-01-01"); + Row row2 = Row.of(2, "bbb", "2021-01-01"); + insertRows(table, row1); + insertRows(table, row2); + + String tagName = "t1"; + long startSnapshotId = table.currentSnapshot().snapshotId(); + table.manageSnapshots().createTag(tagName, startSnapshotId).commit(); + + Row row3 = Row.of(3, "ccc", "2021-01-01"); + Row row4 = Row.of(4, "ddd", "2021-01-01"); + insertRows(table, row3, row4); + + TableResult result = + exec( + "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', " + + "'start-tag'='%s')*/", + TABLE, tagName); + try (CloseableIterator iterator = result.collect()) { + // the start snapshot(row2) is exclusive. + assertRows(ImmutableList.of(row3, row4), iterator); + + Row row5 = Row.of(5, "eee", "2021-01-01"); + Row row6 = Row.of(6, "fff", "2021-01-01"); + insertRows(table, row5, row6); + assertRows(ImmutableList.of(row5, row6), iterator); + + Row row7 = Row.of(7, "ggg", "2021-01-01"); + insertRows(table, row7); + assertRows(ImmutableList.of(row7), iterator); + } + result.getJobClient().ifPresent(JobClient::cancel); + + Assertions.assertThatThrownBy( + () -> + exec( + "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', 'start-tag'='%s', " + + "'start-snapshot-id'='%d' )*/", + TABLE, tagName, startSnapshotId)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("START_SNAPSHOT_ID and START_TAG cannot both be set."); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java new file mode 100644 index 000000000000..494c633088d9 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java @@ -0,0 +1,407 @@ +/* + * 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.iceberg.flink.source; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.time.Duration; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.operators.StreamSource; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.Row; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.flink.TestTableLoader; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.iceberg.util.ThreadPools; +import org.assertj.core.api.Assertions; +import org.awaitility.Awaitility; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestStreamingMonitorFunction extends TableTestBase { + + private static final Schema SCHEMA = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get())); + private static final FileFormat DEFAULT_FORMAT = FileFormat.PARQUET; + private static final long WAIT_TIME_MILLIS = 10 * 1000L; + + @Parameterized.Parameters(name = "FormatVersion={0}") + public static Iterable parameters() { + return ImmutableList.of(new Object[] {1}, new Object[] {2}); + } + + public TestStreamingMonitorFunction(int formatVersion) { + super(formatVersion); + } + + @Before + @Override + public void setupTable() throws IOException { + this.tableDir = temp.newFolder(); + this.metadataDir = new File(tableDir, "metadata"); + Assert.assertTrue(tableDir.delete()); + + // Construct the iceberg table. + table = create(SCHEMA, PartitionSpec.unpartitioned()); + } + + private void runSourceFunctionInTask( + TestSourceContext sourceContext, StreamingMonitorFunction function) { + Thread task = + new Thread( + () -> { + try { + function.run(sourceContext); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + task.start(); + } + + @Test + public void testConsumeWithoutStartSnapshotId() throws Exception { + List> recordsList = generateRecordsAndCommitTxn(10); + ScanContext scanContext = ScanContext.builder().monitorInterval(Duration.ofMillis(100)).build(); + + StreamingMonitorFunction function = createFunction(scanContext); + try (AbstractStreamOperatorTestHarness harness = createHarness(function)) { + harness.setup(); + harness.open(); + + TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); + runSourceFunctionInTask(sourceContext, function); + + awaitExpectedSplits(sourceContext); + + // Stop the stream task. + function.close(); + + TestHelpers.assertRecords( + sourceContext.toRows(), Lists.newArrayList(Iterables.concat(recordsList)), SCHEMA); + } + } + + @Test + public void testConsumeFromStartSnapshotId() throws Exception { + // Commit the first five transactions. + generateRecordsAndCommitTxn(5); + long startSnapshotId = table.currentSnapshot().snapshotId(); + + // Commit the next five transactions. + List> recordsList = generateRecordsAndCommitTxn(5); + + ScanContext scanContext = + ScanContext.builder() + .monitorInterval(Duration.ofMillis(100)) + .startSnapshotId(startSnapshotId) + .build(); + + StreamingMonitorFunction function = createFunction(scanContext); + try (AbstractStreamOperatorTestHarness harness = createHarness(function)) { + harness.setup(); + harness.open(); + + TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); + runSourceFunctionInTask(sourceContext, function); + + awaitExpectedSplits(sourceContext); + + // Stop the stream task. + function.close(); + + TestHelpers.assertRecords( + sourceContext.toRows(), Lists.newArrayList(Iterables.concat(recordsList)), SCHEMA); + } + } + + @Test + public void testConsumeFromStartTag() throws Exception { + // Commit the first five transactions. + generateRecordsAndCommitTxn(5); + long startSnapshotId = table.currentSnapshot().snapshotId(); + String tagName = "t1"; + table.manageSnapshots().createTag(tagName, startSnapshotId).commit(); + + // Commit the next five transactions. + List> recordsList = generateRecordsAndCommitTxn(5); + + ScanContext scanContext = + ScanContext.builder().monitorInterval(Duration.ofMillis(100)).startTag(tagName).build(); + + StreamingMonitorFunction function = createFunction(scanContext); + try (AbstractStreamOperatorTestHarness harness = createHarness(function)) { + harness.setup(); + harness.open(); + + TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); + runSourceFunctionInTask(sourceContext, function); + + awaitExpectedSplits(sourceContext); + + // Stop the stream task. + function.close(); + + TestHelpers.assertRecords( + sourceContext.toRows(), Lists.newArrayList(Iterables.concat(recordsList)), SCHEMA); + } + } + + @Test + public void testCheckpointRestore() throws Exception { + List> recordsList = generateRecordsAndCommitTxn(10); + ScanContext scanContext = ScanContext.builder().monitorInterval(Duration.ofMillis(100)).build(); + + StreamingMonitorFunction func = createFunction(scanContext); + OperatorSubtaskState state; + try (AbstractStreamOperatorTestHarness harness = createHarness(func)) { + harness.setup(); + harness.open(); + + TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); + runSourceFunctionInTask(sourceContext, func); + + awaitExpectedSplits(sourceContext); + + state = harness.snapshot(1, 1); + + // Stop the stream task. + func.close(); + + TestHelpers.assertRecords( + sourceContext.toRows(), Lists.newArrayList(Iterables.concat(recordsList)), SCHEMA); + } + + List> newRecordsList = generateRecordsAndCommitTxn(10); + StreamingMonitorFunction newFunc = createFunction(scanContext); + try (AbstractStreamOperatorTestHarness harness = createHarness(newFunc)) { + harness.setup(); + // Recover to process the remaining snapshots. + harness.initializeState(state); + harness.open(); + + TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); + runSourceFunctionInTask(sourceContext, newFunc); + + awaitExpectedSplits(sourceContext); + + // Stop the stream task. + newFunc.close(); + + TestHelpers.assertRecords( + sourceContext.toRows(), Lists.newArrayList(Iterables.concat(newRecordsList)), SCHEMA); + } + } + + private void awaitExpectedSplits(TestSourceContext sourceContext) { + Awaitility.await("expected splits should be produced") + .atMost(Duration.ofMillis(WAIT_TIME_MILLIS)) + .untilAsserted( + () -> { + assertThat(sourceContext.latch.getCount()).isEqualTo(0); + assertThat(sourceContext.splits).as("Should produce the expected splits").hasSize(1); + }); + } + + @Test + public void testInvalidMaxPlanningSnapshotCount() { + ScanContext scanContext1 = + ScanContext.builder() + .monitorInterval(Duration.ofMillis(100)) + .maxPlanningSnapshotCount(0) + .build(); + + Assertions.assertThatThrownBy(() -> createFunction(scanContext1)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("The max-planning-snapshot-count must be greater than zero"); + + ScanContext scanContext2 = + ScanContext.builder() + .monitorInterval(Duration.ofMillis(100)) + .maxPlanningSnapshotCount(-10) + .build(); + + Assertions.assertThatThrownBy(() -> createFunction(scanContext2)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("The max-planning-snapshot-count must be greater than zero"); + } + + @Test + public void testConsumeWithMaxPlanningSnapshotCount() throws Exception { + generateRecordsAndCommitTxn(10); + + // Use the oldest snapshot as starting to avoid the initial case. + long oldestSnapshotId = SnapshotUtil.oldestAncestor(table).snapshotId(); + + ScanContext scanContext = + ScanContext.builder() + .monitorInterval(Duration.ofMillis(100)) + .splitSize(1000L) + .startSnapshotId(oldestSnapshotId) + .maxPlanningSnapshotCount(Integer.MAX_VALUE) + .build(); + + FlinkInputSplit[] expectedSplits = + FlinkSplitPlanner.planInputSplits(table, scanContext, ThreadPools.getWorkerPool()); + + Assert.assertEquals("should produce 9 splits", 9, expectedSplits.length); + + // This covers three cases that maxPlanningSnapshotCount is less than, equal or greater than the + // total splits number + for (int maxPlanningSnapshotCount : ImmutableList.of(1, 9, 15)) { + scanContext = + ScanContext.builder() + .monitorInterval(Duration.ofMillis(500)) + .startSnapshotId(oldestSnapshotId) + .splitSize(1000L) + .maxPlanningSnapshotCount(maxPlanningSnapshotCount) + .build(); + + StreamingMonitorFunction function = createFunction(scanContext); + try (AbstractStreamOperatorTestHarness harness = createHarness(function)) { + harness.setup(); + harness.open(); + + CountDownLatch latch = new CountDownLatch(1); + TestSourceContext sourceContext = new TestSourceContext(latch); + function.sourceContext(sourceContext); + function.monitorAndForwardSplits(); + + if (maxPlanningSnapshotCount < 10) { + Assert.assertEquals( + "Should produce same splits as max-planning-snapshot-count", + maxPlanningSnapshotCount, + sourceContext.splits.size()); + } + } + } + } + + private List> generateRecordsAndCommitTxn(int commitTimes) throws IOException { + List> expectedRecords = Lists.newArrayList(); + for (int i = 0; i < commitTimes; i++) { + List records = RandomGenericData.generate(SCHEMA, 100, 0L); + expectedRecords.add(records); + + // Commit those records to iceberg table. + writeRecords(records); + } + return expectedRecords; + } + + private void writeRecords(List records) throws IOException { + GenericAppenderHelper appender = new GenericAppenderHelper(table, DEFAULT_FORMAT, temp); + appender.appendToTable(records); + } + + private StreamingMonitorFunction createFunction(ScanContext scanContext) { + return new StreamingMonitorFunction( + TestTableLoader.of(tableDir.getAbsolutePath()), scanContext); + } + + private AbstractStreamOperatorTestHarness createHarness( + StreamingMonitorFunction function) throws Exception { + StreamSource streamSource = + new StreamSource<>(function); + return new AbstractStreamOperatorTestHarness<>(streamSource, 1, 1, 0); + } + + private class TestSourceContext implements SourceFunction.SourceContext { + private final List splits = Lists.newArrayList(); + private final Object checkpointLock = new Object(); + private final CountDownLatch latch; + + TestSourceContext(CountDownLatch latch) { + this.latch = latch; + } + + @Override + public void collect(FlinkInputSplit element) { + splits.add(element); + latch.countDown(); + } + + @Override + public void collectWithTimestamp(FlinkInputSplit element, long timestamp) { + collect(element); + } + + @Override + public void emitWatermark(Watermark mark) {} + + @Override + public void markAsTemporarilyIdle() {} + + @Override + public Object getCheckpointLock() { + return checkpointLock; + } + + @Override + public void close() {} + + private List toRows() throws IOException { + FlinkInputFormat format = + FlinkSource.forRowData() + .tableLoader(TestTableLoader.of(tableDir.getAbsolutePath())) + .buildFormat(); + + List rows = Lists.newArrayList(); + for (FlinkInputSplit split : splits) { + format.open(split); + + RowData element = null; + try { + while (!format.reachedEnd()) { + element = format.nextRecord(element); + rows.add(Row.of(element.getInt(0), element.getString(1).toString())); + } + } finally { + format.close(); + } + } + + return rows; + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java new file mode 100644 index 000000000000..e258a197edf3 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java @@ -0,0 +1,287 @@ +/* + * 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.iceberg.flink.source; + +import java.io.File; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.streaming.api.TimeCharacteristic; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor; +import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxDefaultAction; +import org.apache.flink.streaming.runtime.tasks.mailbox.SteppingMailboxProcessor; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.Row; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.flink.TestTableLoader; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.iceberg.util.ThreadPools; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestStreamingReaderOperator extends TableTestBase { + + private static final Schema SCHEMA = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get())); + private static final FileFormat DEFAULT_FORMAT = FileFormat.PARQUET; + + @Parameterized.Parameters(name = "FormatVersion={0}") + public static Iterable parameters() { + return ImmutableList.of(new Object[] {1}, new Object[] {2}); + } + + public TestStreamingReaderOperator(int formatVersion) { + super(formatVersion); + } + + @Before + @Override + public void setupTable() throws IOException { + this.tableDir = temp.newFolder(); + this.metadataDir = new File(tableDir, "metadata"); + Assert.assertTrue(tableDir.delete()); + + // Construct the iceberg table. + table = create(SCHEMA, PartitionSpec.unpartitioned()); + } + + @Test + public void testProcessAllRecords() throws Exception { + List> expectedRecords = generateRecordsAndCommitTxn(10); + + List splits = generateSplits(); + Assert.assertEquals("Should have 10 splits", 10, splits.size()); + + try (OneInputStreamOperatorTestHarness harness = createReader()) { + harness.setup(); + harness.open(); + + SteppingMailboxProcessor processor = createLocalMailbox(harness); + + List expected = Lists.newArrayList(); + for (int i = 0; i < splits.size(); i++) { + // Process this element to enqueue to mail-box. + harness.processElement(splits.get(i), -1); + + // Run the mail-box once to read all records from the given split. + Assert.assertTrue("Should processed 1 split", processor.runMailboxStep()); + + // Assert the output has expected elements. + expected.addAll(expectedRecords.get(i)); + TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); + } + } + } + + @Test + public void testTriggerCheckpoint() throws Exception { + // Received emitted splits: split1, split2, split3, checkpoint request is triggered when reading + // records from + // split1. + List> expectedRecords = generateRecordsAndCommitTxn(3); + + List splits = generateSplits(); + Assert.assertEquals("Should have 3 splits", 3, splits.size()); + + long timestamp = 0; + try (OneInputStreamOperatorTestHarness harness = createReader()) { + harness.setup(); + harness.open(); + + SteppingMailboxProcessor processor = createLocalMailbox(harness); + + harness.processElement(splits.get(0), ++timestamp); + harness.processElement(splits.get(1), ++timestamp); + harness.processElement(splits.get(2), ++timestamp); + + // Trigger snapshot state, it will start to work once all records from split0 are read. + processor.getMainMailboxExecutor().execute(() -> harness.snapshot(1, 3), "Trigger snapshot"); + + Assert.assertTrue("Should have processed the split0", processor.runMailboxStep()); + Assert.assertTrue( + "Should have processed the snapshot state action", processor.runMailboxStep()); + + TestHelpers.assertRecords(readOutputValues(harness), expectedRecords.get(0), SCHEMA); + + // Read records from split1. + Assert.assertTrue("Should have processed the split1", processor.runMailboxStep()); + + // Read records from split2. + Assert.assertTrue("Should have processed the split2", processor.runMailboxStep()); + + TestHelpers.assertRecords( + readOutputValues(harness), Lists.newArrayList(Iterables.concat(expectedRecords)), SCHEMA); + } + } + + @Test + public void testCheckpointRestore() throws Exception { + List> expectedRecords = generateRecordsAndCommitTxn(15); + + List splits = generateSplits(); + Assert.assertEquals("Should have 10 splits", 15, splits.size()); + + OperatorSubtaskState state; + List expected = Lists.newArrayList(); + try (OneInputStreamOperatorTestHarness harness = createReader()) { + harness.setup(); + harness.open(); + + // Enqueue all the splits. + for (FlinkInputSplit split : splits) { + harness.processElement(split, -1); + } + + // Read all records from the first five splits. + SteppingMailboxProcessor localMailbox = createLocalMailbox(harness); + for (int i = 0; i < 5; i++) { + expected.addAll(expectedRecords.get(i)); + Assert.assertTrue("Should have processed the split#" + i, localMailbox.runMailboxStep()); + + TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); + } + + // Snapshot state now, there're 10 splits left in the state. + state = harness.snapshot(1, 1); + } + + expected.clear(); + try (OneInputStreamOperatorTestHarness harness = createReader()) { + harness.setup(); + // Recover to process the remaining splits. + harness.initializeState(state); + harness.open(); + + SteppingMailboxProcessor localMailbox = createLocalMailbox(harness); + + for (int i = 5; i < 10; i++) { + expected.addAll(expectedRecords.get(i)); + Assert.assertTrue("Should have processed one split#" + i, localMailbox.runMailboxStep()); + + TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); + } + + // Let's process the final 5 splits now. + for (int i = 10; i < 15; i++) { + expected.addAll(expectedRecords.get(i)); + harness.processElement(splits.get(i), 1); + + Assert.assertTrue("Should have processed the split#" + i, localMailbox.runMailboxStep()); + TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); + } + } + } + + private List readOutputValues( + OneInputStreamOperatorTestHarness harness) { + List results = Lists.newArrayList(); + for (RowData rowData : harness.extractOutputValues()) { + results.add(Row.of(rowData.getInt(0), rowData.getString(1).toString())); + } + return results; + } + + private List> generateRecordsAndCommitTxn(int commitTimes) throws IOException { + List> expectedRecords = Lists.newArrayList(); + for (int i = 0; i < commitTimes; i++) { + List records = RandomGenericData.generate(SCHEMA, 100, 0L); + expectedRecords.add(records); + + // Commit those records to iceberg table. + writeRecords(records); + } + return expectedRecords; + } + + private void writeRecords(List records) throws IOException { + GenericAppenderHelper appender = new GenericAppenderHelper(table, DEFAULT_FORMAT, temp); + appender.appendToTable(records); + } + + private List generateSplits() { + List inputSplits = Lists.newArrayList(); + + List snapshotIds = SnapshotUtil.currentAncestorIds(table); + for (int i = snapshotIds.size() - 1; i >= 0; i--) { + ScanContext scanContext; + if (i == snapshotIds.size() - 1) { + // Generate the splits from the first snapshot. + scanContext = ScanContext.builder().useSnapshotId(snapshotIds.get(i)).build(); + } else { + // Generate the splits between the previous snapshot and current snapshot. + scanContext = + ScanContext.builder() + .startSnapshotId(snapshotIds.get(i + 1)) + .endSnapshotId(snapshotIds.get(i)) + .build(); + } + + Collections.addAll( + inputSplits, + FlinkSplitPlanner.planInputSplits(table, scanContext, ThreadPools.getWorkerPool())); + } + + return inputSplits; + } + + private OneInputStreamOperatorTestHarness createReader() + throws Exception { + // This input format is used to opening the emitted split. + FlinkInputFormat inputFormat = + FlinkSource.forRowData() + .tableLoader(TestTableLoader.of(tableDir.getAbsolutePath())) + .buildFormat(); + + OneInputStreamOperatorFactory factory = + StreamingReaderOperator.factory(inputFormat); + OneInputStreamOperatorTestHarness harness = + new OneInputStreamOperatorTestHarness<>(factory, 1, 1, 0); + harness.getStreamConfig().setTimeCharacteristic(TimeCharacteristic.ProcessingTime); + + return harness; + } + + private SteppingMailboxProcessor createLocalMailbox( + OneInputStreamOperatorTestHarness harness) { + return new SteppingMailboxProcessor( + MailboxDefaultAction.Controller::suspendDefaultAction, + harness.getTaskMailbox(), + StreamTaskActionExecutor.IMMEDIATE); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java new file mode 100644 index 000000000000..090b304942c6 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java @@ -0,0 +1,130 @@ +/* + * 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.iceberg.flink.source.assigner; + +import java.util.Collection; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.iceberg.flink.source.SplitHelpers; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public abstract class SplitAssignerTestBase { + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Test + public void testEmptyInitialization() { + SplitAssigner assigner = splitAssigner(); + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + } + + /** Test a sequence of interactions for StaticEnumerator */ + @Test + public void testStaticEnumeratorSequence() throws Exception { + SplitAssigner assigner = splitAssigner(); + assigner.onDiscoveredSplits(createSplits(4, 1, "1")); + + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertSnapshot(assigner, 1); + assigner.onUnassignedSplits(createSplits(1, 1, "1")); + assertSnapshot(assigner, 2); + + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + assertSnapshot(assigner, 0); + } + + /** Test a sequence of interactions for ContinuousEnumerator */ + @Test + public void testContinuousEnumeratorSequence() throws Exception { + SplitAssigner assigner = splitAssigner(); + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + + List splits1 = createSplits(1, 1, "1"); + assertAvailableFuture(assigner, 1, () -> assigner.onDiscoveredSplits(splits1)); + List splits2 = createSplits(1, 1, "1"); + assertAvailableFuture(assigner, 1, () -> assigner.onUnassignedSplits(splits2)); + + assigner.onDiscoveredSplits(createSplits(2, 1, "1")); + assertSnapshot(assigner, 2); + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + assertSnapshot(assigner, 0); + } + + private void assertAvailableFuture( + SplitAssigner assigner, int splitCount, Runnable addSplitsRunnable) { + // register callback + AtomicBoolean futureCompleted = new AtomicBoolean(); + CompletableFuture future = assigner.isAvailable(); + future.thenAccept(ignored -> futureCompleted.set(true)); + // calling isAvailable again should return the same object reference + // note that thenAccept will return a new future. + // we want to assert the same instance on the assigner returned future + Assert.assertSame(future, assigner.isAvailable()); + + // now add some splits + addSplitsRunnable.run(); + Assert.assertEquals(true, futureCompleted.get()); + + for (int i = 0; i < splitCount; ++i) { + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + } + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + assertSnapshot(assigner, 0); + } + + protected void assertGetNext(SplitAssigner assigner, GetSplitResult.Status expectedStatus) { + GetSplitResult result = assigner.getNext(null); + Assert.assertEquals(expectedStatus, result.status()); + switch (expectedStatus) { + case AVAILABLE: + Assert.assertNotNull(result.split()); + break; + case CONSTRAINED: + case UNAVAILABLE: + Assert.assertNull(result.split()); + break; + default: + Assert.fail("Unknown status: " + expectedStatus); + } + } + + protected void assertSnapshot(SplitAssigner assigner, int splitCount) { + Collection stateBeforeGet = assigner.state(); + Assert.assertEquals(splitCount, stateBeforeGet.size()); + } + + protected List createSplits(int fileCount, int filesPerSplit, String version) + throws Exception { + return SplitHelpers.createSplitsFromTransientHadoopTable( + TEMPORARY_FOLDER, fileCount, filesPerSplit, version); + } + + protected abstract SplitAssigner splitAssigner(); +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java new file mode 100644 index 000000000000..8994f3054abe --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.assigner; + +import org.apache.iceberg.flink.source.SplitHelpers; +import org.junit.Test; + +public class TestDefaultSplitAssigner extends SplitAssignerTestBase { + @Override + protected SplitAssigner splitAssigner() { + return new DefaultSplitAssigner(null); + } + + /** Test the assigner when multiple files are in a single split */ + @Test + public void testMultipleFilesInASplit() throws Exception { + SplitAssigner assigner = splitAssigner(); + assigner.onDiscoveredSplits( + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 4, 2)); + + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertSnapshot(assigner, 1); + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + assertSnapshot(assigner, 0); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java new file mode 100644 index 000000000000..e78634e6b873 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java @@ -0,0 +1,80 @@ +/* + * 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.iceberg.flink.source.assigner; + +import java.util.List; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SerializableComparator; +import org.apache.iceberg.flink.source.split.SplitComparators; +import org.apache.iceberg.util.SerializationUtil; +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Test; + +public class TestFileSequenceNumberBasedSplitAssigner extends SplitAssignerTestBase { + @Override + protected SplitAssigner splitAssigner() { + return new OrderedSplitAssignerFactory(SplitComparators.fileSequenceNumber()).createAssigner(); + } + + /** Test the assigner when multiple files are in a single split */ + @Test + public void testMultipleFilesInAnIcebergSplit() { + SplitAssigner assigner = splitAssigner(); + Assertions.assertThatThrownBy( + () -> assigner.onDiscoveredSplits(createSplits(4, 2, "2")), + "Multiple files in a split is not allowed") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Please use 'split-open-file-cost'"); + } + + /** Test sorted splits */ + @Test + public void testSplitSort() throws Exception { + SplitAssigner assigner = splitAssigner(); + List splits = createSplits(5, 1, "2"); + + assigner.onDiscoveredSplits(splits.subList(3, 5)); + assigner.onDiscoveredSplits(splits.subList(0, 1)); + assigner.onDiscoveredSplits(splits.subList(1, 3)); + + assertGetNext(assigner, 1L); + assertGetNext(assigner, 2L); + assertGetNext(assigner, 3L); + assertGetNext(assigner, 4L); + assertGetNext(assigner, 5L); + + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + } + + @Test + public void testSerializable() { + byte[] bytes = SerializationUtil.serializeToBytes(SplitComparators.fileSequenceNumber()); + SerializableComparator comparator = + SerializationUtil.deserializeFromBytes(bytes); + Assert.assertNotNull(comparator); + } + + private void assertGetNext(SplitAssigner assigner, Long expectedSequenceNumber) { + GetSplitResult result = assigner.getNext(null); + ContentFile file = result.split().task().files().iterator().next().file(); + Assert.assertEquals(expectedSequenceNumber, file.fileSequenceNumber()); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java new file mode 100644 index 000000000000..e1fc63fda918 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java @@ -0,0 +1,146 @@ +/* + * 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.iceberg.flink.source.assigner; + +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.io.IOException; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneOffset; +import java.time.temporal.ChronoUnit; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.reader.ColumnStatsWatermarkExtractor; +import org.apache.iceberg.flink.source.reader.ReaderUtil; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SerializableComparator; +import org.apache.iceberg.flink.source.split.SplitComparators; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.SerializationUtil; +import org.junit.Assert; +import org.junit.Test; + +public class TestWatermarkBasedSplitAssigner extends SplitAssignerTestBase { + public static final Schema SCHEMA = + new Schema(required(1, "timestamp_column", Types.TimestampType.withoutZone())); + private static final GenericAppenderFactory APPENDER_FACTORY = new GenericAppenderFactory(SCHEMA); + + @Override + protected SplitAssigner splitAssigner() { + return new OrderedSplitAssignerFactory( + SplitComparators.watermark( + new ColumnStatsWatermarkExtractor(SCHEMA, "timestamp_column", null))) + .createAssigner(); + } + + /** Test the assigner when multiple files are in a single split */ + @Test + public void testMultipleFilesInAnIcebergSplit() { + SplitAssigner assigner = splitAssigner(); + assigner.onDiscoveredSplits(createSplits(4, 2, "2")); + + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + } + + /** Test sorted splits */ + @Test + public void testSplitSort() { + SplitAssigner assigner = splitAssigner(); + + Instant now = Instant.now(); + List splits = + IntStream.range(0, 5) + .mapToObj(i -> splitFromInstant(now.plus(i, ChronoUnit.MINUTES))) + .collect(Collectors.toList()); + + assigner.onDiscoveredSplits(splits.subList(3, 5)); + assigner.onDiscoveredSplits(splits.subList(0, 1)); + assigner.onDiscoveredSplits(splits.subList(1, 3)); + + assertGetNext(assigner, splits.get(0)); + assertGetNext(assigner, splits.get(1)); + assertGetNext(assigner, splits.get(2)); + assertGetNext(assigner, splits.get(3)); + assertGetNext(assigner, splits.get(4)); + + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + } + + @Test + public void testSerializable() { + byte[] bytes = + SerializationUtil.serializeToBytes( + SplitComparators.watermark( + new ColumnStatsWatermarkExtractor( + TestFixtures.SCHEMA, "id", TimeUnit.MILLISECONDS))); + SerializableComparator comparator = + SerializationUtil.deserializeFromBytes(bytes); + Assert.assertNotNull(comparator); + } + + private void assertGetNext(SplitAssigner assigner, IcebergSourceSplit split) { + GetSplitResult result = assigner.getNext(null); + Assert.assertEquals(result.split(), split); + } + + @Override + protected List createSplits( + int fileCount, int filesPerSplit, String version) { + return IntStream.range(0, fileCount / filesPerSplit) + .mapToObj( + splitNum -> + splitFromRecords( + IntStream.range(0, filesPerSplit) + .mapToObj( + fileNum -> + RandomGenericData.generate( + SCHEMA, 2, splitNum * filesPerSplit + fileNum)) + .collect(Collectors.toList()))) + .collect(Collectors.toList()); + } + + private IcebergSourceSplit splitFromInstant(Instant instant) { + Record record = GenericRecord.create(SCHEMA); + record.set(0, LocalDateTime.ofInstant(instant, ZoneOffset.UTC)); + return splitFromRecords(ImmutableList.of(ImmutableList.of(record))); + } + + private IcebergSourceSplit splitFromRecords(List> records) { + try { + return IcebergSourceSplit.fromCombinedScanTask( + ReaderUtil.createCombinedScanTask( + records, TEMPORARY_FOLDER, FileFormat.PARQUET, APPENDER_FACTORY)); + } catch (IOException e) { + throw new RuntimeException("Split creation exception", e); + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java new file mode 100644 index 000000000000..ebc92df02360 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java @@ -0,0 +1,97 @@ +/* + * 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.iceberg.flink.source.enumerator; + +import java.io.IOException; +import java.util.List; +import java.util.NavigableMap; +import java.util.TreeMap; +import org.apache.iceberg.flink.source.ScanContext; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +class ManualContinuousSplitPlanner implements ContinuousSplitPlanner { + private final int maxPlanningSnapshotCount; + // track splits per snapshot + private final NavigableMap> splits; + private long latestSnapshotId; + private int remainingFailures; + + ManualContinuousSplitPlanner(ScanContext scanContext, int expectedFailures) { + this.maxPlanningSnapshotCount = scanContext.maxPlanningSnapshotCount(); + this.splits = new TreeMap<>(); + this.latestSnapshotId = 0L; + this.remainingFailures = expectedFailures; + } + + @Override + public synchronized ContinuousEnumerationResult planSplits( + IcebergEnumeratorPosition lastPosition) { + if (remainingFailures > 0) { + remainingFailures--; + throw new RuntimeException("Expected failure at planning"); + } + + long fromSnapshotIdExclusive = 0; + if (lastPosition != null && lastPosition.snapshotId() != null) { + fromSnapshotIdExclusive = lastPosition.snapshotId(); + } + + Preconditions.checkArgument( + fromSnapshotIdExclusive <= latestSnapshotId, + "last enumerated snapshotId is greater than the latestSnapshotId"); + if (fromSnapshotIdExclusive == latestSnapshotId) { + // already discovered everything. + return new ContinuousEnumerationResult(Lists.newArrayList(), lastPosition, lastPosition); + } + + // find the subset of snapshots to return discovered splits + long toSnapshotIdInclusive; + if (latestSnapshotId - fromSnapshotIdExclusive > maxPlanningSnapshotCount) { + toSnapshotIdInclusive = fromSnapshotIdExclusive + maxPlanningSnapshotCount; + } else { + toSnapshotIdInclusive = latestSnapshotId; + } + + List discoveredSplits = Lists.newArrayList(); + NavigableMap> discoveredView = + splits.subMap(fromSnapshotIdExclusive, false, toSnapshotIdInclusive, true); + discoveredView.forEach((snapshotId, snapshotSplits) -> discoveredSplits.addAll(snapshotSplits)); + ContinuousEnumerationResult result = + new ContinuousEnumerationResult( + discoveredSplits, + lastPosition, + // use the snapshot Id as snapshot timestamp. + IcebergEnumeratorPosition.of(toSnapshotIdInclusive, toSnapshotIdInclusive)); + return result; + } + + /** + * Add a collection of new splits. A monotonically increased snapshotId is assigned to each batch + * of splits added by this method. + */ + public synchronized void addSplits(List newSplits) { + latestSnapshotId += 1; + splits.put(latestSnapshotId, newSplits); + } + + @Override + public void close() throws IOException {} +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java new file mode 100644 index 000000000000..349eb11cf549 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java @@ -0,0 +1,352 @@ +/* + * 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.iceberg.flink.source.enumerator; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.connector.testutils.source.reader.TestingSplitEnumeratorContext; +import org.apache.iceberg.flink.source.ScanContext; +import org.apache.iceberg.flink.source.SplitHelpers; +import org.apache.iceberg.flink.source.StreamingStartingStrategy; +import org.apache.iceberg.flink.source.assigner.DefaultSplitAssigner; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; +import org.apache.iceberg.flink.source.split.SplitRequestEvent; +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestContinuousIcebergEnumerator { + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Test + public void testDiscoverSplitWhenNoReaderRegistered() throws Exception { + TestingSplitEnumeratorContext enumeratorContext = + new TestingSplitEnumeratorContext<>(4); + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 0); + ContinuousIcebergEnumerator enumerator = + createEnumerator(enumeratorContext, scanContext, splitPlanner); + + Collection pendingSplitsEmpty = + enumerator.snapshotState(1).pendingSplits(); + Assert.assertEquals(0, pendingSplitsEmpty.size()); + + // make one split available and trigger the periodic discovery + List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + splitPlanner.addSplits(splits); + enumeratorContext.triggerAllActions(); + + Collection pendingSplits = enumerator.snapshotState(2).pendingSplits(); + Assert.assertEquals(1, pendingSplits.size()); + IcebergSourceSplitState pendingSplit = pendingSplits.iterator().next(); + Assert.assertEquals(splits.get(0).splitId(), pendingSplit.split().splitId()); + Assert.assertEquals(IcebergSourceSplitStatus.UNASSIGNED, pendingSplit.status()); + } + + @Test + public void testDiscoverWhenReaderRegistered() throws Exception { + TestingSplitEnumeratorContext enumeratorContext = + new TestingSplitEnumeratorContext<>(4); + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 0); + ContinuousIcebergEnumerator enumerator = + createEnumerator(enumeratorContext, scanContext, splitPlanner); + + // register one reader, and let it request a split + enumeratorContext.registerReader(2, "localhost"); + enumerator.addReader(2); + enumerator.handleSourceEvent(2, new SplitRequestEvent()); + + // make one split available and trigger the periodic discovery + List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + splitPlanner.addSplits(splits); + enumeratorContext.triggerAllActions(); + + Assert.assertTrue(enumerator.snapshotState(1).pendingSplits().isEmpty()); + Assertions.assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) + .contains(splits.get(0)); + } + + @Test + public void testRequestingReaderUnavailableWhenSplitDiscovered() throws Exception { + TestingSplitEnumeratorContext enumeratorContext = + new TestingSplitEnumeratorContext<>(4); + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 0); + ContinuousIcebergEnumerator enumerator = + createEnumerator(enumeratorContext, scanContext, splitPlanner); + + // register one reader, and let it request a split + enumeratorContext.registerReader(2, "localhost"); + enumerator.addReader(2); + enumerator.handleSourceEvent(2, new SplitRequestEvent()); + + // remove the reader (like in a failure) + enumeratorContext.registeredReaders().remove(2); + + // make one split available and trigger the periodic discovery + List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + Assert.assertEquals(1, splits.size()); + splitPlanner.addSplits(splits); + enumeratorContext.triggerAllActions(); + + Assert.assertFalse(enumeratorContext.getSplitAssignments().containsKey(2)); + List pendingSplitIds = + enumerator.snapshotState(1).pendingSplits().stream() + .map(IcebergSourceSplitState::split) + .map(IcebergSourceSplit::splitId) + .collect(Collectors.toList()); + Assert.assertEquals(splits.size(), pendingSplitIds.size()); + Assert.assertEquals(splits.get(0).splitId(), pendingSplitIds.get(0)); + + // register the reader again, and let it request a split + enumeratorContext.registerReader(2, "localhost"); + enumerator.addReader(2); + enumerator.handleSourceEvent(2, new SplitRequestEvent()); + + Assert.assertTrue(enumerator.snapshotState(2).pendingSplits().isEmpty()); + Assertions.assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) + .contains(splits.get(0)); + } + + @Test + public void testThrottlingDiscovery() throws Exception { + // create 10 splits + List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 10, 1); + + TestingSplitEnumeratorContext enumeratorContext = + new TestingSplitEnumeratorContext<>(4); + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) + // discover one snapshot at a time + .maxPlanningSnapshotCount(1) + .build(); + ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 0); + ContinuousIcebergEnumerator enumerator = + createEnumerator(enumeratorContext, scanContext, splitPlanner); + + // register reader-2, and let it request a split + enumeratorContext.registerReader(2, "localhost"); + enumerator.addReader(2); + enumerator.handleSourceEvent(2, new SplitRequestEvent()); + + // add splits[0] to the planner for next discovery + splitPlanner.addSplits(Arrays.asList(splits.get(0))); + enumeratorContext.triggerAllActions(); + + // because discovered split was assigned to reader, pending splits should be empty + Assert.assertEquals(0, enumerator.snapshotState(1).pendingSplits().size()); + // split assignment to reader-2 should contain splits[0, 1) + Assert.assertEquals( + splits.subList(0, 1), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); + + // add the remaining 9 splits (one for every snapshot) + // run discovery cycles while reader-2 still processing the splits[0] + for (int i = 1; i < 10; ++i) { + splitPlanner.addSplits(Arrays.asList(splits.get(i))); + enumeratorContext.triggerAllActions(); + } + + // can only discover up to 3 snapshots/splits + Assert.assertEquals(3, enumerator.snapshotState(2).pendingSplits().size()); + // split assignment to reader-2 should be splits[0, 1) + Assert.assertEquals( + splits.subList(0, 1), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); + + // now reader-2 finished splits[0] + enumerator.handleSourceEvent(2, new SplitRequestEvent(Arrays.asList(splits.get(0).splitId()))); + enumeratorContext.triggerAllActions(); + // still have 3 pending splits. After assigned splits[1] to reader-2, one more split was + // discovered and added. + Assert.assertEquals(3, enumerator.snapshotState(3).pendingSplits().size()); + // split assignment to reader-2 should be splits[0, 2) + Assert.assertEquals( + splits.subList(0, 2), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); + + // run 3 more split discovery cycles + for (int i = 0; i < 3; ++i) { + enumeratorContext.triggerAllActions(); + } + + // no more splits are discovered due to throttling + Assert.assertEquals(3, enumerator.snapshotState(4).pendingSplits().size()); + // split assignment to reader-2 should still be splits[0, 2) + Assert.assertEquals( + splits.subList(0, 2), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); + + // now reader-2 finished splits[1] + enumerator.handleSourceEvent(2, new SplitRequestEvent(Arrays.asList(splits.get(1).splitId()))); + enumeratorContext.triggerAllActions(); + // still have 3 pending splits. After assigned new splits[2] to reader-2, one more split was + // discovered and added. + Assert.assertEquals(3, enumerator.snapshotState(5).pendingSplits().size()); + // split assignment to reader-2 should be splits[0, 3) + Assert.assertEquals( + splits.subList(0, 3), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); + } + + @Test + public void testTransientPlanningErrorsWithSuccessfulRetry() throws Exception { + TestingSplitEnumeratorContext enumeratorContext = + new TestingSplitEnumeratorContext<>(4); + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) + .maxPlanningSnapshotCount(1) + .maxAllowedPlanningFailures(2) + .build(); + ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 1); + ContinuousIcebergEnumerator enumerator = + createEnumerator(enumeratorContext, scanContext, splitPlanner); + + // Make one split available and trigger the periodic discovery + List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + splitPlanner.addSplits(splits); + + // Trigger a planning and check that no splits returned due to the planning error + enumeratorContext.triggerAllActions(); + Assert.assertEquals(0, enumerator.snapshotState(2).pendingSplits().size()); + + // Second scan planning should succeed and discover the expected splits + enumeratorContext.triggerAllActions(); + Collection pendingSplits = enumerator.snapshotState(3).pendingSplits(); + Assert.assertEquals(1, pendingSplits.size()); + IcebergSourceSplitState pendingSplit = pendingSplits.iterator().next(); + Assert.assertEquals(splits.get(0).splitId(), pendingSplit.split().splitId()); + Assert.assertEquals(IcebergSourceSplitStatus.UNASSIGNED, pendingSplit.status()); + } + + @Test + public void testOverMaxAllowedPlanningErrors() throws Exception { + TestingSplitEnumeratorContext enumeratorContext = + new TestingSplitEnumeratorContext<>(4); + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) + .maxPlanningSnapshotCount(1) + .maxAllowedPlanningFailures(1) + .build(); + ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 2); + createEnumerator(enumeratorContext, scanContext, splitPlanner); + + // Make one split available and trigger the periodic discovery + List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + splitPlanner.addSplits(splits); + + // Check that the scheduler response ignores the current error and continues to run until the + // failure limit is reached + enumeratorContext.triggerAllActions(); + Assert.assertFalse( + enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).isDone()); + + // Check that the task has failed with the expected exception after the failure limit is reached + enumeratorContext.triggerAllActions(); + Assert.assertTrue( + enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).isDone()); + Assertions.assertThatThrownBy( + () -> enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).get()) + .hasCauseInstanceOf(RuntimeException.class) + .hasMessageContaining("Failed to discover new split"); + } + + @Test + public void testPlanningIgnoringErrors() throws Exception { + int expectedFailures = 3; + TestingSplitEnumeratorContext enumeratorContext = + new TestingSplitEnumeratorContext<>(4); + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) + .maxPlanningSnapshotCount(1) + .maxAllowedPlanningFailures(-1) + .build(); + ManualContinuousSplitPlanner splitPlanner = + new ManualContinuousSplitPlanner(scanContext, expectedFailures); + ContinuousIcebergEnumerator enumerator = + createEnumerator(enumeratorContext, scanContext, splitPlanner); + + // Make one split available and trigger the periodic discovery + List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + splitPlanner.addSplits(splits); + + Collection pendingSplits; + // Can not discover the new split with planning failures + for (int i = 0; i < expectedFailures; ++i) { + enumeratorContext.triggerAllActions(); + pendingSplits = enumerator.snapshotState(i).pendingSplits(); + Assert.assertEquals(0, pendingSplits.size()); + } + + // Discovered the new split after a successful scan planning + enumeratorContext.triggerAllActions(); + pendingSplits = enumerator.snapshotState(expectedFailures + 1).pendingSplits(); + Assert.assertEquals(1, pendingSplits.size()); + IcebergSourceSplitState pendingSplit = pendingSplits.iterator().next(); + Assert.assertEquals(splits.get(0).splitId(), pendingSplit.split().splitId()); + Assert.assertEquals(IcebergSourceSplitStatus.UNASSIGNED, pendingSplit.status()); + } + + private static ContinuousIcebergEnumerator createEnumerator( + SplitEnumeratorContext context, + ScanContext scanContext, + ContinuousSplitPlanner splitPlanner) { + + ContinuousIcebergEnumerator enumerator = + new ContinuousIcebergEnumerator( + context, + new DefaultSplitAssigner(null, Collections.emptyList()), + scanContext, + splitPlanner, + null); + enumerator.start(); + return enumerator; + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java new file mode 100644 index 000000000000..1bb2b1c6bf85 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java @@ -0,0 +1,692 @@ +/* + * 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.iceberg.flink.source.enumerator; + +import java.io.IOException; +import java.util.List; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.ScanContext; +import org.apache.iceberg.flink.source.StreamingStartingStrategy; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.rules.TestName; + +public class TestContinuousSplitPlannerImpl { + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + private static final FileFormat fileFormat = FileFormat.PARQUET; + private static final AtomicLong randomSeed = new AtomicLong(); + + @Rule + public final HadoopTableResource tableResource = + new HadoopTableResource( + TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); + + @Rule public TestName testName = new TestName(); + + private GenericAppenderHelper dataAppender; + private DataFile dataFile1; + private Snapshot snapshot1; + private DataFile dataFile2; + private Snapshot snapshot2; + + @Before + public void before() throws IOException { + dataAppender = new GenericAppenderHelper(tableResource.table(), fileFormat, TEMPORARY_FOLDER); + } + + private void appendTwoSnapshots() throws IOException { + // snapshot1 + List batch1 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); + dataFile1 = dataAppender.writeFile(null, batch1); + dataAppender.appendToTable(dataFile1); + snapshot1 = tableResource.table().currentSnapshot(); + + // snapshot2 + List batch2 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 1L); + dataFile2 = dataAppender.writeFile(null, batch2); + dataAppender.appendToTable(dataFile2); + snapshot2 = tableResource.table().currentSnapshot(); + } + + /** @return the last enumerated snapshot id */ + private CycleResult verifyOneCycle( + ContinuousSplitPlannerImpl splitPlanner, IcebergEnumeratorPosition lastPosition) + throws Exception { + List batch = + RandomGenericData.generate(TestFixtures.SCHEMA, 2, randomSeed.incrementAndGet()); + DataFile dataFile = dataAppender.writeFile(null, batch); + dataAppender.appendToTable(dataFile); + Snapshot snapshot = tableResource.table().currentSnapshot(); + + ContinuousEnumerationResult result = splitPlanner.planSplits(lastPosition); + Assert.assertEquals(lastPosition.snapshotId(), result.fromPosition().snapshotId()); + Assert.assertEquals( + lastPosition.snapshotTimestampMs(), result.fromPosition().snapshotTimestampMs()); + Assert.assertEquals(snapshot.snapshotId(), result.toPosition().snapshotId().longValue()); + Assert.assertEquals( + snapshot.timestampMillis(), result.toPosition().snapshotTimestampMs().longValue()); + Assert.assertEquals(1, result.splits().size()); + IcebergSourceSplit split = Iterables.getOnlyElement(result.splits()); + Assert.assertEquals(1, split.task().files().size()); + Assert.assertEquals( + dataFile.path().toString(), + Iterables.getOnlyElement(split.task().files()).file().path().toString()); + return new CycleResult(result.toPosition(), split); + } + + @Test + public void testTableScanThenIncrementalWithEmptyTable() throws Exception { + ScanContext scanContext = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult emptyTableInitialDiscoveryResult = splitPlanner.planSplits(null); + Assert.assertTrue(emptyTableInitialDiscoveryResult.splits().isEmpty()); + Assert.assertNull(emptyTableInitialDiscoveryResult.fromPosition()); + Assert.assertTrue(emptyTableInitialDiscoveryResult.toPosition().isEmpty()); + Assert.assertNull(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()); + + ContinuousEnumerationResult emptyTableSecondDiscoveryResult = + splitPlanner.planSplits(emptyTableInitialDiscoveryResult.toPosition()); + Assert.assertTrue(emptyTableSecondDiscoveryResult.splits().isEmpty()); + Assert.assertTrue(emptyTableSecondDiscoveryResult.fromPosition().isEmpty()); + Assert.assertNull(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()); + Assert.assertTrue(emptyTableSecondDiscoveryResult.toPosition().isEmpty()); + Assert.assertNull(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()); + + // next 3 snapshots + IcebergEnumeratorPosition lastPosition = emptyTableSecondDiscoveryResult.toPosition(); + for (int i = 0; i < 3; ++i) { + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; + } + } + + @Test + public void testTableScanThenIncrementalWithNonEmptyTable() throws Exception { + appendTwoSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + Assert.assertNull(initialResult.fromPosition()); + Assert.assertEquals( + snapshot2.snapshotId(), initialResult.toPosition().snapshotId().longValue()); + Assert.assertEquals( + snapshot2.timestampMillis(), initialResult.toPosition().snapshotTimestampMs().longValue()); + Assert.assertEquals(1, initialResult.splits().size()); + IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); + Assert.assertEquals(2, split.task().files().size()); + Set discoveredFiles = + split.task().files().stream() + .map(fileScanTask -> fileScanTask.file().path().toString()) + .collect(Collectors.toSet()); + Set expectedFiles = + ImmutableSet.of(dataFile1.path().toString(), dataFile2.path().toString()); + Assert.assertEquals(expectedFiles, discoveredFiles); + + IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); + for (int i = 0; i < 3; ++i) { + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; + } + } + + @Test + public void testIncrementalFromLatestSnapshotWithEmptyTable() throws Exception { + ScanContext scanContext = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) + .splitSize(1L) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult emptyTableInitialDiscoveryResult = splitPlanner.planSplits(null); + Assert.assertTrue(emptyTableInitialDiscoveryResult.splits().isEmpty()); + Assert.assertNull(emptyTableInitialDiscoveryResult.fromPosition()); + Assert.assertTrue(emptyTableInitialDiscoveryResult.toPosition().isEmpty()); + Assert.assertNull(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()); + + ContinuousEnumerationResult emptyTableSecondDiscoveryResult = + splitPlanner.planSplits(emptyTableInitialDiscoveryResult.toPosition()); + Assert.assertTrue(emptyTableSecondDiscoveryResult.splits().isEmpty()); + Assert.assertTrue(emptyTableSecondDiscoveryResult.fromPosition().isEmpty()); + Assert.assertNull(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()); + Assert.assertTrue(emptyTableSecondDiscoveryResult.toPosition().isEmpty()); + Assert.assertNull(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()); + + // latest mode should discover both snapshots, as latest position is marked by when job starts + appendTwoSnapshots(); + ContinuousEnumerationResult afterTwoSnapshotsAppended = + splitPlanner.planSplits(emptyTableSecondDiscoveryResult.toPosition()); + Assert.assertEquals(2, afterTwoSnapshotsAppended.splits().size()); + + // next 3 snapshots + IcebergEnumeratorPosition lastPosition = afterTwoSnapshotsAppended.toPosition(); + for (int i = 0; i < 3; ++i) { + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; + } + } + + @Test + public void testIncrementalFromLatestSnapshotWithNonEmptyTable() throws Exception { + appendTwoSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + Assert.assertNull(initialResult.fromPosition()); + // For inclusive behavior, the initial result should point to snapshot1 + // Then the next incremental scan shall discover files from latest snapshot2 (inclusive) + Assert.assertEquals( + snapshot1.snapshotId(), initialResult.toPosition().snapshotId().longValue()); + Assert.assertEquals( + snapshot1.timestampMillis(), initialResult.toPosition().snapshotTimestampMs().longValue()); + Assert.assertEquals(0, initialResult.splits().size()); + + ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); + Assert.assertEquals( + snapshot1.snapshotId(), secondResult.fromPosition().snapshotId().longValue()); + Assert.assertEquals( + snapshot1.timestampMillis(), secondResult.fromPosition().snapshotTimestampMs().longValue()); + Assert.assertEquals(snapshot2.snapshotId(), secondResult.toPosition().snapshotId().longValue()); + Assert.assertEquals( + snapshot2.timestampMillis(), secondResult.toPosition().snapshotTimestampMs().longValue()); + IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); + Assert.assertEquals(1, split.task().files().size()); + Set discoveredFiles = + split.task().files().stream() + .map(fileScanTask -> fileScanTask.file().path().toString()) + .collect(Collectors.toSet()); + // should discover dataFile2 appended in snapshot2 + Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); + Assert.assertEquals(expectedFiles, discoveredFiles); + + IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); + for (int i = 0; i < 3; ++i) { + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; + } + } + + @Test + public void testIncrementalFromEarliestSnapshotWithEmptyTable() throws Exception { + ScanContext scanContext = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult emptyTableInitialDiscoveryResult = splitPlanner.planSplits(null); + Assert.assertTrue(emptyTableInitialDiscoveryResult.splits().isEmpty()); + Assert.assertNull(emptyTableInitialDiscoveryResult.fromPosition()); + Assert.assertNull(emptyTableInitialDiscoveryResult.toPosition().snapshotId()); + Assert.assertNull(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()); + + ContinuousEnumerationResult emptyTableSecondDiscoveryResult = + splitPlanner.planSplits(emptyTableInitialDiscoveryResult.toPosition()); + Assert.assertTrue(emptyTableSecondDiscoveryResult.splits().isEmpty()); + Assert.assertNull(emptyTableSecondDiscoveryResult.fromPosition().snapshotId()); + Assert.assertNull(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()); + Assert.assertNull(emptyTableSecondDiscoveryResult.toPosition().snapshotId()); + Assert.assertNull(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()); + + // next 3 snapshots + IcebergEnumeratorPosition lastPosition = emptyTableSecondDiscoveryResult.toPosition(); + for (int i = 0; i < 3; ++i) { + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; + } + } + + @Test + public void testIncrementalFromEarliestSnapshotWithNonEmptyTable() throws Exception { + appendTwoSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + Assert.assertNull(initialResult.fromPosition()); + // For inclusive behavior, the initial result should point to snapshot1's parent, + // which leads to null snapshotId and snapshotTimestampMs. + Assert.assertNull(initialResult.toPosition().snapshotId()); + Assert.assertNull(initialResult.toPosition().snapshotTimestampMs()); + Assert.assertEquals(0, initialResult.splits().size()); + + ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); + Assert.assertNull(secondResult.fromPosition().snapshotId()); + Assert.assertNull(secondResult.fromPosition().snapshotTimestampMs()); + Assert.assertEquals(snapshot2.snapshotId(), secondResult.toPosition().snapshotId().longValue()); + Assert.assertEquals( + snapshot2.timestampMillis(), secondResult.toPosition().snapshotTimestampMs().longValue()); + IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); + Assert.assertEquals(2, split.task().files().size()); + Set discoveredFiles = + split.task().files().stream() + .map(fileScanTask -> fileScanTask.file().path().toString()) + .collect(Collectors.toSet()); + // should discover files appended in both snapshot1 and snapshot2 + Set expectedFiles = + ImmutableSet.of(dataFile1.path().toString(), dataFile2.path().toString()); + Assert.assertEquals(expectedFiles, discoveredFiles); + + IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); + for (int i = 0; i < 3; ++i) { + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; + } + } + + @Test + public void testIncrementalFromSnapshotIdWithEmptyTable() { + ScanContext scanContextWithInvalidSnapshotId = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) + .startSnapshotId(1L) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl( + tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); + + Assertions.assertThatThrownBy(() -> splitPlanner.planSplits(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Start snapshot id not found in history: 1"); + } + + @Test + public void testIncrementalFromSnapshotIdWithInvalidIds() throws Exception { + appendTwoSnapshots(); + + // find an invalid snapshotId + long invalidSnapshotId = 0L; + while (invalidSnapshotId == snapshot1.snapshotId() + || invalidSnapshotId == snapshot2.snapshotId()) { + invalidSnapshotId++; + } + + ScanContext scanContextWithInvalidSnapshotId = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) + .startSnapshotId(invalidSnapshotId) + .build(); + + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl( + tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); + + Assertions.assertThatThrownBy(() -> splitPlanner.planSplits(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Start snapshot id not found in history: " + invalidSnapshotId); + } + + @Test + public void testIncrementalFromSnapshotId() throws Exception { + appendTwoSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) + .startSnapshotId(snapshot2.snapshotId()) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + Assert.assertNull(initialResult.fromPosition()); + // For inclusive behavior of snapshot2, the initial result should point to snapshot1 (as + // snapshot2's parent) + Assert.assertEquals( + snapshot1.snapshotId(), initialResult.toPosition().snapshotId().longValue()); + Assert.assertEquals( + snapshot1.timestampMillis(), initialResult.toPosition().snapshotTimestampMs().longValue()); + Assert.assertEquals(0, initialResult.splits().size()); + + ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); + Assert.assertEquals( + snapshot1.snapshotId(), secondResult.fromPosition().snapshotId().longValue()); + Assert.assertEquals( + snapshot1.timestampMillis(), secondResult.fromPosition().snapshotTimestampMs().longValue()); + Assert.assertEquals(snapshot2.snapshotId(), secondResult.toPosition().snapshotId().longValue()); + Assert.assertEquals( + snapshot2.timestampMillis(), secondResult.toPosition().snapshotTimestampMs().longValue()); + IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); + Assert.assertEquals(1, split.task().files().size()); + Set discoveredFiles = + split.task().files().stream() + .map(fileScanTask -> fileScanTask.file().path().toString()) + .collect(Collectors.toSet()); + // should discover dataFile2 appended in snapshot2 + Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); + Assert.assertEquals(expectedFiles, discoveredFiles); + + IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); + for (int i = 0; i < 3; ++i) { + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; + } + } + + @Test + public void testIncrementalFromSnapshotTimestampWithEmptyTable() { + ScanContext scanContextWithInvalidSnapshotId = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) + .startSnapshotTimestamp(1L) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl( + tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); + + Assertions.assertThatThrownBy(() -> splitPlanner.planSplits(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot find a snapshot after: 1"); + } + + @Test + public void testIncrementalFromSnapshotTimestampWithInvalidIds() throws Exception { + appendTwoSnapshots(); + + long invalidSnapshotTimestampMs = snapshot2.timestampMillis() + 1000L; + + ScanContext scanContextWithInvalidSnapshotId = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) + .startSnapshotTimestamp(invalidSnapshotTimestampMs) + .build(); + + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl( + tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); + + Assertions.assertThatThrownBy(() -> splitPlanner.planSplits(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot find a snapshot after:"); + } + + @Test + public void testIncrementalFromSnapshotTimestamp() throws Exception { + appendTwoSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) + .startSnapshotTimestamp(snapshot2.timestampMillis()) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + Assert.assertNull(initialResult.fromPosition()); + // For inclusive behavior, the initial result should point to snapshot1 (as snapshot2's parent). + Assert.assertEquals( + snapshot1.snapshotId(), initialResult.toPosition().snapshotId().longValue()); + Assert.assertEquals( + snapshot1.timestampMillis(), initialResult.toPosition().snapshotTimestampMs().longValue()); + Assert.assertEquals(0, initialResult.splits().size()); + + ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); + Assert.assertEquals( + snapshot1.snapshotId(), secondResult.fromPosition().snapshotId().longValue()); + Assert.assertEquals( + snapshot1.timestampMillis(), secondResult.fromPosition().snapshotTimestampMs().longValue()); + Assert.assertEquals(snapshot2.snapshotId(), secondResult.toPosition().snapshotId().longValue()); + Assert.assertEquals( + snapshot2.timestampMillis(), secondResult.toPosition().snapshotTimestampMs().longValue()); + IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); + Assert.assertEquals(1, split.task().files().size()); + Set discoveredFiles = + split.task().files().stream() + .map(fileScanTask -> fileScanTask.file().path().toString()) + .collect(Collectors.toSet()); + // should discover dataFile2 appended in snapshot2 + Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); + Assert.assertEquals(expectedFiles, discoveredFiles); + + IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); + for (int i = 0; i < 3; ++i) { + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; + } + } + + @Test + public void testMaxPlanningSnapshotCount() throws Exception { + appendTwoSnapshots(); + // append 3 more snapshots + for (int i = 2; i < 5; ++i) { + appendSnapshot(i, 2); + } + + ScanContext scanContext = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) + // limit to 1 snapshot per discovery + .maxPlanningSnapshotCount(1) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + Assert.assertNull(initialResult.fromPosition()); + // For inclusive behavior, the initial result should point to snapshot1's parent, + // which leads to null snapshotId and snapshotTimestampMs. + Assert.assertNull(initialResult.toPosition().snapshotId()); + Assert.assertNull(initialResult.toPosition().snapshotTimestampMs()); + Assert.assertEquals(0, initialResult.splits().size()); + + ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); + // should discover dataFile1 appended in snapshot1 + verifyMaxPlanningSnapshotCountResult( + secondResult, null, snapshot1, ImmutableSet.of(dataFile1.path().toString())); + + ContinuousEnumerationResult thirdResult = splitPlanner.planSplits(secondResult.toPosition()); + // should discover dataFile2 appended in snapshot2 + verifyMaxPlanningSnapshotCountResult( + thirdResult, snapshot1, snapshot2, ImmutableSet.of(dataFile2.path().toString())); + } + + @Test + public void testTableScanNoStats() throws Exception { + appendTwoSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .includeColumnStats(false) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + Assert.assertEquals(1, initialResult.splits().size()); + IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); + Assert.assertEquals(2, split.task().files().size()); + verifyStatCount(split, 0); + + IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); + for (int i = 0; i < 3; ++i) { + CycleResult result = verifyOneCycle(splitPlanner, lastPosition); + verifyStatCount(result.split, 0); + lastPosition = result.lastPosition; + } + } + + @Test + public void testTableScanAllStats() throws Exception { + appendTwoSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .includeColumnStats(true) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + Assert.assertEquals(1, initialResult.splits().size()); + IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); + Assert.assertEquals(2, split.task().files().size()); + verifyStatCount(split, 3); + + IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); + for (int i = 0; i < 3; ++i) { + CycleResult result = verifyOneCycle(splitPlanner, lastPosition); + verifyStatCount(result.split, 3); + lastPosition = result.lastPosition; + } + } + + @Test + public void testTableScanSingleStat() throws Exception { + appendTwoSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .includeColumnStats(ImmutableSet.of("data")) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + Assert.assertEquals(1, initialResult.splits().size()); + IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); + Assert.assertEquals(2, split.task().files().size()); + verifyStatCount(split, 1); + + IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); + for (int i = 0; i < 3; ++i) { + CycleResult result = verifyOneCycle(splitPlanner, lastPosition); + verifyStatCount(result.split, 1); + lastPosition = result.lastPosition; + } + } + + private void verifyStatCount(IcebergSourceSplit split, int expected) { + if (expected == 0) { + split + .task() + .files() + .forEach( + f -> { + Assert.assertNull(f.file().valueCounts()); + Assert.assertNull(f.file().columnSizes()); + Assert.assertNull(f.file().lowerBounds()); + Assert.assertNull(f.file().upperBounds()); + Assert.assertNull(f.file().nanValueCounts()); + Assert.assertNull(f.file().nullValueCounts()); + }); + } else { + split + .task() + .files() + .forEach( + f -> { + Assert.assertEquals(expected, f.file().valueCounts().size()); + Assert.assertEquals(expected, f.file().columnSizes().size()); + Assert.assertEquals(expected, f.file().lowerBounds().size()); + Assert.assertEquals(expected, f.file().upperBounds().size()); + Assert.assertEquals(expected, f.file().nullValueCounts().size()); + // The nanValue is not counted for long and string fields + Assert.assertEquals(0, f.file().nanValueCounts().size()); + }); + } + } + + private void verifyMaxPlanningSnapshotCountResult( + ContinuousEnumerationResult result, + Snapshot fromSnapshotExclusive, + Snapshot toSnapshotInclusive, + Set expectedFiles) { + if (fromSnapshotExclusive == null) { + Assert.assertNull(result.fromPosition().snapshotId()); + Assert.assertNull(result.fromPosition().snapshotTimestampMs()); + } else { + Assert.assertEquals( + fromSnapshotExclusive.snapshotId(), result.fromPosition().snapshotId().longValue()); + Assert.assertEquals( + fromSnapshotExclusive.timestampMillis(), + result.fromPosition().snapshotTimestampMs().longValue()); + } + Assert.assertEquals( + toSnapshotInclusive.snapshotId(), result.toPosition().snapshotId().longValue()); + Assert.assertEquals( + toSnapshotInclusive.timestampMillis(), + result.toPosition().snapshotTimestampMs().longValue()); + // should only have one split with one data file, because split discover is limited to + // one snapshot and each snapshot has only one data file appended. + IcebergSourceSplit split = Iterables.getOnlyElement(result.splits()); + Assert.assertEquals(1, split.task().files().size()); + Set discoveredFiles = + split.task().files().stream() + .map(fileScanTask -> fileScanTask.file().path().toString()) + .collect(Collectors.toSet()); + Assert.assertEquals(expectedFiles, discoveredFiles); + } + + private Snapshot appendSnapshot(long seed, int numRecords) throws Exception { + List batch = RandomGenericData.generate(TestFixtures.SCHEMA, numRecords, seed); + DataFile dataFile = dataAppender.writeFile(null, batch); + dataAppender.appendToTable(dataFile); + return tableResource.table().currentSnapshot(); + } + + private static class CycleResult { + IcebergEnumeratorPosition lastPosition; + IcebergSourceSplit split; + + CycleResult(IcebergEnumeratorPosition lastPosition, IcebergSourceSplit split) { + this.lastPosition = lastPosition; + this.split = split; + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java new file mode 100644 index 000000000000..2c94f215908a --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java @@ -0,0 +1,205 @@ +/* + * 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.iceberg.flink.source.enumerator; + +import java.io.IOException; +import java.util.List; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.ScanContext; +import org.apache.iceberg.flink.source.StreamingStartingStrategy; +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.RuleChain; +import org.junit.rules.TemporaryFolder; +import org.junit.rules.TestRule; + +public class TestContinuousSplitPlannerImplStartStrategy { + private static final FileFormat FILE_FORMAT = FileFormat.PARQUET; + + public final TemporaryFolder temporaryFolder = new TemporaryFolder(); + public final HadoopTableResource tableResource = + new HadoopTableResource( + temporaryFolder, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); + @Rule public final TestRule chain = RuleChain.outerRule(temporaryFolder).around(tableResource); + + private GenericAppenderHelper dataAppender; + private Snapshot snapshot1; + private Snapshot snapshot2; + private Snapshot snapshot3; + + @Before + public void before() throws IOException { + dataAppender = new GenericAppenderHelper(tableResource.table(), FILE_FORMAT, temporaryFolder); + } + + private void appendThreeSnapshots() throws IOException { + List batch1 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); + dataAppender.appendToTable(batch1); + snapshot1 = tableResource.table().currentSnapshot(); + + List batch2 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 1L); + dataAppender.appendToTable(batch2); + snapshot2 = tableResource.table().currentSnapshot(); + + List batch3 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 2L); + dataAppender.appendToTable(batch3); + snapshot3 = tableResource.table().currentSnapshot(); + } + + @Test + public void testTableScanThenIncrementalStrategy() throws IOException { + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + + // empty table + Assert.assertFalse( + ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).isPresent()); + + appendThreeSnapshots(); + Snapshot startSnapshot = + ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); + Assert.assertEquals(snapshot3.snapshotId(), startSnapshot.snapshotId()); + } + + @Test + public void testForLatestSnapshotStrategy() throws IOException { + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) + .build(); + + // empty table + Assert.assertFalse( + ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).isPresent()); + + appendThreeSnapshots(); + Snapshot startSnapshot = + ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); + Assert.assertEquals(snapshot3.snapshotId(), startSnapshot.snapshotId()); + } + + @Test + public void testForEarliestSnapshotStrategy() throws IOException { + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) + .build(); + + // empty table + Assert.assertFalse( + ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).isPresent()); + + appendThreeSnapshots(); + Snapshot startSnapshot = + ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); + Assert.assertEquals(snapshot1.snapshotId(), startSnapshot.snapshotId()); + } + + @Test + public void testForSpecificSnapshotIdStrategy() throws IOException { + ScanContext scanContextInvalidSnapshotId = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) + .startSnapshotId(1L) + .build(); + + // empty table + Assertions.assertThatThrownBy( + () -> + ContinuousSplitPlannerImpl.startSnapshot( + tableResource.table(), scanContextInvalidSnapshotId)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Start snapshot id not found in history: 1"); + + appendThreeSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) + .startSnapshotId(snapshot2.snapshotId()) + .build(); + + Snapshot startSnapshot = + ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); + Assert.assertEquals(snapshot2.snapshotId(), startSnapshot.snapshotId()); + } + + @Test + public void testForSpecificSnapshotTimestampStrategySnapshot2() throws IOException { + ScanContext scanContextInvalidSnapshotTimestamp = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) + .startSnapshotTimestamp(1L) + .build(); + + // empty table + Assertions.assertThatThrownBy( + () -> + ContinuousSplitPlannerImpl.startSnapshot( + tableResource.table(), scanContextInvalidSnapshotTimestamp)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot find a snapshot after: "); + + appendThreeSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) + .startSnapshotTimestamp(snapshot2.timestampMillis()) + .build(); + + Snapshot startSnapshot = + ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); + Assert.assertEquals(snapshot2.snapshotId(), startSnapshot.snapshotId()); + } + + @Test + public void testForSpecificSnapshotTimestampStrategySnapshot2Minus1() throws IOException { + appendThreeSnapshots(); + + ScanContext config = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) + .startSnapshotTimestamp(snapshot2.timestampMillis() - 1L) + .build(); + + Snapshot startSnapshot = + ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), config).get(); + Assert.assertEquals(snapshot2.snapshotId(), startSnapshot.snapshotId()); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java new file mode 100644 index 000000000000..e2be0b4b0398 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java @@ -0,0 +1,134 @@ +/* + * 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.iceberg.flink.source.enumerator; + +import org.junit.Assert; +import org.junit.Test; + +public class TestEnumerationHistory { + private static final int MAX_HISTORY_SIZE = 3; + private static final int FEW_PENDING_SPLITS = 2; + private static final int TOO_MANY_PENDING_SPLITS = 100; + + @Test + public void testEmptyHistory() { + EnumerationHistory history = new EnumerationHistory(MAX_HISTORY_SIZE); + int[] expectedHistorySnapshot = new int[0]; + testHistory(history, expectedHistorySnapshot); + } + + @Test + public void testNotFullHistory() { + EnumerationHistory history = new EnumerationHistory(3); + history.add(1); + history.add(2); + int[] expectedHistorySnapshot = {1, 2}; + testHistory(history, expectedHistorySnapshot); + } + + @Test + public void testExactFullHistory() { + EnumerationHistory history = new EnumerationHistory(3); + history.add(1); + history.add(2); + history.add(3); + int[] expectedHistorySnapshot = {1, 2, 3}; + testHistory(history, expectedHistorySnapshot); + } + + @Test + public void testOneMoreThanFullHistory() { + EnumerationHistory history = new EnumerationHistory(3); + history.add(1); + history.add(2); + history.add(3); + history.add(4); + int[] expectedHistorySnapshot = {2, 3, 4}; + testHistory(history, expectedHistorySnapshot); + } + + @Test + public void testTwoMoreThanFullHistory() { + EnumerationHistory history = new EnumerationHistory(3); + history.add(1); + history.add(2); + history.add(3); + history.add(4); + history.add(5); + int[] expectedHistorySnapshot = {3, 4, 5}; + testHistory(history, expectedHistorySnapshot); + } + + @Test + public void testThreeMoreThanFullHistory() { + EnumerationHistory history = new EnumerationHistory(3); + history.add(1); + history.add(2); + history.add(3); + history.add(4); + history.add(5); + history.add(6); + int[] expectedHistorySnapshot = {4, 5, 6}; + testHistory(history, expectedHistorySnapshot); + } + + private void testHistory(EnumerationHistory history, int[] expectedHistorySnapshot) { + Assert.assertFalse(history.shouldPauseSplitDiscovery(FEW_PENDING_SPLITS)); + if (history.hasFullHistory()) { + // throttle because pending split count is more than the sum of enumeration history + Assert.assertTrue(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)); + } else { + // skipped throttling check because there is not enough history + Assert.assertFalse(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)); + } + + int[] historySnapshot = history.snapshot(); + Assert.assertArrayEquals(expectedHistorySnapshot, historySnapshot); + + EnumerationHistory restoredHistory = new EnumerationHistory(MAX_HISTORY_SIZE); + restoredHistory.restore(historySnapshot); + + Assert.assertFalse(history.shouldPauseSplitDiscovery(FEW_PENDING_SPLITS)); + if (history.hasFullHistory()) { + // throttle because pending split count is more than the sum of enumeration history + Assert.assertTrue(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)); + } else { + // skipped throttling check because there is not enough history + Assert.assertFalse(history.shouldPauseSplitDiscovery(30)); + } + } + + @Test + public void testRestoreDifferentSize() { + EnumerationHistory history = new EnumerationHistory(3); + history.add(1); + history.add(2); + history.add(3); + int[] historySnapshot = history.snapshot(); + + EnumerationHistory smallerHistory = new EnumerationHistory(2); + smallerHistory.restore(historySnapshot); + int[] expectedRestoredHistorySnapshot = {2, 3}; + Assert.assertArrayEquals(expectedRestoredHistorySnapshot, smallerHistory.snapshot()); + + EnumerationHistory largerHisotry = new EnumerationHistory(4); + largerHisotry.restore(historySnapshot); + Assert.assertArrayEquals(historySnapshot, largerHisotry.snapshot()); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java new file mode 100644 index 000000000000..1d12d9f66a8a --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.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.iceberg.flink.source.enumerator; + +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import org.apache.iceberg.flink.source.SplitHelpers; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestIcebergEnumeratorStateSerializer { + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + private final IcebergEnumeratorStateSerializer serializer = + new IcebergEnumeratorStateSerializer(true); + + protected final int version; + + @Parameterized.Parameters(name = "version={0}") + public static Object[] parameters() { + return new Object[] {1, 2}; + } + + public TestIcebergEnumeratorStateSerializer(int version) { + this.version = version; + } + + @Test + public void testEmptySnapshotIdAndPendingSplits() throws Exception { + IcebergEnumeratorState enumeratorState = new IcebergEnumeratorState(Collections.emptyList()); + testSerializer(enumeratorState); + } + + @Test + public void testSomeSnapshotIdAndEmptyPendingSplits() throws Exception { + IcebergEnumeratorPosition position = + IcebergEnumeratorPosition.of(1L, System.currentTimeMillis()); + + IcebergEnumeratorState enumeratorState = + new IcebergEnumeratorState(position, Collections.emptyList()); + testSerializer(enumeratorState); + } + + @Test + public void testSomeSnapshotIdAndPendingSplits() throws Exception { + IcebergEnumeratorPosition position = + IcebergEnumeratorPosition.of(2L, System.currentTimeMillis()); + List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 3, 1); + Collection pendingSplits = Lists.newArrayList(); + pendingSplits.add( + new IcebergSourceSplitState(splits.get(0), IcebergSourceSplitStatus.UNASSIGNED)); + pendingSplits.add( + new IcebergSourceSplitState(splits.get(1), IcebergSourceSplitStatus.ASSIGNED)); + pendingSplits.add( + new IcebergSourceSplitState(splits.get(2), IcebergSourceSplitStatus.COMPLETED)); + + IcebergEnumeratorState enumeratorState = new IcebergEnumeratorState(position, pendingSplits); + testSerializer(enumeratorState); + } + + @Test + public void testEnumerationSplitCountHistory() throws Exception { + if (version == 2) { + IcebergEnumeratorPosition position = + IcebergEnumeratorPosition.of(2L, System.currentTimeMillis()); + List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 3, 1); + Collection pendingSplits = Lists.newArrayList(); + pendingSplits.add( + new IcebergSourceSplitState(splits.get(0), IcebergSourceSplitStatus.UNASSIGNED)); + pendingSplits.add( + new IcebergSourceSplitState(splits.get(1), IcebergSourceSplitStatus.ASSIGNED)); + pendingSplits.add( + new IcebergSourceSplitState(splits.get(2), IcebergSourceSplitStatus.COMPLETED)); + int[] enumerationSplitCountHistory = {1, 2, 3}; + + IcebergEnumeratorState enumeratorState = + new IcebergEnumeratorState(position, pendingSplits, enumerationSplitCountHistory); + testSerializer(enumeratorState); + } + } + + private void testSerializer(IcebergEnumeratorState enumeratorState) throws IOException { + byte[] result; + if (version == 1) { + result = serializer.serializeV1(enumeratorState); + } else { + result = serializer.serialize(enumeratorState); + } + + IcebergEnumeratorState deserialized = serializer.deserialize(version, result); + assertEnumeratorStateEquals(enumeratorState, deserialized); + } + + private void assertEnumeratorStateEquals( + IcebergEnumeratorState expected, IcebergEnumeratorState actual) { + Assert.assertEquals(expected.lastEnumeratedPosition(), actual.lastEnumeratedPosition()); + + Assert.assertEquals(expected.pendingSplits().size(), actual.pendingSplits().size()); + Iterator expectedIterator = expected.pendingSplits().iterator(); + Iterator actualIterator = actual.pendingSplits().iterator(); + for (int i = 0; i < expected.pendingSplits().size(); ++i) { + IcebergSourceSplitState expectedSplitState = expectedIterator.next(); + IcebergSourceSplitState actualSplitState = actualIterator.next(); + Assert.assertEquals(expectedSplitState.split().splitId(), actualSplitState.split().splitId()); + Assert.assertEquals( + expectedSplitState.split().fileOffset(), actualSplitState.split().fileOffset()); + Assert.assertEquals( + expectedSplitState.split().recordOffset(), actualSplitState.split().recordOffset()); + Assert.assertEquals(expectedSplitState.status(), actualSplitState.status()); + } + + Assert.assertArrayEquals( + expected.enumerationSplitCountHistory(), actual.enumerationSplitCountHistory()); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java new file mode 100644 index 000000000000..0d33e4ed08ad --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java @@ -0,0 +1,219 @@ +/* + * 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.iceberg.flink.source.reader; + +import java.io.IOException; +import java.util.List; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public abstract class ReaderFunctionTestBase { + + @Parameterized.Parameters(name = "fileFormat={0}") + public static Object[][] parameters() { + return new Object[][] { + new Object[] {FileFormat.AVRO}, + new Object[] {FileFormat.ORC}, + new Object[] {FileFormat.PARQUET} + }; + } + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + protected abstract ReaderFunction readerFunction(); + + protected abstract void assertRecords(List expected, List actual, Schema schema); + + private final FileFormat fileFormat; + private final GenericAppenderFactory appenderFactory; + + public ReaderFunctionTestBase(FileFormat fileFormat) { + this.fileFormat = fileFormat; + this.appenderFactory = new GenericAppenderFactory(TestFixtures.SCHEMA); + } + + private void assertRecordsAndPosition( + List expectedRecords, + int expectedFileOffset, + long startRecordOffset, + RecordsWithSplitIds> batch) { + batch.nextSplit(); + List actualRecords = Lists.newArrayList(); + long recordOffset = startRecordOffset; + RecordAndPosition recordAndPosition; + while ((recordAndPosition = batch.nextRecordFromSplit()) != null) { + actualRecords.add(recordAndPosition.record()); + Assert.assertEquals( + "expected file offset", expectedFileOffset, recordAndPosition.fileOffset()); + Assert.assertEquals( + "expected record offset", recordOffset, recordAndPosition.recordOffset() - 1); + recordOffset++; + } + + Assert.assertEquals("expected record count", expectedRecords.size(), actualRecords.size()); + assertRecords(expectedRecords, actualRecords, TestFixtures.SCHEMA); + } + + @Test + public void testNoCheckpointedPosition() throws IOException { + List> recordBatchList = + ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); + CombinedScanTask combinedScanTask = + ReaderUtil.createCombinedScanTask( + recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); + IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask); + CloseableIterator>> reader = + readerFunction().apply(split); + + RecordsWithSplitIds> batch0 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(0), 0, 0L, batch0); + batch0.recycle(); + + RecordsWithSplitIds> batch1 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(1), 1, 0L, batch1); + batch1.recycle(); + + RecordsWithSplitIds> batch2 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); + batch2.recycle(); + } + + @Test + public void testCheckpointedPositionBeforeFirstFile() throws IOException { + List> recordBatchList = + ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); + CombinedScanTask combinedScanTask = + ReaderUtil.createCombinedScanTask( + recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); + IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 0, 0L); + CloseableIterator>> reader = + readerFunction().apply(split); + + RecordsWithSplitIds> batch0 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(0), 0, 0L, batch0); + batch0.recycle(); + + RecordsWithSplitIds> batch1 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(1), 1, 0L, batch1); + batch1.recycle(); + + RecordsWithSplitIds> batch2 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); + batch2.recycle(); + } + + @Test + public void testCheckpointedPositionMiddleFirstFile() throws IOException { + List> recordBatchList = + ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); + CombinedScanTask combinedScanTask = + ReaderUtil.createCombinedScanTask( + recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); + IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 0, 1L); + CloseableIterator>> reader = + readerFunction().apply(split); + + RecordsWithSplitIds> batch0 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(0).subList(1, 2), 0, 1L, batch0); + batch0.recycle(); + + RecordsWithSplitIds> batch1 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(1), 1, 0L, batch1); + batch1.recycle(); + + RecordsWithSplitIds> batch2 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); + batch2.recycle(); + } + + @Test + public void testCheckpointedPositionAfterFirstFile() throws IOException { + List> recordBatchList = + ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); + CombinedScanTask combinedScanTask = + ReaderUtil.createCombinedScanTask( + recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); + IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 0, 2L); + CloseableIterator>> reader = + readerFunction().apply(split); + + RecordsWithSplitIds> batch1 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(1), 1, 0L, batch1); + batch1.recycle(); + + RecordsWithSplitIds> batch2 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); + batch2.recycle(); + } + + @Test + public void testCheckpointedPositionBeforeSecondFile() throws IOException { + List> recordBatchList = + ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); + CombinedScanTask combinedScanTask = + ReaderUtil.createCombinedScanTask( + recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); + IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 1, 0L); + CloseableIterator>> reader = + readerFunction().apply(split); + + RecordsWithSplitIds> batch1 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(1), 1, 0L, batch1); + batch1.recycle(); + + RecordsWithSplitIds> batch2 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); + batch2.recycle(); + } + + @Test + public void testCheckpointedPositionMidSecondFile() throws IOException { + List> recordBatchList = + ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); + CombinedScanTask combinedScanTask = + ReaderUtil.createCombinedScanTask( + recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); + IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 1, 1L); + CloseableIterator>> reader = + readerFunction().apply(split); + + RecordsWithSplitIds> batch1 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(1).subList(1, 2), 1, 1L, batch1); + batch1.recycle(); + + RecordsWithSplitIds> batch2 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); + batch2.recycle(); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java new file mode 100644 index 000000000000..2a2503ef2478 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import java.io.File; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.BaseCombinedScanTask; +import org.apache.iceberg.BaseFileScanTask; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Files; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.PartitionSpecParser; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.encryption.PlaintextEncryptionManager; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.ResidualEvaluator; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.rules.TemporaryFolder; + +public class ReaderUtil { + + private ReaderUtil() {} + + public static FileScanTask createFileTask( + List records, + File file, + FileFormat fileFormat, + FileAppenderFactory appenderFactory) + throws IOException { + FileAppender appender = + appenderFactory.newAppender(Files.localOutput(file), fileFormat); + try { + appender.addAll(records); + } finally { + appender.close(); + } + + DataFile dataFile = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withRecordCount(records.size()) + .withFileSizeInBytes(file.length()) + .withPath(file.toString()) + .withFormat(fileFormat) + .withMetrics(appender.metrics()) + .build(); + + ResidualEvaluator residuals = ResidualEvaluator.unpartitioned(Expressions.alwaysTrue()); + return new BaseFileScanTask( + dataFile, + null, + SchemaParser.toJson(TestFixtures.SCHEMA), + PartitionSpecParser.toJson(PartitionSpec.unpartitioned()), + residuals); + } + + public static DataIterator createDataIterator(CombinedScanTask combinedTask) { + return new DataIterator<>( + new RowDataFileScanTaskReader( + TestFixtures.SCHEMA, TestFixtures.SCHEMA, null, true, Collections.emptyList()), + combinedTask, + new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), + new PlaintextEncryptionManager()); + } + + public static List> createRecordBatchList( + Schema schema, int listSize, int batchCount) { + return createRecordBatchList(0L, schema, listSize, batchCount); + } + + public static List> createRecordBatchList( + long seed, Schema schema, int listSize, int batchCount) { + List records = RandomGenericData.generate(schema, listSize * batchCount, seed); + return Lists.partition(records, batchCount); + } + + public static CombinedScanTask createCombinedScanTask( + List> recordBatchList, + TemporaryFolder temporaryFolder, + FileFormat fileFormat, + GenericAppenderFactory appenderFactory) + throws IOException { + List fileTasks = Lists.newArrayListWithCapacity(recordBatchList.size()); + for (List recordBatch : recordBatchList) { + FileScanTask fileTask = + ReaderUtil.createFileTask( + recordBatch, temporaryFolder.newFile(), fileFormat, appenderFactory); + fileTasks.add(fileTask); + } + + return new BaseCombinedScanTask(fileTasks); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java new file mode 100644 index 000000000000..644ac2bad6b8 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.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.iceberg.flink.source.reader; + +import java.util.concurrent.atomic.AtomicBoolean; +import org.junit.Assert; +import org.junit.Test; + +public class TestArrayBatchRecords { + + @Test + public void testFullRange() { + String[] elements = new String[] {"0", "1", "2", "3"}; + testArray(elements, elements.length, 2, 119); + } + + @Test + public void testSubRange() { + String[] elements = new String[] {"0", "1", "2", "3"}; + testArray(elements, 2, 0, 0); + } + + private void testArray( + String[] elements, int numberOfRecords, int fileOffset, long startingRecordOffset) { + String splitId = "iceberg_split_1"; + AtomicBoolean recycled = new AtomicBoolean(); + + ArrayBatchRecords recordsWithSplitIds = + ArrayBatchRecords.forRecords( + splitId, + ignored -> recycled.set(true), + elements, + numberOfRecords, + fileOffset, + startingRecordOffset); + + Assert.assertEquals(splitId, recordsWithSplitIds.nextSplit()); + + for (int i = 0; i < numberOfRecords; i++) { + RecordAndPosition recAndPos = recordsWithSplitIds.nextRecordFromSplit(); + Assert.assertEquals(elements[i], recAndPos.record()); + Assert.assertEquals(fileOffset, recAndPos.fileOffset()); + // recordOffset points to the position after this one + Assert.assertEquals(startingRecordOffset + i + 1, recAndPos.recordOffset()); + } + + Assert.assertNull(recordsWithSplitIds.nextRecordFromSplit()); + Assert.assertNull(recordsWithSplitIds.nextSplit()); + recordsWithSplitIds.recycle(); + Assert.assertTrue(recycled.get()); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java new file mode 100644 index 000000000000..f964a7707689 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java @@ -0,0 +1,358 @@ +/* + * 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.iceberg.flink.source.reader; + +import java.util.Arrays; +import java.util.List; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.SourceReaderOptions; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.BaseCombinedScanTask; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.io.CloseableIterator; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestArrayPoolDataIteratorBatcherRowData { + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + private static final FileFormat fileFormat = FileFormat.PARQUET; + + private final GenericAppenderFactory appenderFactory; + private final DataIteratorBatcher batcher; + + public TestArrayPoolDataIteratorBatcherRowData() { + Configuration config = new Configuration(); + // set array pool size to 1 + config.set(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY, 1); + // set batch array size to 2 + config.set(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 2); + this.batcher = + new ArrayPoolDataIteratorBatcher<>(config, new RowDataRecordFactory(TestFixtures.ROW_TYPE)); + this.appenderFactory = new GenericAppenderFactory(TestFixtures.SCHEMA); + } + + /** Read a CombinedScanTask that contains a single file with less than a full batch of records */ + @Test + public void testSingleFileLessThanOneFullBatch() throws Exception { + List records = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 1); + FileScanTask fileTask = + ReaderUtil.createFileTask(records, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); + CombinedScanTask combinedTask = new BaseCombinedScanTask(fileTask); + DataIterator dataIterator = ReaderUtil.createDataIterator(combinedTask); + String splitId = "someSplitId"; + CloseableIterator>> recordBatchIterator = + batcher.batch(splitId, dataIterator); + + ArrayBatchRecords batch = (ArrayBatchRecords) recordBatchIterator.next(); + Assert.assertTrue(batch.finishedSplits().isEmpty()); + Assert.assertEquals(splitId, batch.nextSplit()); + // reusable array size should be the configured value of 2 + Assert.assertEquals(2, batch.records().length); + // assert actual number of records in the array + Assert.assertEquals(1, batch.numberOfRecords()); + + RecordAndPosition recordAndPosition = batch.nextRecordFromSplit(); + + /////////////////////////////// + // assert first record + + Assert.assertEquals(0, recordAndPosition.fileOffset()); + // The position points to where the reader should resume after this record is processed. + Assert.assertEquals(1, recordAndPosition.recordOffset()); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(0), recordAndPosition.record()); + + Assert.assertNull(batch.nextRecordFromSplit()); + Assert.assertNull(batch.nextSplit()); + batch.recycle(); + + // assert end of input + Assert.assertFalse(recordBatchIterator.hasNext()); + } + + /** + * Read a CombinedScanTask that contains a single file with multiple batches. + * + *

    Insert 5 records in a single file that should result in 3 batches + */ + @Test + public void testSingleFileWithMultipleBatches() throws Exception { + List records = RandomGenericData.generate(TestFixtures.SCHEMA, 5, 1); + FileScanTask fileTask = + ReaderUtil.createFileTask(records, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); + CombinedScanTask combinedTask = new BaseCombinedScanTask(fileTask); + DataIterator dataIterator = ReaderUtil.createDataIterator(combinedTask); + String splitId = "someSplitId"; + CloseableIterator>> recordBatchIterator = + batcher.batch(splitId, dataIterator); + + /////////////////////////////// + // assert first batch with full batch of 2 records + + ArrayBatchRecords batch0 = (ArrayBatchRecords) recordBatchIterator.next(); + Assert.assertTrue(batch0.finishedSplits().isEmpty()); + Assert.assertEquals(splitId, batch0.nextSplit()); + // reusable array size should be the configured value of 2 + Assert.assertEquals(2, batch0.records().length); + // assert actual number of records in the array + Assert.assertEquals(2, batch0.numberOfRecords()); + + RecordAndPosition recordAndPosition; + + // assert first record + recordAndPosition = batch0.nextRecordFromSplit(); + Assert.assertEquals(0, recordAndPosition.fileOffset()); + // The position points to where the reader should resume after this record is processed. + Assert.assertEquals(1, recordAndPosition.recordOffset()); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(0), recordAndPosition.record()); + + // assert second record + recordAndPosition = batch0.nextRecordFromSplit(); + Assert.assertEquals(0, recordAndPosition.fileOffset()); + // The position points to where the reader should resume after this record is processed. + Assert.assertEquals(2, recordAndPosition.recordOffset()); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(1), recordAndPosition.record()); + + Assert.assertNull(batch0.nextRecordFromSplit()); + Assert.assertNull(batch0.nextSplit()); + batch0.recycle(); + + /////////////////////////////// + // assert second batch with full batch of 2 records + + ArrayBatchRecords batch1 = (ArrayBatchRecords) recordBatchIterator.next(); + // assert array is reused + Assert.assertSame(batch0.records(), batch1.records()); + Assert.assertTrue(batch1.finishedSplits().isEmpty()); + Assert.assertEquals(splitId, batch1.nextSplit()); + // reusable array size should be the configured value of 2 + Assert.assertEquals(2, batch1.records().length); + // assert actual number of records in the array + Assert.assertEquals(2, batch1.numberOfRecords()); + + // assert third record + recordAndPosition = batch1.nextRecordFromSplit(); + Assert.assertEquals(0, recordAndPosition.fileOffset()); + // The position points to where the reader should resume after this record is processed. + Assert.assertEquals(3, recordAndPosition.recordOffset()); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(2), recordAndPosition.record()); + + // assert fourth record + recordAndPosition = batch1.nextRecordFromSplit(); + Assert.assertEquals(0, recordAndPosition.fileOffset()); + // The position points to where the reader should resume after this record is processed. + Assert.assertEquals(4, recordAndPosition.recordOffset()); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(3), recordAndPosition.record()); + + Assert.assertNull(batch1.nextRecordFromSplit()); + Assert.assertNull(batch1.nextSplit()); + batch1.recycle(); + + /////////////////////////////// + // assert third batch with partial batch of 1 record + + ArrayBatchRecords batch2 = (ArrayBatchRecords) recordBatchIterator.next(); + // assert array is reused + Assert.assertSame(batch0.records(), batch2.records()); + Assert.assertTrue(batch2.finishedSplits().isEmpty()); + Assert.assertEquals(splitId, batch2.nextSplit()); + // reusable array size should be the configured value of 2 + Assert.assertEquals(2, batch2.records().length); + // assert actual number of records in the array + Assert.assertEquals(1, batch2.numberOfRecords()); + + // assert fifth record + recordAndPosition = batch2.nextRecordFromSplit(); + Assert.assertEquals(0, recordAndPosition.fileOffset()); + // The position points to where the reader should resume after this record is processed. + Assert.assertEquals(5, recordAndPosition.recordOffset()); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(4), recordAndPosition.record()); + + Assert.assertNull(batch2.nextRecordFromSplit()); + Assert.assertNull(batch2.nextSplit()); + batch2.recycle(); + + // assert end of input + Assert.assertFalse(recordBatchIterator.hasNext()); + } + + /** + * Read a CombinedScanTask that contains with multiple files. + * + *

    In this test, we also seek the iterator to starting position (1, 1). + */ + @Test + public void testMultipleFilesWithSeekPosition() throws Exception { + List records0 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 1); + FileScanTask fileTask0 = + ReaderUtil.createFileTask( + records0, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); + List records1 = RandomGenericData.generate(TestFixtures.SCHEMA, 4, 2); + FileScanTask fileTask1 = + ReaderUtil.createFileTask( + records1, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); + List records2 = RandomGenericData.generate(TestFixtures.SCHEMA, 3, 3); + FileScanTask fileTask2 = + ReaderUtil.createFileTask( + records2, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); + CombinedScanTask combinedTask = + new BaseCombinedScanTask(Arrays.asList(fileTask0, fileTask1, fileTask2)); + + DataIterator dataIterator = ReaderUtil.createDataIterator(combinedTask); + // seek to file1 and after record 1 + dataIterator.seek(1, 1); + + String splitId = "someSplitId"; + CloseableIterator>> recordBatchIterator = + batcher.batch(splitId, dataIterator); + + /////////////////////////////// + // file0 is skipped by seek + + /////////////////////////////// + // file1 has 4 records. because the seek position, first record is skipped. + // we should read 3 remaining records in 2 batches: + // batch10 with 2 records and batch11 with 1 records. + + // assert first batch from file1 with full batch of 2 records + + // variable naming convention: batch + ArrayBatchRecords batch10 = (ArrayBatchRecords) recordBatchIterator.next(); + Assert.assertTrue(batch10.finishedSplits().isEmpty()); + Assert.assertEquals(splitId, batch10.nextSplit()); + // reusable array size should be the configured value of 2 + Assert.assertEquals(2, batch10.records().length); + // assert actual number of records in the array + Assert.assertEquals(2, batch10.numberOfRecords()); + + RecordAndPosition recordAndPosition; + + recordAndPosition = batch10.nextRecordFromSplit(); + Assert.assertEquals(1, recordAndPosition.fileOffset()); + // seek should skip the first record in file1. starting from the second record + Assert.assertEquals(2, recordAndPosition.recordOffset()); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records1.get(1), recordAndPosition.record()); + + recordAndPosition = batch10.nextRecordFromSplit(); + Assert.assertEquals(1, recordAndPosition.fileOffset()); + // The position points to where the reader should resume after this record is processed. + Assert.assertEquals(3, recordAndPosition.recordOffset()); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records1.get(2), recordAndPosition.record()); + + Assert.assertNull(batch10.nextRecordFromSplit()); + Assert.assertNull(batch10.nextSplit()); + batch10.recycle(); + + // assert second batch from file1 with partial batch of 1 record + + // variable naming convention: batch__ + ArrayBatchRecords batch11 = (ArrayBatchRecords) recordBatchIterator.next(); + // assert array is reused + Assert.assertSame(batch10.records(), batch11.records()); + Assert.assertTrue(batch11.finishedSplits().isEmpty()); + Assert.assertEquals(splitId, batch11.nextSplit()); + // reusable array size should be the configured value of 2 + Assert.assertEquals(2, batch11.records().length); + // assert actual number of records in the array + Assert.assertEquals(1, batch11.numberOfRecords()); + + recordAndPosition = batch11.nextRecordFromSplit(); + Assert.assertEquals(1, recordAndPosition.fileOffset()); + // The position points to where the reader should resume after this record is processed. + Assert.assertEquals(4, recordAndPosition.recordOffset()); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records1.get(3), recordAndPosition.record()); + + Assert.assertNull(batch11.nextRecordFromSplit()); + Assert.assertNull(batch11.nextSplit()); + batch11.recycle(); + + /////////////////////////////// + // file2 has 3 records. + // we should read 3 records in 2 batches: + // batch20 with 2 records and batch21 with 1 records + + // assert first batch from file2 with full batch of 2 records + + // variable naming convention: batch__ + ArrayBatchRecords batch20 = (ArrayBatchRecords) recordBatchIterator.next(); + // assert array is reused + Assert.assertSame(batch10.records(), batch20.records()); + Assert.assertTrue(batch20.finishedSplits().isEmpty()); + Assert.assertEquals(splitId, batch20.nextSplit()); + // reusable array size should be the configured value of 2 + Assert.assertEquals(2, batch20.records().length); + // assert actual number of records in the array + Assert.assertEquals(2, batch20.numberOfRecords()); + + recordAndPosition = batch20.nextRecordFromSplit(); + Assert.assertEquals(2, recordAndPosition.fileOffset()); + // The position points to where the reader should resume after this record is processed. + Assert.assertEquals(1, recordAndPosition.recordOffset()); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records2.get(0), recordAndPosition.record()); + + recordAndPosition = batch20.nextRecordFromSplit(); + Assert.assertEquals(2, recordAndPosition.fileOffset()); + // The position points to where the reader should resume after this record is processed. + Assert.assertEquals(2, recordAndPosition.recordOffset()); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records2.get(1), recordAndPosition.record()); + + Assert.assertNull(batch20.nextRecordFromSplit()); + Assert.assertNull(batch20.nextSplit()); + batch20.recycle(); + + /////////////////////////////// + // assert second batch from file2 with partial batch of 1 record + + // variable naming convention: batch__ + ArrayBatchRecords batch21 = (ArrayBatchRecords) recordBatchIterator.next(); + // assert array is reused + Assert.assertSame(batch10.records(), batch21.records()); + Assert.assertTrue(batch21.finishedSplits().isEmpty()); + Assert.assertEquals(splitId, batch21.nextSplit()); + // reusable array size should be the configured value of 2 + Assert.assertEquals(2, batch21.records().length); + // assert actual number of records in the array + Assert.assertEquals(1, batch21.numberOfRecords()); + + recordAndPosition = batch21.nextRecordFromSplit(); + Assert.assertEquals(2, recordAndPosition.fileOffset()); + // The position points to where the reader should resume after this record is processed. + Assert.assertEquals(3, recordAndPosition.recordOffset()); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records2.get(2), recordAndPosition.record()); + + Assert.assertNull(batch21.nextRecordFromSplit()); + Assert.assertNull(batch21.nextSplit()); + batch21.recycle(); + + // assert end of input + Assert.assertFalse(recordBatchIterator.hasNext()); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java new file mode 100644 index 000000000000..afe8a5d0152c --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java @@ -0,0 +1,178 @@ +/* + * 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.iceberg.flink.source.reader; + +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.io.IOException; +import java.time.LocalDateTime; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Assume; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestColumnStatsWatermarkExtractor { + public static final Schema SCHEMA = + new Schema( + required(1, "timestamp_column", Types.TimestampType.withoutZone()), + required(2, "timestamptz_column", Types.TimestampType.withZone()), + required(3, "long_column", Types.LongType.get()), + required(4, "string_column", Types.StringType.get())); + + private static final GenericAppenderFactory APPENDER_FACTORY = new GenericAppenderFactory(SCHEMA); + + private static final List> TEST_RECORDS = + ImmutableList.of( + RandomGenericData.generate(SCHEMA, 3, 2L), RandomGenericData.generate(SCHEMA, 3, 19L)); + + private static final List> MIN_VALUES = + ImmutableList.of(Maps.newHashMapWithExpectedSize(3), Maps.newHashMapWithExpectedSize(3)); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final HadoopTableResource sourceTableResource = + new HadoopTableResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, SCHEMA); + + private final String columnName; + + @BeforeClass + public static void updateMinValue() { + for (int i = 0; i < TEST_RECORDS.size(); ++i) { + for (Record r : TEST_RECORDS.get(i)) { + Map minValues = MIN_VALUES.get(i); + + LocalDateTime localDateTime = (LocalDateTime) r.get(0); + minValues.merge( + "timestamp_column", localDateTime.toInstant(ZoneOffset.UTC).toEpochMilli(), Math::min); + + OffsetDateTime offsetDateTime = (OffsetDateTime) r.get(1); + minValues.merge("timestamptz_column", offsetDateTime.toInstant().toEpochMilli(), Math::min); + + minValues.merge("long_column", (Long) r.get(2), Math::min); + } + } + } + + @Parameterized.Parameters(name = "{0}") + public static Collection data() { + return ImmutableList.of( + new Object[] {"timestamp_column"}, + new Object[] {"timestamptz_column"}, + new Object[] {"long_column"}); + } + + public TestColumnStatsWatermarkExtractor(String columnName) { + this.columnName = columnName; + } + + @Test + public void testSingle() throws IOException { + ColumnStatsWatermarkExtractor extractor = + new ColumnStatsWatermarkExtractor(SCHEMA, columnName, TimeUnit.MILLISECONDS); + + Assert.assertEquals( + MIN_VALUES.get(0).get(columnName).longValue(), extractor.extractWatermark(split(0))); + } + + @Test + public void testTimeUnit() throws IOException { + Assume.assumeTrue("Run only for long column", columnName.equals("long_column")); + ColumnStatsWatermarkExtractor extractor = + new ColumnStatsWatermarkExtractor(SCHEMA, columnName, TimeUnit.MICROSECONDS); + + Assert.assertEquals( + MIN_VALUES.get(0).get(columnName).longValue() / 1000L, + extractor.extractWatermark(split(0))); + } + + @Test + public void testMultipleFiles() throws IOException { + Assume.assumeTrue("Run only for the timestamp column", columnName.equals("timestamp_column")); + IcebergSourceSplit combinedSplit = + IcebergSourceSplit.fromCombinedScanTask( + ReaderUtil.createCombinedScanTask( + TEST_RECORDS, TEMPORARY_FOLDER, FileFormat.PARQUET, APPENDER_FACTORY)); + + ColumnStatsWatermarkExtractor extractor = + new ColumnStatsWatermarkExtractor(SCHEMA, columnName, null); + + Assert.assertEquals( + MIN_VALUES.get(0).get(columnName).longValue(), extractor.extractWatermark(split(0))); + Assert.assertEquals( + MIN_VALUES.get(1).get(columnName).longValue(), extractor.extractWatermark(split(1))); + Assert.assertEquals( + Math.min(MIN_VALUES.get(0).get(columnName), MIN_VALUES.get(1).get(columnName)), + extractor.extractWatermark(combinedSplit)); + } + + @Test + public void testWrongColumn() { + Assume.assumeTrue("Run only for string column", columnName.equals("string_column")); + Assertions.assertThatThrownBy(() -> new ColumnStatsWatermarkExtractor(SCHEMA, columnName, null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Found STRING, expected a LONG or TIMESTAMP column for watermark generation."); + } + + @Test + public void testEmptyStatistics() throws IOException { + Assume.assumeTrue("Run only for timestamp column", columnName.equals("timestamp_column")); + + // Create an extractor for a column we do not have statistics + ColumnStatsWatermarkExtractor extractor = + new ColumnStatsWatermarkExtractor(10, "missing_field"); + Assertions.assertThatThrownBy(() -> extractor.extractWatermark(split(0))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Missing statistics for column"); + } + + private IcebergSourceSplit split(int id) throws IOException { + return IcebergSourceSplit.fromCombinedScanTask( + ReaderUtil.createCombinedScanTask( + ImmutableList.of(TEST_RECORDS.get(id)), + TEMPORARY_FOLDER, + FileFormat.PARQUET, + APPENDER_FACTORY)); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java new file mode 100644 index 000000000000..88234c61123f --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java @@ -0,0 +1,184 @@ +/* + * 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.iceberg.flink.source.reader; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.testutils.source.reader.TestingReaderContext; +import org.apache.flink.connector.testutils.source.reader.TestingReaderOutput; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.encryption.PlaintextEncryptionManager; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SerializableComparator; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestIcebergSourceReader { + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + private final GenericAppenderFactory appenderFactory; + + public TestIcebergSourceReader() { + this.appenderFactory = new GenericAppenderFactory(TestFixtures.SCHEMA); + } + + @Test + public void testReaderMetrics() throws Exception { + TestingReaderOutput readerOutput = new TestingReaderOutput<>(); + TestingMetricGroup metricGroup = new TestingMetricGroup(); + TestingReaderContext readerContext = new TestingReaderContext(new Configuration(), metricGroup); + IcebergSourceReader reader = createReader(metricGroup, readerContext, null); + reader.start(); + + testOneSplitFetcher(reader, readerOutput, metricGroup, 1); + testOneSplitFetcher(reader, readerOutput, metricGroup, 2); + } + + @Test + public void testReaderOrder() throws Exception { + // Create 2 splits + List> recordBatchList1 = + ReaderUtil.createRecordBatchList(0L, TestFixtures.SCHEMA, 1, 1); + CombinedScanTask task1 = + ReaderUtil.createCombinedScanTask( + recordBatchList1, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); + + List> recordBatchList2 = + ReaderUtil.createRecordBatchList(1L, TestFixtures.SCHEMA, 1, 1); + CombinedScanTask task2 = + ReaderUtil.createCombinedScanTask( + recordBatchList2, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); + + // Sort the splits in one way + List rowDataList1 = + read( + Arrays.asList( + IcebergSourceSplit.fromCombinedScanTask(task1), + IcebergSourceSplit.fromCombinedScanTask(task2)), + 2); + + // Reverse the splits + List rowDataList2 = + read( + Arrays.asList( + IcebergSourceSplit.fromCombinedScanTask(task2), + IcebergSourceSplit.fromCombinedScanTask(task1)), + 2); + + // Check that the order of the elements is not changed + Assert.assertEquals(rowDataList1.get(0), rowDataList2.get(0)); + Assert.assertEquals(rowDataList1.get(1), rowDataList2.get(1)); + } + + private List read(List splits, long expected) throws Exception { + TestingMetricGroup metricGroup = new TestingMetricGroup(); + TestingReaderContext readerContext = new TestingReaderContext(new Configuration(), metricGroup); + // Using IdBasedComparator, so we can have a deterministic order of the splits + IcebergSourceReader reader = createReader(metricGroup, readerContext, new IdBasedComparator()); + reader.start(); + + reader.addSplits(splits); + TestingReaderOutput readerOutput = new TestingReaderOutput<>(); + while (readerOutput.getEmittedRecords().size() < expected) { + reader.pollNext(readerOutput); + } + + reader.pollNext(readerOutput); + + Assert.assertEquals(expected, readerOutput.getEmittedRecords().size()); + return readerOutput.getEmittedRecords(); + } + + private void testOneSplitFetcher( + IcebergSourceReader reader, + TestingReaderOutput readerOutput, + TestingMetricGroup metricGroup, + int expectedCount) + throws Exception { + long seed = expectedCount; + // Each split should contain only one file with one record + List> recordBatchList = + ReaderUtil.createRecordBatchList(seed, TestFixtures.SCHEMA, 1, 1); + CombinedScanTask task = + ReaderUtil.createCombinedScanTask( + recordBatchList, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); + IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(task); + reader.addSplits(Arrays.asList(split)); + + while (readerOutput.getEmittedRecords().size() < expectedCount) { + reader.pollNext(readerOutput); + } + + Assert.assertEquals(expectedCount, readerOutput.getEmittedRecords().size()); + TestHelpers.assertRowData( + TestFixtures.SCHEMA, + recordBatchList.get(0).get(0), + readerOutput.getEmittedRecords().get(expectedCount - 1)); + Assert.assertEquals(expectedCount, metricGroup.counters().get("assignedSplits").getCount()); + + // One more poll will get null record batch. + // That will finish the split and cause split fetcher to be closed due to idleness. + // Then next split will create a new split reader. + reader.pollNext(readerOutput); + } + + private IcebergSourceReader createReader( + MetricGroup metricGroup, + SourceReaderContext readerContext, + SerializableComparator splitComparator) { + IcebergSourceReaderMetrics readerMetrics = + new IcebergSourceReaderMetrics(metricGroup, "db.tbl"); + RowDataReaderFunction readerFunction = + new RowDataReaderFunction( + new Configuration(), + TestFixtures.SCHEMA, + TestFixtures.SCHEMA, + null, + true, + new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), + new PlaintextEncryptionManager(), + Collections.emptyList()); + return new IcebergSourceReader<>( + SerializableRecordEmitter.defaultEmitter(), + readerMetrics, + readerFunction, + splitComparator, + readerContext); + } + + private static class IdBasedComparator implements SerializableComparator { + @Override + public int compare(IcebergSourceSplit o1, IcebergSourceSplit o2) { + return o1.splitId().compareTo(o2.splitId()); + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java new file mode 100644 index 000000000000..d063ad7f4a80 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.conversion.DataStructureConverter; +import org.apache.flink.table.data.conversion.DataStructureConverters; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.flink.types.Row; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.encryption.PlaintextEncryptionManager; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.hadoop.HadoopFileIO; + +public class TestRowDataReaderFunction extends ReaderFunctionTestBase { + + protected static final RowType rowType = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); + private static final DataStructureConverter rowDataConverter = + DataStructureConverters.getConverter(TypeConversions.fromLogicalToDataType(rowType)); + + public TestRowDataReaderFunction(FileFormat fileFormat) { + super(fileFormat); + } + + @Override + protected ReaderFunction readerFunction() { + return new RowDataReaderFunction( + new Configuration(), + TestFixtures.SCHEMA, + TestFixtures.SCHEMA, + null, + true, + new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), + new PlaintextEncryptionManager(), + Collections.emptyList()); + } + + @Override + protected void assertRecords(List expected, List actual, Schema schema) { + List rows = toRows(actual); + TestHelpers.assertRecords(rows, expected, TestFixtures.SCHEMA); + } + + private List toRows(List actual) { + return actual.stream() + .map(rowData -> (Row) rowDataConverter.toExternal(rowData)) + .collect(Collectors.toList()); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java new file mode 100644 index 000000000000..290628c5fc90 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java @@ -0,0 +1,102 @@ +/* + * 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.iceberg.flink.source.reader; + +import java.util.Map; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.SimpleCounter; +import org.apache.flink.metrics.groups.OperatorIOMetricGroup; +import org.apache.flink.metrics.groups.SourceReaderMetricGroup; +import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +class TestingMetricGroup extends UnregisteredMetricsGroup implements SourceReaderMetricGroup { + private final Map counters; + + TestingMetricGroup() { + this.counters = Maps.newHashMap(); + } + + /** Pass along the reference to share the map for child metric groups. */ + private TestingMetricGroup(Map counters) { + this.counters = counters; + } + + Map counters() { + return counters; + } + + @Override + public Counter counter(String name) { + Counter counter = new SimpleCounter(); + counters.put(name, counter); + return counter; + } + + @Override + public MetricGroup addGroup(String name) { + return new TestingMetricGroup(counters); + } + + @Override + public MetricGroup addGroup(String key, String value) { + return new TestingMetricGroup(counters); + } + + @Override + public OperatorIOMetricGroup getIOMetricGroup() { + return new TestingOperatorIOMetricGroup(); + } + + @Override + public Counter getNumRecordsInErrorsCounter() { + return new SimpleCounter(); + } + + @Override + public void setPendingBytesGauge(Gauge pendingBytesGauge) {} + + @Override + public void setPendingRecordsGauge(Gauge pendingRecordsGauge) {} + + private static class TestingOperatorIOMetricGroup extends UnregisteredMetricsGroup + implements OperatorIOMetricGroup { + @Override + public Counter getNumRecordsInCounter() { + return new SimpleCounter(); + } + + @Override + public Counter getNumRecordsOutCounter() { + return new SimpleCounter(); + } + + @Override + public Counter getNumBytesInCounter() { + return new SimpleCounter(); + } + + @Override + public Counter getNumBytesOutCounter() { + return new SimpleCounter(); + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java new file mode 100644 index 000000000000..cd778309f90d --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java @@ -0,0 +1,162 @@ +/* + * 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.iceberg.flink.source.split; + +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.flink.source.SplitHelpers; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestIcebergSourceSplitSerializer { + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + private final IcebergSourceSplitSerializer serializer = new IcebergSourceSplitSerializer(true); + + @Test + public void testLatestVersion() throws Exception { + serializeAndDeserialize(1, 1); + serializeAndDeserialize(10, 2); + } + + private void serializeAndDeserialize(int splitCount, int filesPerSplit) throws Exception { + final List splits = + SplitHelpers.createSplitsFromTransientHadoopTable( + TEMPORARY_FOLDER, splitCount, filesPerSplit); + for (IcebergSourceSplit split : splits) { + byte[] result = serializer.serialize(split); + IcebergSourceSplit deserialized = serializer.deserialize(serializer.getVersion(), result); + assertSplitEquals(split, deserialized); + + byte[] cachedResult = serializer.serialize(split); + Assert.assertSame(result, cachedResult); + IcebergSourceSplit deserialized2 = + serializer.deserialize(serializer.getVersion(), cachedResult); + assertSplitEquals(split, deserialized2); + + split.updatePosition(0, 100); + byte[] resultAfterUpdatePosition = serializer.serialize(split); + // after position change, serialized bytes should have changed + Assert.assertNotSame(cachedResult, resultAfterUpdatePosition); + IcebergSourceSplit deserialized3 = + serializer.deserialize(serializer.getVersion(), resultAfterUpdatePosition); + assertSplitEquals(split, deserialized3); + } + } + + @Test + public void testV1() throws Exception { + serializeAndDeserializeV1(1, 1); + serializeAndDeserializeV1(10, 2); + } + + private void serializeAndDeserializeV1(int splitCount, int filesPerSplit) throws Exception { + final List splits = + SplitHelpers.createSplitsFromTransientHadoopTable( + TEMPORARY_FOLDER, splitCount, filesPerSplit); + for (IcebergSourceSplit split : splits) { + byte[] result = split.serializeV1(); + IcebergSourceSplit deserialized = IcebergSourceSplit.deserializeV1(result); + assertSplitEquals(split, deserialized); + } + } + + @Test + public void testV2() throws Exception { + serializeAndDeserializeV2(1, 1); + serializeAndDeserializeV2(10, 2); + } + + private void serializeAndDeserializeV2(int splitCount, int filesPerSplit) throws Exception { + final List splits = + SplitHelpers.createSplitsFromTransientHadoopTable( + TEMPORARY_FOLDER, splitCount, filesPerSplit); + for (IcebergSourceSplit split : splits) { + byte[] result = split.serializeV2(); + IcebergSourceSplit deserialized = IcebergSourceSplit.deserializeV2(result, true); + assertSplitEquals(split, deserialized); + } + } + + @Test + public void testDeserializeV1() throws Exception { + final List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + for (IcebergSourceSplit split : splits) { + byte[] result = split.serializeV1(); + IcebergSourceSplit deserialized = serializer.deserialize(1, result); + assertSplitEquals(split, deserialized); + } + } + + @Test + public void testCheckpointedPosition() throws Exception { + final AtomicInteger index = new AtomicInteger(); + final List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 10, 2).stream() + .map( + split -> { + IcebergSourceSplit result; + if (index.get() % 2 == 0) { + result = IcebergSourceSplit.fromCombinedScanTask(split.task(), 1, 1); + } else { + result = split; + } + index.incrementAndGet(); + return result; + }) + .collect(Collectors.toList()); + + for (IcebergSourceSplit split : splits) { + byte[] result = serializer.serialize(split); + IcebergSourceSplit deserialized = serializer.deserialize(serializer.getVersion(), result); + assertSplitEquals(split, deserialized); + + byte[] cachedResult = serializer.serialize(split); + Assert.assertSame(result, cachedResult); + IcebergSourceSplit deserialized2 = + serializer.deserialize(serializer.getVersion(), cachedResult); + assertSplitEquals(split, deserialized2); + } + } + + private void assertSplitEquals(IcebergSourceSplit expected, IcebergSourceSplit actual) { + List expectedTasks = Lists.newArrayList(expected.task().tasks().iterator()); + List actualTasks = Lists.newArrayList(actual.task().tasks().iterator()); + Assert.assertEquals(expectedTasks.size(), actualTasks.size()); + for (int i = 0; i < expectedTasks.size(); ++i) { + FileScanTask expectedTask = expectedTasks.get(i); + FileScanTask actualTask = actualTasks.get(i); + Assert.assertEquals(expectedTask.file().path(), actualTask.file().path()); + Assert.assertEquals(expectedTask.sizeBytes(), actualTask.sizeBytes()); + Assert.assertEquals(expectedTask.filesCount(), actualTask.filesCount()); + Assert.assertEquals(expectedTask.start(), actualTask.start()); + Assert.assertEquals(expectedTask.length(), actualTask.length()); + } + + Assert.assertEquals(expected.fileOffset(), actual.fileOffset()); + Assert.assertEquals(expected.recordOffset(), actual.recordOffset()); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java new file mode 100644 index 000000000000..cf244f410288 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java @@ -0,0 +1,31 @@ +/* + * 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.iceberg.flink.util; + +import org.junit.Assert; +import org.junit.Test; + +public class TestFlinkPackage { + + /** This unit test would need to be adjusted as new Flink version is supported. */ + @Test + public void testVersion() { + Assert.assertEquals("1.17.1", FlinkPackage.version()); + } +} diff --git a/flink/v1.17/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink/v1.17/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory new file mode 100644 index 000000000000..47a3c94aa991 --- /dev/null +++ b/flink/v1.17/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -0,0 +1,16 @@ +# 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. + +org.apache.iceberg.flink.source.BoundedTableFactory From 274390f3d65d9f48b26e3c682225d6f9edb35bce Mon Sep 17 00:00:00 2001 From: Rodrigo Meneses Date: Mon, 4 Dec 2023 09:52:05 -0800 Subject: [PATCH 20/47] Remove Flink 1.15 --- flink/v1.15/build.gradle | 263 ---- flink/v1.15/flink-runtime/LICENSE | 502 ------- flink/v1.15/flink-runtime/NOTICE | 91 -- .../flink/IcebergConnectorSmokeTest.java | 29 - .../apache/iceberg/flink/CatalogLoader.java | 215 --- .../apache/iceberg/flink/FlinkCatalog.java | 806 ------------ .../iceberg/flink/FlinkCatalogFactory.java | 213 --- .../apache/iceberg/flink/FlinkConfParser.java | 261 ---- .../iceberg/flink/FlinkConfigOptions.java | 107 -- .../flink/FlinkDynamicTableFactory.java | 209 --- .../flink/FlinkEnvironmentContext.java | 31 - .../apache/iceberg/flink/FlinkFilters.java | 266 ---- .../apache/iceberg/flink/FlinkFixupTypes.java | 50 - .../apache/iceberg/flink/FlinkReadConf.java | 193 --- .../iceberg/flink/FlinkReadOptions.java | 112 -- .../apache/iceberg/flink/FlinkSchemaUtil.java | 181 --- .../iceberg/flink/FlinkSourceFilter.java | 49 - .../apache/iceberg/flink/FlinkTypeToType.java | 199 --- .../iceberg/flink/FlinkTypeVisitor.java | 80 -- .../apache/iceberg/flink/FlinkWriteConf.java | 205 --- .../iceberg/flink/FlinkWriteOptions.java | 73 - .../iceberg/flink/IcebergTableSink.java | 121 -- .../apache/iceberg/flink/RowDataWrapper.java | 142 -- .../org/apache/iceberg/flink/TableLoader.java | 159 --- .../apache/iceberg/flink/TypeToFlinkType.java | 134 -- .../apache/iceberg/flink/actions/Actions.java | 52 - .../flink/actions/RewriteDataFilesAction.java | 72 - .../data/AvroWithFlinkSchemaVisitor.java | 75 -- .../iceberg/flink/data/FlinkAvroReader.java | 169 --- .../iceberg/flink/data/FlinkAvroWriter.java | 165 --- .../iceberg/flink/data/FlinkOrcReader.java | 131 -- .../iceberg/flink/data/FlinkOrcReaders.java | 283 ---- .../iceberg/flink/data/FlinkOrcWriter.java | 163 --- .../iceberg/flink/data/FlinkOrcWriters.java | 317 ----- .../flink/data/FlinkParquetReaders.java | 832 ------------ .../flink/data/FlinkParquetWriters.java | 504 ------- .../flink/data/FlinkSchemaVisitor.java | 161 --- .../iceberg/flink/data/FlinkValueReaders.java | 312 ----- .../iceberg/flink/data/FlinkValueWriters.java | 253 ---- .../data/ParquetWithFlinkSchemaVisitor.java | 222 ---- .../iceberg/flink/data/RowDataProjection.java | 341 ----- .../iceberg/flink/data/RowDataUtil.java | 100 -- .../iceberg/flink/data/StructRowData.java | 300 ----- .../AvroGenericRecordToRowDataMapper.java | 61 - .../flink/sink/BaseDeltaTaskWriter.java | 125 -- .../sink/BucketPartitionKeySelector.java | 70 - .../iceberg/flink/sink/BucketPartitioner.java | 103 -- .../flink/sink/BucketPartitionerUtil.java | 125 -- .../flink/sink/CachingTableSupplier.java | 91 -- .../iceberg/flink/sink/CommitSummary.java | 93 -- .../iceberg/flink/sink/DeltaManifests.java | 71 - .../flink/sink/DeltaManifestsSerializer.java | 122 -- .../flink/sink/EqualityFieldKeySelector.java | 86 -- .../flink/sink/FlinkAppenderFactory.java | 274 ---- .../flink/sink/FlinkFileWriterFactory.java | 293 ----- .../iceberg/flink/sink/FlinkManifestUtil.java | 132 -- .../apache/iceberg/flink/sink/FlinkSink.java | 654 --------- .../flink/sink/IcebergFilesCommitter.java | 516 -------- .../sink/IcebergFilesCommitterMetrics.java | 96 -- .../flink/sink/IcebergStreamWriter.java | 120 -- .../sink/IcebergStreamWriterMetrics.java | 89 -- .../flink/sink/ManifestOutputFileFactory.java | 94 -- .../flink/sink/PartitionKeySelector.java | 64 - .../flink/sink/PartitionedDeltaWriter.java | 97 -- .../flink/sink/RowDataTaskWriterFactory.java | 244 ---- .../iceberg/flink/sink/TaskWriterFactory.java | 45 - .../flink/sink/UnpartitionedDeltaWriter.java | 69 - .../sink/shuffle/AggregatedStatistics.java | 71 - .../shuffle/AggregatedStatisticsTracker.java | 133 -- .../flink/sink/shuffle/DataStatistics.java | 61 - .../shuffle/DataStatisticsCoordinator.java | 322 ----- .../DataStatisticsCoordinatorProvider.java | 51 - .../sink/shuffle/DataStatisticsEvent.java | 57 - .../sink/shuffle/DataStatisticsOperator.java | 188 --- .../sink/shuffle/DataStatisticsOrRecord.java | 113 -- .../DataStatisticsOrRecordSerializer.java | 219 --- .../sink/shuffle/DataStatisticsUtil.java | 97 -- .../flink/sink/shuffle/MapDataStatistics.java | 65 - .../shuffle/MapDataStatisticsSerializer.java | 183 --- .../AvroGenericRecordFileScanTaskReader.java | 42 - .../iceberg/flink/source/DataIterator.java | 156 --- .../iceberg/flink/source/DataTaskReader.java | 47 - .../flink/source/FileScanTaskReader.java | 35 - .../flink/source/FlinkInputFormat.java | 141 -- .../iceberg/flink/source/FlinkInputSplit.java | 48 - .../iceberg/flink/source/FlinkSource.java | 310 ----- .../flink/source/FlinkSplitPlanner.java | 189 --- .../iceberg/flink/source/IcebergSource.java | 558 -------- .../flink/source/IcebergTableSource.java | 229 ---- .../source/RowDataFileScanTaskReader.java | 243 ---- .../iceberg/flink/source/RowDataRewriter.java | 172 --- .../RowDataToAvroGenericRecordConverter.java | 70 - .../iceberg/flink/source/ScanContext.java | 561 -------- .../iceberg/flink/source/SourceUtil.java | 77 -- .../source/StreamingMonitorFunction.java | 269 ---- .../flink/source/StreamingReaderOperator.java | 246 ---- .../source/StreamingStartingStrategy.java | 54 - .../source/assigner/DefaultSplitAssigner.java | 112 -- .../flink/source/assigner/GetSplitResult.java | 77 -- .../assigner/OrderedSplitAssignerFactory.java | 46 - .../assigner/SimpleSplitAssignerFactory.java | 37 - .../flink/source/assigner/SplitAssigner.java | 118 -- .../source/assigner/SplitAssignerFactory.java | 30 - .../source/assigner/SplitAssignerType.java | 33 - .../enumerator/AbstractIcebergEnumerator.java | 173 --- .../ContinuousEnumerationResult.java | 57 - .../ContinuousIcebergEnumerator.java | 177 --- .../enumerator/ContinuousSplitPlanner.java | 30 - .../ContinuousSplitPlannerImpl.java | 236 ---- .../source/enumerator/EnumerationHistory.java | 96 -- .../enumerator/IcebergEnumeratorPosition.java | 79 -- .../IcebergEnumeratorPositionSerializer.java | 90 -- .../enumerator/IcebergEnumeratorState.java | 65 - .../IcebergEnumeratorStateSerializer.java | 196 --- .../enumerator/StaticIcebergEnumerator.java | 51 - .../source/reader/ArrayBatchRecords.java | 171 --- .../reader/ArrayPoolDataIteratorBatcher.java | 130 -- .../AvroGenericRecordReaderFunction.java | 102 -- .../reader/ColumnStatsWatermarkExtractor.java | 98 -- .../source/reader/DataIteratorBatcher.java | 36 - .../reader/DataIteratorReaderFunction.java | 43 - .../source/reader/IcebergSourceReader.java | 77 -- .../reader/IcebergSourceReaderMetrics.java | 61 - .../reader/IcebergSourceSplitReader.java | 147 --- .../flink/source/reader/ListBatchRecords.java | 85 -- .../reader/ListDataIteratorBatcher.java | 94 -- .../source/reader/MetaDataReaderFunction.java | 65 - .../flink/source/reader/ReaderFunction.java | 31 - .../source/reader/RecordAndPosition.java | 78 -- .../flink/source/reader/RecordFactory.java | 34 - .../source/reader/RowDataReaderFunction.java | 79 -- .../source/reader/RowDataRecordFactory.java | 61 - .../reader/SerializableRecordEmitter.java | 40 - .../reader/SplitWatermarkExtractor.java | 28 - .../WatermarkExtractorRecordEmitter.java | 67 - .../source/split/IcebergSourceSplit.java | 177 --- .../split/IcebergSourceSplitSerializer.java | 60 - .../source/split/IcebergSourceSplitState.java | 37 - .../split/IcebergSourceSplitStatus.java | 25 - .../source/split/SerializableComparator.java | 24 - .../flink/source/split/SplitComparators.java | 76 -- .../flink/source/split/SplitRequestEvent.java | 54 - .../flink/util/FlinkCompatibilityUtil.java | 42 - .../iceberg/flink/util/FlinkPackage.java | 33 - .../org.apache.flink.table.factories.Factory | 16 - ....apache.flink.table.factories.TableFactory | 16 - .../flink/AvroGenericRecordConverterBase.java | 90 -- .../apache/iceberg/flink/DataGenerator.java | 42 - .../apache/iceberg/flink/DataGenerators.java | 1172 ----------------- .../iceberg/flink/FlinkCatalogTestBase.java | 155 --- .../apache/iceberg/flink/FlinkTestBase.java | 129 -- .../iceberg/flink/HadoopCatalogExtension.java | 104 -- .../iceberg/flink/HadoopCatalogResource.java | 90 -- .../iceberg/flink/HadoopTableResource.java | 64 - .../iceberg/flink/MiniClusterResource.java | 53 - .../flink/MiniFlinkClusterExtension.java | 53 - .../iceberg/flink/RowDataConverter.java | 135 -- .../apache/iceberg/flink/SimpleDataUtil.java | 443 ------- .../org/apache/iceberg/flink/TestBase.java | 130 -- .../iceberg/flink/TestCatalogLoader.java | 118 -- .../iceberg/flink/TestCatalogTableLoader.java | 116 -- .../iceberg/flink/TestChangeLogTable.java | 305 ----- .../flink/TestDataFileSerialization.java | 207 --- .../apache/iceberg/flink/TestFixtures.java | 59 - .../flink/TestFlinkAnonymousTable.java | 64 - .../flink/TestFlinkCatalogDatabase.java | 293 ----- .../flink/TestFlinkCatalogFactory.java | 115 -- .../iceberg/flink/TestFlinkCatalogTable.java | 430 ------ .../TestFlinkCatalogTablePartitions.java | 120 -- .../iceberg/flink/TestFlinkConfParser.java | 61 - .../iceberg/flink/TestFlinkFilters.java | 467 ------- .../iceberg/flink/TestFlinkHiveCatalog.java | 105 -- .../iceberg/flink/TestFlinkSchemaUtil.java | 415 ------ .../iceberg/flink/TestFlinkTableSink.java | 380 ------ .../apache/iceberg/flink/TestFlinkUpsert.java | 346 ----- .../org/apache/iceberg/flink/TestHelpers.java | 611 --------- .../iceberg/flink/TestIcebergConnector.java | 350 ----- .../flink/TestManifestFileSerialization.java | 174 --- .../iceberg/flink/TestRowDataWrapper.java | 93 -- .../apache/iceberg/flink/TestTableLoader.java | 57 - .../iceberg/flink/TestTableSerialization.java | 110 -- .../actions/TestRewriteDataFilesAction.java | 499 ------- .../iceberg/flink/data/RandomRowData.java | 38 - .../flink/data/RowDataToRowMapper.java | 50 - .../flink/data/TestFlinkAvroReaderWriter.java | 184 --- .../flink/data/TestFlinkOrcReaderWriter.java | 106 -- .../flink/data/TestFlinkParquetReader.java | 138 -- .../flink/data/TestFlinkParquetWriter.java | 93 -- .../flink/data/TestRowDataProjection.java | 593 --------- .../iceberg/flink/data/TestRowProjection.java | 580 -------- .../iceberg/flink/data/TestStructRowData.java | 100 -- .../TestAvroGenericRecordToRowDataMapper.java | 37 - .../sink/TestBucketPartitionKeySelector.java | 65 - .../flink/sink/TestBucketPartitioner.java | 107 -- ...TestBucketPartitionerFlinkIcebergSink.java | 227 ---- .../flink/sink/TestBucketPartitionerUtil.java | 126 -- .../flink/sink/TestCachingTableSupplier.java | 81 -- .../flink/sink/TestCompressionSettings.java | 254 ---- .../flink/sink/TestDeltaTaskWriter.java | 440 ------- .../flink/sink/TestFlinkAppenderFactory.java | 70 - .../sink/TestFlinkFileWriterFactory.java | 71 - .../flink/sink/TestFlinkIcebergSink.java | 394 ------ .../flink/sink/TestFlinkIcebergSinkBase.java | 64 - .../sink/TestFlinkIcebergSinkBranch.java | 137 -- .../flink/sink/TestFlinkIcebergSinkV2.java | 236 ---- .../sink/TestFlinkIcebergSinkV2Base.java | 361 ----- .../sink/TestFlinkIcebergSinkV2Branch.java | 134 -- .../iceberg/flink/sink/TestFlinkManifest.java | 315 ----- .../sink/TestFlinkPartitioningWriters.java | 71 - .../sink/TestFlinkPositionDeltaWriters.java | 71 - .../sink/TestFlinkRollingFileWriters.java | 56 - .../flink/sink/TestFlinkWriterMetrics.java | 60 - .../flink/sink/TestIcebergFilesCommitter.java | 1152 ---------------- .../flink/sink/TestIcebergStreamWriter.java | 388 ------ .../flink/sink/TestRowDataPartitionKey.java | 252 ---- .../iceberg/flink/sink/TestTaskWriters.java | 243 ---- .../shuffle/TestAggregatedStatistics.java | 61 - .../TestAggregatedStatisticsTracker.java | 177 --- .../TestDataStatisticsCoordinator.java | 172 --- ...TestDataStatisticsCoordinatorProvider.java | 147 --- .../shuffle/TestDataStatisticsOperator.java | 227 ---- .../flink/source/BoundedTableFactory.java | 170 --- .../flink/source/BoundedTestSource.java | 108 -- .../flink/source/ChangeLogTableTestBase.java | 87 -- .../iceberg/flink/source/SplitHelpers.java | 132 -- .../iceberg/flink/source/SqlHelpers.java | 60 - .../flink/source/TestBoundedTableFactory.java | 87 -- .../flink/source/TestFlinkInputFormat.java | 193 --- .../TestFlinkInputFormatReaderDeletes.java | 76 -- .../flink/source/TestFlinkMergingMetrics.java | 71 - .../flink/source/TestFlinkMetaDataTable.java | 829 ------------ .../source/TestFlinkReaderDeletesBase.java | 110 -- .../iceberg/flink/source/TestFlinkScan.java | 540 -------- .../flink/source/TestFlinkScanSql.java | 74 -- .../iceberg/flink/source/TestFlinkSource.java | 94 -- .../flink/source/TestFlinkSourceConfig.java | 58 - .../flink/source/TestFlinkSourceSql.java | 85 -- .../flink/source/TestFlinkTableSource.java | 646 --------- .../source/TestIcebergSourceBounded.java | 134 -- ...TestIcebergSourceBoundedGenericRecord.java | 203 --- .../source/TestIcebergSourceBoundedSql.java | 80 -- .../source/TestIcebergSourceContinuous.java | 421 ------ .../source/TestIcebergSourceFailover.java | 297 ----- ...gSourceFailoverWithWatermarkExtractor.java | 112 -- .../TestIcebergSourceReaderDeletes.java | 114 -- .../flink/source/TestIcebergSourceSql.java | 42 - ...stIcebergSourceWithWatermarkExtractor.java | 485 ------- .../TestMetadataTableReadableMetrics.java | 301 ----- .../flink/source/TestProjectMetaColumn.java | 186 --- ...stRowDataToAvroGenericRecordConverter.java | 35 - .../iceberg/flink/source/TestSourceUtil.java | 60 - .../iceberg/flink/source/TestSqlBase.java | 158 --- .../flink/source/TestStreamScanSql.java | 316 ----- .../source/TestStreamingMonitorFunction.java | 405 ------ .../source/TestStreamingReaderOperator.java | 287 ---- .../assigner/SplitAssignerTestBase.java | 130 -- .../assigner/TestDefaultSplitAssigner.java | 43 - ...tFileSequenceNumberBasedSplitAssigner.java | 80 -- .../TestWatermarkBasedSplitAssigner.java | 146 -- .../ManualContinuousSplitPlanner.java | 97 -- .../TestContinuousIcebergEnumerator.java | 353 ----- .../TestContinuousSplitPlannerImpl.java | 688 ---------- ...ntinuousSplitPlannerImplStartStrategy.java | 205 --- .../enumerator/TestEnumerationHistory.java | 134 -- .../TestIcebergEnumeratorStateSerializer.java | 145 -- .../source/reader/ReaderFunctionTestBase.java | 219 --- .../flink/source/reader/ReaderUtil.java | 125 -- .../source/reader/TestArrayBatchRecords.java | 68 - ...stArrayPoolDataIteratorBatcherRowData.java | 358 ----- .../TestColumnStatsWatermarkExtractor.java | 178 --- .../reader/TestIcebergSourceReader.java | 184 --- .../reader/TestRowDataReaderFunction.java | 74 -- .../source/reader/TestingMetricGroup.java | 102 -- .../TestIcebergSourceSplitSerializer.java | 162 --- .../iceberg/flink/util/TestFlinkPackage.java | 31 - .../org.apache.flink.table.factories.Factory | 16 - 276 files changed, 48501 deletions(-) delete mode 100644 flink/v1.15/build.gradle delete mode 100644 flink/v1.15/flink-runtime/LICENSE delete mode 100644 flink/v1.15/flink-runtime/NOTICE delete mode 100644 flink/v1.15/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java delete mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java delete mode 100644 flink/v1.15/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory delete mode 100644 flink/v1.15/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/FlinkCatalogTestBase.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestBase.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java delete mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java delete mode 100644 flink/v1.15/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory diff --git a/flink/v1.15/build.gradle b/flink/v1.15/build.gradle deleted file mode 100644 index a77ec1b50ab3..000000000000 --- a/flink/v1.15/build.gradle +++ /dev/null @@ -1,263 +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. - */ - -String flinkMajorVersion = '1.15' -String scalaVersion = System.getProperty("scalaVersion") != null ? System.getProperty("scalaVersion") : System.getProperty("defaultScalaVersion") - -project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { - - dependencies { - implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow') - api project(':iceberg-api') - implementation project(':iceberg-common') - implementation project(':iceberg-core') - api project(':iceberg-data') - implementation project(':iceberg-orc') - implementation project(':iceberg-parquet') - implementation project(':iceberg-hive-metastore') - - compileOnly libs.flink115.avro - // for dropwizard histogram metrics implementation - compileOnly libs.flink115.metrics.dropwizard - compileOnly libs.flink115.streaming.java - compileOnly "${libs.flink115.streaming.java.get().module}:${libs.flink115.streaming.java.get().getVersion()}:tests" - compileOnly libs.flink115.table.api.java.bridge - compileOnly "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink115.get()}" - compileOnly libs.flink115.connector.base - compileOnly libs.flink115.connector.files - - compileOnly libs.hadoop2.hdfs - compileOnly libs.hadoop2.common - compileOnly(libs.hadoop2.minicluster) { - exclude group: 'org.apache.avro', module: 'avro' - } - - implementation(libs.parquet.avro) { - exclude group: 'org.apache.avro', module: 'avro' - // already shaded by Parquet - exclude group: 'it.unimi.dsi' - exclude group: 'org.codehaus.jackson' - } - - compileOnly libs.avro.avro - - implementation("${libs.orc.core.get().module}:${libs.versions.orc.get()}:nohive") { - exclude group: 'org.apache.hadoop' - exclude group: 'commons-lang' - // These artifacts are shaded and included in the orc-core fat jar - exclude group: 'com.google.protobuf', module: 'protobuf-java' - exclude group: 'org.apache.hive', module: 'hive-storage-api' - } - - testImplementation libs.flink115.connector.test.utils - testImplementation libs.flink115.core - testImplementation libs.flink115.runtime - testImplementation(libs.flink115.test.utilsjunit) { - exclude group: 'junit' - } - testImplementation(libs.flink115.test.utils) { - exclude group: "org.apache.curator", module: 'curator-test' - exclude group: 'junit' - } - - testImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') - testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') - testImplementation project(path: ':iceberg-core', configuration: 'testArtifacts') - testImplementation project(path: ':iceberg-data', configuration: 'testArtifacts') - - // By default, hive-exec is a fat/uber jar and it exports a guava library - // that's really old. We use the core classifier to be able to override our guava - // version. Luckily, hive-exec seems to work okay so far with this version of guava - // See: https://github.com/apache/hive/blob/master/ql/pom.xml#L911 for more context. - testImplementation("${libs.hive2.exec.get().module}:${libs.hive2.exec.get().getVersion()}:core") { - exclude group: 'org.apache.avro', module: 'avro' - exclude group: 'org.slf4j', module: 'slf4j-log4j12' - exclude group: 'org.pentaho' // missing dependency - exclude group: 'org.apache.hive', module: 'hive-llap-tez' - exclude group: 'org.apache.logging.log4j' - exclude group: 'com.google.protobuf', module: 'protobuf-java' - exclude group: 'org.apache.calcite' - exclude group: 'org.apache.calcite.avatica' - exclude group: 'com.google.code.findbugs', module: 'jsr305' - } - - testImplementation(libs.hive2.metastore) { - exclude group: 'org.apache.avro', module: 'avro' - exclude group: 'org.slf4j', module: 'slf4j-log4j12' - exclude group: 'org.pentaho' // missing dependency - exclude group: 'org.apache.hbase' - exclude group: 'org.apache.logging.log4j' - exclude group: 'co.cask.tephra' - exclude group: 'com.google.code.findbugs', module: 'jsr305' - exclude group: 'org.eclipse.jetty.aggregate', module: 'jetty-all' - exclude group: 'org.eclipse.jetty.orbit', module: 'javax.servlet' - exclude group: 'org.apache.parquet', module: 'parquet-hadoop-bundle' - exclude group: 'com.tdunning', module: 'json' - exclude group: 'javax.transaction', module: 'transaction-api' - exclude group: 'com.zaxxer', module: 'HikariCP' - } - - testImplementation libs.awaitility - testImplementation libs.assertj.core - } - - test { - useJUnitPlatform() - } -} - -project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") { - apply plugin: 'com.github.johnrengelman.shadow' - - tasks.jar.dependsOn tasks.shadowJar - - sourceSets { - integration { - java.srcDir "$projectDir/src/integration/java" - resources.srcDir "$projectDir/src/integration/resources" - } - } - - configurations { - implementation { - // included in Flink - exclude group: 'org.slf4j' - exclude group: 'org.apache.commons' - exclude group: 'commons-pool' - exclude group: 'commons-codec' - exclude group: 'org.xerial.snappy' - exclude group: 'javax.xml.bind' - exclude group: 'javax.annotation' - } - } - - dependencies { - implementation(project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}")) { - exclude group: 'org.apache.flink' - } - implementation project(':iceberg-aws') - implementation project(':iceberg-azure') - implementation(project(':iceberg-aliyun')) { - exclude group: 'edu.umd.cs.findbugs', module: 'findbugs' - exclude group: 'org.apache.httpcomponents', module: 'httpclient' - exclude group: 'commons-logging', module: 'commons-logging' - } - implementation project(':iceberg-gcp') - implementation(project(':iceberg-nessie')) { - exclude group: 'com.google.code.findbugs', module: 'jsr305' - } - - // for dropwizard histogram metrics implementation - implementation libs.flink115.metrics.dropwizard - - // for integration testing with the flink-runtime-jar - // all of those dependencies are required because the integration test extends FlinkTestBase - integrationCompileOnly project(':iceberg-api') - integrationImplementation libs.junit.vintage.engine - integrationImplementation libs.assertj.core - integrationImplementation project(path: ":iceberg-flink:iceberg-flink-${flinkMajorVersion}", configuration: "testArtifacts") - integrationImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') - integrationImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') - integrationImplementation(libs.flink115.test.utils) { - exclude group: "org.apache.curator", module: 'curator-test' - exclude group: 'junit' - } - - integrationImplementation libs.flink115.table.api.java.bridge - integrationImplementation "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink115.get()}" - - integrationImplementation libs.hadoop2.common - integrationImplementation libs.hadoop2.hdfs - integrationImplementation(libs.hadoop2.minicluster) { - exclude group: 'org.apache.avro', module: 'avro' - } - - integrationImplementation(libs.hive2.metastore) { - exclude group: 'org.apache.avro', module: 'avro' - exclude group: 'org.slf4j', module: 'slf4j-log4j12' - exclude group: 'org.pentaho' // missing dependency - exclude group: 'org.apache.hbase' - exclude group: 'org.apache.logging.log4j' - exclude group: 'co.cask.tephra' - exclude group: 'com.google.code.findbugs', module: 'jsr305' - exclude group: 'org.eclipse.jetty.aggregate', module: 'jetty-all' - exclude group: 'org.eclipse.jetty.orbit', module: 'javax.servlet' - exclude group: 'org.apache.parquet', module: 'parquet-hadoop-bundle' - exclude group: 'com.tdunning', module: 'json' - exclude group: 'javax.transaction', module: 'transaction-api' - exclude group: 'com.zaxxer', module: 'HikariCP' - } - - integrationImplementation("${libs.hive2.exec.get().module}:${libs.hive2.exec.get().getVersion()}:core") { - exclude group: 'org.apache.avro', module: 'avro' - exclude group: 'org.slf4j', module: 'slf4j-log4j12' - exclude group: 'org.pentaho' // missing dependency - exclude group: 'org.apache.hive', module: 'hive-llap-tez' - exclude group: 'org.apache.logging.log4j' - exclude group: 'com.google.protobuf', module: 'protobuf-java' - exclude group: 'org.apache.calcite' - exclude group: 'org.apache.calcite.avatica' - exclude group: 'com.google.code.findbugs', module: 'jsr305' - } - } - - shadowJar { - configurations = [project.configurations.runtimeClasspath] - - zip64 true - - // include the LICENSE and NOTICE files for the shaded Jar - from(projectDir) { - include 'LICENSE' - include 'NOTICE' - } - - // Relocate dependencies to avoid conflicts - relocate 'org.apache.avro', 'org.apache.iceberg.shaded.org.apache.avro' - relocate 'org.apache.parquet', 'org.apache.iceberg.shaded.org.apache.parquet' - relocate 'com.google.errorprone', 'org.apache.iceberg.shaded.com.google.errorprone' - relocate 'com.google.flatbuffers', 'org.apache.iceberg.shaded.com.google.flatbuffers' - relocate 'com.fasterxml', 'org.apache.iceberg.shaded.com.fasterxml' - relocate 'com.github.benmanes', 'org.apache.iceberg.shaded.com.github.benmanes' - relocate 'org.checkerframework', 'org.apache.iceberg.shaded.org.checkerframework' - relocate 'shaded.parquet', 'org.apache.iceberg.shaded.org.apache.parquet.shaded' - relocate 'org.apache.orc', 'org.apache.iceberg.shaded.org.apache.orc' - relocate 'io.airlift', 'org.apache.iceberg.shaded.io.airlift' - relocate 'org.threeten.extra', 'org.apache.iceberg.shaded.org.threeten.extra' - relocate 'org.apache.hc.client5', 'org.apache.iceberg.shaded.org.apache.hc.client5' - relocate 'org.apache.hc.core5', 'org.apache.iceberg.shaded.org.apache.hc.core5' - - archiveClassifier.set(null) - } - - task integrationTest(type: Test) { - description = "Test Flink Runtime Jar against Flink ${flinkMajorVersion}" - group = "verification" - jvmArgs += project.property('extraJvmArgs') - testClassesDirs = sourceSets.integration.output.classesDirs - classpath = sourceSets.integration.runtimeClasspath + files(shadowJar.archiveFile.get().asFile.path) - inputs.file(shadowJar.archiveFile.get().asFile.path) - } - integrationTest.dependsOn shadowJar - check.dependsOn integrationTest - - jar { - enabled = false - } -} diff --git a/flink/v1.15/flink-runtime/LICENSE b/flink/v1.15/flink-runtime/LICENSE deleted file mode 100644 index 8ab53469eb87..000000000000 --- a/flink/v1.15/flink-runtime/LICENSE +++ /dev/null @@ -1,502 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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 binary artifact contains Apache Avro. - -Copyright: 2014-2020 The Apache Software Foundation. -Home page: https://parquet.apache.org/ -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains the Jackson JSON processor. - -Copyright: 2007-2020 Tatu Saloranta and other contributors -Home page: http://jackson.codehaus.org/ -License: http://www.apache.org/licenses/LICENSE-2.0.txt - --------------------------------------------------------------------------------- - -This binary artifact contains Apache Parquet. - -Copyright: 2014-2020 The Apache Software Foundation. -Home page: https://parquet.apache.org/ -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains Apache Thrift. - -Copyright: 2006-2010 The Apache Software Foundation. -Home page: https://thrift.apache.org/ -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains fastutil. - -Copyright: 2002-2014 Sebastiano Vigna -Home page: http://fastutil.di.unimi.it/ -License: http://www.apache.org/licenses/LICENSE-2.0.html - --------------------------------------------------------------------------------- - -This binary artifact contains Apache ORC. - -Copyright: 2013-2020 The Apache Software Foundation. -Home page: https://orc.apache.org/ -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains Apache Hive's storage API via ORC. - -Copyright: 2013-2020 The Apache Software Foundation. -Home page: https://hive.apache.org/ -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains Airlift Aircompressor. - -Copyright: 2011-2020 Aircompressor authors. -Home page: https://github.com/airlift/aircompressor -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains Airlift Slice. - -Copyright: 2013-2020 Slice authors. -Home page: https://github.com/airlift/slice -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains JetBrains annotations. - -Copyright: 2000-2020 JetBrains s.r.o. -Home page: https://github.com/JetBrains/java-annotations -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains Google Guava. - -Copyright: 2006-2020 The Guava Authors -Home page: https://github.com/google/guava -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains Google Error Prone Annotations. - -Copyright: Copyright 2011-2019 The Error Prone Authors -Home page: https://github.com/google/error-prone -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains checkerframework checker-qual Annotations. - -Copyright: 2004-2020 the Checker Framework developers -Home page: https://github.com/typetools/checker-framework -License: https://github.com/typetools/checker-framework/blob/master/LICENSE.txt (MIT license) - -License text: -| The annotations are licensed under the MIT License. (The text of this -| license appears below.) More specifically, all the parts of the Checker -| Framework that you might want to include with your own program use the -| MIT License. This is the checker-qual.jar file and all the files that -| appear in it: every file in a qual/ directory, plus utility files such -| as NullnessUtil.java, RegexUtil.java, SignednessUtil.java, etc. -| In addition, the cleanroom implementations of third-party annotations, -| which the Checker Framework recognizes as aliases for its own -| annotations, are licensed under the MIT License. -| -| Permission is hereby granted, free of charge, to any person obtaining a copy -| of this software and associated documentation files (the "Software"), to deal -| in the Software without restriction, including without limitation the rights -| to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -| copies of the Software, and to permit persons to whom the Software is -| furnished to do so, subject to the following conditions: -| -| The above copyright notice and this permission notice shall be included in -| all copies or substantial portions of the Software. -| -| THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -| IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -| FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -| AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -| LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -| OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -| THE SOFTWARE. - --------------------------------------------------------------------------------- - -This binary artifact contains Animal Sniffer Annotations. - -Copyright: 2009-2018 codehaus.org -Home page: https://www.mojohaus.org/animal-sniffer/animal-sniffer-annotations/ -License: https://www.mojohaus.org/animal-sniffer/animal-sniffer-annotations/license.html (MIT license) - -License text: -| The MIT License -| -| Copyright (c) 2009 codehaus.org. -| -| Permission is hereby granted, free of charge, to any person obtaining a copy -| of this software and associated documentation files (the "Software"), to deal -| in the Software without restriction, including without limitation the rights -| to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -| copies of the Software, and to permit persons to whom the Software is -| furnished to do so, subject to the following conditions: -| -| The above copyright notice and this permission notice shall be included in -| all copies or substantial portions of the Software. -| -| THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -| IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -| FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -| AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -| LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -| OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -| THE SOFTWARE. - --------------------------------------------------------------------------------- - -This binary artifact contains Caffeine by Ben Manes. - -Copyright: 2014-2020 Ben Manes and contributors -Home page: https://github.com/ben-manes/caffeine -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains Apache Yetus audience annotations. - -Copyright: 2008-2020 The Apache Software Foundation. -Home page: https://yetus.apache.org/ -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains Google protobuf. - -Copyright: 2008 Google Inc. -Home page: https://developers.google.com/protocol-buffers -License: https://github.com/protocolbuffers/protobuf/blob/master/LICENSE (BSD) - -License text: - -| Copyright 2008 Google Inc. All rights reserved. -| -| Redistribution and use in source and binary forms, with or without -| modification, are permitted provided that the following conditions are -| met: -| -| * Redistributions of source code must retain the above copyright -| notice, this list of conditions and the following disclaimer. -| * Redistributions in binary form must reproduce the above -| copyright notice, this list of conditions and the following disclaimer -| in the documentation and/or other materials provided with the -| distribution. -| * Neither the name of Google Inc. nor the names of its -| contributors may be used to endorse or promote products derived from -| this software without specific prior written permission. -| -| THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -| "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -| LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -| A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -| OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -| SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -| LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -| DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -| THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -| (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -| OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. -| -| Code generated by the Protocol Buffer compiler is owned by the owner -| of the input file used when generating it. This code is not -| standalone and requires a support library to be linked with it. This -| support library is itself covered by the above license. - --------------------------------------------------------------------------------- - -This binary artifact contains ThreeTen. - -Copyright: 2007-present, Stephen Colebourne & Michael Nascimento Santos. -Home page: https://www.threeten.org/threeten-extra/ -License: https://github.com/ThreeTen/threeten-extra/blob/master/LICENSE.txt (BSD 3-clause) - -License text: - -| All rights reserved. -| -| * Redistribution and use in source and binary forms, with or without -| modification, are permitted provided that the following conditions are met: -| -| * Redistributions of source code must retain the above copyright notice, -| this list of conditions and the following disclaimer. -| -| * Redistributions in binary form must reproduce the above copyright notice, -| this list of conditions and the following disclaimer in the documentation -| and/or other materials provided with the distribution. -| -| * Neither the name of JSR-310 nor the names of its contributors -| may be used to endorse or promote products derived from this software -| without specific prior written permission. -| -| THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -| "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -| LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -| A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR -| CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, -| EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, -| PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR -| PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF -| LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING -| NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS -| SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - --------------------------------------------------------------------------------- - -This binary artifact includes Project Nessie with the following in its NOTICE -file: - -| Dremio -| Copyright 2015-2017 Dremio Corporation -| -| This product includes software developed at -| The Apache Software Foundation (http://www.apache.org/). - --------------------------------------------------------------------------------- - -This binary includes code from Apache Commons. - -* Core ArrayUtil. - -Copyright: 2020 The Apache Software Foundation -Home page: https://commons.apache.org/ -License: https://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains Apache HttpComponents Client. - -Copyright: 1999-2022 The Apache Software Foundation. -Home page: https://hc.apache.org/ -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This product includes code from Apache HttpComponents Client. - -* retry and error handling logic in ExponentialHttpRequestRetryStrategy.java - -Copyright: 1999-2022 The Apache Software Foundation. -Home page: https://hc.apache.org/ -License: https://www.apache.org/licenses/LICENSE-2.0 diff --git a/flink/v1.15/flink-runtime/NOTICE b/flink/v1.15/flink-runtime/NOTICE deleted file mode 100644 index 81aa1660456a..000000000000 --- a/flink/v1.15/flink-runtime/NOTICE +++ /dev/null @@ -1,91 +0,0 @@ - -Apache Iceberg -Copyright 2017-2022 The Apache Software Foundation - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). - --------------------------------------------------------------------------------- - -This binary artifact includes Apache ORC with the following in its NOTICE file: - -| Apache ORC -| Copyright 2013-2019 The Apache Software Foundation -| -| This product includes software developed by The Apache Software -| Foundation (http://www.apache.org/). -| -| This product includes software developed by Hewlett-Packard: -| (c) Copyright [2014-2015] Hewlett-Packard Development Company, L.P - --------------------------------------------------------------------------------- - -This binary artifact includes Airlift Aircompressor with the following in its -NOTICE file: - -| Snappy Copyright Notices -| ========================= -| -| * Copyright 2011 Dain Sundstrom -| * Copyright 2011, Google Inc. -| -| -| Snappy License -| =============== -| Copyright 2011, Google Inc. -| All rights reserved. -| -| Redistribution and use in source and binary forms, with or without -| modification, are permitted provided that the following conditions are -| met: -| -| * Redistributions of source code must retain the above copyright -| notice, this list of conditions and the following disclaimer. -| * Redistributions in binary form must reproduce the above -| copyright notice, this list of conditions and the following disclaimer -| in the documentation and/or other materials provided with the -| distribution. -| * Neither the name of Google Inc. nor the names of its -| contributors may be used to endorse or promote products derived from -| this software without specific prior written permission. -| -| THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -| "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -| LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -| A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -| OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -| SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -| LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -| DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -| THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -| (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -| OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - --------------------------------------------------------------------------------- - -This binary artifact includes Apache Yetus with the following in its NOTICE -file: - -| Apache Yetus -| Copyright 2008-2020 The Apache Software Foundation -| -| This product includes software developed at -| The Apache Software Foundation (https://www.apache.org/). -| -| --- -| Additional licenses for the Apache Yetus Source/Website: -| --- -| -| -| See LICENSE for terms. - --------------------------------------------------------------------------------- - -This binary artifact includes Project Nessie with the following in its NOTICE -file: - -| Dremio -| Copyright 2015-2017 Dremio Corporation -| -| This product includes software developed at -| The Apache Software Foundation (http://www.apache.org/). diff --git a/flink/v1.15/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java b/flink/v1.15/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java deleted file mode 100644 index 65cfba1ec876..000000000000 --- a/flink/v1.15/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java +++ /dev/null @@ -1,29 +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.iceberg.flink; - -import java.util.Map; - -public class IcebergConnectorSmokeTest extends TestIcebergConnector { - - public IcebergConnectorSmokeTest( - String catalogName, Map properties, boolean isStreaming) { - super(catalogName, properties, isStreaming); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java deleted file mode 100644 index 18473bf4f190..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java +++ /dev/null @@ -1,215 +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.iceberg.flink; - -import java.io.Serializable; -import java.util.Map; -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 org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.rest.RESTCatalog; - -/** Serializable loader to load an Iceberg {@link Catalog}. */ -public interface CatalogLoader extends Serializable, Cloneable { - - /** - * Create a new catalog with the provided properties. NOTICE: for flink, we may initialize the - * {@link CatalogLoader} at flink sql client side or job manager side, and then serialize this - * catalog loader to task manager, finally deserialize it and create a new catalog at task manager - * side. - * - * @return a newly created {@link Catalog} - */ - Catalog loadCatalog(); - - /** Clone a CatalogLoader. */ - @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) - CatalogLoader clone(); - - static CatalogLoader hadoop( - String name, Configuration hadoopConf, Map properties) { - return new HadoopCatalogLoader(name, hadoopConf, properties); - } - - static CatalogLoader hive(String name, Configuration hadoopConf, Map properties) { - return new HiveCatalogLoader(name, hadoopConf, properties); - } - - static CatalogLoader rest(String name, Configuration hadoopConf, Map properties) { - return new RESTCatalogLoader(name, hadoopConf, properties); - } - - static CatalogLoader custom( - String name, Map properties, Configuration hadoopConf, String impl) { - return new CustomCatalogLoader(name, properties, hadoopConf, impl); - } - - class HadoopCatalogLoader implements CatalogLoader { - private final String catalogName; - private final SerializableConfiguration hadoopConf; - private final String warehouseLocation; - private final Map properties; - - private HadoopCatalogLoader( - String catalogName, Configuration conf, Map properties) { - this.catalogName = catalogName; - this.hadoopConf = new SerializableConfiguration(conf); - this.warehouseLocation = properties.get(CatalogProperties.WAREHOUSE_LOCATION); - this.properties = Maps.newHashMap(properties); - } - - @Override - public Catalog loadCatalog() { - return CatalogUtil.loadCatalog( - HadoopCatalog.class.getName(), catalogName, properties, hadoopConf.get()); - } - - @Override - @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) - public CatalogLoader clone() { - return new HadoopCatalogLoader(catalogName, new Configuration(hadoopConf.get()), properties); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("catalogName", catalogName) - .add("warehouseLocation", warehouseLocation) - .toString(); - } - } - - class HiveCatalogLoader implements CatalogLoader { - private final String catalogName; - private final SerializableConfiguration hadoopConf; - private final String uri; - private final String warehouse; - private final int clientPoolSize; - private final Map properties; - - private HiveCatalogLoader( - String catalogName, Configuration conf, Map properties) { - this.catalogName = catalogName; - this.hadoopConf = new SerializableConfiguration(conf); - this.uri = properties.get(CatalogProperties.URI); - this.warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION); - this.clientPoolSize = - properties.containsKey(CatalogProperties.CLIENT_POOL_SIZE) - ? Integer.parseInt(properties.get(CatalogProperties.CLIENT_POOL_SIZE)) - : CatalogProperties.CLIENT_POOL_SIZE_DEFAULT; - this.properties = Maps.newHashMap(properties); - } - - @Override - public Catalog loadCatalog() { - return CatalogUtil.loadCatalog( - HiveCatalog.class.getName(), catalogName, properties, hadoopConf.get()); - } - - @Override - @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) - public CatalogLoader clone() { - return new HiveCatalogLoader(catalogName, new Configuration(hadoopConf.get()), properties); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("catalogName", catalogName) - .add("uri", uri) - .add("warehouse", warehouse) - .add("clientPoolSize", clientPoolSize) - .toString(); - } - } - - class RESTCatalogLoader implements CatalogLoader { - private final String catalogName; - private final SerializableConfiguration hadoopConf; - private final Map properties; - - private RESTCatalogLoader( - String catalogName, Configuration conf, Map properties) { - this.catalogName = catalogName; - this.hadoopConf = new SerializableConfiguration(conf); - this.properties = Maps.newHashMap(properties); - } - - @Override - public Catalog loadCatalog() { - return CatalogUtil.loadCatalog( - RESTCatalog.class.getName(), catalogName, properties, hadoopConf.get()); - } - - @Override - @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) - public CatalogLoader clone() { - return new RESTCatalogLoader(catalogName, new Configuration(hadoopConf.get()), properties); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("catalogName", catalogName) - .add("properties", properties) - .toString(); - } - } - - class CustomCatalogLoader implements CatalogLoader { - - private final SerializableConfiguration hadoopConf; - private final Map properties; - private final String name; - private final String impl; - - private CustomCatalogLoader( - String name, Map properties, Configuration conf, String impl) { - this.hadoopConf = new SerializableConfiguration(conf); - this.properties = Maps.newHashMap(properties); // wrap into a hashmap for serialization - this.name = name; - this.impl = - Preconditions.checkNotNull( - impl, "Cannot initialize custom Catalog, impl class name is null"); - } - - @Override - public Catalog loadCatalog() { - return CatalogUtil.loadCatalog(impl, name, properties, hadoopConf.get()); - } - - @Override - @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) - public CatalogLoader clone() { - return new CustomCatalogLoader(name, properties, new Configuration(hadoopConf.get()), impl); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this).add("name", name).add("impl", impl).toString(); - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java deleted file mode 100644 index 825816fdf416..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java +++ /dev/null @@ -1,806 +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.iceberg.flink; - -import java.io.Closeable; -import java.io.IOException; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Optional; -import java.util.Set; -import java.util.stream.Collectors; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.catalog.AbstractCatalog; -import org.apache.flink.table.catalog.CatalogBaseTable; -import org.apache.flink.table.catalog.CatalogDatabase; -import org.apache.flink.table.catalog.CatalogDatabaseImpl; -import org.apache.flink.table.catalog.CatalogFunction; -import org.apache.flink.table.catalog.CatalogPartition; -import org.apache.flink.table.catalog.CatalogPartitionSpec; -import org.apache.flink.table.catalog.CatalogTable; -import org.apache.flink.table.catalog.CatalogTableImpl; -import org.apache.flink.table.catalog.ObjectPath; -import org.apache.flink.table.catalog.exceptions.CatalogException; -import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; -import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException; -import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; -import org.apache.flink.table.catalog.exceptions.FunctionNotExistException; -import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; -import org.apache.flink.table.catalog.exceptions.TableNotExistException; -import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException; -import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; -import org.apache.flink.table.catalog.stats.CatalogTableStatistics; -import org.apache.flink.table.expressions.Expression; -import org.apache.flink.table.factories.Factory; -import org.apache.flink.util.StringUtils; -import org.apache.iceberg.CachingCatalog; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.MetadataTableType; -import org.apache.iceberg.PartitionField; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.Table; -import org.apache.iceberg.Transaction; -import org.apache.iceberg.UpdateProperties; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.catalog.SupportsNamespaces; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.exceptions.AlreadyExistsException; -import org.apache.iceberg.exceptions.NamespaceNotEmptyException; -import org.apache.iceberg.exceptions.NoSuchNamespaceException; -import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.base.Splitter; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; - -/** - * A Flink Catalog implementation that wraps an Iceberg {@link Catalog}. - * - *

    The mapping between Flink database and Iceberg namespace: Supplying a base namespace for a - * given catalog, so if you have a catalog that supports a 2-level namespace, you would supply the - * first level in the catalog configuration and the second level would be exposed as Flink - * databases. - * - *

    The Iceberg table manages its partitions by itself. The partition of the Iceberg table is - * independent of the partition of Flink. - */ -public class FlinkCatalog extends AbstractCatalog { - - private final CatalogLoader catalogLoader; - private final Catalog icebergCatalog; - private final Namespace baseNamespace; - private final SupportsNamespaces asNamespaceCatalog; - private final Closeable closeable; - private final boolean cacheEnabled; - - public FlinkCatalog( - String catalogName, - String defaultDatabase, - Namespace baseNamespace, - CatalogLoader catalogLoader, - boolean cacheEnabled, - long cacheExpirationIntervalMs) { - super(catalogName, defaultDatabase); - this.catalogLoader = catalogLoader; - this.baseNamespace = baseNamespace; - this.cacheEnabled = cacheEnabled; - - Catalog originalCatalog = catalogLoader.loadCatalog(); - icebergCatalog = - cacheEnabled - ? CachingCatalog.wrap(originalCatalog, cacheExpirationIntervalMs) - : originalCatalog; - asNamespaceCatalog = - originalCatalog instanceof SupportsNamespaces ? (SupportsNamespaces) originalCatalog : null; - closeable = originalCatalog instanceof Closeable ? (Closeable) originalCatalog : null; - - FlinkEnvironmentContext.init(); - } - - @Override - public void open() throws CatalogException {} - - @Override - public void close() throws CatalogException { - if (closeable != null) { - try { - closeable.close(); - } catch (IOException e) { - throw new CatalogException(e); - } - } - } - - public Catalog catalog() { - return icebergCatalog; - } - - /** Append a new level to the base namespace */ - private static Namespace appendLevel(Namespace baseNamespace, String newLevel) { - String[] namespace = new String[baseNamespace.levels().length + 1]; - System.arraycopy(baseNamespace.levels(), 0, namespace, 0, baseNamespace.levels().length); - namespace[baseNamespace.levels().length] = newLevel; - return Namespace.of(namespace); - } - - TableIdentifier toIdentifier(ObjectPath path) { - String objectName = path.getObjectName(); - List tableName = Splitter.on('$').splitToList(objectName); - - if (tableName.size() == 1) { - return TableIdentifier.of( - appendLevel(baseNamespace, path.getDatabaseName()), path.getObjectName()); - } else if (tableName.size() == 2 && MetadataTableType.from(tableName.get(1)) != null) { - return TableIdentifier.of( - appendLevel(appendLevel(baseNamespace, path.getDatabaseName()), tableName.get(0)), - tableName.get(1)); - } else { - throw new IllegalArgumentException("Illegal table name:" + objectName); - } - } - - @Override - public List listDatabases() throws CatalogException { - if (asNamespaceCatalog == null) { - return Collections.singletonList(getDefaultDatabase()); - } - - return asNamespaceCatalog.listNamespaces(baseNamespace).stream() - .map(n -> n.level(n.levels().length - 1)) - .collect(Collectors.toList()); - } - - @Override - public CatalogDatabase getDatabase(String databaseName) - throws DatabaseNotExistException, CatalogException { - if (asNamespaceCatalog == null) { - if (!getDefaultDatabase().equals(databaseName)) { - throw new DatabaseNotExistException(getName(), databaseName); - } else { - return new CatalogDatabaseImpl(Maps.newHashMap(), ""); - } - } else { - try { - Map metadata = - Maps.newHashMap( - asNamespaceCatalog.loadNamespaceMetadata(appendLevel(baseNamespace, databaseName))); - String comment = metadata.remove("comment"); - return new CatalogDatabaseImpl(metadata, comment); - } catch (NoSuchNamespaceException e) { - throw new DatabaseNotExistException(getName(), databaseName, e); - } - } - } - - @Override - public boolean databaseExists(String databaseName) throws CatalogException { - try { - getDatabase(databaseName); - return true; - } catch (DatabaseNotExistException ignore) { - return false; - } - } - - @Override - public void createDatabase(String name, CatalogDatabase database, boolean ignoreIfExists) - throws DatabaseAlreadyExistException, CatalogException { - createDatabase( - name, mergeComment(database.getProperties(), database.getComment()), ignoreIfExists); - } - - private void createDatabase( - String databaseName, Map metadata, boolean ignoreIfExists) - throws DatabaseAlreadyExistException, CatalogException { - if (asNamespaceCatalog != null) { - try { - asNamespaceCatalog.createNamespace(appendLevel(baseNamespace, databaseName), metadata); - } catch (AlreadyExistsException e) { - if (!ignoreIfExists) { - throw new DatabaseAlreadyExistException(getName(), databaseName, e); - } - } - } else { - throw new UnsupportedOperationException( - "Namespaces are not supported by catalog: " + getName()); - } - } - - private Map mergeComment(Map metadata, String comment) { - Map ret = Maps.newHashMap(metadata); - if (metadata.containsKey("comment")) { - throw new CatalogException("Database properties should not contain key: 'comment'."); - } - - if (!StringUtils.isNullOrWhitespaceOnly(comment)) { - ret.put("comment", comment); - } - return ret; - } - - @Override - public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade) - throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException { - if (asNamespaceCatalog != null) { - try { - boolean success = asNamespaceCatalog.dropNamespace(appendLevel(baseNamespace, name)); - if (!success && !ignoreIfNotExists) { - throw new DatabaseNotExistException(getName(), name); - } - } catch (NoSuchNamespaceException e) { - if (!ignoreIfNotExists) { - throw new DatabaseNotExistException(getName(), name, e); - } - } catch (NamespaceNotEmptyException e) { - throw new DatabaseNotEmptyException(getName(), name, e); - } - } else { - if (!ignoreIfNotExists) { - throw new DatabaseNotExistException(getName(), name); - } - } - } - - @Override - public void alterDatabase(String name, CatalogDatabase newDatabase, boolean ignoreIfNotExists) - throws DatabaseNotExistException, CatalogException { - if (asNamespaceCatalog != null) { - Namespace namespace = appendLevel(baseNamespace, name); - Map updates = Maps.newHashMap(); - Set removals = Sets.newHashSet(); - - try { - Map oldProperties = asNamespaceCatalog.loadNamespaceMetadata(namespace); - Map newProperties = - mergeComment(newDatabase.getProperties(), newDatabase.getComment()); - - for (String key : oldProperties.keySet()) { - if (!newProperties.containsKey(key)) { - removals.add(key); - } - } - - for (Map.Entry entry : newProperties.entrySet()) { - if (!entry.getValue().equals(oldProperties.get(entry.getKey()))) { - updates.put(entry.getKey(), entry.getValue()); - } - } - - if (!updates.isEmpty()) { - asNamespaceCatalog.setProperties(namespace, updates); - } - - if (!removals.isEmpty()) { - asNamespaceCatalog.removeProperties(namespace, removals); - } - - } catch (NoSuchNamespaceException e) { - if (!ignoreIfNotExists) { - throw new DatabaseNotExistException(getName(), name, e); - } - } - } else { - if (getDefaultDatabase().equals(name)) { - throw new CatalogException( - "Can not alter the default database when the iceberg catalog doesn't support namespaces."); - } - if (!ignoreIfNotExists) { - throw new DatabaseNotExistException(getName(), name); - } - } - } - - @Override - public List listTables(String databaseName) - throws DatabaseNotExistException, CatalogException { - try { - return icebergCatalog.listTables(appendLevel(baseNamespace, databaseName)).stream() - .map(TableIdentifier::name) - .collect(Collectors.toList()); - } catch (NoSuchNamespaceException e) { - throw new DatabaseNotExistException(getName(), databaseName, e); - } - } - - @Override - public CatalogTable getTable(ObjectPath tablePath) - throws TableNotExistException, CatalogException { - Table table = loadIcebergTable(tablePath); - return toCatalogTable(table); - } - - private Table loadIcebergTable(ObjectPath tablePath) throws TableNotExistException { - try { - Table table = icebergCatalog.loadTable(toIdentifier(tablePath)); - if (cacheEnabled) { - table.refresh(); - } - - return table; - } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { - throw new TableNotExistException(getName(), tablePath, e); - } - } - - @Override - public boolean tableExists(ObjectPath tablePath) throws CatalogException { - return icebergCatalog.tableExists(toIdentifier(tablePath)); - } - - @Override - public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists) - throws TableNotExistException, CatalogException { - try { - icebergCatalog.dropTable(toIdentifier(tablePath)); - } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { - if (!ignoreIfNotExists) { - throw new TableNotExistException(getName(), tablePath, e); - } - } - } - - @Override - public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists) - throws TableNotExistException, TableAlreadyExistException, CatalogException { - try { - icebergCatalog.renameTable( - toIdentifier(tablePath), - toIdentifier(new ObjectPath(tablePath.getDatabaseName(), newTableName))); - } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { - if (!ignoreIfNotExists) { - throw new TableNotExistException(getName(), tablePath, e); - } - } catch (AlreadyExistsException e) { - throw new TableAlreadyExistException(getName(), tablePath, e); - } - } - - @Override - public void createTable(ObjectPath tablePath, CatalogBaseTable table, boolean ignoreIfExists) - throws CatalogException, TableAlreadyExistException { - if (Objects.equals( - table.getOptions().get("connector"), FlinkDynamicTableFactory.FACTORY_IDENTIFIER)) { - throw new IllegalArgumentException( - "Cannot create the table with 'connector'='iceberg' table property in " - + "an iceberg catalog, Please create table with 'connector'='iceberg' property in a non-iceberg catalog or " - + "create table without 'connector'='iceberg' related properties in an iceberg table."); - } - - createIcebergTable(tablePath, table, ignoreIfExists); - } - - void createIcebergTable(ObjectPath tablePath, CatalogBaseTable table, boolean ignoreIfExists) - throws CatalogException, TableAlreadyExistException { - validateFlinkTable(table); - - Schema icebergSchema = FlinkSchemaUtil.convert(table.getSchema()); - PartitionSpec spec = toPartitionSpec(((CatalogTable) table).getPartitionKeys(), icebergSchema); - - ImmutableMap.Builder properties = ImmutableMap.builder(); - String location = null; - for (Map.Entry entry : table.getOptions().entrySet()) { - if ("location".equalsIgnoreCase(entry.getKey())) { - location = entry.getValue(); - } else { - properties.put(entry.getKey(), entry.getValue()); - } - } - - try { - icebergCatalog.createTable( - toIdentifier(tablePath), icebergSchema, spec, location, properties.build()); - } catch (AlreadyExistsException e) { - if (!ignoreIfExists) { - throw new TableAlreadyExistException(getName(), tablePath, e); - } - } - } - - private static void validateTableSchemaAndPartition(CatalogTable ct1, CatalogTable ct2) { - TableSchema ts1 = ct1.getSchema(); - TableSchema ts2 = ct2.getSchema(); - boolean equalsPrimary = false; - - if (ts1.getPrimaryKey().isPresent() && ts2.getPrimaryKey().isPresent()) { - equalsPrimary = - Objects.equals(ts1.getPrimaryKey().get().getType(), ts2.getPrimaryKey().get().getType()) - && Objects.equals( - ts1.getPrimaryKey().get().getColumns(), ts2.getPrimaryKey().get().getColumns()); - } else if (!ts1.getPrimaryKey().isPresent() && !ts2.getPrimaryKey().isPresent()) { - equalsPrimary = true; - } - - if (!(Objects.equals(ts1.getTableColumns(), ts2.getTableColumns()) - && Objects.equals(ts1.getWatermarkSpecs(), ts2.getWatermarkSpecs()) - && equalsPrimary)) { - throw new UnsupportedOperationException("Altering schema is not supported yet."); - } - - if (!ct1.getPartitionKeys().equals(ct2.getPartitionKeys())) { - throw new UnsupportedOperationException("Altering partition keys is not supported yet."); - } - } - - @Override - public void alterTable(ObjectPath tablePath, CatalogBaseTable newTable, boolean ignoreIfNotExists) - throws CatalogException, TableNotExistException { - validateFlinkTable(newTable); - - Table icebergTable; - try { - icebergTable = loadIcebergTable(tablePath); - } catch (TableNotExistException e) { - if (!ignoreIfNotExists) { - throw e; - } else { - return; - } - } - - CatalogTable table = toCatalogTable(icebergTable); - - // Currently, Flink SQL only support altering table properties. - - // For current Flink Catalog API, support for adding/removing/renaming columns cannot be done by - // comparing - // CatalogTable instances, unless the Flink schema contains Iceberg column IDs. - validateTableSchemaAndPartition(table, (CatalogTable) newTable); - - Map oldProperties = table.getOptions(); - Map setProperties = Maps.newHashMap(); - - String setLocation = null; - String setSnapshotId = null; - String pickSnapshotId = null; - - for (Map.Entry entry : newTable.getOptions().entrySet()) { - String key = entry.getKey(); - String value = entry.getValue(); - - if (Objects.equals(value, oldProperties.get(key))) { - continue; - } - - if ("location".equalsIgnoreCase(key)) { - setLocation = value; - } else if ("current-snapshot-id".equalsIgnoreCase(key)) { - setSnapshotId = value; - } else if ("cherry-pick-snapshot-id".equalsIgnoreCase(key)) { - pickSnapshotId = value; - } else { - setProperties.put(key, value); - } - } - - oldProperties - .keySet() - .forEach( - k -> { - if (!newTable.getOptions().containsKey(k)) { - setProperties.put(k, null); - } - }); - - commitChanges(icebergTable, setLocation, setSnapshotId, pickSnapshotId, setProperties); - } - - private static void validateFlinkTable(CatalogBaseTable table) { - Preconditions.checkArgument( - table instanceof CatalogTable, "The Table should be a CatalogTable."); - - TableSchema schema = table.getSchema(); - schema - .getTableColumns() - .forEach( - column -> { - if (!FlinkCompatibilityUtil.isPhysicalColumn(column)) { - throw new UnsupportedOperationException( - "Creating table with computed columns is not supported yet."); - } - }); - - if (!schema.getWatermarkSpecs().isEmpty()) { - throw new UnsupportedOperationException( - "Creating table with watermark specs is not supported yet."); - } - } - - private static PartitionSpec toPartitionSpec(List partitionKeys, Schema icebergSchema) { - PartitionSpec.Builder builder = PartitionSpec.builderFor(icebergSchema); - partitionKeys.forEach(builder::identity); - return builder.build(); - } - - private static List toPartitionKeys(PartitionSpec spec, Schema icebergSchema) { - ImmutableList.Builder partitionKeysBuilder = ImmutableList.builder(); - for (PartitionField field : spec.fields()) { - if (field.transform().isIdentity()) { - partitionKeysBuilder.add(icebergSchema.findColumnName(field.sourceId())); - } else { - // Not created by Flink SQL. - // For compatibility with iceberg tables, return empty. - // TODO modify this after Flink support partition transform. - return Collections.emptyList(); - } - } - return partitionKeysBuilder.build(); - } - - private static void commitChanges( - Table table, - String setLocation, - String setSnapshotId, - String pickSnapshotId, - Map setProperties) { - // don't allow setting the snapshot and picking a commit at the same time because order is - // ambiguous and choosing - // one order leads to different results - Preconditions.checkArgument( - setSnapshotId == null || pickSnapshotId == null, - "Cannot set the current snapshot ID and cherry-pick snapshot changes"); - - if (setSnapshotId != null) { - long newSnapshotId = Long.parseLong(setSnapshotId); - table.manageSnapshots().setCurrentSnapshot(newSnapshotId).commit(); - } - - // if updating the table snapshot, perform that update first in case it fails - if (pickSnapshotId != null) { - long newSnapshotId = Long.parseLong(pickSnapshotId); - table.manageSnapshots().cherrypick(newSnapshotId).commit(); - } - - Transaction transaction = table.newTransaction(); - - if (setLocation != null) { - transaction.updateLocation().setLocation(setLocation).commit(); - } - - if (!setProperties.isEmpty()) { - UpdateProperties updateProperties = transaction.updateProperties(); - setProperties.forEach( - (k, v) -> { - if (v == null) { - updateProperties.remove(k); - } else { - updateProperties.set(k, v); - } - }); - updateProperties.commit(); - } - - transaction.commitTransaction(); - } - - static CatalogTable toCatalogTable(Table table) { - TableSchema schema = FlinkSchemaUtil.toSchema(table.schema()); - List partitionKeys = toPartitionKeys(table.spec(), table.schema()); - - // NOTE: We can not create a IcebergCatalogTable extends CatalogTable, because Flink optimizer - // may use - // CatalogTableImpl to copy a new catalog table. - // Let's re-loading table from Iceberg catalog when creating source/sink operators. - // Iceberg does not have Table comment, so pass a null (Default comment value in Flink). - return new CatalogTableImpl(schema, partitionKeys, table.properties(), null); - } - - @Override - public Optional getFactory() { - return Optional.of(new FlinkDynamicTableFactory(this)); - } - - CatalogLoader getCatalogLoader() { - return catalogLoader; - } - - // ------------------------------ Unsupported methods - // --------------------------------------------- - - @Override - public List listViews(String databaseName) throws CatalogException { - return Collections.emptyList(); - } - - @Override - public CatalogPartition getPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public boolean partitionExists(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public void createPartition( - ObjectPath tablePath, - CatalogPartitionSpec partitionSpec, - CatalogPartition partition, - boolean ignoreIfExists) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public void dropPartition( - ObjectPath tablePath, CatalogPartitionSpec partitionSpec, boolean ignoreIfNotExists) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public void alterPartition( - ObjectPath tablePath, - CatalogPartitionSpec partitionSpec, - CatalogPartition newPartition, - boolean ignoreIfNotExists) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public List listFunctions(String dbName) throws CatalogException { - return Collections.emptyList(); - } - - @Override - public CatalogFunction getFunction(ObjectPath functionPath) - throws FunctionNotExistException, CatalogException { - throw new FunctionNotExistException(getName(), functionPath); - } - - @Override - public boolean functionExists(ObjectPath functionPath) throws CatalogException { - return false; - } - - @Override - public void createFunction( - ObjectPath functionPath, CatalogFunction function, boolean ignoreIfExists) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public void alterFunction( - ObjectPath functionPath, CatalogFunction newFunction, boolean ignoreIfNotExists) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public void dropFunction(ObjectPath functionPath, boolean ignoreIfNotExists) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public void alterTableStatistics( - ObjectPath tablePath, CatalogTableStatistics tableStatistics, boolean ignoreIfNotExists) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public void alterTableColumnStatistics( - ObjectPath tablePath, CatalogColumnStatistics columnStatistics, boolean ignoreIfNotExists) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public void alterPartitionStatistics( - ObjectPath tablePath, - CatalogPartitionSpec partitionSpec, - CatalogTableStatistics partitionStatistics, - boolean ignoreIfNotExists) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public void alterPartitionColumnStatistics( - ObjectPath tablePath, - CatalogPartitionSpec partitionSpec, - CatalogColumnStatistics columnStatistics, - boolean ignoreIfNotExists) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public List listPartitions(ObjectPath tablePath) - throws TableNotExistException, TableNotPartitionedException, CatalogException { - Table table = loadIcebergTable(tablePath); - - if (table.spec().isUnpartitioned()) { - throw new TableNotPartitionedException(icebergCatalog.name(), tablePath); - } - - Set set = Sets.newHashSet(); - try (CloseableIterable tasks = table.newScan().planFiles()) { - for (DataFile dataFile : CloseableIterable.transform(tasks, FileScanTask::file)) { - Map map = Maps.newHashMap(); - StructLike structLike = dataFile.partition(); - PartitionSpec spec = table.specs().get(dataFile.specId()); - for (int i = 0; i < structLike.size(); i++) { - map.put(spec.fields().get(i).name(), String.valueOf(structLike.get(i, Object.class))); - } - set.add(new CatalogPartitionSpec(map)); - } - } catch (IOException e) { - throw new CatalogException( - String.format("Failed to list partitions of table %s", tablePath), e); - } - - return Lists.newArrayList(set); - } - - @Override - public List listPartitions( - ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public List listPartitionsByFilter( - ObjectPath tablePath, List filters) throws CatalogException { - throw new UnsupportedOperationException(); - } - - // After partition pruning and filter push down, the statistics have become very inaccurate, so - // the statistics from - // here are of little significance. - // Flink will support something like SupportsReportStatistics in future. - - @Override - public CatalogTableStatistics getTableStatistics(ObjectPath tablePath) throws CatalogException { - return CatalogTableStatistics.UNKNOWN; - } - - @Override - public CatalogColumnStatistics getTableColumnStatistics(ObjectPath tablePath) - throws CatalogException { - return CatalogColumnStatistics.UNKNOWN; - } - - @Override - public CatalogTableStatistics getPartitionStatistics( - ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws CatalogException { - return CatalogTableStatistics.UNKNOWN; - } - - @Override - public CatalogColumnStatistics getPartitionColumnStatistics( - ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws CatalogException { - return CatalogColumnStatistics.UNKNOWN; - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java deleted file mode 100644 index 1453753849ec..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java +++ /dev/null @@ -1,213 +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.iceberg.flink; - -import java.net.URL; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import org.apache.flink.configuration.GlobalConfiguration; -import org.apache.flink.runtime.util.HadoopUtils; -import org.apache.flink.table.catalog.Catalog; -import org.apache.flink.table.factories.CatalogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.base.Strings; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.util.PropertyUtil; - -/** - * A Flink Catalog factory implementation that creates {@link FlinkCatalog}. - * - *

    This supports the following catalog configuration options: - * - *

      - *
    • type - Flink catalog factory key, should be "iceberg" - *
    • catalog-type - iceberg catalog type, "hive", "hadoop" or "rest" - *
    • uri - the Hive Metastore URI (Hive catalog only) - *
    • clients - the Hive Client Pool Size (Hive catalog only) - *
    • warehouse - the warehouse path (Hadoop catalog only) - *
    • default-database - a database name to use as the default - *
    • base-namespace - a base namespace as the prefix for all databases (Hadoop - * catalog only) - *
    • cache-enabled - whether to enable catalog cache - *
    - * - *

    To use a custom catalog that is not a Hive or Hadoop catalog, extend this class and override - * {@link #createCatalogLoader(String, Map, Configuration)}. - */ -public class FlinkCatalogFactory implements CatalogFactory { - - // Can not just use "type", it conflicts with CATALOG_TYPE. - public static final String ICEBERG_CATALOG_TYPE = "catalog-type"; - public static final String ICEBERG_CATALOG_TYPE_HADOOP = "hadoop"; - public static final String ICEBERG_CATALOG_TYPE_HIVE = "hive"; - public static final String ICEBERG_CATALOG_TYPE_REST = "rest"; - - public static final String HIVE_CONF_DIR = "hive-conf-dir"; - public static final String HADOOP_CONF_DIR = "hadoop-conf-dir"; - public static final String DEFAULT_DATABASE = "default-database"; - public static final String DEFAULT_DATABASE_NAME = "default"; - public static final String BASE_NAMESPACE = "base-namespace"; - - public static final String TYPE = "type"; - public static final String PROPERTY_VERSION = "property-version"; - - /** - * Create an Iceberg {@link org.apache.iceberg.catalog.Catalog} loader to be used by this Flink - * catalog adapter. - * - * @param name Flink's catalog name - * @param properties Flink's catalog properties - * @param hadoopConf Hadoop configuration for catalog - * @return an Iceberg catalog loader - */ - static CatalogLoader createCatalogLoader( - String name, Map properties, Configuration hadoopConf) { - String catalogImpl = properties.get(CatalogProperties.CATALOG_IMPL); - if (catalogImpl != null) { - String catalogType = properties.get(ICEBERG_CATALOG_TYPE); - Preconditions.checkArgument( - catalogType == null, - "Cannot create catalog %s, both catalog-type and catalog-impl are set: catalog-type=%s, catalog-impl=%s", - name, - catalogType, - catalogImpl); - return CatalogLoader.custom(name, properties, hadoopConf, catalogImpl); - } - - String catalogType = properties.getOrDefault(ICEBERG_CATALOG_TYPE, ICEBERG_CATALOG_TYPE_HIVE); - switch (catalogType.toLowerCase(Locale.ENGLISH)) { - case ICEBERG_CATALOG_TYPE_HIVE: - // The values of properties 'uri', 'warehouse', 'hive-conf-dir' are allowed to be null, in - // that case it will - // fallback to parse those values from hadoop configuration which is loaded from classpath. - String hiveConfDir = properties.get(HIVE_CONF_DIR); - String hadoopConfDir = properties.get(HADOOP_CONF_DIR); - Configuration newHadoopConf = mergeHiveConf(hadoopConf, hiveConfDir, hadoopConfDir); - return CatalogLoader.hive(name, newHadoopConf, properties); - - case ICEBERG_CATALOG_TYPE_HADOOP: - return CatalogLoader.hadoop(name, hadoopConf, properties); - - case ICEBERG_CATALOG_TYPE_REST: - return CatalogLoader.rest(name, hadoopConf, properties); - - default: - throw new UnsupportedOperationException( - "Unknown catalog-type: " + catalogType + " (Must be 'hive', 'hadoop' or 'rest')"); - } - } - - @Override - public Map requiredContext() { - Map context = Maps.newHashMap(); - context.put(TYPE, "iceberg"); - context.put(PROPERTY_VERSION, "1"); - return context; - } - - @Override - public List supportedProperties() { - return ImmutableList.of("*"); - } - - @Override - public Catalog createCatalog(String name, Map properties) { - return createCatalog(name, properties, clusterHadoopConf()); - } - - protected Catalog createCatalog( - String name, Map properties, Configuration hadoopConf) { - CatalogLoader catalogLoader = createCatalogLoader(name, properties, hadoopConf); - String defaultDatabase = properties.getOrDefault(DEFAULT_DATABASE, DEFAULT_DATABASE_NAME); - - Namespace baseNamespace = Namespace.empty(); - if (properties.containsKey(BASE_NAMESPACE)) { - baseNamespace = Namespace.of(properties.get(BASE_NAMESPACE).split("\\.")); - } - - boolean cacheEnabled = - PropertyUtil.propertyAsBoolean( - properties, CatalogProperties.CACHE_ENABLED, CatalogProperties.CACHE_ENABLED_DEFAULT); - - long cacheExpirationIntervalMs = - PropertyUtil.propertyAsLong( - properties, - CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS, - CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS_OFF); - Preconditions.checkArgument( - cacheExpirationIntervalMs != 0, - "%s is not allowed to be 0.", - CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS); - - return new FlinkCatalog( - name, - defaultDatabase, - baseNamespace, - catalogLoader, - cacheEnabled, - cacheExpirationIntervalMs); - } - - private static Configuration mergeHiveConf( - Configuration hadoopConf, String hiveConfDir, String hadoopConfDir) { - Configuration newConf = new Configuration(hadoopConf); - if (!Strings.isNullOrEmpty(hiveConfDir)) { - Preconditions.checkState( - Files.exists(Paths.get(hiveConfDir, "hive-site.xml")), - "There should be a hive-site.xml file under the directory %s", - hiveConfDir); - newConf.addResource(new Path(hiveConfDir, "hive-site.xml")); - } else { - // If don't provide the hive-site.xml path explicitly, it will try to load resource from - // classpath. If still - // couldn't load the configuration file, then it will throw exception in HiveCatalog. - URL configFile = CatalogLoader.class.getClassLoader().getResource("hive-site.xml"); - if (configFile != null) { - newConf.addResource(configFile); - } - } - - if (!Strings.isNullOrEmpty(hadoopConfDir)) { - Preconditions.checkState( - Files.exists(Paths.get(hadoopConfDir, "hdfs-site.xml")), - "Failed to load Hadoop configuration: missing %s", - Paths.get(hadoopConfDir, "hdfs-site.xml")); - newConf.addResource(new Path(hadoopConfDir, "hdfs-site.xml")); - Preconditions.checkState( - Files.exists(Paths.get(hadoopConfDir, "core-site.xml")), - "Failed to load Hadoop configuration: missing %s", - Paths.get(hadoopConfDir, "core-site.xml")); - newConf.addResource(new Path(hadoopConfDir, "core-site.xml")); - } - - return newConf; - } - - public static Configuration clusterHadoopConf() { - return HadoopUtils.getHadoopConfiguration(GlobalConfiguration.loadConfiguration()); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java deleted file mode 100644 index 7167859e600c..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java +++ /dev/null @@ -1,261 +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.iceberg.flink; - -import java.time.Duration; -import java.util.List; -import java.util.Map; -import java.util.function.Function; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.util.TimeUtils; -import org.apache.iceberg.Table; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -class FlinkConfParser { - - private final Map tableProperties; - private final Map options; - private final ReadableConfig readableConfig; - - FlinkConfParser(Table table, Map options, ReadableConfig readableConfig) { - this.tableProperties = table.properties(); - this.options = options; - this.readableConfig = readableConfig; - } - - public BooleanConfParser booleanConf() { - return new BooleanConfParser(); - } - - public IntConfParser intConf() { - return new IntConfParser(); - } - - public LongConfParser longConf() { - return new LongConfParser(); - } - - public > EnumConfParser enumConfParser(Class enumClass) { - return new EnumConfParser<>(enumClass); - } - - public StringConfParser stringConf() { - return new StringConfParser(); - } - - public DurationConfParser durationConf() { - return new DurationConfParser(); - } - - class BooleanConfParser extends ConfParser { - private Boolean defaultValue; - - @Override - protected BooleanConfParser self() { - return this; - } - - public BooleanConfParser defaultValue(boolean value) { - this.defaultValue = value; - return self(); - } - - public BooleanConfParser defaultValue(String value) { - this.defaultValue = Boolean.parseBoolean(value); - return self(); - } - - public boolean parse() { - Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); - return parse(Boolean::parseBoolean, defaultValue); - } - } - - class IntConfParser extends ConfParser { - private Integer defaultValue; - - @Override - protected IntConfParser self() { - return this; - } - - public IntConfParser defaultValue(int value) { - this.defaultValue = value; - return self(); - } - - public int parse() { - Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); - return parse(Integer::parseInt, defaultValue); - } - - public Integer parseOptional() { - return parse(Integer::parseInt, null); - } - } - - class LongConfParser extends ConfParser { - private Long defaultValue; - - @Override - protected LongConfParser self() { - return this; - } - - public LongConfParser defaultValue(long value) { - this.defaultValue = value; - return self(); - } - - public long parse() { - Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); - return parse(Long::parseLong, defaultValue); - } - - public Long parseOptional() { - return parse(Long::parseLong, null); - } - } - - class StringConfParser extends ConfParser { - private String defaultValue; - - @Override - protected StringConfParser self() { - return this; - } - - public StringConfParser defaultValue(String value) { - this.defaultValue = value; - return self(); - } - - public String parse() { - Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); - return parse(Function.identity(), defaultValue); - } - - public String parseOptional() { - return parse(Function.identity(), null); - } - } - - class EnumConfParser> extends ConfParser, E> { - private E defaultValue; - private final Class enumClass; - - EnumConfParser(Class enumClass) { - this.enumClass = enumClass; - } - - @Override - protected EnumConfParser self() { - return this; - } - - public EnumConfParser defaultValue(E value) { - this.defaultValue = value; - return self(); - } - - public E parse() { - Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); - return parse(s -> Enum.valueOf(enumClass, s), defaultValue); - } - - public E parseOptional() { - return parse(s -> Enum.valueOf(enumClass, s), null); - } - } - - class DurationConfParser extends ConfParser { - private Duration defaultValue; - - @Override - protected DurationConfParser self() { - return this; - } - - public DurationConfParser defaultValue(Duration value) { - this.defaultValue = value; - return self(); - } - - public Duration parse() { - Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); - return parse(TimeUtils::parseDuration, defaultValue); - } - - public Duration parseOptional() { - return parse(TimeUtils::parseDuration, null); - } - } - - abstract class ConfParser { - private final List optionNames = Lists.newArrayList(); - private String tablePropertyName; - private ConfigOption configOption; - - protected abstract ThisT self(); - - public ThisT option(String name) { - this.optionNames.add(name); - return self(); - } - - public ThisT flinkConfig(ConfigOption newConfigOption) { - this.configOption = newConfigOption; - return self(); - } - - public ThisT tableProperty(String name) { - this.tablePropertyName = name; - return self(); - } - - protected T parse(Function conversion, T defaultValue) { - if (!optionNames.isEmpty()) { - for (String optionName : optionNames) { - String optionValue = options.get(optionName); - if (optionValue != null) { - return conversion.apply(optionValue); - } - } - } - - if (configOption != null) { - T propertyValue = readableConfig.get(configOption); - if (propertyValue != null) { - return propertyValue; - } - } - - if (tablePropertyName != null) { - String propertyValue = tableProperties.get(tablePropertyName); - if (propertyValue != null) { - return conversion.apply(propertyValue); - } - } - - return defaultValue; - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java deleted file mode 100644 index 7c7afd24ed8e..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java +++ /dev/null @@ -1,107 +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.iceberg.flink; - -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.description.Description; -import org.apache.flink.configuration.description.TextElement; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.iceberg.flink.source.assigner.SplitAssignerType; -import org.apache.iceberg.util.ThreadPools; - -/** - * When constructing Flink Iceberg source via Java API, configs can be set in {@link Configuration} - * passed to source builder. E.g. - * - *

    - *   configuration.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, true);
    - *   FlinkSource.forRowData()
    - *       .flinkConf(configuration)
    - *       ...
    - * 
    - * - *

    When using Flink SQL/table API, connector options can be set in Flink's {@link - * TableEnvironment}. - * - *

    - *   TableEnvironment tEnv = createTableEnv();
    - *   tEnv.getConfig()
    - *        .getConfiguration()
    - *        .setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, true);
    - * 
    - */ -public class FlinkConfigOptions { - - private FlinkConfigOptions() {} - - public static final ConfigOption TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM = - ConfigOptions.key("table.exec.iceberg.infer-source-parallelism") - .booleanType() - .defaultValue(true) - .withDescription( - "If is false, parallelism of source are set by config.\n" - + "If is true, source parallelism is inferred according to splits number.\n"); - - public static final ConfigOption TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX = - ConfigOptions.key("table.exec.iceberg.infer-source-parallelism.max") - .intType() - .defaultValue(100) - .withDescription("Sets max infer parallelism for source operator."); - - public static final ConfigOption TABLE_EXEC_ICEBERG_EXPOSE_SPLIT_LOCALITY_INFO = - ConfigOptions.key("table.exec.iceberg.expose-split-locality-info") - .booleanType() - .noDefaultValue() - .withDescription( - "Expose split host information to use Flink's locality aware split assigner."); - - public static final ConfigOption SOURCE_READER_FETCH_BATCH_RECORD_COUNT = - ConfigOptions.key("table.exec.iceberg.fetch-batch-record-count") - .intType() - .defaultValue(2048) - .withDescription("The target number of records for Iceberg reader fetch batch."); - - public static final ConfigOption TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE = - ConfigOptions.key("table.exec.iceberg.worker-pool-size") - .intType() - .defaultValue(ThreadPools.WORKER_THREAD_POOL_SIZE) - .withDescription("The size of workers pool used to plan or scan manifests."); - - public static final ConfigOption TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE = - ConfigOptions.key("table.exec.iceberg.use-flip27-source") - .booleanType() - .defaultValue(false) - .withDescription("Use the FLIP-27 based Iceberg source implementation."); - - public static final ConfigOption TABLE_EXEC_SPLIT_ASSIGNER_TYPE = - ConfigOptions.key("table.exec.iceberg.split-assigner-type") - .enumType(SplitAssignerType.class) - .defaultValue(SplitAssignerType.SIMPLE) - .withDescription( - Description.builder() - .text("Split assigner type that determine how splits are assigned to readers.") - .linebreak() - .list( - TextElement.text( - SplitAssignerType.SIMPLE - + ": simple assigner that doesn't provide any guarantee on order or locality.")) - .build()); -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java deleted file mode 100644 index 8e1f420b722d..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java +++ /dev/null @@ -1,209 +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.iceberg.flink; - -import java.util.Map; -import java.util.Set; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.catalog.CatalogBaseTable; -import org.apache.flink.table.catalog.CatalogDatabaseImpl; -import org.apache.flink.table.catalog.CatalogTable; -import org.apache.flink.table.catalog.ObjectIdentifier; -import org.apache.flink.table.catalog.ObjectPath; -import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; -import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; -import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.factories.DynamicTableSinkFactory; -import org.apache.flink.table.factories.DynamicTableSourceFactory; -import org.apache.flink.table.utils.TableSchemaUtils; -import org.apache.flink.util.Preconditions; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.exceptions.AlreadyExistsException; -import org.apache.iceberg.flink.source.IcebergTableSource; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; - -public class FlinkDynamicTableFactory - implements DynamicTableSinkFactory, DynamicTableSourceFactory { - static final String FACTORY_IDENTIFIER = "iceberg"; - - private static final ConfigOption CATALOG_NAME = - ConfigOptions.key("catalog-name") - .stringType() - .noDefaultValue() - .withDescription("Catalog name"); - - private static final ConfigOption CATALOG_TYPE = - ConfigOptions.key(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE) - .stringType() - .noDefaultValue() - .withDescription("Catalog type, the optional types are: custom, hadoop, hive."); - - private static final ConfigOption CATALOG_DATABASE = - ConfigOptions.key("catalog-database") - .stringType() - .defaultValue(FlinkCatalogFactory.DEFAULT_DATABASE_NAME) - .withDescription("Database name managed in the iceberg catalog."); - - private static final ConfigOption CATALOG_TABLE = - ConfigOptions.key("catalog-table") - .stringType() - .noDefaultValue() - .withDescription("Table name managed in the underlying iceberg catalog and database."); - - private final FlinkCatalog catalog; - - public FlinkDynamicTableFactory() { - this.catalog = null; - } - - public FlinkDynamicTableFactory(FlinkCatalog catalog) { - this.catalog = catalog; - } - - @Override - public DynamicTableSource createDynamicTableSource(Context context) { - ObjectIdentifier objectIdentifier = context.getObjectIdentifier(); - CatalogTable catalogTable = context.getCatalogTable(); - Map tableProps = catalogTable.getOptions(); - TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(catalogTable.getSchema()); - - TableLoader tableLoader; - if (catalog != null) { - tableLoader = createTableLoader(catalog, objectIdentifier.toObjectPath()); - } else { - tableLoader = - createTableLoader( - catalogTable, - tableProps, - objectIdentifier.getDatabaseName(), - objectIdentifier.getObjectName()); - } - - return new IcebergTableSource(tableLoader, tableSchema, tableProps, context.getConfiguration()); - } - - @Override - public DynamicTableSink createDynamicTableSink(Context context) { - ObjectIdentifier objectIdentifier = context.getObjectIdentifier(); - CatalogTable catalogTable = context.getCatalogTable(); - Map writeProps = catalogTable.getOptions(); - TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(catalogTable.getSchema()); - - TableLoader tableLoader; - if (catalog != null) { - tableLoader = createTableLoader(catalog, objectIdentifier.toObjectPath()); - } else { - tableLoader = - createTableLoader( - catalogTable, - writeProps, - objectIdentifier.getDatabaseName(), - objectIdentifier.getObjectName()); - } - - return new IcebergTableSink(tableLoader, tableSchema, context.getConfiguration(), writeProps); - } - - @Override - public Set> requiredOptions() { - Set> options = Sets.newHashSet(); - options.add(CATALOG_TYPE); - options.add(CATALOG_NAME); - return options; - } - - @Override - public Set> optionalOptions() { - Set> options = Sets.newHashSet(); - options.add(CATALOG_DATABASE); - options.add(CATALOG_TABLE); - return options; - } - - @Override - public String factoryIdentifier() { - return FACTORY_IDENTIFIER; - } - - private static TableLoader createTableLoader( - CatalogBaseTable catalogBaseTable, - Map tableProps, - String databaseName, - String tableName) { - Configuration flinkConf = new Configuration(); - tableProps.forEach(flinkConf::setString); - - String catalogName = flinkConf.getString(CATALOG_NAME); - Preconditions.checkNotNull( - catalogName, "Table property '%s' cannot be null", CATALOG_NAME.key()); - - String catalogDatabase = flinkConf.getString(CATALOG_DATABASE, databaseName); - Preconditions.checkNotNull(catalogDatabase, "The iceberg database name cannot be null"); - - String catalogTable = flinkConf.getString(CATALOG_TABLE, tableName); - Preconditions.checkNotNull(catalogTable, "The iceberg table name cannot be null"); - - org.apache.hadoop.conf.Configuration hadoopConf = FlinkCatalogFactory.clusterHadoopConf(); - FlinkCatalogFactory factory = new FlinkCatalogFactory(); - FlinkCatalog flinkCatalog = - (FlinkCatalog) factory.createCatalog(catalogName, tableProps, hadoopConf); - ObjectPath objectPath = new ObjectPath(catalogDatabase, catalogTable); - - // Create database if not exists in the external catalog. - if (!flinkCatalog.databaseExists(catalogDatabase)) { - try { - flinkCatalog.createDatabase( - catalogDatabase, new CatalogDatabaseImpl(Maps.newHashMap(), null), true); - } catch (DatabaseAlreadyExistException e) { - throw new AlreadyExistsException( - e, - "Database %s already exists in the iceberg catalog %s.", - catalogName, - catalogDatabase); - } - } - - // Create table if not exists in the external catalog. - if (!flinkCatalog.tableExists(objectPath)) { - try { - flinkCatalog.createIcebergTable(objectPath, catalogBaseTable, true); - } catch (TableAlreadyExistException e) { - throw new AlreadyExistsException( - e, - "Table %s already exists in the database %s and catalog %s", - catalogTable, - catalogDatabase, - catalogName); - } - } - - return TableLoader.fromCatalog( - flinkCatalog.getCatalogLoader(), TableIdentifier.of(catalogDatabase, catalogTable)); - } - - private static TableLoader createTableLoader(FlinkCatalog catalog, ObjectPath objectPath) { - Preconditions.checkNotNull(catalog, "Flink catalog cannot be null"); - return TableLoader.fromCatalog(catalog.getCatalogLoader(), catalog.toIdentifier(objectPath)); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java deleted file mode 100644 index f35bb577fbba..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java +++ /dev/null @@ -1,31 +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.iceberg.flink; - -import org.apache.iceberg.EnvironmentContext; -import org.apache.iceberg.flink.util.FlinkPackage; - -class FlinkEnvironmentContext { - private FlinkEnvironmentContext() {} - - public static void init() { - EnvironmentContext.put(EnvironmentContext.ENGINE_NAME, "flink"); - EnvironmentContext.put(EnvironmentContext.ENGINE_VERSION, FlinkPackage.version()); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java deleted file mode 100644 index f2244d5137a1..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java +++ /dev/null @@ -1,266 +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.iceberg.flink; - -import java.time.Instant; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.function.BiFunction; -import java.util.regex.Matcher; -import java.util.regex.Pattern; -import org.apache.flink.table.expressions.CallExpression; -import org.apache.flink.table.expressions.FieldReferenceExpression; -import org.apache.flink.table.expressions.ResolvedExpression; -import org.apache.flink.table.expressions.ValueLiteralExpression; -import org.apache.flink.table.functions.BuiltInFunctionDefinitions; -import org.apache.flink.table.functions.FunctionDefinition; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.expressions.Expression.Operation; -import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.util.DateTimeUtil; -import org.apache.iceberg.util.NaNUtil; - -public class FlinkFilters { - private FlinkFilters() {} - - private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%"); - - private static final Map FILTERS = - ImmutableMap.builder() - .put(BuiltInFunctionDefinitions.EQUALS, Operation.EQ) - .put(BuiltInFunctionDefinitions.NOT_EQUALS, Operation.NOT_EQ) - .put(BuiltInFunctionDefinitions.GREATER_THAN, Operation.GT) - .put(BuiltInFunctionDefinitions.GREATER_THAN_OR_EQUAL, Operation.GT_EQ) - .put(BuiltInFunctionDefinitions.LESS_THAN, Operation.LT) - .put(BuiltInFunctionDefinitions.LESS_THAN_OR_EQUAL, Operation.LT_EQ) - .put(BuiltInFunctionDefinitions.IS_NULL, Operation.IS_NULL) - .put(BuiltInFunctionDefinitions.IS_NOT_NULL, Operation.NOT_NULL) - .put(BuiltInFunctionDefinitions.AND, Operation.AND) - .put(BuiltInFunctionDefinitions.OR, Operation.OR) - .put(BuiltInFunctionDefinitions.NOT, Operation.NOT) - .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH) - .buildOrThrow(); - - /** - * Convert flink expression to iceberg expression. - * - *

    the BETWEEN, NOT_BETWEEN, IN expression will be converted by flink automatically. the - * BETWEEN will be converted to (GT_EQ AND LT_EQ), the NOT_BETWEEN will be converted to (LT_EQ OR - * GT_EQ), the IN will be converted to OR, so we do not add the conversion here - * - * @param flinkExpression the flink expression - * @return the iceberg expression - */ - public static Optional convert( - org.apache.flink.table.expressions.Expression flinkExpression) { - if (!(flinkExpression instanceof CallExpression)) { - return Optional.empty(); - } - - CallExpression call = (CallExpression) flinkExpression; - Operation op = FILTERS.get(call.getFunctionDefinition()); - if (op != null) { - switch (op) { - case IS_NULL: - return onlyChildAs(call, FieldReferenceExpression.class) - .map(FieldReferenceExpression::getName) - .map(Expressions::isNull); - - case NOT_NULL: - return onlyChildAs(call, FieldReferenceExpression.class) - .map(FieldReferenceExpression::getName) - .map(Expressions::notNull); - - case LT: - return convertFieldAndLiteral(Expressions::lessThan, Expressions::greaterThan, call); - - case LT_EQ: - return convertFieldAndLiteral( - Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call); - - case GT: - return convertFieldAndLiteral(Expressions::greaterThan, Expressions::lessThan, call); - - case GT_EQ: - return convertFieldAndLiteral( - Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call); - - case EQ: - return convertFieldAndLiteral( - (ref, lit) -> { - if (NaNUtil.isNaN(lit)) { - return Expressions.isNaN(ref); - } else { - return Expressions.equal(ref, lit); - } - }, - call); - - case NOT_EQ: - return convertFieldAndLiteral( - (ref, lit) -> { - if (NaNUtil.isNaN(lit)) { - return Expressions.notNaN(ref); - } else { - return Expressions.notEqual(ref, lit); - } - }, - call); - - case NOT: - return onlyChildAs(call, CallExpression.class) - .flatMap(FlinkFilters::convert) - .map(Expressions::not); - - case AND: - return convertLogicExpression(Expressions::and, call); - - case OR: - return convertLogicExpression(Expressions::or, call); - - case STARTS_WITH: - return convertLike(call); - } - } - - return Optional.empty(); - } - - private static Optional onlyChildAs( - CallExpression call, Class expectedChildClass) { - List children = call.getResolvedChildren(); - if (children.size() != 1) { - return Optional.empty(); - } - - ResolvedExpression child = children.get(0); - if (!expectedChildClass.isInstance(child)) { - return Optional.empty(); - } - - return Optional.of(expectedChildClass.cast(child)); - } - - private static Optional convertLike(CallExpression call) { - List args = call.getResolvedChildren(); - if (args.size() != 2) { - return Optional.empty(); - } - - org.apache.flink.table.expressions.Expression left = args.get(0); - org.apache.flink.table.expressions.Expression right = args.get(1); - - if (left instanceof FieldReferenceExpression && right instanceof ValueLiteralExpression) { - String name = ((FieldReferenceExpression) left).getName(); - return convertLiteral((ValueLiteralExpression) right) - .flatMap( - lit -> { - if (lit instanceof String) { - String pattern = (String) lit; - Matcher matcher = STARTS_WITH_PATTERN.matcher(pattern); - // exclude special char of LIKE - // '_' is the wildcard of the SQL LIKE - if (!pattern.contains("_") && matcher.matches()) { - return Optional.of(Expressions.startsWith(name, matcher.group(1))); - } - } - - return Optional.empty(); - }); - } - - return Optional.empty(); - } - - private static Optional convertLogicExpression( - BiFunction function, CallExpression call) { - List args = call.getResolvedChildren(); - if (args == null || args.size() != 2) { - return Optional.empty(); - } - - Optional left = convert(args.get(0)); - Optional right = convert(args.get(1)); - if (left.isPresent() && right.isPresent()) { - return Optional.of(function.apply(left.get(), right.get())); - } - - return Optional.empty(); - } - - private static Optional convertLiteral(ValueLiteralExpression expression) { - Optional value = - expression.getValueAs( - expression.getOutputDataType().getLogicalType().getDefaultConversion()); - return value.map( - o -> { - if (o instanceof LocalDateTime) { - return DateTimeUtil.microsFromTimestamp((LocalDateTime) o); - } else if (o instanceof Instant) { - return DateTimeUtil.microsFromInstant((Instant) o); - } else if (o instanceof LocalTime) { - return DateTimeUtil.microsFromTime((LocalTime) o); - } else if (o instanceof LocalDate) { - return DateTimeUtil.daysFromDate((LocalDate) o); - } - - return o; - }); - } - - private static Optional convertFieldAndLiteral( - BiFunction expr, CallExpression call) { - return convertFieldAndLiteral(expr, expr, call); - } - - private static Optional convertFieldAndLiteral( - BiFunction convertLR, - BiFunction convertRL, - CallExpression call) { - List args = call.getResolvedChildren(); - if (args.size() != 2) { - return Optional.empty(); - } - - org.apache.flink.table.expressions.Expression left = args.get(0); - org.apache.flink.table.expressions.Expression right = args.get(1); - - if (left instanceof FieldReferenceExpression && right instanceof ValueLiteralExpression) { - String name = ((FieldReferenceExpression) left).getName(); - Optional lit = convertLiteral((ValueLiteralExpression) right); - if (lit.isPresent()) { - return Optional.of(convertLR.apply(name, lit.get())); - } - } else if (left instanceof ValueLiteralExpression - && right instanceof FieldReferenceExpression) { - Optional lit = convertLiteral((ValueLiteralExpression) left); - String name = ((FieldReferenceExpression) right).getName(); - if (lit.isPresent()) { - return Optional.of(convertRL.apply(name, lit.get())); - } - } - - return Optional.empty(); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java deleted file mode 100644 index 767d4497ac91..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java +++ /dev/null @@ -1,50 +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.iceberg.flink; - -import org.apache.iceberg.Schema; -import org.apache.iceberg.types.FixupTypes; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.types.Types; - -/** - * The uuid and fixed are converted to the same Flink type. Conversion back can produce only one, - * which may not be correct. - */ -class FlinkFixupTypes extends FixupTypes { - - private FlinkFixupTypes(Schema referenceSchema) { - super(referenceSchema); - } - - static Schema fixup(Schema schema, Schema referenceSchema) { - return new Schema( - TypeUtil.visit(schema, new FlinkFixupTypes(referenceSchema)).asStructType().fields()); - } - - @Override - protected boolean fixupPrimitive(Type.PrimitiveType type, Type source) { - if (type instanceof Types.FixedType) { - int length = ((Types.FixedType) type).length(); - return source.typeId() == Type.TypeID.UUID && length == 16; - } - return false; - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java deleted file mode 100644 index 0e04c9affb19..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java +++ /dev/null @@ -1,193 +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.iceberg.flink; - -import java.time.Duration; -import java.util.Map; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.util.TimeUtils; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.flink.source.StreamingStartingStrategy; - -public class FlinkReadConf { - - private final FlinkConfParser confParser; - - public FlinkReadConf( - Table table, Map readOptions, ReadableConfig readableConfig) { - this.confParser = new FlinkConfParser(table, readOptions, readableConfig); - } - - public Long snapshotId() { - return confParser.longConf().option(FlinkReadOptions.SNAPSHOT_ID.key()).parseOptional(); - } - - public String tag() { - return confParser.stringConf().option(FlinkReadOptions.TAG.key()).parseOptional(); - } - - public String startTag() { - return confParser.stringConf().option(FlinkReadOptions.START_TAG.key()).parseOptional(); - } - - public String endTag() { - return confParser.stringConf().option(FlinkReadOptions.END_TAG.key()).parseOptional(); - } - - public String branch() { - return confParser.stringConf().option(FlinkReadOptions.BRANCH.key()).parseOptional(); - } - - public boolean caseSensitive() { - return confParser - .booleanConf() - .option(FlinkReadOptions.CASE_SENSITIVE) - .flinkConfig(FlinkReadOptions.CASE_SENSITIVE_OPTION) - .defaultValue(FlinkReadOptions.CASE_SENSITIVE_OPTION.defaultValue()) - .parse(); - } - - public Long asOfTimestamp() { - return confParser.longConf().option(FlinkReadOptions.AS_OF_TIMESTAMP.key()).parseOptional(); - } - - public StreamingStartingStrategy startingStrategy() { - return confParser - .enumConfParser(StreamingStartingStrategy.class) - .option(FlinkReadOptions.STARTING_STRATEGY) - .flinkConfig(FlinkReadOptions.STARTING_STRATEGY_OPTION) - .defaultValue(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) - .parse(); - } - - public Long startSnapshotTimestamp() { - return confParser - .longConf() - .option(FlinkReadOptions.START_SNAPSHOT_TIMESTAMP.key()) - .parseOptional(); - } - - public Long startSnapshotId() { - return confParser.longConf().option(FlinkReadOptions.START_SNAPSHOT_ID.key()).parseOptional(); - } - - public Long endSnapshotId() { - return confParser.longConf().option(FlinkReadOptions.END_SNAPSHOT_ID.key()).parseOptional(); - } - - public long splitSize() { - return confParser - .longConf() - .option(FlinkReadOptions.SPLIT_SIZE) - .flinkConfig(FlinkReadOptions.SPLIT_SIZE_OPTION) - .tableProperty(TableProperties.SPLIT_SIZE) - .defaultValue(TableProperties.SPLIT_SIZE_DEFAULT) - .parse(); - } - - public int splitLookback() { - return confParser - .intConf() - .option(FlinkReadOptions.SPLIT_LOOKBACK) - .flinkConfig(FlinkReadOptions.SPLIT_LOOKBACK_OPTION) - .tableProperty(TableProperties.SPLIT_LOOKBACK) - .defaultValue(TableProperties.SPLIT_LOOKBACK_DEFAULT) - .parse(); - } - - public long splitFileOpenCost() { - return confParser - .longConf() - .option(FlinkReadOptions.SPLIT_FILE_OPEN_COST) - .flinkConfig(FlinkReadOptions.SPLIT_FILE_OPEN_COST_OPTION) - .tableProperty(TableProperties.SPLIT_OPEN_FILE_COST) - .defaultValue(TableProperties.SPLIT_OPEN_FILE_COST_DEFAULT) - .parse(); - } - - public boolean streaming() { - return confParser - .booleanConf() - .option(FlinkReadOptions.STREAMING) - .flinkConfig(FlinkReadOptions.STREAMING_OPTION) - .defaultValue(FlinkReadOptions.STREAMING_OPTION.defaultValue()) - .parse(); - } - - public Duration monitorInterval() { - String duration = - confParser - .stringConf() - .option(FlinkReadOptions.MONITOR_INTERVAL) - .flinkConfig(FlinkReadOptions.MONITOR_INTERVAL_OPTION) - .defaultValue(FlinkReadOptions.MONITOR_INTERVAL_OPTION.defaultValue()) - .parse(); - - return TimeUtils.parseDuration(duration); - } - - public boolean includeColumnStats() { - return confParser - .booleanConf() - .option(FlinkReadOptions.INCLUDE_COLUMN_STATS) - .flinkConfig(FlinkReadOptions.INCLUDE_COLUMN_STATS_OPTION) - .defaultValue(FlinkReadOptions.INCLUDE_COLUMN_STATS_OPTION.defaultValue()) - .parse(); - } - - public int maxPlanningSnapshotCount() { - return confParser - .intConf() - .option(FlinkReadOptions.MAX_PLANNING_SNAPSHOT_COUNT) - .flinkConfig(FlinkReadOptions.MAX_PLANNING_SNAPSHOT_COUNT_OPTION) - .defaultValue(FlinkReadOptions.MAX_PLANNING_SNAPSHOT_COUNT_OPTION.defaultValue()) - .parse(); - } - - public String nameMapping() { - return confParser.stringConf().option(TableProperties.DEFAULT_NAME_MAPPING).parseOptional(); - } - - public long limit() { - return confParser - .longConf() - .option(FlinkReadOptions.LIMIT) - .flinkConfig(FlinkReadOptions.LIMIT_OPTION) - .defaultValue(FlinkReadOptions.LIMIT_OPTION.defaultValue()) - .parse(); - } - - public int workerPoolSize() { - return confParser - .intConf() - .flinkConfig(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE) - .defaultValue(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.defaultValue()) - .parse(); - } - - public int maxAllowedPlanningFailures() { - return confParser - .intConf() - .option(FlinkReadOptions.MAX_ALLOWED_PLANNING_FAILURES) - .flinkConfig(FlinkReadOptions.MAX_ALLOWED_PLANNING_FAILURES_OPTION) - .defaultValue(FlinkReadOptions.MAX_ALLOWED_PLANNING_FAILURES_OPTION.defaultValue()) - .parse(); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java deleted file mode 100644 index 55c5aca3b677..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java +++ /dev/null @@ -1,112 +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.iceberg.flink; - -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.flink.source.StreamingStartingStrategy; - -/** Flink source read options */ -public class FlinkReadOptions { - private static final String PREFIX = "connector.iceberg."; - - private FlinkReadOptions() {} - - public static final ConfigOption SNAPSHOT_ID = - ConfigOptions.key("snapshot-id").longType().defaultValue(null); - - public static final ConfigOption TAG = - ConfigOptions.key("tag").stringType().defaultValue(null); - - public static final ConfigOption BRANCH = - ConfigOptions.key("branch").stringType().defaultValue(null); - - public static final ConfigOption START_TAG = - ConfigOptions.key("start-tag").stringType().defaultValue(null); - - public static final ConfigOption END_TAG = - ConfigOptions.key("end-tag").stringType().defaultValue(null); - - public static final String CASE_SENSITIVE = "case-sensitive"; - public static final ConfigOption CASE_SENSITIVE_OPTION = - ConfigOptions.key(PREFIX + CASE_SENSITIVE).booleanType().defaultValue(false); - - public static final ConfigOption AS_OF_TIMESTAMP = - ConfigOptions.key("as-of-timestamp").longType().defaultValue(null); - - public static final String STARTING_STRATEGY = "starting-strategy"; - public static final ConfigOption STARTING_STRATEGY_OPTION = - ConfigOptions.key(PREFIX + STARTING_STRATEGY) - .enumType(StreamingStartingStrategy.class) - .defaultValue(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT); - - public static final ConfigOption START_SNAPSHOT_TIMESTAMP = - ConfigOptions.key("start-snapshot-timestamp").longType().defaultValue(null); - - public static final ConfigOption START_SNAPSHOT_ID = - ConfigOptions.key("start-snapshot-id").longType().defaultValue(null); - - public static final ConfigOption END_SNAPSHOT_ID = - ConfigOptions.key("end-snapshot-id").longType().defaultValue(null); - - public static final String SPLIT_SIZE = "split-size"; - public static final ConfigOption SPLIT_SIZE_OPTION = - ConfigOptions.key(PREFIX + SPLIT_SIZE) - .longType() - .defaultValue(TableProperties.SPLIT_SIZE_DEFAULT); - - public static final String SPLIT_LOOKBACK = "split-lookback"; - public static final ConfigOption SPLIT_LOOKBACK_OPTION = - ConfigOptions.key(PREFIX + SPLIT_LOOKBACK) - .intType() - .defaultValue(TableProperties.SPLIT_LOOKBACK_DEFAULT); - - public static final String SPLIT_FILE_OPEN_COST = "split-file-open-cost"; - public static final ConfigOption SPLIT_FILE_OPEN_COST_OPTION = - ConfigOptions.key(PREFIX + SPLIT_FILE_OPEN_COST) - .longType() - .defaultValue(TableProperties.SPLIT_OPEN_FILE_COST_DEFAULT); - - public static final String STREAMING = "streaming"; - public static final ConfigOption STREAMING_OPTION = - ConfigOptions.key(PREFIX + STREAMING).booleanType().defaultValue(false); - - public static final String MONITOR_INTERVAL = "monitor-interval"; - public static final ConfigOption MONITOR_INTERVAL_OPTION = - ConfigOptions.key(PREFIX + MONITOR_INTERVAL).stringType().defaultValue("60s"); - - public static final String INCLUDE_COLUMN_STATS = "include-column-stats"; - public static final ConfigOption INCLUDE_COLUMN_STATS_OPTION = - ConfigOptions.key(PREFIX + INCLUDE_COLUMN_STATS).booleanType().defaultValue(false); - - public static final String MAX_PLANNING_SNAPSHOT_COUNT = "max-planning-snapshot-count"; - public static final ConfigOption MAX_PLANNING_SNAPSHOT_COUNT_OPTION = - ConfigOptions.key(PREFIX + MAX_PLANNING_SNAPSHOT_COUNT) - .intType() - .defaultValue(Integer.MAX_VALUE); - - public static final String LIMIT = "limit"; - public static final ConfigOption LIMIT_OPTION = - ConfigOptions.key(PREFIX + LIMIT).longType().defaultValue(-1L); - - public static final String MAX_ALLOWED_PLANNING_FAILURES = "max-allowed-planning-failures"; - public static final ConfigOption MAX_ALLOWED_PLANNING_FAILURES_OPTION = - ConfigOptions.key(PREFIX + MAX_ALLOWED_PLANNING_FAILURES).intType().defaultValue(3); -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java deleted file mode 100644 index 25725639c330..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java +++ /dev/null @@ -1,181 +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.iceberg.flink; - -import java.util.List; -import java.util.Set; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.utils.TypeConversions; -import org.apache.iceberg.Schema; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.types.Types; - -/** - * Converter between Flink types and Iceberg type. The conversion is not a 1:1 mapping that not - * allows back-and-forth conversion. So some information might get lost during the back-and-forth - * conversion. - * - *

    This inconsistent types: - * - *

      - *
    • map Iceberg UUID type to Flink BinaryType(16) - *
    • map Flink VarCharType(_) and CharType(_) to Iceberg String type - *
    • map Flink VarBinaryType(_) to Iceberg Binary type - *
    • map Flink TimeType(_) to Iceberg Time type (microseconds) - *
    • map Flink TimestampType(_) to Iceberg Timestamp without zone type (microseconds) - *
    • map Flink LocalZonedTimestampType(_) to Iceberg Timestamp with zone type (microseconds) - *
    • map Flink MultiSetType to Iceberg Map type(element, int) - *
    - * - *

    - */ -public class FlinkSchemaUtil { - - private FlinkSchemaUtil() {} - - /** Convert the flink table schema to apache iceberg schema. */ - public static Schema convert(TableSchema schema) { - LogicalType schemaType = schema.toRowDataType().getLogicalType(); - Preconditions.checkArgument( - schemaType instanceof RowType, "Schema logical type should be RowType."); - - RowType root = (RowType) schemaType; - Type converted = root.accept(new FlinkTypeToType(root)); - - Schema iSchema = new Schema(converted.asStructType().fields()); - return freshIdentifierFieldIds(iSchema, schema); - } - - private static Schema freshIdentifierFieldIds(Schema iSchema, TableSchema schema) { - // Locate the identifier field id list. - Set identifierFieldIds = Sets.newHashSet(); - if (schema.getPrimaryKey().isPresent()) { - for (String column : schema.getPrimaryKey().get().getColumns()) { - Types.NestedField field = iSchema.findField(column); - Preconditions.checkNotNull( - field, - "Cannot find field ID for the primary key column %s in schema %s", - column, - iSchema); - identifierFieldIds.add(field.fieldId()); - } - } - - return new Schema(iSchema.schemaId(), iSchema.asStruct().fields(), identifierFieldIds); - } - - /** - * Convert a Flink {@link TableSchema} to a {@link Schema} based on the given schema. - * - *

    This conversion does not assign new ids; it uses ids from the base schema. - * - *

    Data types, field order, and nullability will match the Flink type. This conversion may - * return a schema that is not compatible with base schema. - * - * @param baseSchema a Schema on which conversion is based - * @param flinkSchema a Flink TableSchema - * @return the equivalent Schema - * @throws IllegalArgumentException if the type cannot be converted or there are missing ids - */ - public static Schema convert(Schema baseSchema, TableSchema flinkSchema) { - // convert to a type with fresh ids - Types.StructType struct = convert(flinkSchema).asStruct(); - // reassign ids to match the base schema - Schema schema = TypeUtil.reassignIds(new Schema(struct.fields()), baseSchema); - // reassign doc to match the base schema - schema = TypeUtil.reassignDoc(schema, baseSchema); - - // fix types that can't be represented in Flink (UUID) - Schema fixedSchema = FlinkFixupTypes.fixup(schema, baseSchema); - return freshIdentifierFieldIds(fixedSchema, flinkSchema); - } - - /** - * Convert a {@link Schema} to a {@link RowType Flink type}. - * - * @param schema a Schema - * @return the equivalent Flink type - * @throws IllegalArgumentException if the type cannot be converted to Flink - */ - public static RowType convert(Schema schema) { - return (RowType) TypeUtil.visit(schema, new TypeToFlinkType()); - } - - /** - * Convert a {@link Type} to a {@link LogicalType Flink type}. - * - * @param type a Type - * @return the equivalent Flink type - * @throws IllegalArgumentException if the type cannot be converted to Flink - */ - public static LogicalType convert(Type type) { - return TypeUtil.visit(type, new TypeToFlinkType()); - } - - /** - * Convert a {@link RowType} to a {@link TableSchema}. - * - * @param rowType a RowType - * @return Flink TableSchema - */ - public static TableSchema toSchema(RowType rowType) { - TableSchema.Builder builder = TableSchema.builder(); - for (RowType.RowField field : rowType.getFields()) { - builder.field(field.getName(), TypeConversions.fromLogicalToDataType(field.getType())); - } - return builder.build(); - } - - /** - * Convert a {@link Schema} to a {@link TableSchema}. - * - * @param schema iceberg schema to convert. - * @return Flink TableSchema. - */ - public static TableSchema toSchema(Schema schema) { - TableSchema.Builder builder = TableSchema.builder(); - - // Add columns. - for (RowType.RowField field : convert(schema).getFields()) { - builder.field(field.getName(), TypeConversions.fromLogicalToDataType(field.getType())); - } - - // Add primary key. - Set identifierFieldIds = schema.identifierFieldIds(); - if (!identifierFieldIds.isEmpty()) { - List columns = Lists.newArrayListWithExpectedSize(identifierFieldIds.size()); - for (Integer identifierFieldId : identifierFieldIds) { - String columnName = schema.findColumnName(identifierFieldId); - Preconditions.checkNotNull( - columnName, "Cannot find field with id %s in schema %s", identifierFieldId, schema); - - columns.add(columnName); - } - builder.primaryKey(columns.toArray(new String[0])); - } - - return builder.build(); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java deleted file mode 100644 index 5fbd84909d69..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java +++ /dev/null @@ -1,49 +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.iceberg.flink; - -import org.apache.flink.api.common.functions.FilterFunction; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.Schema; -import org.apache.iceberg.expressions.Evaluator; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.types.Types; - -public class FlinkSourceFilter implements FilterFunction { - - private final RowType rowType; - private final Evaluator evaluator; - private final Types.StructType struct; - private volatile RowDataWrapper wrapper; - - public FlinkSourceFilter(Schema schema, Expression expr, boolean caseSensitive) { - this.rowType = FlinkSchemaUtil.convert(schema); - this.struct = schema.asStruct(); - this.evaluator = new Evaluator(struct, expr, caseSensitive); - } - - @Override - public boolean filter(RowData value) { - if (wrapper == null) { - this.wrapper = new RowDataWrapper(rowType, struct); - } - return evaluator.eval(wrapper.wrap(value)); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java deleted file mode 100644 index 6f8bfef2ef44..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java +++ /dev/null @@ -1,199 +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.iceberg.flink; - -import java.util.List; -import java.util.stream.Collectors; -import org.apache.flink.table.types.logical.ArrayType; -import org.apache.flink.table.types.logical.BigIntType; -import org.apache.flink.table.types.logical.BinaryType; -import org.apache.flink.table.types.logical.BooleanType; -import org.apache.flink.table.types.logical.CharType; -import org.apache.flink.table.types.logical.DateType; -import org.apache.flink.table.types.logical.DecimalType; -import org.apache.flink.table.types.logical.DoubleType; -import org.apache.flink.table.types.logical.FloatType; -import org.apache.flink.table.types.logical.IntType; -import org.apache.flink.table.types.logical.LocalZonedTimestampType; -import org.apache.flink.table.types.logical.MapType; -import org.apache.flink.table.types.logical.MultisetType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.SmallIntType; -import org.apache.flink.table.types.logical.TimeType; -import org.apache.flink.table.types.logical.TimestampType; -import org.apache.flink.table.types.logical.TinyIntType; -import org.apache.flink.table.types.logical.VarBinaryType; -import org.apache.flink.table.types.logical.VarCharType; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; - -class FlinkTypeToType extends FlinkTypeVisitor { - - private final RowType root; - private int nextId; - - FlinkTypeToType(RowType root) { - this.root = root; - // the root struct's fields use the first ids - this.nextId = root.getFieldCount(); - } - - private int getNextId() { - int next = nextId; - nextId += 1; - return next; - } - - @Override - public Type visit(CharType charType) { - return Types.StringType.get(); - } - - @Override - public Type visit(VarCharType varCharType) { - return Types.StringType.get(); - } - - @Override - public Type visit(BooleanType booleanType) { - return Types.BooleanType.get(); - } - - @Override - public Type visit(BinaryType binaryType) { - return Types.FixedType.ofLength(binaryType.getLength()); - } - - @Override - public Type visit(VarBinaryType varBinaryType) { - return Types.BinaryType.get(); - } - - @Override - public Type visit(DecimalType decimalType) { - return Types.DecimalType.of(decimalType.getPrecision(), decimalType.getScale()); - } - - @Override - public Type visit(TinyIntType tinyIntType) { - return Types.IntegerType.get(); - } - - @Override - public Type visit(SmallIntType smallIntType) { - return Types.IntegerType.get(); - } - - @Override - public Type visit(IntType intType) { - return Types.IntegerType.get(); - } - - @Override - public Type visit(BigIntType bigIntType) { - return Types.LongType.get(); - } - - @Override - public Type visit(FloatType floatType) { - return Types.FloatType.get(); - } - - @Override - public Type visit(DoubleType doubleType) { - return Types.DoubleType.get(); - } - - @Override - public Type visit(DateType dateType) { - return Types.DateType.get(); - } - - @Override - public Type visit(TimeType timeType) { - return Types.TimeType.get(); - } - - @Override - public Type visit(TimestampType timestampType) { - return Types.TimestampType.withoutZone(); - } - - @Override - public Type visit(LocalZonedTimestampType localZonedTimestampType) { - return Types.TimestampType.withZone(); - } - - @Override - public Type visit(ArrayType arrayType) { - Type elementType = arrayType.getElementType().accept(this); - if (arrayType.getElementType().isNullable()) { - return Types.ListType.ofOptional(getNextId(), elementType); - } else { - return Types.ListType.ofRequired(getNextId(), elementType); - } - } - - @Override - public Type visit(MultisetType multisetType) { - Type elementType = multisetType.getElementType().accept(this); - return Types.MapType.ofRequired(getNextId(), getNextId(), elementType, Types.IntegerType.get()); - } - - @Override - public Type visit(MapType mapType) { - // keys in map are not allowed to be null. - Type keyType = mapType.getKeyType().accept(this); - Type valueType = mapType.getValueType().accept(this); - if (mapType.getValueType().isNullable()) { - return Types.MapType.ofOptional(getNextId(), getNextId(), keyType, valueType); - } else { - return Types.MapType.ofRequired(getNextId(), getNextId(), keyType, valueType); - } - } - - @Override - @SuppressWarnings("ReferenceEquality") - public Type visit(RowType rowType) { - List newFields = Lists.newArrayListWithExpectedSize(rowType.getFieldCount()); - boolean isRoot = root == rowType; - - List types = - rowType.getFields().stream() - .map(f -> f.getType().accept(this)) - .collect(Collectors.toList()); - - for (int i = 0; i < rowType.getFieldCount(); i++) { - int id = isRoot ? i : getNextId(); - - RowType.RowField field = rowType.getFields().get(i); - String name = field.getName(); - String comment = field.getDescription().orElse(null); - - if (field.getType().isNullable()) { - newFields.add(Types.NestedField.optional(id, name, types.get(i), comment)); - } else { - newFields.add(Types.NestedField.required(id, name, types.get(i), comment)); - } - } - - return Types.StructType.of(newFields); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java deleted file mode 100644 index f3de2416088c..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java +++ /dev/null @@ -1,80 +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.iceberg.flink; - -import org.apache.flink.table.types.logical.DayTimeIntervalType; -import org.apache.flink.table.types.logical.DistinctType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.LogicalTypeVisitor; -import org.apache.flink.table.types.logical.NullType; -import org.apache.flink.table.types.logical.RawType; -import org.apache.flink.table.types.logical.StructuredType; -import org.apache.flink.table.types.logical.SymbolType; -import org.apache.flink.table.types.logical.YearMonthIntervalType; -import org.apache.flink.table.types.logical.ZonedTimestampType; - -public abstract class FlinkTypeVisitor implements LogicalTypeVisitor { - - // ------------------------- Unsupported types ------------------------------ - - @Override - public T visit(ZonedTimestampType zonedTimestampType) { - throw new UnsupportedOperationException("Unsupported ZonedTimestampType."); - } - - @Override - public T visit(YearMonthIntervalType yearMonthIntervalType) { - throw new UnsupportedOperationException("Unsupported YearMonthIntervalType."); - } - - @Override - public T visit(DayTimeIntervalType dayTimeIntervalType) { - throw new UnsupportedOperationException("Unsupported DayTimeIntervalType."); - } - - @Override - public T visit(DistinctType distinctType) { - throw new UnsupportedOperationException("Unsupported DistinctType."); - } - - @Override - public T visit(StructuredType structuredType) { - throw new UnsupportedOperationException("Unsupported StructuredType."); - } - - @Override - public T visit(NullType nullType) { - throw new UnsupportedOperationException("Unsupported NullType."); - } - - @Override - public T visit(RawType rawType) { - throw new UnsupportedOperationException("Unsupported RawType."); - } - - @Override - public T visit(SymbolType symbolType) { - throw new UnsupportedOperationException("Unsupported SymbolType."); - } - - @Override - public T visit(LogicalType other) { - throw new UnsupportedOperationException("Unsupported type: " + other); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java deleted file mode 100644 index ca7b1120bc81..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java +++ /dev/null @@ -1,205 +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.iceberg.flink; - -import java.time.Duration; -import java.util.Map; -import org.apache.flink.annotation.Experimental; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.iceberg.DistributionMode; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; - -/** - * A class for common Iceberg configs for Flink writes. - * - *

    If a config is set at multiple levels, the following order of precedence is used (top to - * bottom): - * - *

      - *
    1. Write options - *
    2. flink ReadableConfig - *
    3. Table metadata - *
    - * - * The most specific value is set in write options and takes precedence over all other configs. If - * no write option is provided, this class checks the flink configuration for any overrides. If no - * applicable value is found in the write options, this class uses the table metadata. - * - *

    Note this class is NOT meant to be serialized. - */ -public class FlinkWriteConf { - - private final FlinkConfParser confParser; - - public FlinkWriteConf( - Table table, Map writeOptions, ReadableConfig readableConfig) { - this.confParser = new FlinkConfParser(table, writeOptions, readableConfig); - } - - public boolean overwriteMode() { - return confParser - .booleanConf() - .option(FlinkWriteOptions.OVERWRITE_MODE.key()) - .flinkConfig(FlinkWriteOptions.OVERWRITE_MODE) - .defaultValue(FlinkWriteOptions.OVERWRITE_MODE.defaultValue()) - .parse(); - } - - public boolean upsertMode() { - return confParser - .booleanConf() - .option(FlinkWriteOptions.WRITE_UPSERT_ENABLED.key()) - .flinkConfig(FlinkWriteOptions.WRITE_UPSERT_ENABLED) - .tableProperty(TableProperties.UPSERT_ENABLED) - .defaultValue(TableProperties.UPSERT_ENABLED_DEFAULT) - .parse(); - } - - public FileFormat dataFileFormat() { - String valueAsString = - confParser - .stringConf() - .option(FlinkWriteOptions.WRITE_FORMAT.key()) - .flinkConfig(FlinkWriteOptions.WRITE_FORMAT) - .tableProperty(TableProperties.DEFAULT_FILE_FORMAT) - .defaultValue(TableProperties.DEFAULT_FILE_FORMAT_DEFAULT) - .parse(); - return FileFormat.fromString(valueAsString); - } - - public long targetDataFileSize() { - return confParser - .longConf() - .option(FlinkWriteOptions.TARGET_FILE_SIZE_BYTES.key()) - .flinkConfig(FlinkWriteOptions.TARGET_FILE_SIZE_BYTES) - .tableProperty(TableProperties.WRITE_TARGET_FILE_SIZE_BYTES) - .defaultValue(TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT) - .parse(); - } - - public String parquetCompressionCodec() { - return confParser - .stringConf() - .option(FlinkWriteOptions.COMPRESSION_CODEC.key()) - .flinkConfig(FlinkWriteOptions.COMPRESSION_CODEC) - .tableProperty(TableProperties.PARQUET_COMPRESSION) - .defaultValue(TableProperties.PARQUET_COMPRESSION_DEFAULT) - .parse(); - } - - public String parquetCompressionLevel() { - return confParser - .stringConf() - .option(FlinkWriteOptions.COMPRESSION_LEVEL.key()) - .flinkConfig(FlinkWriteOptions.COMPRESSION_LEVEL) - .tableProperty(TableProperties.PARQUET_COMPRESSION_LEVEL) - .defaultValue(TableProperties.PARQUET_COMPRESSION_LEVEL_DEFAULT) - .parseOptional(); - } - - public String avroCompressionCodec() { - return confParser - .stringConf() - .option(FlinkWriteOptions.COMPRESSION_CODEC.key()) - .flinkConfig(FlinkWriteOptions.COMPRESSION_CODEC) - .tableProperty(TableProperties.AVRO_COMPRESSION) - .defaultValue(TableProperties.AVRO_COMPRESSION_DEFAULT) - .parse(); - } - - public String avroCompressionLevel() { - return confParser - .stringConf() - .option(FlinkWriteOptions.COMPRESSION_LEVEL.key()) - .flinkConfig(FlinkWriteOptions.COMPRESSION_LEVEL) - .tableProperty(TableProperties.AVRO_COMPRESSION_LEVEL) - .defaultValue(TableProperties.AVRO_COMPRESSION_LEVEL_DEFAULT) - .parseOptional(); - } - - public String orcCompressionCodec() { - return confParser - .stringConf() - .option(FlinkWriteOptions.COMPRESSION_CODEC.key()) - .flinkConfig(FlinkWriteOptions.COMPRESSION_CODEC) - .tableProperty(TableProperties.ORC_COMPRESSION) - .defaultValue(TableProperties.ORC_COMPRESSION_DEFAULT) - .parse(); - } - - public String orcCompressionStrategy() { - return confParser - .stringConf() - .option(FlinkWriteOptions.COMPRESSION_STRATEGY.key()) - .flinkConfig(FlinkWriteOptions.COMPRESSION_STRATEGY) - .tableProperty(TableProperties.ORC_COMPRESSION_STRATEGY) - .defaultValue(TableProperties.ORC_COMPRESSION_STRATEGY_DEFAULT) - .parse(); - } - - public DistributionMode distributionMode() { - String modeName = - confParser - .stringConf() - .option(FlinkWriteOptions.DISTRIBUTION_MODE.key()) - .flinkConfig(FlinkWriteOptions.DISTRIBUTION_MODE) - .tableProperty(TableProperties.WRITE_DISTRIBUTION_MODE) - .defaultValue(TableProperties.WRITE_DISTRIBUTION_MODE_NONE) - .parse(); - return DistributionMode.fromName(modeName); - } - - public int workerPoolSize() { - return confParser - .intConf() - .flinkConfig(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE) - .defaultValue(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.defaultValue()) - .parse(); - } - - public String branch() { - return confParser - .stringConf() - .option(FlinkWriteOptions.BRANCH.key()) - .defaultValue(FlinkWriteOptions.BRANCH.defaultValue()) - .parse(); - } - - public Integer writeParallelism() { - return confParser.intConf().option(FlinkWriteOptions.WRITE_PARALLELISM.key()).parseOptional(); - } - - /** - * NOTE: This may be removed or changed in a future release. This value specifies the interval for - * refreshing the table instances in sink writer subtasks. If not specified then the default - * behavior is to not refresh the table. - * - * @return the interval for refreshing the table in sink writer subtasks - */ - @Experimental - public Duration tableRefreshInterval() { - return confParser - .durationConf() - .option(FlinkWriteOptions.TABLE_REFRESH_INTERVAL.key()) - .flinkConfig(FlinkWriteOptions.TABLE_REFRESH_INTERVAL) - .parseOptional(); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java deleted file mode 100644 index df73f2e09cac..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java +++ /dev/null @@ -1,73 +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.iceberg.flink; - -import java.time.Duration; -import org.apache.flink.annotation.Experimental; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; -import org.apache.iceberg.SnapshotRef; - -/** Flink sink write options */ -public class FlinkWriteOptions { - - private FlinkWriteOptions() {} - - // File format for write operations(default: Table write.format.default ) - public static final ConfigOption WRITE_FORMAT = - ConfigOptions.key("write-format").stringType().noDefaultValue(); - - // Overrides this table's write.target-file-size-bytes - public static final ConfigOption TARGET_FILE_SIZE_BYTES = - ConfigOptions.key("target-file-size-bytes").longType().noDefaultValue(); - - // Overrides this table's write..compression-codec - public static final ConfigOption COMPRESSION_CODEC = - ConfigOptions.key("compression-codec").stringType().noDefaultValue(); - - // Overrides this table's write..compression-level - public static final ConfigOption COMPRESSION_LEVEL = - ConfigOptions.key("compression-level").stringType().noDefaultValue(); - - // Overrides this table's write..compression-strategy - public static final ConfigOption COMPRESSION_STRATEGY = - ConfigOptions.key("compression-strategy").stringType().noDefaultValue(); - - // Overrides this table's write.upsert.enabled - public static final ConfigOption WRITE_UPSERT_ENABLED = - ConfigOptions.key("upsert-enabled").booleanType().noDefaultValue(); - - public static final ConfigOption OVERWRITE_MODE = - ConfigOptions.key("overwrite-enabled").booleanType().defaultValue(false); - - // Overrides the table's write.distribution-mode - public static final ConfigOption DISTRIBUTION_MODE = - ConfigOptions.key("distribution-mode").stringType().noDefaultValue(); - - // Branch to write to - public static final ConfigOption BRANCH = - ConfigOptions.key("branch").stringType().defaultValue(SnapshotRef.MAIN_BRANCH); - - public static final ConfigOption WRITE_PARALLELISM = - ConfigOptions.key("write-parallelism").intType().noDefaultValue(); - - @Experimental - public static final ConfigOption TABLE_REFRESH_INTERVAL = - ConfigOptions.key("table-refresh-interval").durationType().noDefaultValue(); -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java deleted file mode 100644 index 1b9268569d9a..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java +++ /dev/null @@ -1,121 +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.iceberg.flink; - -import java.util.List; -import java.util.Map; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSink; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.api.constraints.UniqueConstraint; -import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.ProviderContext; -import org.apache.flink.table.connector.sink.DataStreamSinkProvider; -import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.connector.sink.abilities.SupportsOverwrite; -import org.apache.flink.table.connector.sink.abilities.SupportsPartitioning; -import org.apache.flink.table.data.RowData; -import org.apache.flink.types.RowKind; -import org.apache.flink.util.Preconditions; -import org.apache.iceberg.flink.sink.FlinkSink; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; - -public class IcebergTableSink implements DynamicTableSink, SupportsPartitioning, SupportsOverwrite { - private final TableLoader tableLoader; - private final TableSchema tableSchema; - private final ReadableConfig readableConfig; - private final Map writeProps; - - private boolean overwrite = false; - - private IcebergTableSink(IcebergTableSink toCopy) { - this.tableLoader = toCopy.tableLoader; - this.tableSchema = toCopy.tableSchema; - this.overwrite = toCopy.overwrite; - this.readableConfig = toCopy.readableConfig; - this.writeProps = toCopy.writeProps; - } - - public IcebergTableSink( - TableLoader tableLoader, - TableSchema tableSchema, - ReadableConfig readableConfig, - Map writeProps) { - this.tableLoader = tableLoader; - this.tableSchema = tableSchema; - this.readableConfig = readableConfig; - this.writeProps = writeProps; - } - - @Override - public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { - Preconditions.checkState( - !overwrite || context.isBounded(), - "Unbounded data stream doesn't support overwrite operation."); - - List equalityColumns = - tableSchema.getPrimaryKey().map(UniqueConstraint::getColumns).orElseGet(ImmutableList::of); - - return new DataStreamSinkProvider() { - @Override - public DataStreamSink consumeDataStream( - ProviderContext providerContext, DataStream dataStream) { - return FlinkSink.forRowData(dataStream) - .tableLoader(tableLoader) - .tableSchema(tableSchema) - .equalityFieldColumns(equalityColumns) - .overwrite(overwrite) - .setAll(writeProps) - .flinkConf(readableConfig) - .append(); - } - }; - } - - @Override - public void applyStaticPartition(Map partition) { - // The flink's PartitionFanoutWriter will handle the static partition write policy - // automatically. - } - - @Override - public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { - ChangelogMode.Builder builder = ChangelogMode.newBuilder(); - for (RowKind kind : requestedMode.getContainedKinds()) { - builder.addContainedKind(kind); - } - return builder.build(); - } - - @Override - public DynamicTableSink copy() { - return new IcebergTableSink(this); - } - - @Override - public String asSummaryString() { - return "Iceberg table sink"; - } - - @Override - public void applyOverwrite(boolean newOverwrite) { - this.overwrite = newOverwrite; - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java deleted file mode 100644 index d4cec7a3e80b..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java +++ /dev/null @@ -1,142 +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.iceberg.flink; - -import java.lang.reflect.Array; -import java.nio.ByteBuffer; -import java.time.LocalDateTime; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.types.logical.DecimalType; -import org.apache.flink.table.types.logical.LocalZonedTimestampType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.TimestampType; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.DateTimeUtil; -import org.apache.iceberg.util.UUIDUtil; - -public class RowDataWrapper implements StructLike { - - private final LogicalType[] types; - private final PositionalGetter[] getters; - private RowData rowData = null; - - public RowDataWrapper(RowType rowType, Types.StructType struct) { - int size = rowType.getFieldCount(); - - types = (LogicalType[]) Array.newInstance(LogicalType.class, size); - getters = (PositionalGetter[]) Array.newInstance(PositionalGetter.class, size); - - for (int i = 0; i < size; i++) { - types[i] = rowType.getTypeAt(i); - getters[i] = buildGetter(types[i], struct.fields().get(i).type()); - } - } - - public RowDataWrapper wrap(RowData data) { - this.rowData = data; - return this; - } - - @Override - public int size() { - return types.length; - } - - @Override - public T get(int pos, Class javaClass) { - if (rowData.isNullAt(pos)) { - return null; - } else if (getters[pos] != null) { - return javaClass.cast(getters[pos].get(rowData, pos)); - } - - Object value = RowData.createFieldGetter(types[pos], pos).getFieldOrNull(rowData); - return javaClass.cast(value); - } - - @Override - public void set(int pos, T value) { - throw new UnsupportedOperationException( - "Could not set a field in the RowDataWrapper because rowData is read-only"); - } - - private interface PositionalGetter { - T get(RowData data, int pos); - } - - private static PositionalGetter buildGetter(LogicalType logicalType, Type type) { - switch (logicalType.getTypeRoot()) { - case TINYINT: - return (row, pos) -> (int) row.getByte(pos); - case SMALLINT: - return (row, pos) -> (int) row.getShort(pos); - case CHAR: - case VARCHAR: - return (row, pos) -> row.getString(pos).toString(); - - case BINARY: - case VARBINARY: - if (Type.TypeID.UUID == type.typeId()) { - return (row, pos) -> UUIDUtil.convert(row.getBinary(pos)); - } else { - return (row, pos) -> ByteBuffer.wrap(row.getBinary(pos)); - } - - case DECIMAL: - DecimalType decimalType = (DecimalType) logicalType; - return (row, pos) -> - row.getDecimal(pos, decimalType.getPrecision(), decimalType.getScale()).toBigDecimal(); - - case TIME_WITHOUT_TIME_ZONE: - // Time in RowData is in milliseconds (Integer), while iceberg's time is microseconds - // (Long). - return (row, pos) -> ((long) row.getInt(pos)) * 1_000; - - case TIMESTAMP_WITHOUT_TIME_ZONE: - TimestampType timestampType = (TimestampType) logicalType; - return (row, pos) -> { - LocalDateTime localDateTime = - row.getTimestamp(pos, timestampType.getPrecision()).toLocalDateTime(); - return DateTimeUtil.microsFromTimestamp(localDateTime); - }; - - case TIMESTAMP_WITH_LOCAL_TIME_ZONE: - LocalZonedTimestampType lzTs = (LocalZonedTimestampType) logicalType; - return (row, pos) -> { - TimestampData timestampData = row.getTimestamp(pos, lzTs.getPrecision()); - return timestampData.getMillisecond() * 1000 - + timestampData.getNanoOfMillisecond() / 1000; - }; - - case ROW: - RowType rowType = (RowType) logicalType; - Types.StructType structType = (Types.StructType) type; - - RowDataWrapper nestedWrapper = new RowDataWrapper(rowType, structType); - return (row, pos) -> nestedWrapper.wrap(row.getRow(pos, rowType.getFieldCount())); - - default: - return null; - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java deleted file mode 100644 index da509451fee7..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java +++ /dev/null @@ -1,159 +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.iceberg.flink; - -import java.io.Closeable; -import java.io.IOException; -import java.io.Serializable; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.hadoop.HadoopTables; -import org.apache.iceberg.hadoop.SerializableConfiguration; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; - -/** - * Serializable loader to load an Iceberg {@link Table}. Flink needs to get {@link Table} objects in - * the cluster (for example, to get splits), not just on the client side. So we need an Iceberg - * table loader to get the {@link Table} object. - */ -public interface TableLoader extends Closeable, Serializable, Cloneable { - - void open(); - - boolean isOpen(); - - Table loadTable(); - - /** Clone a TableLoader */ - @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) - TableLoader clone(); - - static TableLoader fromCatalog(CatalogLoader catalogLoader, TableIdentifier identifier) { - return new CatalogTableLoader(catalogLoader, identifier); - } - - static TableLoader fromHadoopTable(String location) { - return fromHadoopTable(location, FlinkCatalogFactory.clusterHadoopConf()); - } - - static TableLoader fromHadoopTable(String location, Configuration hadoopConf) { - return new HadoopTableLoader(location, hadoopConf); - } - - class HadoopTableLoader implements TableLoader { - - private static final long serialVersionUID = 1L; - - private final String location; - private final SerializableConfiguration hadoopConf; - - private transient HadoopTables tables; - - private HadoopTableLoader(String location, Configuration conf) { - this.location = location; - this.hadoopConf = new SerializableConfiguration(conf); - } - - @Override - public void open() { - tables = new HadoopTables(hadoopConf.get()); - } - - @Override - public boolean isOpen() { - return tables != null; - } - - @Override - public Table loadTable() { - FlinkEnvironmentContext.init(); - return tables.load(location); - } - - @Override - @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) - public TableLoader clone() { - return new HadoopTableLoader(location, new Configuration(hadoopConf.get())); - } - - @Override - public void close() {} - - @Override - public String toString() { - return MoreObjects.toStringHelper(this).add("location", location).toString(); - } - } - - class CatalogTableLoader implements TableLoader { - - private static final long serialVersionUID = 1L; - - private final CatalogLoader catalogLoader; - private final String identifier; - - private transient Catalog catalog; - - private CatalogTableLoader(CatalogLoader catalogLoader, TableIdentifier tableIdentifier) { - this.catalogLoader = catalogLoader; - this.identifier = tableIdentifier.toString(); - } - - @Override - public void open() { - catalog = catalogLoader.loadCatalog(); - } - - @Override - public boolean isOpen() { - return catalog != null; - } - - @Override - public Table loadTable() { - FlinkEnvironmentContext.init(); - return catalog.loadTable(TableIdentifier.parse(identifier)); - } - - @Override - public void close() throws IOException { - if (catalog instanceof Closeable) { - ((Closeable) catalog).close(); - } - - catalog = null; - } - - @Override - @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) - public TableLoader clone() { - return new CatalogTableLoader(catalogLoader.clone(), TableIdentifier.parse(identifier)); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("tableIdentifier", identifier) - .add("catalogLoader", catalogLoader) - .toString(); - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java deleted file mode 100644 index f8f1b74b1ceb..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java +++ /dev/null @@ -1,134 +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.iceberg.flink; - -import java.util.List; -import org.apache.flink.table.types.logical.ArrayType; -import org.apache.flink.table.types.logical.BigIntType; -import org.apache.flink.table.types.logical.BinaryType; -import org.apache.flink.table.types.logical.BooleanType; -import org.apache.flink.table.types.logical.DateType; -import org.apache.flink.table.types.logical.DecimalType; -import org.apache.flink.table.types.logical.DoubleType; -import org.apache.flink.table.types.logical.FloatType; -import org.apache.flink.table.types.logical.IntType; -import org.apache.flink.table.types.logical.LocalZonedTimestampType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.MapType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.TimeType; -import org.apache.flink.table.types.logical.TimestampType; -import org.apache.flink.table.types.logical.VarBinaryType; -import org.apache.flink.table.types.logical.VarCharType; -import org.apache.iceberg.Schema; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.types.Types; - -class TypeToFlinkType extends TypeUtil.SchemaVisitor { - TypeToFlinkType() {} - - @Override - public LogicalType schema(Schema schema, LogicalType structType) { - return structType; - } - - @Override - public LogicalType struct(Types.StructType struct, List fieldResults) { - List fields = struct.fields(); - - List flinkFields = Lists.newArrayListWithExpectedSize(fieldResults.size()); - for (int i = 0; i < fields.size(); i += 1) { - Types.NestedField field = fields.get(i); - LogicalType type = fieldResults.get(i); - RowType.RowField flinkField = - new RowType.RowField(field.name(), type.copy(field.isOptional()), field.doc()); - flinkFields.add(flinkField); - } - - return new RowType(flinkFields); - } - - @Override - public LogicalType field(Types.NestedField field, LogicalType fieldResult) { - return fieldResult; - } - - @Override - public LogicalType list(Types.ListType list, LogicalType elementResult) { - return new ArrayType(elementResult.copy(list.isElementOptional())); - } - - @Override - public LogicalType map(Types.MapType map, LogicalType keyResult, LogicalType valueResult) { - // keys in map are not allowed to be null. - return new MapType(keyResult.copy(false), valueResult.copy(map.isValueOptional())); - } - - @Override - public LogicalType primitive(Type.PrimitiveType primitive) { - switch (primitive.typeId()) { - case BOOLEAN: - return new BooleanType(); - case INTEGER: - return new IntType(); - case LONG: - return new BigIntType(); - case FLOAT: - return new FloatType(); - case DOUBLE: - return new DoubleType(); - case DATE: - return new DateType(); - case TIME: - // For the type: Flink only support TimeType with default precision (second) now. The - // precision of time is - // not supported in Flink, so we can think of it as a simple time type directly. - // For the data: Flink uses int that support mills to represent time data, so it supports - // mills precision. - return new TimeType(); - case TIMESTAMP: - Types.TimestampType timestamp = (Types.TimestampType) primitive; - if (timestamp.shouldAdjustToUTC()) { - // MICROS - return new LocalZonedTimestampType(6); - } else { - // MICROS - return new TimestampType(6); - } - case STRING: - return new VarCharType(VarCharType.MAX_LENGTH); - case UUID: - // UUID length is 16 - return new BinaryType(16); - case FIXED: - Types.FixedType fixedType = (Types.FixedType) primitive; - return new BinaryType(fixedType.length()); - case BINARY: - return new VarBinaryType(VarBinaryType.MAX_LENGTH); - case DECIMAL: - Types.DecimalType decimal = (Types.DecimalType) primitive; - return new DecimalType(decimal.precision(), decimal.scale()); - default: - throw new UnsupportedOperationException( - "Cannot convert unknown type to Flink: " + primitive); - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java deleted file mode 100644 index 06ac54617ae6..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java +++ /dev/null @@ -1,52 +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.iceberg.flink.actions; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.iceberg.Table; - -public class Actions { - - public static final Configuration CONFIG = - new Configuration() - // disable classloader check as Avro may cache class/object in the serializers. - .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); - - private StreamExecutionEnvironment env; - private Table table; - - private Actions(StreamExecutionEnvironment env, Table table) { - this.env = env; - this.table = table; - } - - public static Actions forTable(StreamExecutionEnvironment env, Table table) { - return new Actions(env, table); - } - - public static Actions forTable(Table table) { - return new Actions(StreamExecutionEnvironment.getExecutionEnvironment(CONFIG), table); - } - - public RewriteDataFilesAction rewriteDataFiles() { - return new RewriteDataFilesAction(env, table); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java deleted file mode 100644 index 9876bb3861c4..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java +++ /dev/null @@ -1,72 +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.iceberg.flink.actions; - -import java.util.List; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.Table; -import org.apache.iceberg.actions.BaseRewriteDataFilesAction; -import org.apache.iceberg.flink.source.RowDataRewriter; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -public class RewriteDataFilesAction extends BaseRewriteDataFilesAction { - - private StreamExecutionEnvironment env; - private int maxParallelism; - - public RewriteDataFilesAction(StreamExecutionEnvironment env, Table table) { - super(table); - this.env = env; - this.maxParallelism = env.getParallelism(); - } - - @Override - protected FileIO fileIO() { - return table().io(); - } - - @Override - protected List rewriteDataForTasks(List combinedScanTasks) { - int size = combinedScanTasks.size(); - int parallelism = Math.min(size, maxParallelism); - DataStream dataStream = env.fromCollection(combinedScanTasks); - RowDataRewriter rowDataRewriter = - new RowDataRewriter(table(), caseSensitive(), fileIO(), encryptionManager()); - try { - return rowDataRewriter.rewriteDataForTasks(dataStream, parallelism); - } catch (Exception e) { - throw new RuntimeException("Rewrite data file error.", e); - } - } - - @Override - protected RewriteDataFilesAction self() { - return this; - } - - public RewriteDataFilesAction maxParallelism(int parallelism) { - Preconditions.checkArgument(parallelism > 0, "Invalid max parallelism %s", parallelism); - this.maxParallelism = parallelism; - return this; - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java deleted file mode 100644 index 8103224a0b6c..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java +++ /dev/null @@ -1,75 +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.iceberg.flink.data; - -import org.apache.flink.table.types.logical.ArrayType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.LogicalTypeFamily; -import org.apache.flink.table.types.logical.MapType; -import org.apache.flink.table.types.logical.NullType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.avro.AvroWithPartnerByStructureVisitor; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.util.Pair; - -public abstract class AvroWithFlinkSchemaVisitor - extends AvroWithPartnerByStructureVisitor { - - @Override - protected boolean isStringType(LogicalType logicalType) { - return logicalType.getTypeRoot().getFamilies().contains(LogicalTypeFamily.CHARACTER_STRING); - } - - @Override - protected boolean isMapType(LogicalType logicalType) { - return logicalType instanceof MapType; - } - - @Override - protected LogicalType arrayElementType(LogicalType arrayType) { - Preconditions.checkArgument( - arrayType instanceof ArrayType, "Invalid array: %s is not an array", arrayType); - return ((ArrayType) arrayType).getElementType(); - } - - @Override - protected LogicalType mapKeyType(LogicalType mapType) { - Preconditions.checkArgument(isMapType(mapType), "Invalid map: %s is not a map", mapType); - return ((MapType) mapType).getKeyType(); - } - - @Override - protected LogicalType mapValueType(LogicalType mapType) { - Preconditions.checkArgument(isMapType(mapType), "Invalid map: %s is not a map", mapType); - return ((MapType) mapType).getValueType(); - } - - @Override - protected Pair fieldNameAndType(LogicalType structType, int pos) { - Preconditions.checkArgument( - structType instanceof RowType, "Invalid struct: %s is not a struct", structType); - RowType.RowField field = ((RowType) structType).getFields().get(pos); - return Pair.of(field.getName(), field.getType()); - } - - @Override - protected LogicalType nullType() { - return new NullType(); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java deleted file mode 100644 index 86404959735a..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java +++ /dev/null @@ -1,169 +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.iceberg.flink.data; - -import java.io.IOException; -import java.util.List; -import java.util.Map; -import java.util.function.Supplier; -import org.apache.avro.LogicalType; -import org.apache.avro.LogicalTypes; -import org.apache.avro.Schema; -import org.apache.avro.io.DatumReader; -import org.apache.avro.io.Decoder; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.avro.AvroSchemaWithTypeVisitor; -import org.apache.iceberg.avro.SupportsRowPosition; -import org.apache.iceberg.avro.ValueReader; -import org.apache.iceberg.avro.ValueReaders; -import org.apache.iceberg.data.avro.DecoderResolver; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; - -public class FlinkAvroReader implements DatumReader, SupportsRowPosition { - - private final Schema readSchema; - private final ValueReader reader; - private Schema fileSchema = null; - - public FlinkAvroReader(org.apache.iceberg.Schema expectedSchema, Schema readSchema) { - this(expectedSchema, readSchema, ImmutableMap.of()); - } - - @SuppressWarnings("unchecked") - public FlinkAvroReader( - org.apache.iceberg.Schema expectedSchema, Schema readSchema, Map constants) { - this.readSchema = readSchema; - this.reader = - (ValueReader) - AvroSchemaWithTypeVisitor.visit(expectedSchema, readSchema, new ReadBuilder(constants)); - } - - @Override - public void setSchema(Schema newFileSchema) { - this.fileSchema = Schema.applyAliases(newFileSchema, readSchema); - } - - @Override - public RowData read(RowData reuse, Decoder decoder) throws IOException { - return DecoderResolver.resolveAndRead(decoder, readSchema, fileSchema, reader, reuse); - } - - @Override - public void setRowPositionSupplier(Supplier posSupplier) { - if (reader instanceof SupportsRowPosition) { - ((SupportsRowPosition) reader).setRowPositionSupplier(posSupplier); - } - } - - private static class ReadBuilder extends AvroSchemaWithTypeVisitor> { - private final Map idToConstant; - - private ReadBuilder(Map idToConstant) { - this.idToConstant = idToConstant; - } - - @Override - public ValueReader record( - Types.StructType expected, Schema record, List names, List> fields) { - return FlinkValueReaders.struct(fields, expected.asStructType(), idToConstant); - } - - @Override - public ValueReader union(Type expected, Schema union, List> options) { - return ValueReaders.union(options); - } - - @Override - public ValueReader array( - Types.ListType expected, Schema array, ValueReader elementReader) { - return FlinkValueReaders.array(elementReader); - } - - @Override - public ValueReader map( - Types.MapType expected, Schema map, ValueReader keyReader, ValueReader valueReader) { - return FlinkValueReaders.arrayMap(keyReader, valueReader); - } - - @Override - public ValueReader map(Types.MapType expected, Schema map, ValueReader valueReader) { - return FlinkValueReaders.map(FlinkValueReaders.strings(), valueReader); - } - - @Override - public ValueReader primitive(Type.PrimitiveType expected, Schema primitive) { - LogicalType logicalType = primitive.getLogicalType(); - if (logicalType != null) { - switch (logicalType.getName()) { - case "date": - return ValueReaders.ints(); - - case "time-micros": - return FlinkValueReaders.timeMicros(); - - case "timestamp-millis": - return FlinkValueReaders.timestampMills(); - - case "timestamp-micros": - return FlinkValueReaders.timestampMicros(); - - case "decimal": - LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) logicalType; - return FlinkValueReaders.decimal( - ValueReaders.decimalBytesReader(primitive), - decimal.getPrecision(), - decimal.getScale()); - - case "uuid": - return FlinkValueReaders.uuids(); - - default: - throw new IllegalArgumentException("Unknown logical type: " + logicalType); - } - } - - switch (primitive.getType()) { - case NULL: - return ValueReaders.nulls(); - case BOOLEAN: - return ValueReaders.booleans(); - case INT: - return ValueReaders.ints(); - case LONG: - return ValueReaders.longs(); - case FLOAT: - return ValueReaders.floats(); - case DOUBLE: - return ValueReaders.doubles(); - case STRING: - return FlinkValueReaders.strings(); - case FIXED: - return ValueReaders.fixed(primitive.getFixedSize()); - case BYTES: - return ValueReaders.bytes(); - case ENUM: - return FlinkValueReaders.enums(primitive.getEnumSymbols()); - default: - throw new IllegalArgumentException("Unsupported type: " + primitive); - } - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java deleted file mode 100644 index 873e65783119..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java +++ /dev/null @@ -1,165 +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.iceberg.flink.data; - -import java.io.IOException; -import java.util.List; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import java.util.stream.Stream; -import org.apache.avro.LogicalTypes; -import org.apache.avro.Schema; -import org.apache.avro.io.Encoder; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FieldMetrics; -import org.apache.iceberg.avro.MetricsAwareDatumWriter; -import org.apache.iceberg.avro.ValueWriter; -import org.apache.iceberg.avro.ValueWriters; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -public class FlinkAvroWriter implements MetricsAwareDatumWriter { - private final RowType rowType; - private ValueWriter writer = null; - - public FlinkAvroWriter(RowType rowType) { - this.rowType = rowType; - } - - @Override - @SuppressWarnings("unchecked") - public void setSchema(Schema schema) { - this.writer = - (ValueWriter) - AvroWithFlinkSchemaVisitor.visit(rowType, schema, new WriteBuilder()); - } - - @Override - public void write(RowData datum, Encoder out) throws IOException { - writer.write(datum, out); - } - - @Override - public Stream metrics() { - return writer.metrics(); - } - - private static class WriteBuilder extends AvroWithFlinkSchemaVisitor> { - @Override - public ValueWriter record( - LogicalType struct, Schema record, List names, List> fields) { - return FlinkValueWriters.row( - fields, - IntStream.range(0, names.size()) - .mapToObj(i -> fieldNameAndType(struct, i).second()) - .collect(Collectors.toList())); - } - - @Override - public ValueWriter union(LogicalType type, Schema union, List> options) { - Preconditions.checkArgument( - options.contains(ValueWriters.nulls()), - "Cannot create writer for non-option union: %s", - union); - Preconditions.checkArgument( - options.size() == 2, "Cannot create writer for non-option union: %s", union); - if (union.getTypes().get(0).getType() == Schema.Type.NULL) { - return ValueWriters.option(0, options.get(1)); - } else { - return ValueWriters.option(1, options.get(0)); - } - } - - @Override - public ValueWriter array(LogicalType sArray, Schema array, ValueWriter elementWriter) { - return FlinkValueWriters.array(elementWriter, arrayElementType(sArray)); - } - - @Override - public ValueWriter map(LogicalType sMap, Schema map, ValueWriter valueReader) { - return FlinkValueWriters.map( - FlinkValueWriters.strings(), mapKeyType(sMap), valueReader, mapValueType(sMap)); - } - - @Override - public ValueWriter map( - LogicalType sMap, Schema map, ValueWriter keyWriter, ValueWriter valueWriter) { - return FlinkValueWriters.arrayMap( - keyWriter, mapKeyType(sMap), valueWriter, mapValueType(sMap)); - } - - @Override - public ValueWriter primitive(LogicalType type, Schema primitive) { - org.apache.avro.LogicalType logicalType = primitive.getLogicalType(); - if (logicalType != null) { - switch (logicalType.getName()) { - case "date": - return ValueWriters.ints(); - - case "time-micros": - return FlinkValueWriters.timeMicros(); - - case "timestamp-micros": - return FlinkValueWriters.timestampMicros(); - - case "decimal": - LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) logicalType; - return FlinkValueWriters.decimal(decimal.getPrecision(), decimal.getScale()); - - case "uuid": - return ValueWriters.uuids(); - - default: - throw new IllegalArgumentException("Unsupported logical type: " + logicalType); - } - } - - switch (primitive.getType()) { - case NULL: - return ValueWriters.nulls(); - case BOOLEAN: - return ValueWriters.booleans(); - case INT: - switch (type.getTypeRoot()) { - case TINYINT: - return ValueWriters.tinyints(); - case SMALLINT: - return ValueWriters.shorts(); - default: - return ValueWriters.ints(); - } - case LONG: - return ValueWriters.longs(); - case FLOAT: - return ValueWriters.floats(); - case DOUBLE: - return ValueWriters.doubles(); - case STRING: - return FlinkValueWriters.strings(); - case FIXED: - return ValueWriters.fixed(primitive.getFixedSize()); - case BYTES: - return ValueWriters.bytes(); - default: - throw new IllegalArgumentException("Unsupported type: " + primitive); - } - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java deleted file mode 100644 index 65b9d44ad4b8..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java +++ /dev/null @@ -1,131 +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.iceberg.flink.data; - -import java.util.List; -import java.util.Map; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.orc.OrcRowReader; -import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor; -import org.apache.iceberg.orc.OrcValueReader; -import org.apache.iceberg.orc.OrcValueReaders; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; -import org.apache.orc.TypeDescription; -import org.apache.orc.storage.ql.exec.vector.StructColumnVector; -import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; - -public class FlinkOrcReader implements OrcRowReader { - private final OrcValueReader reader; - - public FlinkOrcReader(Schema iSchema, TypeDescription readSchema) { - this(iSchema, readSchema, ImmutableMap.of()); - } - - public FlinkOrcReader(Schema iSchema, TypeDescription readSchema, Map idToConstant) { - this.reader = - OrcSchemaWithTypeVisitor.visit(iSchema, readSchema, new ReadBuilder(idToConstant)); - } - - @Override - public RowData read(VectorizedRowBatch batch, int row) { - return (RowData) reader.read(new StructColumnVector(batch.size, batch.cols), row); - } - - @Override - public void setBatchContext(long batchOffsetInFile) { - reader.setBatchContext(batchOffsetInFile); - } - - private static class ReadBuilder extends OrcSchemaWithTypeVisitor> { - private final Map idToConstant; - - private ReadBuilder(Map idToConstant) { - this.idToConstant = idToConstant; - } - - @Override - public OrcValueReader record( - Types.StructType iStruct, - TypeDescription record, - List names, - List> fields) { - return FlinkOrcReaders.struct(fields, iStruct, idToConstant); - } - - @Override - public OrcValueReader list( - Types.ListType iList, TypeDescription array, OrcValueReader elementReader) { - return FlinkOrcReaders.array(elementReader); - } - - @Override - public OrcValueReader map( - Types.MapType iMap, - TypeDescription map, - OrcValueReader keyReader, - OrcValueReader valueReader) { - return FlinkOrcReaders.map(keyReader, valueReader); - } - - @Override - public OrcValueReader primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) { - switch (iPrimitive.typeId()) { - case BOOLEAN: - return OrcValueReaders.booleans(); - case INTEGER: - return OrcValueReaders.ints(); - case LONG: - return OrcValueReaders.longs(); - case FLOAT: - return OrcValueReaders.floats(); - case DOUBLE: - return OrcValueReaders.doubles(); - case DATE: - return FlinkOrcReaders.dates(); - case TIME: - return FlinkOrcReaders.times(); - case TIMESTAMP: - Types.TimestampType timestampType = (Types.TimestampType) iPrimitive; - if (timestampType.shouldAdjustToUTC()) { - return FlinkOrcReaders.timestampTzs(); - } else { - return FlinkOrcReaders.timestamps(); - } - case STRING: - return FlinkOrcReaders.strings(); - case UUID: - case FIXED: - case BINARY: - return OrcValueReaders.bytes(); - case DECIMAL: - Types.DecimalType decimalType = (Types.DecimalType) iPrimitive; - return FlinkOrcReaders.decimals(decimalType.precision(), decimalType.scale()); - default: - throw new IllegalArgumentException( - String.format( - "Invalid iceberg type %s corresponding to ORC type %s", iPrimitive, primitive)); - } - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java deleted file mode 100644 index 7a4a15c7e600..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java +++ /dev/null @@ -1,283 +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.iceberg.flink.data; - -import java.math.BigDecimal; -import java.time.Instant; -import java.time.LocalDateTime; -import java.time.ZoneOffset; -import java.util.List; -import java.util.Map; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.GenericArrayData; -import org.apache.flink.table.data.GenericMapData; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.iceberg.orc.OrcValueReader; -import org.apache.iceberg.orc.OrcValueReaders; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Types; -import org.apache.orc.storage.ql.exec.vector.BytesColumnVector; -import org.apache.orc.storage.ql.exec.vector.ColumnVector; -import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector; -import org.apache.orc.storage.ql.exec.vector.ListColumnVector; -import org.apache.orc.storage.ql.exec.vector.LongColumnVector; -import org.apache.orc.storage.ql.exec.vector.MapColumnVector; -import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector; -import org.apache.orc.storage.serde2.io.HiveDecimalWritable; - -class FlinkOrcReaders { - private FlinkOrcReaders() {} - - static OrcValueReader strings() { - return StringReader.INSTANCE; - } - - static OrcValueReader dates() { - return DateReader.INSTANCE; - } - - static OrcValueReader decimals(int precision, int scale) { - if (precision <= 18) { - return new Decimal18Reader(precision, scale); - } else if (precision <= 38) { - return new Decimal38Reader(precision, scale); - } else { - throw new IllegalArgumentException("Invalid precision: " + precision); - } - } - - static OrcValueReader times() { - return TimeReader.INSTANCE; - } - - static OrcValueReader timestamps() { - return TimestampReader.INSTANCE; - } - - static OrcValueReader timestampTzs() { - return TimestampTzReader.INSTANCE; - } - - static OrcValueReader array(OrcValueReader elementReader) { - return new ArrayReader<>(elementReader); - } - - public static OrcValueReader map( - OrcValueReader keyReader, OrcValueReader valueReader) { - return new MapReader<>(keyReader, valueReader); - } - - public static OrcValueReader struct( - List> readers, Types.StructType struct, Map idToConstant) { - return new StructReader(readers, struct, idToConstant); - } - - private static class StringReader implements OrcValueReader { - private static final StringReader INSTANCE = new StringReader(); - - @Override - public StringData nonNullRead(ColumnVector vector, int row) { - BytesColumnVector bytesVector = (BytesColumnVector) vector; - return StringData.fromBytes( - bytesVector.vector[row], bytesVector.start[row], bytesVector.length[row]); - } - } - - private static class DateReader implements OrcValueReader { - private static final DateReader INSTANCE = new DateReader(); - - @Override - public Integer nonNullRead(ColumnVector vector, int row) { - return (int) ((LongColumnVector) vector).vector[row]; - } - } - - private static class Decimal18Reader implements OrcValueReader { - private final int precision; - private final int scale; - - Decimal18Reader(int precision, int scale) { - this.precision = precision; - this.scale = scale; - } - - @Override - public DecimalData nonNullRead(ColumnVector vector, int row) { - HiveDecimalWritable value = ((DecimalColumnVector) vector).vector[row]; - - // The hive ORC writer may will adjust the scale of decimal data. - Preconditions.checkArgument( - value.precision() <= precision, - "Cannot read value as decimal(%s,%s), too large: %s", - precision, - scale, - value); - - return DecimalData.fromUnscaledLong(value.serialize64(scale), precision, scale); - } - } - - private static class Decimal38Reader implements OrcValueReader { - private final int precision; - private final int scale; - - Decimal38Reader(int precision, int scale) { - this.precision = precision; - this.scale = scale; - } - - @Override - public DecimalData nonNullRead(ColumnVector vector, int row) { - BigDecimal value = - ((DecimalColumnVector) vector).vector[row].getHiveDecimal().bigDecimalValue(); - - Preconditions.checkArgument( - value.precision() <= precision, - "Cannot read value as decimal(%s,%s), too large: %s", - precision, - scale, - value); - - return DecimalData.fromBigDecimal(value, precision, scale); - } - } - - private static class TimeReader implements OrcValueReader { - private static final TimeReader INSTANCE = new TimeReader(); - - @Override - public Integer nonNullRead(ColumnVector vector, int row) { - long micros = ((LongColumnVector) vector).vector[row]; - // Flink only support time mills, just erase micros. - return (int) (micros / 1000); - } - } - - private static class TimestampReader implements OrcValueReader { - private static final TimestampReader INSTANCE = new TimestampReader(); - - @Override - public TimestampData nonNullRead(ColumnVector vector, int row) { - TimestampColumnVector tcv = (TimestampColumnVector) vector; - LocalDateTime localDate = - Instant.ofEpochSecond(Math.floorDiv(tcv.time[row], 1_000), tcv.nanos[row]) - .atOffset(ZoneOffset.UTC) - .toLocalDateTime(); - return TimestampData.fromLocalDateTime(localDate); - } - } - - private static class TimestampTzReader implements OrcValueReader { - private static final TimestampTzReader INSTANCE = new TimestampTzReader(); - - @Override - public TimestampData nonNullRead(ColumnVector vector, int row) { - TimestampColumnVector tcv = (TimestampColumnVector) vector; - Instant instant = - Instant.ofEpochSecond(Math.floorDiv(tcv.time[row], 1_000), tcv.nanos[row]) - .atOffset(ZoneOffset.UTC) - .toInstant(); - return TimestampData.fromInstant(instant); - } - } - - private static class ArrayReader implements OrcValueReader { - private final OrcValueReader elementReader; - - private ArrayReader(OrcValueReader elementReader) { - this.elementReader = elementReader; - } - - @Override - public ArrayData nonNullRead(ColumnVector vector, int row) { - ListColumnVector listVector = (ListColumnVector) vector; - int offset = (int) listVector.offsets[row]; - int length = (int) listVector.lengths[row]; - List elements = Lists.newArrayListWithExpectedSize(length); - for (int c = 0; c < length; ++c) { - elements.add(elementReader.read(listVector.child, offset + c)); - } - return new GenericArrayData(elements.toArray()); - } - - @Override - public void setBatchContext(long batchOffsetInFile) { - elementReader.setBatchContext(batchOffsetInFile); - } - } - - private static class MapReader implements OrcValueReader { - private final OrcValueReader keyReader; - private final OrcValueReader valueReader; - - private MapReader(OrcValueReader keyReader, OrcValueReader valueReader) { - this.keyReader = keyReader; - this.valueReader = valueReader; - } - - @Override - public MapData nonNullRead(ColumnVector vector, int row) { - MapColumnVector mapVector = (MapColumnVector) vector; - int offset = (int) mapVector.offsets[row]; - long length = mapVector.lengths[row]; - - Map map = Maps.newHashMap(); - for (int c = 0; c < length; c++) { - K key = keyReader.read(mapVector.keys, offset + c); - V value = valueReader.read(mapVector.values, offset + c); - map.put(key, value); - } - - return new GenericMapData(map); - } - - @Override - public void setBatchContext(long batchOffsetInFile) { - keyReader.setBatchContext(batchOffsetInFile); - valueReader.setBatchContext(batchOffsetInFile); - } - } - - private static class StructReader extends OrcValueReaders.StructReader { - private final int numFields; - - StructReader( - List> readers, Types.StructType struct, Map idToConstant) { - super(readers, struct, idToConstant); - this.numFields = struct.fields().size(); - } - - @Override - protected RowData create() { - return new GenericRowData(numFields); - } - - @Override - protected void set(RowData struct, int pos, Object value) { - ((GenericRowData) struct).setField(pos, value); - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java deleted file mode 100644 index 6a31accffd22..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java +++ /dev/null @@ -1,163 +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.iceberg.flink.data; - -import java.util.Deque; -import java.util.List; -import java.util.stream.Stream; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FieldMetrics; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.orc.GenericOrcWriters; -import org.apache.iceberg.orc.OrcRowWriter; -import org.apache.iceberg.orc.OrcValueWriter; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; -import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; - -public class FlinkOrcWriter implements OrcRowWriter { - private final FlinkOrcWriters.RowDataWriter writer; - - private FlinkOrcWriter(RowType rowType, Schema iSchema) { - this.writer = - (FlinkOrcWriters.RowDataWriter) - FlinkSchemaVisitor.visit(rowType, iSchema, new WriteBuilder()); - } - - public static OrcRowWriter buildWriter(RowType rowType, Schema iSchema) { - return new FlinkOrcWriter(rowType, iSchema); - } - - @Override - public void write(RowData row, VectorizedRowBatch output) { - Preconditions.checkArgument(row != null, "value must not be null"); - writer.writeRow(row, output); - } - - @Override - public List> writers() { - return writer.writers(); - } - - @Override - public Stream> metrics() { - return writer.metrics(); - } - - private static class WriteBuilder extends FlinkSchemaVisitor> { - private final Deque fieldIds = Lists.newLinkedList(); - - private WriteBuilder() {} - - @Override - public void beforeField(Types.NestedField field) { - fieldIds.push(field.fieldId()); - } - - @Override - public void afterField(Types.NestedField field) { - fieldIds.pop(); - } - - @Override - public OrcValueWriter record( - Types.StructType iStruct, List> results, List fieldType) { - return FlinkOrcWriters.struct(results, fieldType); - } - - @Override - public OrcValueWriter map( - Types.MapType iMap, - OrcValueWriter key, - OrcValueWriter value, - LogicalType keyType, - LogicalType valueType) { - return FlinkOrcWriters.map(key, value, keyType, valueType); - } - - @Override - public OrcValueWriter list( - Types.ListType iList, OrcValueWriter element, LogicalType elementType) { - return FlinkOrcWriters.list(element, elementType); - } - - @Override - public OrcValueWriter primitive(Type.PrimitiveType iPrimitive, LogicalType flinkPrimitive) { - switch (iPrimitive.typeId()) { - case BOOLEAN: - return GenericOrcWriters.booleans(); - case INTEGER: - switch (flinkPrimitive.getTypeRoot()) { - case TINYINT: - return GenericOrcWriters.bytes(); - case SMALLINT: - return GenericOrcWriters.shorts(); - } - return GenericOrcWriters.ints(); - case LONG: - return GenericOrcWriters.longs(); - case FLOAT: - Preconditions.checkArgument( - fieldIds.peek() != null, - String.format( - "[BUG] Cannot find field id for primitive field with type %s. This is likely because id " - + "information is not properly pushed during schema visiting.", - iPrimitive)); - return GenericOrcWriters.floats(fieldIds.peek()); - case DOUBLE: - Preconditions.checkArgument( - fieldIds.peek() != null, - String.format( - "[BUG] Cannot find field id for primitive field with type %s. This is likely because id " - + "information is not properly pushed during schema visiting.", - iPrimitive)); - return GenericOrcWriters.doubles(fieldIds.peek()); - case DATE: - return FlinkOrcWriters.dates(); - case TIME: - return FlinkOrcWriters.times(); - case TIMESTAMP: - Types.TimestampType timestampType = (Types.TimestampType) iPrimitive; - if (timestampType.shouldAdjustToUTC()) { - return FlinkOrcWriters.timestampTzs(); - } else { - return FlinkOrcWriters.timestamps(); - } - case STRING: - return FlinkOrcWriters.strings(); - case UUID: - case FIXED: - case BINARY: - return GenericOrcWriters.byteArrays(); - case DECIMAL: - Types.DecimalType decimalType = (Types.DecimalType) iPrimitive; - return FlinkOrcWriters.decimals(decimalType.precision(), decimalType.scale()); - default: - throw new IllegalArgumentException( - String.format( - "Invalid iceberg type %s corresponding to Flink logical type %s", - iPrimitive, flinkPrimitive)); - } - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java deleted file mode 100644 index da2f95cf822f..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java +++ /dev/null @@ -1,317 +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.iceberg.flink.data; - -import java.time.Instant; -import java.time.OffsetDateTime; -import java.time.ZoneOffset; -import java.util.List; -import java.util.stream.Stream; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.iceberg.FieldMetrics; -import org.apache.iceberg.data.orc.GenericOrcWriters; -import org.apache.iceberg.orc.OrcValueWriter; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.orc.storage.common.type.HiveDecimal; -import org.apache.orc.storage.ql.exec.vector.BytesColumnVector; -import org.apache.orc.storage.ql.exec.vector.ColumnVector; -import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector; -import org.apache.orc.storage.ql.exec.vector.ListColumnVector; -import org.apache.orc.storage.ql.exec.vector.LongColumnVector; -import org.apache.orc.storage.ql.exec.vector.MapColumnVector; -import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector; - -class FlinkOrcWriters { - - private FlinkOrcWriters() {} - - static OrcValueWriter strings() { - return StringWriter.INSTANCE; - } - - static OrcValueWriter dates() { - return DateWriter.INSTANCE; - } - - static OrcValueWriter times() { - return TimeWriter.INSTANCE; - } - - static OrcValueWriter timestamps() { - return TimestampWriter.INSTANCE; - } - - static OrcValueWriter timestampTzs() { - return TimestampTzWriter.INSTANCE; - } - - static OrcValueWriter decimals(int precision, int scale) { - if (precision <= 18) { - return new Decimal18Writer(precision, scale); - } else if (precision <= 38) { - return new Decimal38Writer(precision, scale); - } else { - throw new IllegalArgumentException("Invalid precision: " + precision); - } - } - - static OrcValueWriter list( - OrcValueWriter elementWriter, LogicalType elementType) { - return new ListWriter<>(elementWriter, elementType); - } - - static OrcValueWriter map( - OrcValueWriter keyWriter, - OrcValueWriter valueWriter, - LogicalType keyType, - LogicalType valueType) { - return new MapWriter<>(keyWriter, valueWriter, keyType, valueType); - } - - static OrcValueWriter struct(List> writers, List types) { - return new RowDataWriter(writers, types); - } - - private static class StringWriter implements OrcValueWriter { - private static final StringWriter INSTANCE = new StringWriter(); - - @Override - public void nonNullWrite(int rowId, StringData data, ColumnVector output) { - byte[] value = data.toBytes(); - ((BytesColumnVector) output).setRef(rowId, value, 0, value.length); - } - } - - private static class DateWriter implements OrcValueWriter { - private static final DateWriter INSTANCE = new DateWriter(); - - @Override - public void nonNullWrite(int rowId, Integer data, ColumnVector output) { - ((LongColumnVector) output).vector[rowId] = data; - } - } - - private static class TimeWriter implements OrcValueWriter { - private static final TimeWriter INSTANCE = new TimeWriter(); - - @Override - public void nonNullWrite(int rowId, Integer millis, ColumnVector output) { - // The time in flink is in millisecond, while the standard time in iceberg is microsecond. - // So we need to transform it to microsecond. - ((LongColumnVector) output).vector[rowId] = millis * 1000L; - } - } - - private static class TimestampWriter implements OrcValueWriter { - private static final TimestampWriter INSTANCE = new TimestampWriter(); - - @Override - public void nonNullWrite(int rowId, TimestampData data, ColumnVector output) { - TimestampColumnVector cv = (TimestampColumnVector) output; - cv.setIsUTC(true); - // millis - OffsetDateTime offsetDateTime = data.toInstant().atOffset(ZoneOffset.UTC); - cv.time[rowId] = - offsetDateTime.toEpochSecond() * 1_000 + offsetDateTime.getNano() / 1_000_000; - // truncate nanos to only keep microsecond precision. - cv.nanos[rowId] = (offsetDateTime.getNano() / 1_000) * 1_000; - } - } - - private static class TimestampTzWriter implements OrcValueWriter { - private static final TimestampTzWriter INSTANCE = new TimestampTzWriter(); - - @SuppressWarnings("JavaInstantGetSecondsGetNano") - @Override - public void nonNullWrite(int rowId, TimestampData data, ColumnVector output) { - TimestampColumnVector cv = (TimestampColumnVector) output; - // millis - Instant instant = data.toInstant(); - cv.time[rowId] = instant.toEpochMilli(); - // truncate nanos to only keep microsecond precision. - cv.nanos[rowId] = (instant.getNano() / 1_000) * 1_000; - } - } - - private static class Decimal18Writer implements OrcValueWriter { - private final int precision; - private final int scale; - - Decimal18Writer(int precision, int scale) { - this.precision = precision; - this.scale = scale; - } - - @Override - public void nonNullWrite(int rowId, DecimalData data, ColumnVector output) { - Preconditions.checkArgument( - scale == data.scale(), - "Cannot write value as decimal(%s,%s), wrong scale: %s", - precision, - scale, - data); - Preconditions.checkArgument( - data.precision() <= precision, - "Cannot write value as decimal(%s,%s), too large: %s", - precision, - scale, - data); - - ((DecimalColumnVector) output) - .vector[rowId].setFromLongAndScale(data.toUnscaledLong(), data.scale()); - } - } - - private static class Decimal38Writer implements OrcValueWriter { - private final int precision; - private final int scale; - - Decimal38Writer(int precision, int scale) { - this.precision = precision; - this.scale = scale; - } - - @Override - public void nonNullWrite(int rowId, DecimalData data, ColumnVector output) { - Preconditions.checkArgument( - scale == data.scale(), - "Cannot write value as decimal(%s,%s), wrong scale: %s", - precision, - scale, - data); - Preconditions.checkArgument( - data.precision() <= precision, - "Cannot write value as decimal(%s,%s), too large: %s", - precision, - scale, - data); - - ((DecimalColumnVector) output) - .vector[rowId].set(HiveDecimal.create(data.toBigDecimal(), false)); - } - } - - static class ListWriter implements OrcValueWriter { - private final OrcValueWriter elementWriter; - private final ArrayData.ElementGetter elementGetter; - - ListWriter(OrcValueWriter elementWriter, LogicalType elementType) { - this.elementWriter = elementWriter; - this.elementGetter = ArrayData.createElementGetter(elementType); - } - - @Override - @SuppressWarnings("unchecked") - public void nonNullWrite(int rowId, ArrayData data, ColumnVector output) { - ListColumnVector cv = (ListColumnVector) output; - cv.lengths[rowId] = data.size(); - cv.offsets[rowId] = cv.childCount; - cv.childCount = (int) (cv.childCount + cv.lengths[rowId]); - // make sure the child is big enough. - growColumnVector(cv.child, cv.childCount); - - for (int e = 0; e < cv.lengths[rowId]; ++e) { - Object value = elementGetter.getElementOrNull(data, e); - elementWriter.write((int) (e + cv.offsets[rowId]), (T) value, cv.child); - } - } - - @Override - public Stream> metrics() { - return elementWriter.metrics(); - } - } - - static class MapWriter implements OrcValueWriter { - private final OrcValueWriter keyWriter; - private final OrcValueWriter valueWriter; - private final ArrayData.ElementGetter keyGetter; - private final ArrayData.ElementGetter valueGetter; - - MapWriter( - OrcValueWriter keyWriter, - OrcValueWriter valueWriter, - LogicalType keyType, - LogicalType valueType) { - this.keyWriter = keyWriter; - this.valueWriter = valueWriter; - this.keyGetter = ArrayData.createElementGetter(keyType); - this.valueGetter = ArrayData.createElementGetter(valueType); - } - - @Override - @SuppressWarnings("unchecked") - public void nonNullWrite(int rowId, MapData data, ColumnVector output) { - MapColumnVector cv = (MapColumnVector) output; - ArrayData keyArray = data.keyArray(); - ArrayData valArray = data.valueArray(); - - // record the length and start of the list elements - cv.lengths[rowId] = data.size(); - cv.offsets[rowId] = cv.childCount; - cv.childCount = (int) (cv.childCount + cv.lengths[rowId]); - // make sure the child is big enough - growColumnVector(cv.keys, cv.childCount); - growColumnVector(cv.values, cv.childCount); - // Add each element - for (int e = 0; e < cv.lengths[rowId]; ++e) { - int pos = (int) (e + cv.offsets[rowId]); - keyWriter.write(pos, (K) keyGetter.getElementOrNull(keyArray, e), cv.keys); - valueWriter.write(pos, (V) valueGetter.getElementOrNull(valArray, e), cv.values); - } - } - - @Override - public Stream> metrics() { - return Stream.concat(keyWriter.metrics(), valueWriter.metrics()); - } - } - - static class RowDataWriter extends GenericOrcWriters.StructWriter { - private final List fieldGetters; - - RowDataWriter(List> writers, List types) { - super(writers); - - this.fieldGetters = Lists.newArrayListWithExpectedSize(types.size()); - for (int i = 0; i < types.size(); i++) { - fieldGetters.add(RowData.createFieldGetter(types.get(i), i)); - } - } - - @Override - protected Object get(RowData struct, int index) { - return fieldGetters.get(index).getFieldOrNull(struct); - } - } - - private static void growColumnVector(ColumnVector cv, int requestedSize) { - if (cv.isNull.length < requestedSize) { - // Use growth factor of 3 to avoid frequent array allocations - cv.ensureSize(requestedSize * 3, true); - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java deleted file mode 100644 index ab7b1174c9f3..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java +++ /dev/null @@ -1,832 +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.iceberg.flink.data; - -import java.math.BigDecimal; -import java.math.BigInteger; -import java.nio.ByteBuffer; -import java.time.Instant; -import java.time.ZoneOffset; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.GenericArrayData; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RawValueData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.iceberg.MetadataColumns; -import org.apache.iceberg.Schema; -import org.apache.iceberg.parquet.ParquetSchemaUtil; -import org.apache.iceberg.parquet.ParquetValueReader; -import org.apache.iceberg.parquet.ParquetValueReaders; -import org.apache.iceberg.parquet.TypeWithSchemaVisitor; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.ArrayUtil; -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.io.api.Binary; -import org.apache.parquet.schema.GroupType; -import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation; -import org.apache.parquet.schema.MessageType; -import org.apache.parquet.schema.PrimitiveType; -import org.apache.parquet.schema.Type; - -public class FlinkParquetReaders { - private FlinkParquetReaders() {} - - public static ParquetValueReader buildReader( - Schema expectedSchema, MessageType fileSchema) { - return buildReader(expectedSchema, fileSchema, ImmutableMap.of()); - } - - @SuppressWarnings("unchecked") - public static ParquetValueReader buildReader( - Schema expectedSchema, MessageType fileSchema, Map idToConstant) { - return (ParquetValueReader) - TypeWithSchemaVisitor.visit( - expectedSchema.asStruct(), fileSchema, new ReadBuilder(fileSchema, idToConstant)); - } - - private static class ReadBuilder extends TypeWithSchemaVisitor> { - private final MessageType type; - private final Map idToConstant; - - ReadBuilder(MessageType type, Map idToConstant) { - this.type = type; - this.idToConstant = idToConstant; - } - - @Override - public ParquetValueReader message( - Types.StructType expected, MessageType message, List> fieldReaders) { - return struct(expected, message.asGroupType(), fieldReaders); - } - - @Override - public ParquetValueReader struct( - Types.StructType expected, GroupType struct, List> fieldReaders) { - // match the expected struct's order - Map> readersById = Maps.newHashMap(); - Map typesById = Maps.newHashMap(); - Map maxDefinitionLevelsById = Maps.newHashMap(); - List fields = struct.getFields(); - for (int i = 0; i < fields.size(); i += 1) { - Type fieldType = fields.get(i); - if (fieldReaders.get(i) != null) { - int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1; - if (fieldType.getId() != null) { - int id = fieldType.getId().intValue(); - readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i))); - typesById.put(id, fieldType); - if (idToConstant.containsKey(id)) { - maxDefinitionLevelsById.put(id, fieldD); - } - } - } - } - - List expectedFields = - expected != null ? expected.fields() : ImmutableList.of(); - List> reorderedFields = - Lists.newArrayListWithExpectedSize(expectedFields.size()); - List types = Lists.newArrayListWithExpectedSize(expectedFields.size()); - // Defaulting to parent max definition level - int defaultMaxDefinitionLevel = type.getMaxDefinitionLevel(currentPath()); - for (Types.NestedField field : expectedFields) { - int id = field.fieldId(); - if (idToConstant.containsKey(id)) { - // containsKey is used because the constant may be null - int fieldMaxDefinitionLevel = - maxDefinitionLevelsById.getOrDefault(id, defaultMaxDefinitionLevel); - reorderedFields.add( - ParquetValueReaders.constant(idToConstant.get(id), fieldMaxDefinitionLevel)); - types.add(null); - } else if (id == MetadataColumns.ROW_POSITION.fieldId()) { - reorderedFields.add(ParquetValueReaders.position()); - types.add(null); - } else if (id == MetadataColumns.IS_DELETED.fieldId()) { - reorderedFields.add(ParquetValueReaders.constant(false)); - types.add(null); - } else { - ParquetValueReader reader = readersById.get(id); - if (reader != null) { - reorderedFields.add(reader); - types.add(typesById.get(id)); - } else { - reorderedFields.add(ParquetValueReaders.nulls()); - types.add(null); - } - } - } - - return new RowDataReader(types, reorderedFields); - } - - @Override - public ParquetValueReader list( - Types.ListType expectedList, GroupType array, ParquetValueReader elementReader) { - if (expectedList == null) { - return null; - } - - String[] repeatedPath = currentPath(); - - int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1; - int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1; - - Type elementType = ParquetSchemaUtil.determineListElementType(array); - int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1; - - return new ArrayReader<>( - repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader)); - } - - @Override - public ParquetValueReader map( - Types.MapType expectedMap, - GroupType map, - ParquetValueReader keyReader, - ParquetValueReader valueReader) { - if (expectedMap == null) { - return null; - } - - GroupType repeatedKeyValue = map.getFields().get(0).asGroupType(); - String[] repeatedPath = currentPath(); - - int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1; - int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1; - - Type keyType = repeatedKeyValue.getType(0); - int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1; - Type valueType = repeatedKeyValue.getType(1); - int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1; - - return new MapReader<>( - repeatedD, - repeatedR, - ParquetValueReaders.option(keyType, keyD, keyReader), - ParquetValueReaders.option(valueType, valueD, valueReader)); - } - - @Override - @SuppressWarnings("CyclomaticComplexity") - public ParquetValueReader primitive( - org.apache.iceberg.types.Type.PrimitiveType expected, PrimitiveType primitive) { - if (expected == null) { - return null; - } - - ColumnDescriptor desc = type.getColumnDescription(currentPath()); - - if (primitive.getOriginalType() != null) { - switch (primitive.getOriginalType()) { - case ENUM: - case JSON: - case UTF8: - return new StringReader(desc); - case INT_8: - case INT_16: - case INT_32: - if (expected.typeId() == Types.LongType.get().typeId()) { - return new ParquetValueReaders.IntAsLongReader(desc); - } else { - return new ParquetValueReaders.UnboxedReader<>(desc); - } - case TIME_MICROS: - return new LossyMicrosToMillisTimeReader(desc); - case TIME_MILLIS: - return new MillisTimeReader(desc); - case DATE: - case INT_64: - return new ParquetValueReaders.UnboxedReader<>(desc); - case TIMESTAMP_MICROS: - if (((Types.TimestampType) expected).shouldAdjustToUTC()) { - return new MicrosToTimestampTzReader(desc); - } else { - return new MicrosToTimestampReader(desc); - } - case TIMESTAMP_MILLIS: - if (((Types.TimestampType) expected).shouldAdjustToUTC()) { - return new MillisToTimestampTzReader(desc); - } else { - return new MillisToTimestampReader(desc); - } - case DECIMAL: - DecimalLogicalTypeAnnotation decimal = - (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation(); - switch (primitive.getPrimitiveTypeName()) { - case BINARY: - case FIXED_LEN_BYTE_ARRAY: - return new BinaryDecimalReader(desc, decimal.getPrecision(), decimal.getScale()); - case INT64: - return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale()); - case INT32: - return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale()); - default: - throw new UnsupportedOperationException( - "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName()); - } - case BSON: - return new ParquetValueReaders.ByteArrayReader(desc); - default: - throw new UnsupportedOperationException( - "Unsupported logical type: " + primitive.getOriginalType()); - } - } - - switch (primitive.getPrimitiveTypeName()) { - case FIXED_LEN_BYTE_ARRAY: - case BINARY: - return new ParquetValueReaders.ByteArrayReader(desc); - case INT32: - if (expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) { - return new ParquetValueReaders.IntAsLongReader(desc); - } else { - return new ParquetValueReaders.UnboxedReader<>(desc); - } - case FLOAT: - if (expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) { - return new ParquetValueReaders.FloatAsDoubleReader(desc); - } else { - return new ParquetValueReaders.UnboxedReader<>(desc); - } - case BOOLEAN: - case INT64: - case DOUBLE: - return new ParquetValueReaders.UnboxedReader<>(desc); - default: - throw new UnsupportedOperationException("Unsupported type: " + primitive); - } - } - } - - private static class BinaryDecimalReader - extends ParquetValueReaders.PrimitiveReader { - private final int precision; - private final int scale; - - BinaryDecimalReader(ColumnDescriptor desc, int precision, int scale) { - super(desc); - this.precision = precision; - this.scale = scale; - } - - @Override - public DecimalData read(DecimalData ignored) { - Binary binary = column.nextBinary(); - BigDecimal bigDecimal = new BigDecimal(new BigInteger(binary.getBytes()), scale); - // TODO: need a unit test to write-read-validate decimal via FlinkParquetWrite/Reader - return DecimalData.fromBigDecimal(bigDecimal, precision, scale); - } - } - - private static class IntegerDecimalReader - extends ParquetValueReaders.PrimitiveReader { - private final int precision; - private final int scale; - - IntegerDecimalReader(ColumnDescriptor desc, int precision, int scale) { - super(desc); - this.precision = precision; - this.scale = scale; - } - - @Override - public DecimalData read(DecimalData ignored) { - return DecimalData.fromUnscaledLong(column.nextInteger(), precision, scale); - } - } - - private static class LongDecimalReader extends ParquetValueReaders.PrimitiveReader { - private final int precision; - private final int scale; - - LongDecimalReader(ColumnDescriptor desc, int precision, int scale) { - super(desc); - this.precision = precision; - this.scale = scale; - } - - @Override - public DecimalData read(DecimalData ignored) { - return DecimalData.fromUnscaledLong(column.nextLong(), precision, scale); - } - } - - private static class MicrosToTimestampTzReader - extends ParquetValueReaders.UnboxedReader { - MicrosToTimestampTzReader(ColumnDescriptor desc) { - super(desc); - } - - @Override - public TimestampData read(TimestampData ignored) { - long value = readLong(); - return TimestampData.fromLocalDateTime( - Instant.ofEpochSecond( - Math.floorDiv(value, 1000_000), Math.floorMod(value, 1000_000) * 1000) - .atOffset(ZoneOffset.UTC) - .toLocalDateTime()); - } - - @Override - public long readLong() { - return column.nextLong(); - } - } - - private static class MicrosToTimestampReader - extends ParquetValueReaders.UnboxedReader { - MicrosToTimestampReader(ColumnDescriptor desc) { - super(desc); - } - - @Override - public TimestampData read(TimestampData ignored) { - long value = readLong(); - return TimestampData.fromInstant( - Instant.ofEpochSecond( - Math.floorDiv(value, 1000_000), Math.floorMod(value, 1000_000) * 1000)); - } - - @Override - public long readLong() { - return column.nextLong(); - } - } - - private static class MillisToTimestampReader - extends ParquetValueReaders.UnboxedReader { - MillisToTimestampReader(ColumnDescriptor desc) { - super(desc); - } - - @Override - public TimestampData read(TimestampData ignored) { - long millis = readLong(); - return TimestampData.fromEpochMillis(millis); - } - - @Override - public long readLong() { - return column.nextLong(); - } - } - - private static class MillisToTimestampTzReader - extends ParquetValueReaders.UnboxedReader { - MillisToTimestampTzReader(ColumnDescriptor desc) { - super(desc); - } - - @Override - public TimestampData read(TimestampData ignored) { - long millis = readLong(); - return TimestampData.fromLocalDateTime( - Instant.ofEpochMilli(millis).atOffset(ZoneOffset.UTC).toLocalDateTime()); - } - - @Override - public long readLong() { - return column.nextLong(); - } - } - - private static class StringReader extends ParquetValueReaders.PrimitiveReader { - StringReader(ColumnDescriptor desc) { - super(desc); - } - - @Override - public StringData read(StringData ignored) { - Binary binary = column.nextBinary(); - ByteBuffer buffer = binary.toByteBuffer(); - if (buffer.hasArray()) { - return StringData.fromBytes( - buffer.array(), buffer.arrayOffset() + buffer.position(), buffer.remaining()); - } else { - return StringData.fromBytes(binary.getBytes()); - } - } - } - - private static class LossyMicrosToMillisTimeReader - extends ParquetValueReaders.PrimitiveReader { - LossyMicrosToMillisTimeReader(ColumnDescriptor desc) { - super(desc); - } - - @Override - public Integer read(Integer reuse) { - // Discard microseconds since Flink uses millisecond unit for TIME type. - return (int) Math.floorDiv(column.nextLong(), 1000); - } - } - - private static class MillisTimeReader extends ParquetValueReaders.PrimitiveReader { - MillisTimeReader(ColumnDescriptor desc) { - super(desc); - } - - @Override - public Integer read(Integer reuse) { - return (int) column.nextLong(); - } - } - - private static class ArrayReader - extends ParquetValueReaders.RepeatedReader { - private int readPos = 0; - private int writePos = 0; - - ArrayReader(int definitionLevel, int repetitionLevel, ParquetValueReader reader) { - super(definitionLevel, repetitionLevel, reader); - } - - @Override - protected ReusableArrayData newListData(ArrayData reuse) { - this.readPos = 0; - this.writePos = 0; - - if (reuse instanceof ReusableArrayData) { - return (ReusableArrayData) reuse; - } else { - return new ReusableArrayData(); - } - } - - @Override - @SuppressWarnings("unchecked") - protected E getElement(ReusableArrayData list) { - E value = null; - if (readPos < list.capacity()) { - value = (E) list.values[readPos]; - } - - readPos += 1; - - return value; - } - - @Override - protected void addElement(ReusableArrayData reused, E element) { - if (writePos >= reused.capacity()) { - reused.grow(); - } - - reused.values[writePos] = element; - - writePos += 1; - } - - @Override - protected ArrayData buildList(ReusableArrayData list) { - // Since ReusableArrayData is not accepted by Flink, use GenericArrayData temporarily to walk - // around it. - // Revert this to use ReusableArrayData once it is fixed in Flink. - // For your reference, https://issues.apache.org/jira/browse/FLINK-25238. - return new GenericArrayData(Arrays.copyOf(list.values, writePos)); - } - } - - private static class MapReader - extends ParquetValueReaders.RepeatedKeyValueReader { - private int readPos = 0; - private int writePos = 0; - - private final ParquetValueReaders.ReusableEntry entry = - new ParquetValueReaders.ReusableEntry<>(); - private final ParquetValueReaders.ReusableEntry nullEntry = - new ParquetValueReaders.ReusableEntry<>(); - - MapReader( - int definitionLevel, - int repetitionLevel, - ParquetValueReader keyReader, - ParquetValueReader valueReader) { - super(definitionLevel, repetitionLevel, keyReader, valueReader); - } - - @Override - protected ReusableMapData newMapData(MapData reuse) { - this.readPos = 0; - this.writePos = 0; - - if (reuse instanceof ReusableMapData) { - return (ReusableMapData) reuse; - } else { - return new ReusableMapData(); - } - } - - @Override - @SuppressWarnings("unchecked") - protected Map.Entry getPair(ReusableMapData map) { - Map.Entry kv = nullEntry; - if (readPos < map.capacity()) { - entry.set((K) map.keys.values[readPos], (V) map.values.values[readPos]); - kv = entry; - } - - readPos += 1; - - return kv; - } - - @Override - protected void addPair(ReusableMapData map, K key, V value) { - if (writePos >= map.capacity()) { - map.grow(); - } - - map.keys.values[writePos] = key; - map.values.values[writePos] = value; - - writePos += 1; - } - - @Override - protected MapData buildMap(ReusableMapData map) { - map.setNumElements(writePos); - return map; - } - } - - private static class RowDataReader - extends ParquetValueReaders.StructReader { - private final int numFields; - - RowDataReader(List types, List> readers) { - super(types, readers); - this.numFields = readers.size(); - } - - @Override - protected GenericRowData newStructData(RowData reuse) { - if (reuse instanceof GenericRowData) { - return (GenericRowData) reuse; - } else { - return new GenericRowData(numFields); - } - } - - @Override - protected Object getField(GenericRowData intermediate, int pos) { - return intermediate.getField(pos); - } - - @Override - protected RowData buildStruct(GenericRowData struct) { - return struct; - } - - @Override - protected void set(GenericRowData row, int pos, Object value) { - row.setField(pos, value); - } - - @Override - protected void setNull(GenericRowData row, int pos) { - row.setField(pos, null); - } - - @Override - protected void setBoolean(GenericRowData row, int pos, boolean value) { - row.setField(pos, value); - } - - @Override - protected void setInteger(GenericRowData row, int pos, int value) { - row.setField(pos, value); - } - - @Override - protected void setLong(GenericRowData row, int pos, long value) { - row.setField(pos, value); - } - - @Override - protected void setFloat(GenericRowData row, int pos, float value) { - row.setField(pos, value); - } - - @Override - protected void setDouble(GenericRowData row, int pos, double value) { - row.setField(pos, value); - } - } - - private static class ReusableMapData implements MapData { - private final ReusableArrayData keys; - private final ReusableArrayData values; - - private int numElements; - - private ReusableMapData() { - this.keys = new ReusableArrayData(); - this.values = new ReusableArrayData(); - } - - private void grow() { - keys.grow(); - values.grow(); - } - - private int capacity() { - return keys.capacity(); - } - - public void setNumElements(int numElements) { - this.numElements = numElements; - keys.setNumElements(numElements); - values.setNumElements(numElements); - } - - @Override - public int size() { - return numElements; - } - - @Override - public ReusableArrayData keyArray() { - return keys; - } - - @Override - public ReusableArrayData valueArray() { - return values; - } - } - - private static class ReusableArrayData implements ArrayData { - private static final Object[] EMPTY = new Object[0]; - - private Object[] values = EMPTY; - private int numElements = 0; - - private void grow() { - if (values.length == 0) { - this.values = new Object[20]; - } else { - Object[] old = values; - this.values = new Object[old.length << 1]; - // copy the old array in case it has values that can be reused - System.arraycopy(old, 0, values, 0, old.length); - } - } - - private int capacity() { - return values.length; - } - - public void setNumElements(int numElements) { - this.numElements = numElements; - } - - @Override - public int size() { - return numElements; - } - - @Override - public boolean isNullAt(int ordinal) { - return null == values[ordinal]; - } - - @Override - public boolean getBoolean(int ordinal) { - return (boolean) values[ordinal]; - } - - @Override - public byte getByte(int ordinal) { - return (byte) values[ordinal]; - } - - @Override - public short getShort(int ordinal) { - return (short) values[ordinal]; - } - - @Override - public int getInt(int ordinal) { - return (int) values[ordinal]; - } - - @Override - public long getLong(int ordinal) { - return (long) values[ordinal]; - } - - @Override - public float getFloat(int ordinal) { - return (float) values[ordinal]; - } - - @Override - public double getDouble(int ordinal) { - return (double) values[ordinal]; - } - - @Override - public StringData getString(int pos) { - return (StringData) values[pos]; - } - - @Override - public DecimalData getDecimal(int pos, int precision, int scale) { - return (DecimalData) values[pos]; - } - - @Override - public TimestampData getTimestamp(int pos, int precision) { - return (TimestampData) values[pos]; - } - - @SuppressWarnings("unchecked") - @Override - public RawValueData getRawValue(int pos) { - return (RawValueData) values[pos]; - } - - @Override - public byte[] getBinary(int ordinal) { - return (byte[]) values[ordinal]; - } - - @Override - public ArrayData getArray(int ordinal) { - return (ArrayData) values[ordinal]; - } - - @Override - public MapData getMap(int ordinal) { - return (MapData) values[ordinal]; - } - - @Override - public RowData getRow(int pos, int numFields) { - return (RowData) values[pos]; - } - - @Override - public boolean[] toBooleanArray() { - return ArrayUtil.toPrimitive((Boolean[]) values); - } - - @Override - public byte[] toByteArray() { - return ArrayUtil.toPrimitive((Byte[]) values); - } - - @Override - public short[] toShortArray() { - return ArrayUtil.toPrimitive((Short[]) values); - } - - @Override - public int[] toIntArray() { - return ArrayUtil.toPrimitive((Integer[]) values); - } - - @Override - public long[] toLongArray() { - return ArrayUtil.toPrimitive((Long[]) values); - } - - @Override - public float[] toFloatArray() { - return ArrayUtil.toPrimitive((Float[]) values); - } - - @Override - public double[] toDoubleArray() { - return ArrayUtil.toPrimitive((Double[]) values); - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java deleted file mode 100644 index db4f1730a134..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java +++ /dev/null @@ -1,504 +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.iceberg.flink.data; - -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.NoSuchElementException; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.types.logical.ArrayType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.MapType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.RowType.RowField; -import org.apache.flink.table.types.logical.SmallIntType; -import org.apache.flink.table.types.logical.TinyIntType; -import org.apache.iceberg.parquet.ParquetValueReaders; -import org.apache.iceberg.parquet.ParquetValueWriter; -import org.apache.iceberg.parquet.ParquetValueWriters; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.util.DecimalUtil; -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.io.api.Binary; -import org.apache.parquet.schema.GroupType; -import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation; -import org.apache.parquet.schema.MessageType; -import org.apache.parquet.schema.PrimitiveType; -import org.apache.parquet.schema.Type; - -public class FlinkParquetWriters { - private FlinkParquetWriters() {} - - @SuppressWarnings("unchecked") - public static ParquetValueWriter buildWriter(LogicalType schema, MessageType type) { - return (ParquetValueWriter) - ParquetWithFlinkSchemaVisitor.visit(schema, type, new WriteBuilder(type)); - } - - private static class WriteBuilder extends ParquetWithFlinkSchemaVisitor> { - private final MessageType type; - - WriteBuilder(MessageType type) { - this.type = type; - } - - @Override - public ParquetValueWriter message( - RowType sStruct, MessageType message, List> fields) { - return struct(sStruct, message.asGroupType(), fields); - } - - @Override - public ParquetValueWriter struct( - RowType sStruct, GroupType struct, List> fieldWriters) { - List fields = struct.getFields(); - List flinkFields = sStruct.getFields(); - List> writers = Lists.newArrayListWithExpectedSize(fieldWriters.size()); - List flinkTypes = Lists.newArrayList(); - for (int i = 0; i < fields.size(); i += 1) { - writers.add(newOption(struct.getType(i), fieldWriters.get(i))); - flinkTypes.add(flinkFields.get(i).getType()); - } - - return new RowDataWriter(writers, flinkTypes); - } - - @Override - public ParquetValueWriter list( - ArrayType sArray, GroupType array, ParquetValueWriter elementWriter) { - GroupType repeated = array.getFields().get(0).asGroupType(); - String[] repeatedPath = currentPath(); - - int repeatedD = type.getMaxDefinitionLevel(repeatedPath); - int repeatedR = type.getMaxRepetitionLevel(repeatedPath); - - return new ArrayDataWriter<>( - repeatedD, - repeatedR, - newOption(repeated.getType(0), elementWriter), - sArray.getElementType()); - } - - @Override - public ParquetValueWriter map( - MapType sMap, - GroupType map, - ParquetValueWriter keyWriter, - ParquetValueWriter valueWriter) { - GroupType repeatedKeyValue = map.getFields().get(0).asGroupType(); - String[] repeatedPath = currentPath(); - - int repeatedD = type.getMaxDefinitionLevel(repeatedPath); - int repeatedR = type.getMaxRepetitionLevel(repeatedPath); - - return new MapDataWriter<>( - repeatedD, - repeatedR, - newOption(repeatedKeyValue.getType(0), keyWriter), - newOption(repeatedKeyValue.getType(1), valueWriter), - sMap.getKeyType(), - sMap.getValueType()); - } - - private ParquetValueWriter newOption(Type fieldType, ParquetValueWriter writer) { - int maxD = type.getMaxDefinitionLevel(path(fieldType.getName())); - return ParquetValueWriters.option(fieldType, maxD, writer); - } - - @Override - public ParquetValueWriter primitive(LogicalType fType, PrimitiveType primitive) { - ColumnDescriptor desc = type.getColumnDescription(currentPath()); - - if (primitive.getOriginalType() != null) { - switch (primitive.getOriginalType()) { - case ENUM: - case JSON: - case UTF8: - return strings(desc); - case DATE: - case INT_8: - case INT_16: - case INT_32: - return ints(fType, desc); - case INT_64: - return ParquetValueWriters.longs(desc); - case TIME_MICROS: - return timeMicros(desc); - case TIMESTAMP_MICROS: - return timestamps(desc); - case DECIMAL: - DecimalLogicalTypeAnnotation decimal = - (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation(); - switch (primitive.getPrimitiveTypeName()) { - case INT32: - return decimalAsInteger(desc, decimal.getPrecision(), decimal.getScale()); - case INT64: - return decimalAsLong(desc, decimal.getPrecision(), decimal.getScale()); - case BINARY: - case FIXED_LEN_BYTE_ARRAY: - return decimalAsFixed(desc, decimal.getPrecision(), decimal.getScale()); - default: - throw new UnsupportedOperationException( - "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName()); - } - case BSON: - return byteArrays(desc); - default: - throw new UnsupportedOperationException( - "Unsupported logical type: " + primitive.getOriginalType()); - } - } - - switch (primitive.getPrimitiveTypeName()) { - case FIXED_LEN_BYTE_ARRAY: - case BINARY: - return byteArrays(desc); - case BOOLEAN: - return ParquetValueWriters.booleans(desc); - case INT32: - return ints(fType, desc); - case INT64: - return ParquetValueWriters.longs(desc); - case FLOAT: - return ParquetValueWriters.floats(desc); - case DOUBLE: - return ParquetValueWriters.doubles(desc); - default: - throw new UnsupportedOperationException("Unsupported type: " + primitive); - } - } - } - - private static ParquetValueWriters.PrimitiveWriter ints( - LogicalType type, ColumnDescriptor desc) { - if (type instanceof TinyIntType) { - return ParquetValueWriters.tinyints(desc); - } else if (type instanceof SmallIntType) { - return ParquetValueWriters.shorts(desc); - } - return ParquetValueWriters.ints(desc); - } - - private static ParquetValueWriters.PrimitiveWriter strings(ColumnDescriptor desc) { - return new StringDataWriter(desc); - } - - private static ParquetValueWriters.PrimitiveWriter timeMicros(ColumnDescriptor desc) { - return new TimeMicrosWriter(desc); - } - - private static ParquetValueWriters.PrimitiveWriter decimalAsInteger( - ColumnDescriptor desc, int precision, int scale) { - Preconditions.checkArgument( - precision <= 9, - "Cannot write decimal value as integer with precision larger than 9," - + " wrong precision %s", - precision); - return new IntegerDecimalWriter(desc, precision, scale); - } - - private static ParquetValueWriters.PrimitiveWriter decimalAsLong( - ColumnDescriptor desc, int precision, int scale) { - Preconditions.checkArgument( - precision <= 18, - "Cannot write decimal value as long with precision larger than 18, " - + " wrong precision %s", - precision); - return new LongDecimalWriter(desc, precision, scale); - } - - private static ParquetValueWriters.PrimitiveWriter decimalAsFixed( - ColumnDescriptor desc, int precision, int scale) { - return new FixedDecimalWriter(desc, precision, scale); - } - - private static ParquetValueWriters.PrimitiveWriter timestamps( - ColumnDescriptor desc) { - return new TimestampDataWriter(desc); - } - - private static ParquetValueWriters.PrimitiveWriter byteArrays(ColumnDescriptor desc) { - return new ByteArrayWriter(desc); - } - - private static class StringDataWriter extends ParquetValueWriters.PrimitiveWriter { - private StringDataWriter(ColumnDescriptor desc) { - super(desc); - } - - @Override - public void write(int repetitionLevel, StringData value) { - column.writeBinary(repetitionLevel, Binary.fromReusedByteArray(value.toBytes())); - } - } - - private static class TimeMicrosWriter extends ParquetValueWriters.PrimitiveWriter { - private TimeMicrosWriter(ColumnDescriptor desc) { - super(desc); - } - - @Override - public void write(int repetitionLevel, Integer value) { - long micros = value.longValue() * 1000; - column.writeLong(repetitionLevel, micros); - } - } - - private static class IntegerDecimalWriter - extends ParquetValueWriters.PrimitiveWriter { - private final int precision; - private final int scale; - - private IntegerDecimalWriter(ColumnDescriptor desc, int precision, int scale) { - super(desc); - this.precision = precision; - this.scale = scale; - } - - @Override - public void write(int repetitionLevel, DecimalData decimal) { - Preconditions.checkArgument( - decimal.scale() == scale, - "Cannot write value as decimal(%s,%s), wrong scale: %s", - precision, - scale, - decimal); - Preconditions.checkArgument( - decimal.precision() <= precision, - "Cannot write value as decimal(%s,%s), too large: %s", - precision, - scale, - decimal); - - column.writeInteger(repetitionLevel, (int) decimal.toUnscaledLong()); - } - } - - private static class LongDecimalWriter extends ParquetValueWriters.PrimitiveWriter { - private final int precision; - private final int scale; - - private LongDecimalWriter(ColumnDescriptor desc, int precision, int scale) { - super(desc); - this.precision = precision; - this.scale = scale; - } - - @Override - public void write(int repetitionLevel, DecimalData decimal) { - Preconditions.checkArgument( - decimal.scale() == scale, - "Cannot write value as decimal(%s,%s), wrong scale: %s", - precision, - scale, - decimal); - Preconditions.checkArgument( - decimal.precision() <= precision, - "Cannot write value as decimal(%s,%s), too large: %s", - precision, - scale, - decimal); - - column.writeLong(repetitionLevel, decimal.toUnscaledLong()); - } - } - - private static class FixedDecimalWriter extends ParquetValueWriters.PrimitiveWriter { - private final int precision; - private final int scale; - private final ThreadLocal bytes; - - private FixedDecimalWriter(ColumnDescriptor desc, int precision, int scale) { - super(desc); - this.precision = precision; - this.scale = scale; - this.bytes = - ThreadLocal.withInitial(() -> new byte[TypeUtil.decimalRequiredBytes(precision)]); - } - - @Override - public void write(int repetitionLevel, DecimalData decimal) { - byte[] binary = - DecimalUtil.toReusedFixLengthBytes(precision, scale, decimal.toBigDecimal(), bytes.get()); - column.writeBinary(repetitionLevel, Binary.fromReusedByteArray(binary)); - } - } - - private static class TimestampDataWriter - extends ParquetValueWriters.PrimitiveWriter { - private TimestampDataWriter(ColumnDescriptor desc) { - super(desc); - } - - @Override - public void write(int repetitionLevel, TimestampData value) { - column.writeLong( - repetitionLevel, value.getMillisecond() * 1000 + value.getNanoOfMillisecond() / 1000); - } - } - - private static class ByteArrayWriter extends ParquetValueWriters.PrimitiveWriter { - private ByteArrayWriter(ColumnDescriptor desc) { - super(desc); - } - - @Override - public void write(int repetitionLevel, byte[] bytes) { - column.writeBinary(repetitionLevel, Binary.fromReusedByteArray(bytes)); - } - } - - private static class ArrayDataWriter extends ParquetValueWriters.RepeatedWriter { - private final LogicalType elementType; - - private ArrayDataWriter( - int definitionLevel, - int repetitionLevel, - ParquetValueWriter writer, - LogicalType elementType) { - super(definitionLevel, repetitionLevel, writer); - this.elementType = elementType; - } - - @Override - protected Iterator elements(ArrayData list) { - return new ElementIterator<>(list); - } - - private class ElementIterator implements Iterator { - private final int size; - private final ArrayData list; - private final ArrayData.ElementGetter getter; - private int index; - - private ElementIterator(ArrayData list) { - this.list = list; - size = list.size(); - getter = ArrayData.createElementGetter(elementType); - index = 0; - } - - @Override - public boolean hasNext() { - return index != size; - } - - @Override - @SuppressWarnings("unchecked") - public E next() { - if (index >= size) { - throw new NoSuchElementException(); - } - - E element = (E) getter.getElementOrNull(list, index); - index += 1; - - return element; - } - } - } - - private static class MapDataWriter - extends ParquetValueWriters.RepeatedKeyValueWriter { - private final LogicalType keyType; - private final LogicalType valueType; - - private MapDataWriter( - int definitionLevel, - int repetitionLevel, - ParquetValueWriter keyWriter, - ParquetValueWriter valueWriter, - LogicalType keyType, - LogicalType valueType) { - super(definitionLevel, repetitionLevel, keyWriter, valueWriter); - this.keyType = keyType; - this.valueType = valueType; - } - - @Override - protected Iterator> pairs(MapData map) { - return new EntryIterator<>(map); - } - - private class EntryIterator implements Iterator> { - private final int size; - private final ArrayData keys; - private final ArrayData values; - private final ParquetValueReaders.ReusableEntry entry; - private final ArrayData.ElementGetter keyGetter; - private final ArrayData.ElementGetter valueGetter; - private int index; - - private EntryIterator(MapData map) { - size = map.size(); - keys = map.keyArray(); - values = map.valueArray(); - entry = new ParquetValueReaders.ReusableEntry<>(); - keyGetter = ArrayData.createElementGetter(keyType); - valueGetter = ArrayData.createElementGetter(valueType); - index = 0; - } - - @Override - public boolean hasNext() { - return index != size; - } - - @Override - @SuppressWarnings("unchecked") - public Map.Entry next() { - if (index >= size) { - throw new NoSuchElementException(); - } - - entry.set( - (K) keyGetter.getElementOrNull(keys, index), - (V) valueGetter.getElementOrNull(values, index)); - index += 1; - - return entry; - } - } - } - - private static class RowDataWriter extends ParquetValueWriters.StructWriter { - private final RowData.FieldGetter[] fieldGetter; - - RowDataWriter(List> writers, List types) { - super(writers); - fieldGetter = new RowData.FieldGetter[types.size()]; - for (int i = 0; i < types.size(); i += 1) { - fieldGetter[i] = RowData.createFieldGetter(types.get(i), i); - } - } - - @Override - protected Object get(RowData struct, int index) { - return fieldGetter[index].getFieldOrNull(struct); - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java deleted file mode 100644 index ba4e1a7a7aec..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java +++ /dev/null @@ -1,161 +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.iceberg.flink.data; - -import java.util.List; -import org.apache.flink.table.types.logical.ArrayType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.MapType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.Schema; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; - -abstract class FlinkSchemaVisitor { - - static T visit(RowType flinkType, Schema schema, FlinkSchemaVisitor visitor) { - return visit(flinkType, schema.asStruct(), visitor); - } - - private static T visit(LogicalType flinkType, Type iType, FlinkSchemaVisitor visitor) { - switch (iType.typeId()) { - case STRUCT: - return visitRecord(flinkType, iType.asStructType(), visitor); - - case MAP: - MapType mapType = (MapType) flinkType; - Types.MapType iMapType = iType.asMapType(); - T key; - T value; - - Types.NestedField keyField = iMapType.field(iMapType.keyId()); - visitor.beforeMapKey(keyField); - try { - key = visit(mapType.getKeyType(), iMapType.keyType(), visitor); - } finally { - visitor.afterMapKey(keyField); - } - - Types.NestedField valueField = iMapType.field(iMapType.valueId()); - visitor.beforeMapValue(valueField); - try { - value = visit(mapType.getValueType(), iMapType.valueType(), visitor); - } finally { - visitor.afterMapValue(valueField); - } - - return visitor.map(iMapType, key, value, mapType.getKeyType(), mapType.getValueType()); - - case LIST: - ArrayType listType = (ArrayType) flinkType; - Types.ListType iListType = iType.asListType(); - T element; - - Types.NestedField elementField = iListType.field(iListType.elementId()); - visitor.beforeListElement(elementField); - try { - element = visit(listType.getElementType(), iListType.elementType(), visitor); - } finally { - visitor.afterListElement(elementField); - } - - return visitor.list(iListType, element, listType.getElementType()); - - default: - return visitor.primitive(iType.asPrimitiveType(), flinkType); - } - } - - private static T visitRecord( - LogicalType flinkType, Types.StructType struct, FlinkSchemaVisitor visitor) { - Preconditions.checkArgument(flinkType instanceof RowType, "%s is not a RowType.", flinkType); - RowType rowType = (RowType) flinkType; - - int fieldSize = struct.fields().size(); - List results = Lists.newArrayListWithExpectedSize(fieldSize); - List fieldTypes = Lists.newArrayListWithExpectedSize(fieldSize); - List nestedFields = struct.fields(); - - for (int i = 0; i < fieldSize; i++) { - Types.NestedField iField = nestedFields.get(i); - int fieldIndex = rowType.getFieldIndex(iField.name()); - Preconditions.checkArgument( - fieldIndex >= 0, "NestedField: %s is not found in flink RowType: %s", iField, rowType); - - LogicalType fieldFlinkType = rowType.getTypeAt(fieldIndex); - - fieldTypes.add(fieldFlinkType); - - visitor.beforeField(iField); - try { - results.add(visit(fieldFlinkType, iField.type(), visitor)); - } finally { - visitor.afterField(iField); - } - } - - return visitor.record(struct, results, fieldTypes); - } - - public T record(Types.StructType iStruct, List results, List fieldTypes) { - return null; - } - - public T list(Types.ListType iList, T element, LogicalType elementType) { - return null; - } - - public T map(Types.MapType iMap, T key, T value, LogicalType keyType, LogicalType valueType) { - return null; - } - - public T primitive(Type.PrimitiveType iPrimitive, LogicalType flinkPrimitive) { - return null; - } - - public void beforeField(Types.NestedField field) {} - - public void afterField(Types.NestedField field) {} - - public void beforeListElement(Types.NestedField elementField) { - beforeField(elementField); - } - - public void afterListElement(Types.NestedField elementField) { - afterField(elementField); - } - - public void beforeMapKey(Types.NestedField keyField) { - beforeField(keyField); - } - - public void afterMapKey(Types.NestedField keyField) { - afterField(keyField); - } - - public void beforeMapValue(Types.NestedField valueField) { - beforeField(valueField); - } - - public void afterMapValue(Types.NestedField valueField) { - afterField(valueField); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java deleted file mode 100644 index 32f6c3a2ccfd..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java +++ /dev/null @@ -1,312 +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.iceberg.flink.data; - -import java.io.IOException; -import java.math.BigDecimal; -import java.math.BigInteger; -import java.nio.charset.StandardCharsets; -import java.util.List; -import java.util.Map; -import org.apache.avro.io.Decoder; -import org.apache.avro.util.Utf8; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.GenericArrayData; -import org.apache.flink.table.data.GenericMapData; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.iceberg.avro.ValueReader; -import org.apache.iceberg.avro.ValueReaders; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Types; - -public class FlinkValueReaders { - - private FlinkValueReaders() {} - - static ValueReader strings() { - return StringReader.INSTANCE; - } - - static ValueReader enums(List symbols) { - return new EnumReader(symbols); - } - - static ValueReader uuids() { - return ValueReaders.fixed(16); - } - - static ValueReader timeMicros() { - return TimeMicrosReader.INSTANCE; - } - - static ValueReader timestampMills() { - return TimestampMillsReader.INSTANCE; - } - - static ValueReader timestampMicros() { - return TimestampMicrosReader.INSTANCE; - } - - static ValueReader decimal( - ValueReader unscaledReader, int precision, int scale) { - return new DecimalReader(unscaledReader, precision, scale); - } - - static ValueReader array(ValueReader elementReader) { - return new ArrayReader(elementReader); - } - - static ValueReader arrayMap(ValueReader keyReader, ValueReader valueReader) { - return new ArrayMapReader(keyReader, valueReader); - } - - static ValueReader map(ValueReader keyReader, ValueReader valueReader) { - return new MapReader(keyReader, valueReader); - } - - static ValueReader struct( - List> readers, Types.StructType struct, Map idToConstant) { - return new StructReader(readers, struct, idToConstant); - } - - private static class StringReader implements ValueReader { - private static final StringReader INSTANCE = new StringReader(); - - private StringReader() {} - - @Override - public StringData read(Decoder decoder, Object reuse) throws IOException { - // use the decoder's readString(Utf8) method because it may be a resolving decoder - Utf8 utf8 = null; - if (reuse instanceof StringData) { - utf8 = new Utf8(((StringData) reuse).toBytes()); - } - - Utf8 string = decoder.readString(utf8); - return StringData.fromBytes(string.getBytes(), 0, string.getByteLength()); - } - } - - private static class EnumReader implements ValueReader { - private final StringData[] symbols; - - private EnumReader(List symbols) { - this.symbols = new StringData[symbols.size()]; - for (int i = 0; i < this.symbols.length; i += 1) { - this.symbols[i] = StringData.fromBytes(symbols.get(i).getBytes(StandardCharsets.UTF_8)); - } - } - - @Override - public StringData read(Decoder decoder, Object ignore) throws IOException { - int index = decoder.readEnum(); - return symbols[index]; - } - } - - private static class DecimalReader implements ValueReader { - private final ValueReader bytesReader; - private final int precision; - private final int scale; - - private DecimalReader(ValueReader bytesReader, int precision, int scale) { - this.bytesReader = bytesReader; - this.precision = precision; - this.scale = scale; - } - - @Override - public DecimalData read(Decoder decoder, Object reuse) throws IOException { - byte[] bytes = bytesReader.read(decoder, null); - return DecimalData.fromBigDecimal( - new BigDecimal(new BigInteger(bytes), scale), precision, scale); - } - } - - private static class TimeMicrosReader implements ValueReader { - private static final TimeMicrosReader INSTANCE = new TimeMicrosReader(); - - @Override - public Integer read(Decoder decoder, Object reuse) throws IOException { - long micros = decoder.readLong(); - // Flink only support time mills, just erase micros. - return (int) (micros / 1000); - } - } - - private static class TimestampMillsReader implements ValueReader { - private static final TimestampMillsReader INSTANCE = new TimestampMillsReader(); - - @Override - public TimestampData read(Decoder decoder, Object reuse) throws IOException { - return TimestampData.fromEpochMillis(decoder.readLong()); - } - } - - private static class TimestampMicrosReader implements ValueReader { - private static final TimestampMicrosReader INSTANCE = new TimestampMicrosReader(); - - @Override - public TimestampData read(Decoder decoder, Object reuse) throws IOException { - long micros = decoder.readLong(); - long mills = micros / 1000; - int nanos = ((int) (micros % 1000)) * 1000; - if (nanos < 0) { - nanos += 1_000_000; - mills -= 1; - } - return TimestampData.fromEpochMillis(mills, nanos); - } - } - - private static class ArrayReader implements ValueReader { - private final ValueReader elementReader; - private final List reusedList = Lists.newArrayList(); - - private ArrayReader(ValueReader elementReader) { - this.elementReader = elementReader; - } - - @Override - public GenericArrayData read(Decoder decoder, Object reuse) throws IOException { - reusedList.clear(); - long chunkLength = decoder.readArrayStart(); - - while (chunkLength > 0) { - for (int i = 0; i < chunkLength; i += 1) { - reusedList.add(elementReader.read(decoder, null)); - } - - chunkLength = decoder.arrayNext(); - } - - // this will convert the list to an array so it is okay to reuse the list - return new GenericArrayData(reusedList.toArray()); - } - } - - private static MapData kvArrayToMap(List keyList, List valueList) { - Map map = Maps.newHashMap(); - Object[] keys = keyList.toArray(); - Object[] values = valueList.toArray(); - for (int i = 0; i < keys.length; i++) { - map.put(keys[i], values[i]); - } - - return new GenericMapData(map); - } - - private static class ArrayMapReader implements ValueReader { - private final ValueReader keyReader; - private final ValueReader valueReader; - - private final List reusedKeyList = Lists.newArrayList(); - private final List reusedValueList = Lists.newArrayList(); - - private ArrayMapReader(ValueReader keyReader, ValueReader valueReader) { - this.keyReader = keyReader; - this.valueReader = valueReader; - } - - @Override - public MapData read(Decoder decoder, Object reuse) throws IOException { - reusedKeyList.clear(); - reusedValueList.clear(); - - long chunkLength = decoder.readArrayStart(); - - while (chunkLength > 0) { - for (int i = 0; i < chunkLength; i += 1) { - reusedKeyList.add(keyReader.read(decoder, null)); - reusedValueList.add(valueReader.read(decoder, null)); - } - - chunkLength = decoder.arrayNext(); - } - - return kvArrayToMap(reusedKeyList, reusedValueList); - } - } - - private static class MapReader implements ValueReader { - private final ValueReader keyReader; - private final ValueReader valueReader; - - private final List reusedKeyList = Lists.newArrayList(); - private final List reusedValueList = Lists.newArrayList(); - - private MapReader(ValueReader keyReader, ValueReader valueReader) { - this.keyReader = keyReader; - this.valueReader = valueReader; - } - - @Override - public MapData read(Decoder decoder, Object reuse) throws IOException { - reusedKeyList.clear(); - reusedValueList.clear(); - - long chunkLength = decoder.readMapStart(); - - while (chunkLength > 0) { - for (int i = 0; i < chunkLength; i += 1) { - reusedKeyList.add(keyReader.read(decoder, null)); - reusedValueList.add(valueReader.read(decoder, null)); - } - - chunkLength = decoder.mapNext(); - } - - return kvArrayToMap(reusedKeyList, reusedValueList); - } - } - - private static class StructReader extends ValueReaders.StructReader { - private final int numFields; - - private StructReader( - List> readers, Types.StructType struct, Map idToConstant) { - super(readers, struct, idToConstant); - this.numFields = readers.size(); - } - - @Override - protected RowData reuseOrCreate(Object reuse) { - if (reuse instanceof GenericRowData && ((GenericRowData) reuse).getArity() == numFields) { - return (GenericRowData) reuse; - } - return new GenericRowData(numFields); - } - - @Override - protected Object get(RowData struct, int pos) { - return null; - } - - @Override - protected void set(RowData struct, int pos, Object value) { - ((GenericRowData) struct).setField(pos, value); - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java deleted file mode 100644 index 4e86ecce28b5..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java +++ /dev/null @@ -1,253 +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.iceberg.flink.data; - -import java.io.IOException; -import java.lang.reflect.Array; -import java.util.List; -import org.apache.avro.io.Encoder; -import org.apache.avro.util.Utf8; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.iceberg.avro.ValueWriter; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.util.DecimalUtil; - -public class FlinkValueWriters { - - private FlinkValueWriters() {} - - static ValueWriter strings() { - return StringWriter.INSTANCE; - } - - static ValueWriter timeMicros() { - return TimeMicrosWriter.INSTANCE; - } - - static ValueWriter timestampMicros() { - return TimestampMicrosWriter.INSTANCE; - } - - static ValueWriter decimal(int precision, int scale) { - return new DecimalWriter(precision, scale); - } - - static ValueWriter array(ValueWriter elementWriter, LogicalType elementType) { - return new ArrayWriter<>(elementWriter, elementType); - } - - static ValueWriter arrayMap( - ValueWriter keyWriter, - LogicalType keyType, - ValueWriter valueWriter, - LogicalType valueType) { - return new ArrayMapWriter<>(keyWriter, keyType, valueWriter, valueType); - } - - static ValueWriter map( - ValueWriter keyWriter, - LogicalType keyType, - ValueWriter valueWriter, - LogicalType valueType) { - return new MapWriter<>(keyWriter, keyType, valueWriter, valueType); - } - - static ValueWriter row(List> writers, List types) { - return new RowWriter(writers, types); - } - - private static class StringWriter implements ValueWriter { - private static final StringWriter INSTANCE = new StringWriter(); - - private StringWriter() {} - - @Override - public void write(StringData s, Encoder encoder) throws IOException { - // toBytes is cheaper than Avro calling toString, which incurs encoding costs - encoder.writeString(new Utf8(s.toBytes())); - } - } - - private static class DecimalWriter implements ValueWriter { - private final int precision; - private final int scale; - private final ThreadLocal bytes; - - private DecimalWriter(int precision, int scale) { - this.precision = precision; - this.scale = scale; - this.bytes = - ThreadLocal.withInitial(() -> new byte[TypeUtil.decimalRequiredBytes(precision)]); - } - - @Override - public void write(DecimalData d, Encoder encoder) throws IOException { - encoder.writeFixed( - DecimalUtil.toReusedFixLengthBytes(precision, scale, d.toBigDecimal(), bytes.get())); - } - } - - private static class TimeMicrosWriter implements ValueWriter { - private static final TimeMicrosWriter INSTANCE = new TimeMicrosWriter(); - - @Override - public void write(Integer timeMills, Encoder encoder) throws IOException { - encoder.writeLong(timeMills * 1000L); - } - } - - private static class TimestampMicrosWriter implements ValueWriter { - private static final TimestampMicrosWriter INSTANCE = new TimestampMicrosWriter(); - - @Override - public void write(TimestampData timestampData, Encoder encoder) throws IOException { - long micros = - timestampData.getMillisecond() * 1000 + timestampData.getNanoOfMillisecond() / 1000; - encoder.writeLong(micros); - } - } - - private static class ArrayWriter implements ValueWriter { - private final ValueWriter elementWriter; - private final ArrayData.ElementGetter elementGetter; - - private ArrayWriter(ValueWriter elementWriter, LogicalType elementType) { - this.elementWriter = elementWriter; - this.elementGetter = ArrayData.createElementGetter(elementType); - } - - @Override - @SuppressWarnings("unchecked") - public void write(ArrayData array, Encoder encoder) throws IOException { - encoder.writeArrayStart(); - int numElements = array.size(); - encoder.setItemCount(numElements); - for (int i = 0; i < numElements; i += 1) { - encoder.startItem(); - elementWriter.write((T) elementGetter.getElementOrNull(array, i), encoder); - } - encoder.writeArrayEnd(); - } - } - - private static class ArrayMapWriter implements ValueWriter { - private final ValueWriter keyWriter; - private final ValueWriter valueWriter; - private final ArrayData.ElementGetter keyGetter; - private final ArrayData.ElementGetter valueGetter; - - private ArrayMapWriter( - ValueWriter keyWriter, - LogicalType keyType, - ValueWriter valueWriter, - LogicalType valueType) { - this.keyWriter = keyWriter; - this.keyGetter = ArrayData.createElementGetter(keyType); - this.valueWriter = valueWriter; - this.valueGetter = ArrayData.createElementGetter(valueType); - } - - @Override - @SuppressWarnings("unchecked") - public void write(MapData map, Encoder encoder) throws IOException { - encoder.writeArrayStart(); - int numElements = map.size(); - encoder.setItemCount(numElements); - ArrayData keyArray = map.keyArray(); - ArrayData valueArray = map.valueArray(); - for (int i = 0; i < numElements; i += 1) { - encoder.startItem(); - keyWriter.write((K) keyGetter.getElementOrNull(keyArray, i), encoder); - valueWriter.write((V) valueGetter.getElementOrNull(valueArray, i), encoder); - } - encoder.writeArrayEnd(); - } - } - - private static class MapWriter implements ValueWriter { - private final ValueWriter keyWriter; - private final ValueWriter valueWriter; - private final ArrayData.ElementGetter keyGetter; - private final ArrayData.ElementGetter valueGetter; - - private MapWriter( - ValueWriter keyWriter, - LogicalType keyType, - ValueWriter valueWriter, - LogicalType valueType) { - this.keyWriter = keyWriter; - this.keyGetter = ArrayData.createElementGetter(keyType); - this.valueWriter = valueWriter; - this.valueGetter = ArrayData.createElementGetter(valueType); - } - - @Override - @SuppressWarnings("unchecked") - public void write(MapData map, Encoder encoder) throws IOException { - encoder.writeMapStart(); - int numElements = map.size(); - encoder.setItemCount(numElements); - ArrayData keyArray = map.keyArray(); - ArrayData valueArray = map.valueArray(); - for (int i = 0; i < numElements; i += 1) { - encoder.startItem(); - keyWriter.write((K) keyGetter.getElementOrNull(keyArray, i), encoder); - valueWriter.write((V) valueGetter.getElementOrNull(valueArray, i), encoder); - } - encoder.writeMapEnd(); - } - } - - static class RowWriter implements ValueWriter { - private final ValueWriter[] writers; - private final RowData.FieldGetter[] getters; - - private RowWriter(List> writers, List types) { - this.writers = (ValueWriter[]) Array.newInstance(ValueWriter.class, writers.size()); - this.getters = new RowData.FieldGetter[writers.size()]; - for (int i = 0; i < writers.size(); i += 1) { - this.writers[i] = writers.get(i); - this.getters[i] = RowData.createFieldGetter(types.get(i), i); - } - } - - @Override - public void write(RowData row, Encoder encoder) throws IOException { - for (int i = 0; i < writers.length; i += 1) { - if (row.isNullAt(i)) { - writers[i].write(null, encoder); - } else { - write(row, i, writers[i], encoder); - } - } - } - - @SuppressWarnings("unchecked") - private void write(RowData row, int pos, ValueWriter writer, Encoder encoder) - throws IOException { - writer.write((T) getters[pos].getFieldOrNull(row), encoder); - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java deleted file mode 100644 index 33feb2e32118..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java +++ /dev/null @@ -1,222 +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.iceberg.flink.data; - -import java.util.Deque; -import java.util.List; -import org.apache.flink.table.types.logical.ArrayType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.MapType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.RowType.RowField; -import org.apache.iceberg.avro.AvroSchemaUtil; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.parquet.schema.GroupType; -import org.apache.parquet.schema.MessageType; -import org.apache.parquet.schema.OriginalType; -import org.apache.parquet.schema.PrimitiveType; -import org.apache.parquet.schema.Type; - -public class ParquetWithFlinkSchemaVisitor { - private final Deque fieldNames = Lists.newLinkedList(); - - public static T visit( - LogicalType sType, Type type, ParquetWithFlinkSchemaVisitor visitor) { - Preconditions.checkArgument(sType != null, "Invalid DataType: null"); - if (type instanceof MessageType) { - Preconditions.checkArgument( - sType instanceof RowType, "Invalid struct: %s is not a struct", sType); - RowType struct = (RowType) sType; - return visitor.message( - struct, (MessageType) type, visitFields(struct, type.asGroupType(), visitor)); - } else if (type.isPrimitive()) { - return visitor.primitive(sType, type.asPrimitiveType()); - } else { - // if not a primitive, the typeId must be a group - GroupType group = type.asGroupType(); - OriginalType annotation = group.getOriginalType(); - if (annotation != null) { - switch (annotation) { - case LIST: - Preconditions.checkArgument( - !group.isRepetition(Type.Repetition.REPEATED), - "Invalid list: top-level group is repeated: %s", - group); - Preconditions.checkArgument( - group.getFieldCount() == 1, - "Invalid list: does not contain single repeated field: %s", - group); - - GroupType repeatedElement = group.getFields().get(0).asGroupType(); - Preconditions.checkArgument( - repeatedElement.isRepetition(Type.Repetition.REPEATED), - "Invalid list: inner group is not repeated"); - Preconditions.checkArgument( - repeatedElement.getFieldCount() <= 1, - "Invalid list: repeated group is not a single field: %s", - group); - - Preconditions.checkArgument( - sType instanceof ArrayType, "Invalid list: %s is not an array", sType); - ArrayType array = (ArrayType) sType; - RowType.RowField element = - new RowField( - "element", array.getElementType(), "element of " + array.asSummaryString()); - - visitor.fieldNames.push(repeatedElement.getName()); - try { - T elementResult = null; - if (repeatedElement.getFieldCount() > 0) { - elementResult = visitField(element, repeatedElement.getType(0), visitor); - } - - return visitor.list(array, group, elementResult); - - } finally { - visitor.fieldNames.pop(); - } - - case MAP: - Preconditions.checkArgument( - !group.isRepetition(Type.Repetition.REPEATED), - "Invalid map: top-level group is repeated: %s", - group); - Preconditions.checkArgument( - group.getFieldCount() == 1, - "Invalid map: does not contain single repeated field: %s", - group); - - GroupType repeatedKeyValue = group.getType(0).asGroupType(); - Preconditions.checkArgument( - repeatedKeyValue.isRepetition(Type.Repetition.REPEATED), - "Invalid map: inner group is not repeated"); - Preconditions.checkArgument( - repeatedKeyValue.getFieldCount() <= 2, - "Invalid map: repeated group does not have 2 fields"); - - Preconditions.checkArgument( - sType instanceof MapType, "Invalid map: %s is not a map", sType); - MapType map = (MapType) sType; - RowField keyField = - new RowField("key", map.getKeyType(), "key of " + map.asSummaryString()); - RowField valueField = - new RowField("value", map.getValueType(), "value of " + map.asSummaryString()); - - visitor.fieldNames.push(repeatedKeyValue.getName()); - try { - T keyResult = null; - T valueResult = null; - switch (repeatedKeyValue.getFieldCount()) { - case 2: - // if there are 2 fields, both key and value are projected - keyResult = visitField(keyField, repeatedKeyValue.getType(0), visitor); - valueResult = visitField(valueField, repeatedKeyValue.getType(1), visitor); - break; - case 1: - // if there is just one, use the name to determine what it is - Type keyOrValue = repeatedKeyValue.getType(0); - if (keyOrValue.getName().equalsIgnoreCase("key")) { - keyResult = visitField(keyField, keyOrValue, visitor); - // value result remains null - } else { - valueResult = visitField(valueField, keyOrValue, visitor); - // key result remains null - } - break; - default: - // both results will remain null - } - - return visitor.map(map, group, keyResult, valueResult); - - } finally { - visitor.fieldNames.pop(); - } - - default: - } - } - Preconditions.checkArgument( - sType instanceof RowType, "Invalid struct: %s is not a struct", sType); - RowType struct = (RowType) sType; - return visitor.struct(struct, group, visitFields(struct, group, visitor)); - } - } - - private static T visitField( - RowType.RowField sField, Type field, ParquetWithFlinkSchemaVisitor visitor) { - visitor.fieldNames.push(field.getName()); - try { - return visit(sField.getType(), field, visitor); - } finally { - visitor.fieldNames.pop(); - } - } - - private static List visitFields( - RowType struct, GroupType group, ParquetWithFlinkSchemaVisitor visitor) { - List sFields = struct.getFields(); - Preconditions.checkArgument( - sFields.size() == group.getFieldCount(), "Structs do not match: %s and %s", struct, group); - List results = Lists.newArrayListWithExpectedSize(group.getFieldCount()); - for (int i = 0; i < sFields.size(); i += 1) { - Type field = group.getFields().get(i); - RowType.RowField sField = sFields.get(i); - Preconditions.checkArgument( - field.getName().equals(AvroSchemaUtil.makeCompatibleName(sField.getName())), - "Structs do not match: field %s != %s", - field.getName(), - sField.getName()); - results.add(visitField(sField, field, visitor)); - } - - return results; - } - - public T message(RowType sStruct, MessageType message, List fields) { - return null; - } - - public T struct(RowType sStruct, GroupType struct, List fields) { - return null; - } - - public T list(ArrayType sArray, GroupType array, T element) { - return null; - } - - public T map(MapType sMap, GroupType map, T key, T value) { - return null; - } - - public T primitive(LogicalType sPrimitive, PrimitiveType primitive) { - return null; - } - - protected String[] currentPath() { - return Lists.newArrayList(fieldNames.descendingIterator()).toArray(new String[0]); - } - - protected String[] path(String name) { - List list = Lists.newArrayList(fieldNames.descendingIterator()); - list.add(name); - return list.toArray(new String[0]); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java deleted file mode 100644 index 33816c97ac29..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java +++ /dev/null @@ -1,341 +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.iceberg.flink.data; - -import java.util.Arrays; -import java.util.Map; -import java.util.Objects; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RawValueData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.types.RowKind; -import org.apache.flink.util.StringUtils; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Types; - -public class RowDataProjection implements RowData { - /** - * Creates a projecting wrapper for {@link RowData} rows. - * - *

    This projection will not project the nested children types of repeated types like lists and - * maps. - * - * @param schema schema of rows wrapped by this projection - * @param projectedSchema result schema of the projected rows - * @return a wrapper to project rows - */ - public static RowDataProjection create(Schema schema, Schema projectedSchema) { - return RowDataProjection.create( - FlinkSchemaUtil.convert(schema), schema.asStruct(), projectedSchema.asStruct()); - } - - /** - * Creates a projecting wrapper for {@link RowData} rows. - * - *

    This projection will not project the nested children types of repeated types like lists and - * maps. - * - * @param rowType flink row type of rows wrapped by this projection - * @param schema schema of rows wrapped by this projection - * @param projectedSchema result schema of the projected rows - * @return a wrapper to project rows - */ - public static RowDataProjection create( - RowType rowType, Types.StructType schema, Types.StructType projectedSchema) { - return new RowDataProjection(rowType, schema, projectedSchema); - } - - private final RowData.FieldGetter[] getters; - private RowData rowData; - - private RowDataProjection( - RowType rowType, Types.StructType rowStruct, Types.StructType projectType) { - Map fieldIdToPosition = Maps.newHashMap(); - for (int i = 0; i < rowStruct.fields().size(); i++) { - fieldIdToPosition.put(rowStruct.fields().get(i).fieldId(), i); - } - - this.getters = new RowData.FieldGetter[projectType.fields().size()]; - for (int i = 0; i < getters.length; i++) { - Types.NestedField projectField = projectType.fields().get(i); - Types.NestedField rowField = rowStruct.field(projectField.fieldId()); - - Preconditions.checkNotNull( - rowField, - "Cannot locate the project field <%s> in the iceberg struct <%s>", - projectField, - rowStruct); - - getters[i] = - createFieldGetter( - rowType, fieldIdToPosition.get(projectField.fieldId()), rowField, projectField); - } - } - - private static RowData.FieldGetter createFieldGetter( - RowType rowType, int position, Types.NestedField rowField, Types.NestedField projectField) { - Preconditions.checkArgument( - rowField.type().typeId() == projectField.type().typeId(), - "Different iceberg type between row field <%s> and project field <%s>", - rowField, - projectField); - - switch (projectField.type().typeId()) { - case STRUCT: - RowType nestedRowType = (RowType) rowType.getTypeAt(position); - return row -> { - // null nested struct value - if (row.isNullAt(position)) { - return null; - } - - RowData nestedRow = row.getRow(position, nestedRowType.getFieldCount()); - return RowDataProjection.create( - nestedRowType, rowField.type().asStructType(), projectField.type().asStructType()) - .wrap(nestedRow); - }; - - case MAP: - Types.MapType projectedMap = projectField.type().asMapType(); - Types.MapType originalMap = rowField.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 with non-primitive type. Trying to project <%s> out of <%s>", - projectField, - rowField); - - return RowData.createFieldGetter(rowType.getTypeAt(position), position); - - case LIST: - Types.ListType projectedList = projectField.type().asListType(); - Types.ListType originalList = rowField.type().asListType(); - - boolean elementProjectable = - !projectedList.elementType().isNestedType() - || projectedList.elementType().equals(originalList.elementType()); - Preconditions.checkArgument( - elementProjectable, - "Cannot project a partial list element with non-primitive type. Trying to project <%s> out of <%s>", - projectField, - rowField); - - return RowData.createFieldGetter(rowType.getTypeAt(position), position); - - default: - return RowData.createFieldGetter(rowType.getTypeAt(position), position); - } - } - - public RowData wrap(RowData row) { - // StructProjection allow wrapping null root struct object. - // See more discussions in https://github.com/apache/iceberg/pull/7517. - // RowDataProjection never allowed null root object to be wrapped. - // Hence, it is fine to enforce strict Preconditions check here. - Preconditions.checkArgument(row != null, "Invalid row data: null"); - this.rowData = row; - return this; - } - - private Object getValue(int pos) { - Preconditions.checkState(rowData != null, "Row data not wrapped"); - return getters[pos].getFieldOrNull(rowData); - } - - @Override - public int getArity() { - return getters.length; - } - - @Override - public RowKind getRowKind() { - Preconditions.checkState(rowData != null, "Row data not wrapped"); - return rowData.getRowKind(); - } - - @Override - public void setRowKind(RowKind kind) { - throw new UnsupportedOperationException("Cannot set row kind in the RowDataProjection"); - } - - @Override - public boolean isNullAt(int pos) { - return getValue(pos) == null; - } - - @Override - public boolean getBoolean(int pos) { - return (boolean) getValue(pos); - } - - @Override - public byte getByte(int pos) { - return (byte) getValue(pos); - } - - @Override - public short getShort(int pos) { - return (short) getValue(pos); - } - - @Override - public int getInt(int pos) { - return (int) getValue(pos); - } - - @Override - public long getLong(int pos) { - return (long) getValue(pos); - } - - @Override - public float getFloat(int pos) { - return (float) getValue(pos); - } - - @Override - public double getDouble(int pos) { - return (double) getValue(pos); - } - - @Override - public StringData getString(int pos) { - return (StringData) getValue(pos); - } - - @Override - public DecimalData getDecimal(int pos, int precision, int scale) { - return (DecimalData) getValue(pos); - } - - @Override - public TimestampData getTimestamp(int pos, int precision) { - return (TimestampData) getValue(pos); - } - - @Override - @SuppressWarnings("unchecked") - public RawValueData getRawValue(int pos) { - return (RawValueData) getValue(pos); - } - - @Override - public byte[] getBinary(int pos) { - return (byte[]) getValue(pos); - } - - @Override - public ArrayData getArray(int pos) { - return (ArrayData) getValue(pos); - } - - @Override - public MapData getMap(int pos) { - return (MapData) getValue(pos); - } - - @Override - public RowData getRow(int pos, int numFields) { - return (RowData) getValue(pos); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - - if (!(o instanceof RowDataProjection)) { - return false; - } - - RowDataProjection that = (RowDataProjection) o; - return deepEquals(that); - } - - @Override - public int hashCode() { - int result = Objects.hashCode(getRowKind()); - for (int pos = 0; pos < getArity(); pos++) { - if (!isNullAt(pos)) { - // Arrays.deepHashCode handles array object properly - result = 31 * result + Arrays.deepHashCode(new Object[] {getValue(pos)}); - } - } - - return result; - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append(getRowKind().shortString()).append("("); - for (int pos = 0; pos < getArity(); pos++) { - if (pos != 0) { - sb.append(","); - } - // copied the behavior from Flink GenericRowData - sb.append(StringUtils.arrayAwareToString(getValue(pos))); - } - - sb.append(")"); - return sb.toString(); - } - - private boolean deepEquals(RowDataProjection other) { - if (getRowKind() != other.getRowKind()) { - return false; - } - - if (getArity() != other.getArity()) { - return false; - } - - for (int pos = 0; pos < getArity(); ++pos) { - if (isNullAt(pos) && other.isNullAt(pos)) { - continue; - } - - if ((isNullAt(pos) && !other.isNullAt(pos)) || (!isNullAt(pos) && other.isNullAt(pos))) { - return false; - } - - // Objects.deepEquals handles array object properly - if (!Objects.deepEquals(getValue(pos), other.getValue(pos))) { - return false; - } - } - - return true; - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java deleted file mode 100644 index c5cb51b7eae4..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java +++ /dev/null @@ -1,100 +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.iceberg.flink.data; - -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import org.apache.avro.generic.GenericData; -import org.apache.avro.util.Utf8; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.ByteBuffers; -import org.apache.iceberg.util.DateTimeUtil; - -public class RowDataUtil { - - private RowDataUtil() {} - - public static Object convertConstant(Type type, Object value) { - if (value == null) { - return null; - } - - switch (type.typeId()) { - case DECIMAL: // DecimalData - Types.DecimalType decimal = (Types.DecimalType) type; - return DecimalData.fromBigDecimal((BigDecimal) value, decimal.precision(), decimal.scale()); - case STRING: // StringData - if (value instanceof Utf8) { - Utf8 utf8 = (Utf8) value; - return StringData.fromBytes(utf8.getBytes(), 0, utf8.getByteLength()); - } - return StringData.fromString(value.toString()); - case FIXED: // byte[] - if (value instanceof byte[]) { - return value; - } else if (value instanceof GenericData.Fixed) { - return ((GenericData.Fixed) value).bytes(); - } - return ByteBuffers.toByteArray((ByteBuffer) value); - case BINARY: // byte[] - return ByteBuffers.toByteArray((ByteBuffer) value); - case TIME: // int mills instead of long - return (int) ((Long) value / 1000); - case TIMESTAMP: // TimestampData - return TimestampData.fromLocalDateTime(DateTimeUtil.timestampFromMicros((Long) value)); - default: - } - return value; - } - - /** - * Similar to the private {@link RowDataSerializer#copyRowData(RowData, RowData)} method. This - * skips the check the arity of rowType and from, because the from RowData may contains additional - * column for position deletes. Using {@link RowDataSerializer#copy(RowData, RowData)} will fail - * the arity check. - */ - public static RowData clone( - RowData from, RowData reuse, RowType rowType, TypeSerializer[] fieldSerializers) { - GenericRowData ret; - if (reuse instanceof GenericRowData) { - ret = (GenericRowData) reuse; - } else { - ret = new GenericRowData(from.getArity()); - } - ret.setRowKind(from.getRowKind()); - for (int i = 0; i < rowType.getFieldCount(); i++) { - if (!from.isNullAt(i)) { - RowData.FieldGetter getter = RowData.createFieldGetter(rowType.getTypeAt(i), i); - ret.setField(i, fieldSerializers[i].copy(getter.getFieldOrNull(from))); - } else { - ret.setField(i, null); - } - } - return ret; - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java deleted file mode 100644 index 1019285018d0..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java +++ /dev/null @@ -1,300 +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.iceberg.flink.data; - -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import java.time.Duration; -import java.time.Instant; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.time.OffsetDateTime; -import java.time.ZoneOffset; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.UUID; -import org.apache.flink.annotation.Internal; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.GenericArrayData; -import org.apache.flink.table.data.GenericMapData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RawValueData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.types.RowKind; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.ByteBuffers; - -@Internal -public class StructRowData implements RowData { - private final Types.StructType type; - private RowKind kind; - private StructLike struct; - - public StructRowData(Types.StructType type) { - this(type, RowKind.INSERT); - } - - public StructRowData(Types.StructType type, RowKind kind) { - this(type, null, kind); - } - - private StructRowData(Types.StructType type, StructLike struct) { - this(type, struct, RowKind.INSERT); - } - - private StructRowData(Types.StructType type, StructLike struct, RowKind kind) { - this.type = type; - this.struct = struct; - this.kind = kind; - } - - public StructRowData setStruct(StructLike newStruct) { - this.struct = newStruct; - return this; - } - - @Override - public int getArity() { - return struct.size(); - } - - @Override - public RowKind getRowKind() { - return kind; - } - - @Override - public void setRowKind(RowKind newKind) { - Preconditions.checkNotNull(newKind, "kind can not be null"); - this.kind = newKind; - } - - @Override - public boolean isNullAt(int pos) { - return struct.get(pos, Object.class) == null; - } - - @Override - public boolean getBoolean(int pos) { - return struct.get(pos, Boolean.class); - } - - @Override - public byte getByte(int pos) { - return (byte) (int) struct.get(pos, Integer.class); - } - - @Override - public short getShort(int pos) { - return (short) (int) struct.get(pos, Integer.class); - } - - @Override - public int getInt(int pos) { - Object integer = struct.get(pos, Object.class); - - if (integer instanceof Integer) { - return (int) integer; - } else if (integer instanceof LocalDate) { - return (int) ((LocalDate) integer).toEpochDay(); - } else if (integer instanceof LocalTime) { - return (int) (((LocalTime) integer).toNanoOfDay() / 1000_000); - } else { - throw new IllegalStateException( - "Unknown type for int field. Type name: " + integer.getClass().getName()); - } - } - - @Override - public long getLong(int pos) { - Object longVal = struct.get(pos, Object.class); - - if (longVal instanceof Long) { - return (long) longVal; - } else if (longVal instanceof OffsetDateTime) { - return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000; - } else if (longVal instanceof LocalDate) { - return ((LocalDate) longVal).toEpochDay(); - } else if (longVal instanceof LocalTime) { - return ((LocalTime) longVal).toNanoOfDay(); - } else if (longVal instanceof LocalDateTime) { - return Duration.between(Instant.EPOCH, ((LocalDateTime) longVal).atOffset(ZoneOffset.UTC)) - .toNanos() - / 1000; - } else { - throw new IllegalStateException( - "Unknown type for long field. Type name: " + longVal.getClass().getName()); - } - } - - @Override - public float getFloat(int pos) { - return struct.get(pos, Float.class); - } - - @Override - public double getDouble(int pos) { - return struct.get(pos, Double.class); - } - - @Override - public StringData getString(int pos) { - return isNullAt(pos) ? null : getStringDataInternal(pos); - } - - private StringData getStringDataInternal(int pos) { - CharSequence seq = struct.get(pos, CharSequence.class); - return StringData.fromString(seq.toString()); - } - - @Override - public DecimalData getDecimal(int pos, int precision, int scale) { - return isNullAt(pos) - ? null - : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale); - } - - private BigDecimal getDecimalInternal(int pos) { - return struct.get(pos, BigDecimal.class); - } - - @Override - public TimestampData getTimestamp(int pos, int precision) { - long timeLong = getLong(pos); - return TimestampData.fromEpochMillis(timeLong / 1000, (int) (timeLong % 1000) * 1000); - } - - @Override - public RawValueData getRawValue(int pos) { - throw new UnsupportedOperationException("Not supported yet."); - } - - @Override - public byte[] getBinary(int pos) { - return isNullAt(pos) ? null : getBinaryInternal(pos); - } - - private byte[] getBinaryInternal(int pos) { - Object bytes = struct.get(pos, Object.class); - - // should only be either ByteBuffer or byte[] - if (bytes instanceof ByteBuffer) { - return ByteBuffers.toByteArray((ByteBuffer) bytes); - } else if (bytes instanceof byte[]) { - return (byte[]) bytes; - } else if (bytes instanceof UUID) { - UUID uuid = (UUID) bytes; - ByteBuffer bb = ByteBuffer.allocate(16); - bb.putLong(uuid.getMostSignificantBits()); - bb.putLong(uuid.getLeastSignificantBits()); - return bb.array(); - } else { - throw new IllegalStateException( - "Unknown type for binary field. Type name: " + bytes.getClass().getName()); - } - } - - @Override - public ArrayData getArray(int pos) { - return isNullAt(pos) - ? null - : (ArrayData) - convertValue(type.fields().get(pos).type().asListType(), struct.get(pos, List.class)); - } - - @Override - public MapData getMap(int pos) { - return isNullAt(pos) - ? null - : (MapData) - convertValue(type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class)); - } - - @Override - public RowData getRow(int pos, int numFields) { - return isNullAt(pos) ? null : getStructRowData(pos, numFields); - } - - private StructRowData getStructRowData(int pos, int numFields) { - return new StructRowData( - type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class)); - } - - private Object convertValue(Type elementType, Object value) { - switch (elementType.typeId()) { - case BOOLEAN: - case INTEGER: - case DATE: - case TIME: - case LONG: - case FLOAT: - case DOUBLE: - case DECIMAL: - return value; - case TIMESTAMP: - long millisecond = (long) value / 1000; - int nanoOfMillisecond = (int) ((Long) value % 1000) * 1000; - return TimestampData.fromEpochMillis(millisecond, nanoOfMillisecond); - case STRING: - return StringData.fromString(value.toString()); - case FIXED: - case BINARY: - return ByteBuffers.toByteArray((ByteBuffer) value); - case STRUCT: - return new StructRowData(elementType.asStructType(), (StructLike) value); - case LIST: - List list = (List) value; - Object[] array = new Object[list.size()]; - - int index = 0; - for (Object element : list) { - if (element == null) { - array[index] = null; - } else { - array[index] = convertValue(elementType.asListType().elementType(), element); - } - - index += 1; - } - return new GenericArrayData(array); - case MAP: - Types.MapType mapType = elementType.asMapType(); - Set> entries = ((Map) value).entrySet(); - Map result = Maps.newHashMap(); - for (Map.Entry entry : entries) { - final Object keyValue = convertValue(mapType.keyType(), entry.getKey()); - final Object valueValue = convertValue(mapType.valueType(), entry.getValue()); - result.put(keyValue, valueValue); - } - - return new GenericMapData(result); - default: - throw new UnsupportedOperationException("Unsupported element type: " + elementType); - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java deleted file mode 100644 index 04e168385a36..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java +++ /dev/null @@ -1,61 +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.iceberg.flink.sink; - -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.formats.avro.AvroToRowDataConverters; -import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.utils.TypeConversions; -import org.apache.iceberg.avro.AvroSchemaUtil; - -/** - * This util class converts Avro GenericRecord to Flink RowData.
    - *
    - * Internally it uses Flink {@link AvroToRowDataConverters}. Because of the precision difference - * between how Iceberg schema (micro) and Flink {@link AvroToRowDataConverters} (milli) deal with - * time type, we can't directly use the Avro Schema converted from Iceberg schema via {@link - * AvroSchemaUtil#convert(org.apache.iceberg.Schema, String)}. - */ -public class AvroGenericRecordToRowDataMapper implements MapFunction { - - private final AvroToRowDataConverters.AvroToRowDataConverter converter; - - AvroGenericRecordToRowDataMapper(RowType rowType) { - this.converter = AvroToRowDataConverters.createRowConverter(rowType); - } - - @Override - public RowData map(GenericRecord genericRecord) throws Exception { - return (RowData) converter.convert(genericRecord); - } - - /** Create a mapper based on Avro schema. */ - public static AvroGenericRecordToRowDataMapper forAvroSchema(Schema avroSchema) { - DataType dataType = AvroSchemaConverter.convertToDataType(avroSchema.toString()); - LogicalType logicalType = TypeConversions.fromDataToLogicalType(dataType); - RowType rowType = RowType.of(logicalType.getChildren().stream().toArray(LogicalType[]::new)); - return new AvroGenericRecordToRowDataMapper(rowType); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java deleted file mode 100644 index 40e0b5f2a34e..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java +++ /dev/null @@ -1,125 +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.iceberg.flink.sink; - -import java.io.IOException; -import java.util.List; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -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.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.data.RowDataProjection; -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.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.types.TypeUtil; - -abstract class BaseDeltaTaskWriter extends BaseTaskWriter { - - private final Schema schema; - private final Schema deleteSchema; - private final RowDataWrapper wrapper; - private final RowDataWrapper keyWrapper; - private final RowDataProjection keyProjection; - private final boolean upsert; - - BaseDeltaTaskWriter( - PartitionSpec spec, - FileFormat format, - FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, - FileIO io, - long targetFileSize, - Schema schema, - RowType flinkSchema, - List equalityFieldIds, - boolean upsert) { - super(spec, format, appenderFactory, fileFactory, io, targetFileSize); - this.schema = schema; - this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds)); - this.wrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); - this.keyWrapper = - new RowDataWrapper(FlinkSchemaUtil.convert(deleteSchema), deleteSchema.asStruct()); - this.keyProjection = - RowDataProjection.create(flinkSchema, schema.asStruct(), deleteSchema.asStruct()); - this.upsert = upsert; - } - - abstract RowDataDeltaWriter route(RowData row); - - RowDataWrapper wrapper() { - return wrapper; - } - - @Override - public void write(RowData row) throws IOException { - 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; // UPDATE_BEFORE is not necessary for UPSERT, we do nothing to prevent delete one - // row twice - } - 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()); - } - } - - protected class RowDataDeltaWriter extends BaseEqualityDeltaWriter { - RowDataDeltaWriter(PartitionKey partition) { - super(partition, schema, deleteSchema); - } - - @Override - protected StructLike asStructLike(RowData data) { - return wrapper.wrap(data); - } - - @Override - protected StructLike asStructLikeKey(RowData data) { - return keyWrapper.wrap(data); - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java deleted file mode 100644 index 1cb6e013bd2c..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java +++ /dev/null @@ -1,70 +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.iceberg.flink.sink; - -import java.util.stream.IntStream; -import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.PartitionKey; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.RowDataWrapper; - -/** - * A {@link KeySelector} that extracts the bucketId from a data row's bucket partition as the key. - * To be used with the {@link BucketPartitioner}. - */ -class BucketPartitionKeySelector implements KeySelector { - - private final Schema schema; - private final PartitionKey partitionKey; - private final RowType flinkSchema; - private final int bucketFieldPosition; - - private transient RowDataWrapper rowDataWrapper; - - BucketPartitionKeySelector(PartitionSpec partitionSpec, Schema schema, RowType flinkSchema) { - this.schema = schema; - this.partitionKey = new PartitionKey(partitionSpec, schema); - this.flinkSchema = flinkSchema; - this.bucketFieldPosition = getBucketFieldPosition(partitionSpec); - } - - private int getBucketFieldPosition(PartitionSpec partitionSpec) { - int bucketFieldId = BucketPartitionerUtil.getBucketFieldId(partitionSpec); - return IntStream.range(0, partitionSpec.fields().size()) - .filter(i -> partitionSpec.fields().get(i).fieldId() == bucketFieldId) - .toArray()[0]; - } - - private RowDataWrapper lazyRowDataWrapper() { - if (rowDataWrapper == null) { - rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); - } - - return rowDataWrapper; - } - - @Override - public Integer getKey(RowData rowData) { - partitionKey.partition(lazyRowDataWrapper().wrap(rowData)); - return partitionKey.get(bucketFieldPosition, Integer.class); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java deleted file mode 100644 index 9c9a117906e2..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java +++ /dev/null @@ -1,103 +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.iceberg.flink.sink; - -import org.apache.flink.api.common.functions.Partitioner; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** - * This partitioner will redirect records to writers deterministically based on the Bucket partition - * spec. It'll attempt to optimize the file size written depending on whether numPartitions is - * greater, less or equal than the maxNumBuckets. Note: The current implementation only supports ONE - * bucket in the partition spec. - */ -class BucketPartitioner implements Partitioner { - - static final String BUCKET_NULL_MESSAGE = "bucketId cannot be null"; - static final String BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE = - "Invalid bucket ID %s: must be non-negative."; - static final String BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE = - "Invalid bucket ID %s: must be less than bucket limit: %s."; - - private final int maxNumBuckets; - - // To hold the OFFSET of the next writer to use for any bucket, only used when writers > the - // number of buckets - private final int[] currentBucketWriterOffset; - - BucketPartitioner(PartitionSpec partitionSpec) { - this.maxNumBuckets = BucketPartitionerUtil.getMaxNumBuckets(partitionSpec); - this.currentBucketWriterOffset = new int[maxNumBuckets]; - } - - /** - * Determine the partition id based on the following criteria: If the number of writers <= the - * number of buckets, an evenly distributed number of buckets will be assigned to each writer (one - * writer -> many buckets). Conversely, if the number of writers > the number of buckets the logic - * is handled by the {@link #getPartitionWithMoreWritersThanBuckets - * getPartitionWritersGreaterThanBuckets} method. - * - * @param bucketId the bucketId for each request - * @param numPartitions the total number of partitions - * @return the partition id (writer) to use for each request - */ - @Override - public int partition(Integer bucketId, int numPartitions) { - Preconditions.checkNotNull(bucketId, BUCKET_NULL_MESSAGE); - Preconditions.checkArgument(bucketId >= 0, BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE, bucketId); - Preconditions.checkArgument( - bucketId < maxNumBuckets, BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE, bucketId, maxNumBuckets); - - if (numPartitions <= maxNumBuckets) { - return bucketId % numPartitions; - } else { - return getPartitionWithMoreWritersThanBuckets(bucketId, numPartitions); - } - } - - /*- - * If the number of writers > the number of buckets each partitioner will keep a state of multiple - * writers per bucket as evenly as possible, and will round-robin the requests across them, in this - * case each writer will target only one bucket at all times (many writers -> one bucket). Example: - * Configuration: numPartitions (writers) = 5, maxBuckets = 2 - * Expected behavior: - * - Records for Bucket 0 will be "round robin" between Writers 0, 2 and 4 - * - Records for Bucket 1 will always use Writer 1 and 3 - * Notes: - * - maxNumWritersPerBucket determines when to reset the currentBucketWriterOffset to 0 for this bucketId - * - When numPartitions is not evenly divisible by maxBuckets, some buckets will have one more writer (extraWriter). - * In this example Bucket 0 has an "extra writer" to consider before resetting its offset to 0. - * - * @return the destination partition index (writer subtask id) - */ - private int getPartitionWithMoreWritersThanBuckets(int bucketId, int numPartitions) { - int currentOffset = currentBucketWriterOffset[bucketId]; - // Determine if this bucket requires an "extra writer" - int extraWriter = bucketId < (numPartitions % maxNumBuckets) ? 1 : 0; - // The max number of writers this bucket can have - int maxNumWritersPerBucket = (numPartitions / maxNumBuckets) + extraWriter; - - // Increment the writer offset or reset if it's reached the max for this bucket - int nextOffset = currentOffset == maxNumWritersPerBucket - 1 ? 0 : currentOffset + 1; - currentBucketWriterOffset[bucketId] = nextOffset; - - return bucketId + (maxNumBuckets * currentOffset); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java deleted file mode 100644 index c33207728d3e..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java +++ /dev/null @@ -1,125 +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.iceberg.flink.sink; - -import java.util.List; -import java.util.Objects; -import java.util.stream.Collectors; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.transforms.PartitionSpecVisitor; - -final class BucketPartitionerUtil { - static final String BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE = - "Invalid number of buckets: %s (must be 1)"; - - private BucketPartitionerUtil() {} - - /** - * Determines whether the PartitionSpec has one and only one Bucket definition - * - * @param partitionSpec the partition spec in question - * @return whether the PartitionSpec has only one Bucket - */ - static boolean hasOneBucketField(PartitionSpec partitionSpec) { - List> bucketFields = getBucketFields(partitionSpec); - return bucketFields != null && bucketFields.size() == 1; - } - - /** - * Extracts the Bucket definition from a PartitionSpec. - * - * @param partitionSpec the partition spec in question - * @return the Bucket definition in the form of a tuple (fieldId, maxNumBuckets) - */ - private static Tuple2 getBucketFieldInfo(PartitionSpec partitionSpec) { - List> bucketFields = getBucketFields(partitionSpec); - Preconditions.checkArgument( - bucketFields.size() == 1, - BucketPartitionerUtil.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE, - bucketFields.size()); - return bucketFields.get(0); - } - - static int getBucketFieldId(PartitionSpec partitionSpec) { - return getBucketFieldInfo(partitionSpec).f0; - } - - static int getMaxNumBuckets(PartitionSpec partitionSpec) { - return getBucketFieldInfo(partitionSpec).f1; - } - - private static List> getBucketFields(PartitionSpec spec) { - return PartitionSpecVisitor.visit(spec, new BucketPartitionSpecVisitor()).stream() - .filter(Objects::nonNull) - .collect(Collectors.toList()); - } - - private static class BucketPartitionSpecVisitor - implements PartitionSpecVisitor> { - @Override - public Tuple2 identity(int fieldId, String sourceName, int sourceId) { - return null; - } - - @Override - public Tuple2 bucket( - int fieldId, String sourceName, int sourceId, int numBuckets) { - return new Tuple2<>(fieldId, numBuckets); - } - - @Override - public Tuple2 truncate( - int fieldId, String sourceName, int sourceId, int width) { - return null; - } - - @Override - public Tuple2 year(int fieldId, String sourceName, int sourceId) { - return null; - } - - @Override - public Tuple2 month(int fieldId, String sourceName, int sourceId) { - return null; - } - - @Override - public Tuple2 day(int fieldId, String sourceName, int sourceId) { - return null; - } - - @Override - public Tuple2 hour(int fieldId, String sourceName, int sourceId) { - return null; - } - - @Override - public Tuple2 alwaysNull(int fieldId, String sourceName, int sourceId) { - return null; - } - - @Override - public Tuple2 unknown( - int fieldId, String sourceName, int sourceId, String transform) { - return null; - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java deleted file mode 100644 index e9f9786f9190..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java +++ /dev/null @@ -1,91 +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.iceberg.flink.sink; - -import java.time.Duration; -import org.apache.flink.util.Preconditions; -import org.apache.iceberg.SerializableTable; -import org.apache.iceberg.Table; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.util.DateTimeUtil; -import org.apache.iceberg.util.SerializableSupplier; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * A table loader that will only reload a table after a certain interval has passed. WARNING: This - * table loader should be used carefully when used with writer tasks. It could result in heavy load - * on a catalog for jobs with many writers. - */ -class CachingTableSupplier implements SerializableSupplier

  • { - - private static final Logger LOG = LoggerFactory.getLogger(CachingTableSupplier.class); - - private final Table initialTable; - private final TableLoader tableLoader; - private final Duration tableRefreshInterval; - private long lastLoadTimeMillis; - private transient Table table; - - CachingTableSupplier( - SerializableTable initialTable, TableLoader tableLoader, Duration tableRefreshInterval) { - Preconditions.checkArgument(initialTable != null, "initialTable cannot be null"); - Preconditions.checkArgument(tableLoader != null, "tableLoader cannot be null"); - Preconditions.checkArgument( - tableRefreshInterval != null, "tableRefreshInterval cannot be null"); - this.initialTable = initialTable; - this.table = initialTable; - this.tableLoader = tableLoader; - this.tableRefreshInterval = tableRefreshInterval; - this.lastLoadTimeMillis = System.currentTimeMillis(); - } - - @Override - public Table get() { - if (table == null) { - this.table = initialTable; - } - return table; - } - - Table initialTable() { - return initialTable; - } - - void refreshTable() { - if (System.currentTimeMillis() > lastLoadTimeMillis + tableRefreshInterval.toMillis()) { - try { - if (!tableLoader.isOpen()) { - tableLoader.open(); - } - - this.table = tableLoader.loadTable(); - this.lastLoadTimeMillis = System.currentTimeMillis(); - - LOG.info( - "Table {} reloaded, next min load time threshold is {}", - table.name(), - DateTimeUtil.formatTimestampMillis( - lastLoadTimeMillis + tableRefreshInterval.toMillis())); - } catch (Exception e) { - LOG.warn("An error occurred reloading table {}, table was not reloaded", table.name(), e); - } - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java deleted file mode 100644 index 9a2f57181708..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java +++ /dev/null @@ -1,93 +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.iceberg.flink.sink; - -import java.util.Arrays; -import java.util.NavigableMap; -import java.util.concurrent.atomic.AtomicLong; -import org.apache.iceberg.io.WriteResult; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; - -class CommitSummary { - - private final AtomicLong dataFilesCount = new AtomicLong(); - private final AtomicLong dataFilesRecordCount = new AtomicLong(); - private final AtomicLong dataFilesByteCount = new AtomicLong(); - private final AtomicLong deleteFilesCount = new AtomicLong(); - private final AtomicLong deleteFilesRecordCount = new AtomicLong(); - private final AtomicLong deleteFilesByteCount = new AtomicLong(); - - CommitSummary(NavigableMap pendingResults) { - pendingResults - .values() - .forEach( - writeResult -> { - dataFilesCount.addAndGet(writeResult.dataFiles().length); - Arrays.stream(writeResult.dataFiles()) - .forEach( - dataFile -> { - dataFilesRecordCount.addAndGet(dataFile.recordCount()); - dataFilesByteCount.addAndGet(dataFile.fileSizeInBytes()); - }); - deleteFilesCount.addAndGet(writeResult.deleteFiles().length); - Arrays.stream(writeResult.deleteFiles()) - .forEach( - deleteFile -> { - deleteFilesRecordCount.addAndGet(deleteFile.recordCount()); - deleteFilesByteCount.addAndGet(deleteFile.fileSizeInBytes()); - }); - }); - } - - long dataFilesCount() { - return dataFilesCount.get(); - } - - long dataFilesRecordCount() { - return dataFilesRecordCount.get(); - } - - long dataFilesByteCount() { - return dataFilesByteCount.get(); - } - - long deleteFilesCount() { - return deleteFilesCount.get(); - } - - long deleteFilesRecordCount() { - return deleteFilesRecordCount.get(); - } - - long deleteFilesByteCount() { - return deleteFilesByteCount.get(); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("dataFilesCount", dataFilesCount) - .add("dataFilesRecordCount", dataFilesRecordCount) - .add("dataFilesByteCount", dataFilesByteCount) - .add("deleteFilesCount", deleteFilesCount) - .add("deleteFilesRecordCount", deleteFilesRecordCount) - .add("deleteFilesByteCount", deleteFilesByteCount) - .toString(); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java deleted file mode 100644 index 036970c06d5b..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java +++ /dev/null @@ -1,71 +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.iceberg.flink.sink; - -import java.util.List; -import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -class DeltaManifests { - - private static final CharSequence[] EMPTY_REF_DATA_FILES = new CharSequence[0]; - - private final ManifestFile dataManifest; - private final ManifestFile deleteManifest; - private final CharSequence[] referencedDataFiles; - - DeltaManifests(ManifestFile dataManifest, ManifestFile deleteManifest) { - this(dataManifest, deleteManifest, EMPTY_REF_DATA_FILES); - } - - DeltaManifests( - ManifestFile dataManifest, ManifestFile deleteManifest, CharSequence[] referencedDataFiles) { - Preconditions.checkNotNull(referencedDataFiles, "Referenced data files shouldn't be null."); - - this.dataManifest = dataManifest; - this.deleteManifest = deleteManifest; - this.referencedDataFiles = referencedDataFiles; - } - - ManifestFile dataManifest() { - return dataManifest; - } - - ManifestFile deleteManifest() { - return deleteManifest; - } - - CharSequence[] referencedDataFiles() { - return referencedDataFiles; - } - - List manifests() { - List manifests = Lists.newArrayListWithCapacity(2); - if (dataManifest != null) { - manifests.add(dataManifest); - } - - if (deleteManifest != null) { - manifests.add(deleteManifest); - } - - return manifests; - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java deleted file mode 100644 index 92ca284b12ba..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java +++ /dev/null @@ -1,122 +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.iceberg.flink.sink; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import org.apache.flink.core.io.SimpleVersionedSerializer; -import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.ManifestFiles; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -class DeltaManifestsSerializer implements SimpleVersionedSerializer { - private static final int VERSION_1 = 1; - private static final int VERSION_2 = 2; - private static final byte[] EMPTY_BINARY = new byte[0]; - - static final DeltaManifestsSerializer INSTANCE = new DeltaManifestsSerializer(); - - @Override - public int getVersion() { - return VERSION_2; - } - - @Override - public byte[] serialize(DeltaManifests deltaManifests) throws IOException { - Preconditions.checkNotNull( - deltaManifests, "DeltaManifests to be serialized should not be null"); - - ByteArrayOutputStream binaryOut = new ByteArrayOutputStream(); - DataOutputStream out = new DataOutputStream(binaryOut); - - byte[] dataManifestBinary = EMPTY_BINARY; - if (deltaManifests.dataManifest() != null) { - dataManifestBinary = ManifestFiles.encode(deltaManifests.dataManifest()); - } - - out.writeInt(dataManifestBinary.length); - out.write(dataManifestBinary); - - byte[] deleteManifestBinary = EMPTY_BINARY; - if (deltaManifests.deleteManifest() != null) { - deleteManifestBinary = ManifestFiles.encode(deltaManifests.deleteManifest()); - } - - out.writeInt(deleteManifestBinary.length); - out.write(deleteManifestBinary); - - CharSequence[] referencedDataFiles = deltaManifests.referencedDataFiles(); - out.writeInt(referencedDataFiles.length); - for (CharSequence referencedDataFile : referencedDataFiles) { - out.writeUTF(referencedDataFile.toString()); - } - - return binaryOut.toByteArray(); - } - - @Override - public DeltaManifests deserialize(int version, byte[] serialized) throws IOException { - if (version == VERSION_1) { - return deserializeV1(serialized); - } else if (version == VERSION_2) { - return deserializeV2(serialized); - } else { - throw new RuntimeException("Unknown serialize version: " + version); - } - } - - private DeltaManifests deserializeV1(byte[] serialized) throws IOException { - return new DeltaManifests(ManifestFiles.decode(serialized), null); - } - - private DeltaManifests deserializeV2(byte[] serialized) throws IOException { - ManifestFile dataManifest = null; - ManifestFile deleteManifest = null; - - ByteArrayInputStream binaryIn = new ByteArrayInputStream(serialized); - DataInputStream in = new DataInputStream(binaryIn); - - int dataManifestSize = in.readInt(); - if (dataManifestSize > 0) { - byte[] dataManifestBinary = new byte[dataManifestSize]; - Preconditions.checkState(in.read(dataManifestBinary) == dataManifestSize); - - dataManifest = ManifestFiles.decode(dataManifestBinary); - } - - int deleteManifestSize = in.readInt(); - if (deleteManifestSize > 0) { - byte[] deleteManifestBinary = new byte[deleteManifestSize]; - Preconditions.checkState(in.read(deleteManifestBinary) == deleteManifestSize); - - deleteManifest = ManifestFiles.decode(deleteManifestBinary); - } - - int referenceDataFileNum = in.readInt(); - CharSequence[] referencedDataFiles = new CharSequence[referenceDataFileNum]; - for (int i = 0; i < referenceDataFileNum; i++) { - referencedDataFiles[i] = in.readUTF(); - } - - return new DeltaManifests(dataManifest, deleteManifest, referencedDataFiles); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java deleted file mode 100644 index 18b269d6c3e9..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java +++ /dev/null @@ -1,86 +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.iceberg.flink.sink; - -import java.util.List; -import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.util.StructLikeWrapper; -import org.apache.iceberg.util.StructProjection; - -/** - * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record - * will be emitted to same writer in order. - */ -class EqualityFieldKeySelector implements KeySelector { - - private final Schema schema; - private final RowType flinkSchema; - private final Schema deleteSchema; - - private transient RowDataWrapper rowDataWrapper; - private transient StructProjection structProjection; - private transient StructLikeWrapper structLikeWrapper; - - EqualityFieldKeySelector(Schema schema, RowType flinkSchema, List equalityFieldIds) { - this.schema = schema; - this.flinkSchema = flinkSchema; - this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds)); - } - - /** - * Construct the {@link RowDataWrapper} lazily here because few members in it are not - * serializable. In this way, we don't have to serialize them with forcing. - */ - protected RowDataWrapper lazyRowDataWrapper() { - if (rowDataWrapper == null) { - rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); - } - return rowDataWrapper; - } - - /** Construct the {@link StructProjection} lazily because it is not serializable. */ - protected StructProjection lazyStructProjection() { - if (structProjection == null) { - structProjection = StructProjection.create(schema, deleteSchema); - } - return structProjection; - } - - /** Construct the {@link StructLikeWrapper} lazily because it is not serializable. */ - protected StructLikeWrapper lazyStructLikeWrapper() { - if (structLikeWrapper == null) { - structLikeWrapper = StructLikeWrapper.forType(deleteSchema.asStruct()); - } - return structLikeWrapper; - } - - @Override - public Integer getKey(RowData row) { - RowDataWrapper wrappedRowData = lazyRowDataWrapper().wrap(row); - StructProjection projectedRowData = lazyStructProjection().wrap(wrappedRowData); - StructLikeWrapper wrapper = lazyStructLikeWrapper().set(projectedRowData); - return wrapper.hashCode(); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java deleted file mode 100644 index b6f1392d1562..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java +++ /dev/null @@ -1,274 +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.iceberg.flink.sink; - -import java.io.IOException; -import java.io.Serializable; -import java.io.UncheckedIOException; -import java.util.Map; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.MetricsConfig; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.Table; -import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.deletes.EqualityDeleteWriter; -import org.apache.iceberg.deletes.PositionDeleteWriter; -import org.apache.iceberg.encryption.EncryptedOutputFile; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.data.FlinkAvroWriter; -import org.apache.iceberg.flink.data.FlinkOrcWriter; -import org.apache.iceberg.flink.data.FlinkParquetWriters; -import org.apache.iceberg.io.DataWriter; -import org.apache.iceberg.io.DeleteSchemaUtil; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.io.OutputFile; -import org.apache.iceberg.orc.ORC; -import org.apache.iceberg.parquet.Parquet; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -public class FlinkAppenderFactory implements FileAppenderFactory, Serializable { - private final Schema schema; - private final RowType flinkSchema; - private final Map props; - private final PartitionSpec spec; - private final int[] equalityFieldIds; - private final Schema eqDeleteRowSchema; - private final Schema posDeleteRowSchema; - private final Table table; - - private RowType eqDeleteFlinkSchema = null; - private RowType posDeleteFlinkSchema = null; - - public FlinkAppenderFactory( - Table table, - Schema schema, - RowType flinkSchema, - Map props, - PartitionSpec spec, - int[] equalityFieldIds, - Schema eqDeleteRowSchema, - Schema posDeleteRowSchema) { - Preconditions.checkNotNull(table, "Table shouldn't be null"); - this.table = table; - this.schema = schema; - this.flinkSchema = flinkSchema; - this.props = props; - this.spec = spec; - this.equalityFieldIds = equalityFieldIds; - this.eqDeleteRowSchema = eqDeleteRowSchema; - this.posDeleteRowSchema = posDeleteRowSchema; - } - - private RowType lazyEqDeleteFlinkSchema() { - if (eqDeleteFlinkSchema == null) { - Preconditions.checkNotNull(eqDeleteRowSchema, "Equality delete row schema shouldn't be null"); - this.eqDeleteFlinkSchema = FlinkSchemaUtil.convert(eqDeleteRowSchema); - } - return eqDeleteFlinkSchema; - } - - private RowType lazyPosDeleteFlinkSchema() { - if (posDeleteFlinkSchema == null) { - Preconditions.checkNotNull(posDeleteRowSchema, "Pos-delete row schema shouldn't be null"); - this.posDeleteFlinkSchema = FlinkSchemaUtil.convert(posDeleteRowSchema); - } - return this.posDeleteFlinkSchema; - } - - @Override - public FileAppender newAppender(OutputFile outputFile, FileFormat format) { - MetricsConfig metricsConfig = MetricsConfig.forTable(table); - try { - switch (format) { - case AVRO: - return Avro.write(outputFile) - .createWriterFunc(ignore -> new FlinkAvroWriter(flinkSchema)) - .setAll(props) - .schema(schema) - .metricsConfig(metricsConfig) - .overwrite() - .build(); - - case ORC: - return ORC.write(outputFile) - .createWriterFunc( - (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(flinkSchema, iSchema)) - .setAll(props) - .metricsConfig(metricsConfig) - .schema(schema) - .overwrite() - .build(); - - case PARQUET: - return Parquet.write(outputFile) - .createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(flinkSchema, msgType)) - .setAll(props) - .metricsConfig(metricsConfig) - .schema(schema) - .overwrite() - .build(); - - default: - throw new UnsupportedOperationException("Cannot write unknown file format: " + format); - } - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - @Override - public DataWriter newDataWriter( - EncryptedOutputFile file, FileFormat format, StructLike partition) { - return new DataWriter<>( - newAppender(file.encryptingOutputFile(), format), - format, - file.encryptingOutputFile().location(), - spec, - partition, - file.keyMetadata()); - } - - @Override - public EqualityDeleteWriter newEqDeleteWriter( - EncryptedOutputFile outputFile, FileFormat format, StructLike partition) { - Preconditions.checkState( - equalityFieldIds != null && equalityFieldIds.length > 0, - "Equality field ids shouldn't be null or empty when creating equality-delete writer"); - Preconditions.checkNotNull( - eqDeleteRowSchema, - "Equality delete row schema shouldn't be null when creating equality-delete writer"); - - MetricsConfig metricsConfig = MetricsConfig.forTable(table); - try { - switch (format) { - case AVRO: - return Avro.writeDeletes(outputFile.encryptingOutputFile()) - .createWriterFunc(ignore -> new FlinkAvroWriter(lazyEqDeleteFlinkSchema())) - .withPartition(partition) - .overwrite() - .setAll(props) - .metricsConfig(metricsConfig) - .rowSchema(eqDeleteRowSchema) - .withSpec(spec) - .withKeyMetadata(outputFile.keyMetadata()) - .equalityFieldIds(equalityFieldIds) - .buildEqualityWriter(); - - case ORC: - return ORC.writeDeletes(outputFile.encryptingOutputFile()) - .createWriterFunc( - (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(flinkSchema, iSchema)) - .withPartition(partition) - .overwrite() - .setAll(props) - .metricsConfig(metricsConfig) - .rowSchema(eqDeleteRowSchema) - .withSpec(spec) - .withKeyMetadata(outputFile.keyMetadata()) - .equalityFieldIds(equalityFieldIds) - .buildEqualityWriter(); - - case PARQUET: - return Parquet.writeDeletes(outputFile.encryptingOutputFile()) - .createWriterFunc( - msgType -> FlinkParquetWriters.buildWriter(lazyEqDeleteFlinkSchema(), msgType)) - .withPartition(partition) - .overwrite() - .setAll(props) - .metricsConfig(metricsConfig) - .rowSchema(eqDeleteRowSchema) - .withSpec(spec) - .withKeyMetadata(outputFile.keyMetadata()) - .equalityFieldIds(equalityFieldIds) - .buildEqualityWriter(); - - default: - throw new UnsupportedOperationException( - "Cannot write equality-deletes for unsupported file format: " + format); - } - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - @Override - public PositionDeleteWriter newPosDeleteWriter( - EncryptedOutputFile outputFile, FileFormat format, StructLike partition) { - MetricsConfig metricsConfig = MetricsConfig.forPositionDelete(table); - try { - switch (format) { - case AVRO: - return Avro.writeDeletes(outputFile.encryptingOutputFile()) - .createWriterFunc(ignore -> new FlinkAvroWriter(lazyPosDeleteFlinkSchema())) - .withPartition(partition) - .overwrite() - .setAll(props) - .metricsConfig(metricsConfig) - .rowSchema(posDeleteRowSchema) - .withSpec(spec) - .withKeyMetadata(outputFile.keyMetadata()) - .buildPositionWriter(); - - case ORC: - RowType orcPosDeleteSchema = - FlinkSchemaUtil.convert(DeleteSchemaUtil.posDeleteSchema(posDeleteRowSchema)); - return ORC.writeDeletes(outputFile.encryptingOutputFile()) - .createWriterFunc( - (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(orcPosDeleteSchema, iSchema)) - .withPartition(partition) - .overwrite() - .setAll(props) - .metricsConfig(metricsConfig) - .rowSchema(posDeleteRowSchema) - .withSpec(spec) - .withKeyMetadata(outputFile.keyMetadata()) - .transformPaths(path -> StringData.fromString(path.toString())) - .buildPositionWriter(); - - case PARQUET: - RowType flinkPosDeleteSchema = - FlinkSchemaUtil.convert(DeleteSchemaUtil.posDeleteSchema(posDeleteRowSchema)); - return Parquet.writeDeletes(outputFile.encryptingOutputFile()) - .createWriterFunc( - msgType -> FlinkParquetWriters.buildWriter(flinkPosDeleteSchema, msgType)) - .withPartition(partition) - .overwrite() - .setAll(props) - .metricsConfig(metricsConfig) - .rowSchema(posDeleteRowSchema) - .withSpec(spec) - .withKeyMetadata(outputFile.keyMetadata()) - .transformPaths(path -> StringData.fromString(path.toString())) - .buildPositionWriter(); - - default: - throw new UnsupportedOperationException( - "Cannot write pos-deletes for unsupported file format: " + format); - } - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java deleted file mode 100644 index 2183fe062af4..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java +++ /dev/null @@ -1,293 +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.iceberg.flink.sink; - -import static org.apache.iceberg.MetadataColumns.DELETE_FILE_ROW_FIELD_NAME; -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.DELETE_DEFAULT_FILE_FORMAT; - -import java.io.Serializable; -import java.util.Map; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.Table; -import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.data.BaseFileWriterFactory; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.data.FlinkAvroWriter; -import org.apache.iceberg.flink.data.FlinkOrcWriter; -import org.apache.iceberg.flink.data.FlinkParquetWriters; -import org.apache.iceberg.io.DeleteSchemaUtil; -import org.apache.iceberg.orc.ORC; -import org.apache.iceberg.parquet.Parquet; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -class FlinkFileWriterFactory extends BaseFileWriterFactory implements Serializable { - private RowType dataFlinkType; - private RowType equalityDeleteFlinkType; - private RowType positionDeleteFlinkType; - - FlinkFileWriterFactory( - Table table, - FileFormat dataFileFormat, - Schema dataSchema, - RowType dataFlinkType, - SortOrder dataSortOrder, - FileFormat deleteFileFormat, - int[] equalityFieldIds, - Schema equalityDeleteRowSchema, - RowType equalityDeleteFlinkType, - SortOrder equalityDeleteSortOrder, - Schema positionDeleteRowSchema, - RowType positionDeleteFlinkType) { - - super( - table, - dataFileFormat, - dataSchema, - dataSortOrder, - deleteFileFormat, - equalityFieldIds, - equalityDeleteRowSchema, - equalityDeleteSortOrder, - positionDeleteRowSchema); - - this.dataFlinkType = dataFlinkType; - this.equalityDeleteFlinkType = equalityDeleteFlinkType; - this.positionDeleteFlinkType = positionDeleteFlinkType; - } - - static Builder builderFor(Table table) { - return new Builder(table); - } - - @Override - protected void configureDataWrite(Avro.DataWriteBuilder builder) { - builder.createWriterFunc(ignore -> new FlinkAvroWriter(dataFlinkType())); - } - - @Override - protected void configureEqualityDelete(Avro.DeleteWriteBuilder builder) { - builder.createWriterFunc(ignored -> new FlinkAvroWriter(equalityDeleteFlinkType())); - } - - @Override - protected void configurePositionDelete(Avro.DeleteWriteBuilder builder) { - int rowFieldIndex = positionDeleteFlinkType().getFieldIndex(DELETE_FILE_ROW_FIELD_NAME); - if (rowFieldIndex >= 0) { - // FlinkAvroWriter accepts just the Flink type of the row ignoring the path and pos - RowType positionDeleteRowFlinkType = - (RowType) positionDeleteFlinkType().getTypeAt(rowFieldIndex); - builder.createWriterFunc(ignored -> new FlinkAvroWriter(positionDeleteRowFlinkType)); - } - } - - @Override - protected void configureDataWrite(Parquet.DataWriteBuilder builder) { - builder.createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(dataFlinkType(), msgType)); - } - - @Override - protected void configureEqualityDelete(Parquet.DeleteWriteBuilder builder) { - builder.createWriterFunc( - msgType -> FlinkParquetWriters.buildWriter(equalityDeleteFlinkType(), msgType)); - } - - @Override - protected void configurePositionDelete(Parquet.DeleteWriteBuilder builder) { - builder.createWriterFunc( - msgType -> FlinkParquetWriters.buildWriter(positionDeleteFlinkType(), msgType)); - builder.transformPaths(path -> StringData.fromString(path.toString())); - } - - @Override - protected void configureDataWrite(ORC.DataWriteBuilder builder) { - builder.createWriterFunc( - (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(dataFlinkType(), iSchema)); - } - - @Override - protected void configureEqualityDelete(ORC.DeleteWriteBuilder builder) { - builder.createWriterFunc( - (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(equalityDeleteFlinkType(), iSchema)); - } - - @Override - protected void configurePositionDelete(ORC.DeleteWriteBuilder builder) { - builder.createWriterFunc( - (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(positionDeleteFlinkType(), iSchema)); - builder.transformPaths(path -> StringData.fromString(path.toString())); - } - - private RowType dataFlinkType() { - if (dataFlinkType == null) { - Preconditions.checkNotNull(dataSchema(), "Data schema must not be null"); - this.dataFlinkType = FlinkSchemaUtil.convert(dataSchema()); - } - - return dataFlinkType; - } - - private RowType equalityDeleteFlinkType() { - if (equalityDeleteFlinkType == null) { - Preconditions.checkNotNull( - equalityDeleteRowSchema(), "Equality delete schema must not be null"); - this.equalityDeleteFlinkType = FlinkSchemaUtil.convert(equalityDeleteRowSchema()); - } - - return equalityDeleteFlinkType; - } - - private RowType positionDeleteFlinkType() { - if (positionDeleteFlinkType == null) { - // wrap the optional row schema into the position delete schema that contains path and - // position - Schema positionDeleteSchema = DeleteSchemaUtil.posDeleteSchema(positionDeleteRowSchema()); - this.positionDeleteFlinkType = FlinkSchemaUtil.convert(positionDeleteSchema); - } - - return positionDeleteFlinkType; - } - - static class Builder { - private final Table table; - private FileFormat dataFileFormat; - private Schema dataSchema; - private RowType dataFlinkType; - private SortOrder dataSortOrder; - private FileFormat deleteFileFormat; - private int[] equalityFieldIds; - private Schema equalityDeleteRowSchema; - private RowType equalityDeleteFlinkType; - private SortOrder equalityDeleteSortOrder; - private Schema positionDeleteRowSchema; - private RowType positionDeleteFlinkType; - - Builder(Table table) { - this.table = table; - - Map properties = table.properties(); - - String dataFileFormatName = - properties.getOrDefault(DEFAULT_FILE_FORMAT, DEFAULT_FILE_FORMAT_DEFAULT); - this.dataFileFormat = FileFormat.fromString(dataFileFormatName); - - String deleteFileFormatName = - properties.getOrDefault(DELETE_DEFAULT_FILE_FORMAT, dataFileFormatName); - this.deleteFileFormat = FileFormat.fromString(deleteFileFormatName); - } - - Builder dataFileFormat(FileFormat newDataFileFormat) { - this.dataFileFormat = newDataFileFormat; - return this; - } - - Builder dataSchema(Schema newDataSchema) { - this.dataSchema = newDataSchema; - return this; - } - - /** - * Sets a Flink type for data. - * - *

    If not set, the value is derived from the provided Iceberg schema. - */ - Builder dataFlinkType(RowType newDataFlinkType) { - this.dataFlinkType = newDataFlinkType; - return this; - } - - Builder dataSortOrder(SortOrder newDataSortOrder) { - this.dataSortOrder = newDataSortOrder; - return this; - } - - Builder deleteFileFormat(FileFormat newDeleteFileFormat) { - this.deleteFileFormat = newDeleteFileFormat; - return this; - } - - Builder equalityFieldIds(int[] newEqualityFieldIds) { - this.equalityFieldIds = newEqualityFieldIds; - return this; - } - - Builder equalityDeleteRowSchema(Schema newEqualityDeleteRowSchema) { - this.equalityDeleteRowSchema = newEqualityDeleteRowSchema; - return this; - } - - /** - * Sets a Flink type for equality deletes. - * - *

    If not set, the value is derived from the provided Iceberg schema. - */ - Builder equalityDeleteFlinkType(RowType newEqualityDeleteFlinkType) { - this.equalityDeleteFlinkType = newEqualityDeleteFlinkType; - return this; - } - - Builder equalityDeleteSortOrder(SortOrder newEqualityDeleteSortOrder) { - this.equalityDeleteSortOrder = newEqualityDeleteSortOrder; - return this; - } - - Builder positionDeleteRowSchema(Schema newPositionDeleteRowSchema) { - this.positionDeleteRowSchema = newPositionDeleteRowSchema; - return this; - } - - /** - * Sets a Flink type for position deletes. - * - *

    If not set, the value is derived from the provided Iceberg schema. - */ - Builder positionDeleteFlinkType(RowType newPositionDeleteFlinkType) { - this.positionDeleteFlinkType = newPositionDeleteFlinkType; - return this; - } - - FlinkFileWriterFactory build() { - boolean noEqualityDeleteConf = equalityFieldIds == null && equalityDeleteRowSchema == null; - boolean fullEqualityDeleteConf = equalityFieldIds != null && equalityDeleteRowSchema != null; - Preconditions.checkArgument( - noEqualityDeleteConf || fullEqualityDeleteConf, - "Equality field IDs and equality delete row schema must be set together"); - - return new FlinkFileWriterFactory( - table, - dataFileFormat, - dataSchema, - dataFlinkType, - dataSortOrder, - deleteFileFormat, - equalityFieldIds, - equalityDeleteRowSchema, - equalityDeleteFlinkType, - equalityDeleteSortOrder, - positionDeleteRowSchema, - positionDeleteFlinkType); - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java deleted file mode 100644 index c7e8a2dea7cb..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java +++ /dev/null @@ -1,132 +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.iceberg.flink.sink; - -import java.io.IOException; -import java.util.List; -import java.util.Map; -import java.util.function.Supplier; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.ManifestFiles; -import org.apache.iceberg.ManifestWriter; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Table; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.io.OutputFile; -import org.apache.iceberg.io.WriteResult; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -class FlinkManifestUtil { - private static final int FORMAT_V2 = 2; - private static final Long DUMMY_SNAPSHOT_ID = 0L; - - private FlinkManifestUtil() {} - - static ManifestFile writeDataFiles( - OutputFile outputFile, PartitionSpec spec, List dataFiles) throws IOException { - ManifestWriter writer = - ManifestFiles.write(FORMAT_V2, spec, outputFile, DUMMY_SNAPSHOT_ID); - - try (ManifestWriter closeableWriter = writer) { - closeableWriter.addAll(dataFiles); - } - - return writer.toManifestFile(); - } - - static List readDataFiles( - ManifestFile manifestFile, FileIO io, Map specsById) - throws IOException { - try (CloseableIterable dataFiles = ManifestFiles.read(manifestFile, io, specsById)) { - return Lists.newArrayList(dataFiles); - } - } - - static ManifestOutputFileFactory createOutputFileFactory( - Supplier

    tableSupplier, - Map tableProps, - String flinkJobId, - String operatorUniqueId, - int subTaskId, - long attemptNumber) { - return new ManifestOutputFileFactory( - tableSupplier, tableProps, flinkJobId, operatorUniqueId, subTaskId, attemptNumber); - } - - /** - * Write the {@link WriteResult} to temporary manifest files. - * - * @param result all those DataFiles/DeleteFiles in this WriteResult should be written with same - * partition spec - */ - static DeltaManifests writeCompletedFiles( - WriteResult result, Supplier outputFileSupplier, PartitionSpec spec) - throws IOException { - - ManifestFile dataManifest = null; - ManifestFile deleteManifest = null; - - // Write the completed data files into a newly created data manifest file. - if (result.dataFiles() != null && result.dataFiles().length > 0) { - dataManifest = - writeDataFiles(outputFileSupplier.get(), spec, Lists.newArrayList(result.dataFiles())); - } - - // Write the completed delete files into a newly created delete manifest file. - if (result.deleteFiles() != null && result.deleteFiles().length > 0) { - OutputFile deleteManifestFile = outputFileSupplier.get(); - - ManifestWriter deleteManifestWriter = - ManifestFiles.writeDeleteManifest(FORMAT_V2, spec, deleteManifestFile, DUMMY_SNAPSHOT_ID); - try (ManifestWriter writer = deleteManifestWriter) { - for (DeleteFile deleteFile : result.deleteFiles()) { - writer.add(deleteFile); - } - } - - deleteManifest = deleteManifestWriter.toManifestFile(); - } - - return new DeltaManifests(dataManifest, deleteManifest, result.referencedDataFiles()); - } - - static WriteResult readCompletedFiles( - DeltaManifests deltaManifests, FileIO io, Map specsById) - throws IOException { - WriteResult.Builder builder = WriteResult.builder(); - - // Read the completed data files from persisted data manifest file. - if (deltaManifests.dataManifest() != null) { - builder.addDataFiles(readDataFiles(deltaManifests.dataManifest(), io, specsById)); - } - - // Read the completed delete files from persisted delete manifests file. - if (deltaManifests.deleteManifest() != null) { - try (CloseableIterable deleteFiles = - ManifestFiles.readDeleteManifest(deltaManifests.deleteManifest(), io, specsById)) { - builder.addDeleteFiles(deleteFiles); - } - } - - return builder.addReferencedDataFiles(deltaManifests.referencedDataFiles()).build(); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java deleted file mode 100644 index 769af7d77140..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java +++ /dev/null @@ -1,654 +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.iceberg.flink.sink; - -import static org.apache.iceberg.TableProperties.AVRO_COMPRESSION; -import static org.apache.iceberg.TableProperties.AVRO_COMPRESSION_LEVEL; -import static org.apache.iceberg.TableProperties.ORC_COMPRESSION; -import static org.apache.iceberg.TableProperties.ORC_COMPRESSION_STRATEGY; -import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION; -import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION_LEVEL; -import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.time.Duration; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.function.Function; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSink; -import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; -import org.apache.flink.streaming.api.functions.sink.DiscardingSink; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.util.DataFormatConverters; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.types.Row; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DistributionMode; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionField; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SerializableTable; -import org.apache.iceberg.Table; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.FlinkWriteConf; -import org.apache.iceberg.flink.FlinkWriteOptions; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; -import org.apache.iceberg.io.WriteResult; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.util.SerializableSupplier; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class FlinkSink { - private static final Logger LOG = LoggerFactory.getLogger(FlinkSink.class); - - private static final String ICEBERG_STREAM_WRITER_NAME = - IcebergStreamWriter.class.getSimpleName(); - private static final String ICEBERG_FILES_COMMITTER_NAME = - IcebergFilesCommitter.class.getSimpleName(); - - private FlinkSink() {} - - /** - * Initialize a {@link Builder} to export the data from generic input data stream into iceberg - * table. We use {@link RowData} inside the sink connector, so users need to provide a mapper - * function and a {@link TypeInformation} to convert those generic records to a RowData - * DataStream. - * - * @param input the generic source input data stream. - * @param mapper function to convert the generic data to {@link RowData} - * @param outputType to define the {@link TypeInformation} for the input data. - * @param the data type of records. - * @return {@link Builder} to connect the iceberg table. - */ - public static Builder builderFor( - DataStream input, MapFunction mapper, TypeInformation outputType) { - return new Builder().forMapperOutputType(input, mapper, outputType); - } - - /** - * Initialize a {@link Builder} to export the data from input data stream with {@link Row}s into - * iceberg table. We use {@link RowData} inside the sink connector, so users need to provide a - * {@link TableSchema} for builder to convert those {@link Row}s to a {@link RowData} DataStream. - * - * @param input the source input data stream with {@link Row}s. - * @param tableSchema defines the {@link TypeInformation} for input data. - * @return {@link Builder} to connect the iceberg table. - */ - public static Builder forRow(DataStream input, TableSchema tableSchema) { - RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType(); - DataType[] fieldDataTypes = tableSchema.getFieldDataTypes(); - - DataFormatConverters.RowConverter rowConverter = - new DataFormatConverters.RowConverter(fieldDataTypes); - return builderFor(input, rowConverter::toInternal, FlinkCompatibilityUtil.toTypeInfo(rowType)) - .tableSchema(tableSchema); - } - - /** - * Initialize a {@link Builder} to export the data from input data stream with {@link RowData}s - * into iceberg table. - * - * @param input the source input data stream with {@link RowData}s. - * @return {@link Builder} to connect the iceberg table. - */ - public static Builder forRowData(DataStream input) { - return new Builder().forRowData(input); - } - - public static class Builder { - private Function> inputCreator = null; - private TableLoader tableLoader; - private Table table; - private TableSchema tableSchema; - private List equalityFieldColumns = null; - private String uidPrefix = null; - private final Map snapshotProperties = Maps.newHashMap(); - private ReadableConfig readableConfig = new Configuration(); - private final Map writeOptions = Maps.newHashMap(); - private FlinkWriteConf flinkWriteConf = null; - - private Builder() {} - - private Builder forRowData(DataStream newRowDataInput) { - this.inputCreator = ignored -> newRowDataInput; - return this; - } - - private Builder forMapperOutputType( - DataStream input, MapFunction mapper, TypeInformation outputType) { - this.inputCreator = - newUidPrefix -> { - // Input stream order is crucial for some situation(e.g. in cdc case). Therefore, we - // need to set the parallelism - // of map operator same as its input to keep map operator chaining its input, and avoid - // rebalanced by default. - SingleOutputStreamOperator inputStream = - input.map(mapper, outputType).setParallelism(input.getParallelism()); - if (newUidPrefix != null) { - inputStream.name(operatorName(newUidPrefix)).uid(newUidPrefix + "-mapper"); - } - return inputStream; - }; - return this; - } - - /** - * This iceberg {@link Table} instance is used for initializing {@link IcebergStreamWriter} - * which will write all the records into {@link DataFile}s and emit them to downstream operator. - * Providing a table would avoid so many table loading from each separate task. - * - * @param newTable the loaded iceberg table instance. - * @return {@link Builder} to connect the iceberg table. - */ - public Builder table(Table newTable) { - this.table = newTable; - return this; - } - - /** - * The table loader is used for loading tables in {@link IcebergFilesCommitter} lazily, we need - * this loader because {@link Table} is not serializable and could not just use the loaded table - * from Builder#table in the remote task manager. - * - * @param newTableLoader to load iceberg table inside tasks. - * @return {@link Builder} to connect the iceberg table. - */ - public Builder tableLoader(TableLoader newTableLoader) { - this.tableLoader = newTableLoader; - return this; - } - - /** - * Set the write properties for Flink sink. View the supported properties in {@link - * FlinkWriteOptions} - */ - public Builder set(String property, String value) { - writeOptions.put(property, value); - return this; - } - - /** - * Set the write properties for Flink sink. View the supported properties in {@link - * FlinkWriteOptions} - */ - public Builder setAll(Map properties) { - writeOptions.putAll(properties); - return this; - } - - public Builder tableSchema(TableSchema newTableSchema) { - this.tableSchema = newTableSchema; - return this; - } - - public Builder overwrite(boolean newOverwrite) { - writeOptions.put(FlinkWriteOptions.OVERWRITE_MODE.key(), Boolean.toString(newOverwrite)); - return this; - } - - public Builder flinkConf(ReadableConfig config) { - this.readableConfig = config; - return this; - } - - /** - * Configure the write {@link DistributionMode} that the flink sink will use. Currently, flink - * support {@link DistributionMode#NONE} and {@link DistributionMode#HASH}. - * - * @param mode to specify the write distribution mode. - * @return {@link Builder} to connect the iceberg table. - */ - public Builder distributionMode(DistributionMode mode) { - Preconditions.checkArgument( - !DistributionMode.RANGE.equals(mode), - "Flink does not support 'range' write distribution mode now."); - if (mode != null) { - writeOptions.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), mode.modeName()); - } - return this; - } - - /** - * Configuring the write parallel number for iceberg stream writer. - * - * @param newWriteParallelism the number of parallel iceberg stream writer. - * @return {@link Builder} to connect the iceberg table. - */ - public Builder writeParallelism(int newWriteParallelism) { - writeOptions.put( - FlinkWriteOptions.WRITE_PARALLELISM.key(), Integer.toString(newWriteParallelism)); - return this; - } - - /** - * All INSERT/UPDATE_AFTER events from input stream will be transformed to UPSERT events, which - * means it will DELETE the old records and then INSERT the new records. In partitioned table, - * the partition fields should be a subset of equality fields, otherwise the old row that - * located in partition-A could not be deleted by the new row that located in partition-B. - * - * @param enabled indicate whether it should transform all INSERT/UPDATE_AFTER events to UPSERT. - * @return {@link Builder} to connect the iceberg table. - */ - public Builder upsert(boolean enabled) { - writeOptions.put(FlinkWriteOptions.WRITE_UPSERT_ENABLED.key(), Boolean.toString(enabled)); - return this; - } - - /** - * Configuring the equality field columns for iceberg table that accept CDC or UPSERT events. - * - * @param columns defines the iceberg table's key. - * @return {@link Builder} to connect the iceberg table. - */ - public Builder equalityFieldColumns(List columns) { - this.equalityFieldColumns = columns; - return this; - } - - /** - * Set the uid prefix for FlinkSink operators. Note that FlinkSink internally consists of - * multiple operators (like writer, committer, dummy sink etc.) Actually operator uid will be - * appended with a suffix like "uidPrefix-writer".
    - *
    - * If provided, this prefix is also applied to operator names.
    - *
    - * Flink auto generates operator uid if not set explicitly. It is a recommended - * best-practice to set uid for all operators before deploying to production. Flink has an - * option to {@code pipeline.auto-generate-uid=false} to disable auto-generation and force - * explicit setting of all operator uid.
    - *
    - * Be careful with setting this for an existing job, because now we are changing the operator - * uid from an auto-generated one to this new value. When deploying the change with a - * checkpoint, Flink won't be able to restore the previous Flink sink operator state (more - * specifically the committer operator state). You need to use {@code --allowNonRestoredState} - * to ignore the previous sink state. During restore Flink sink state is used to check if last - * commit was actually successful or not. {@code --allowNonRestoredState} can lead to data loss - * if the Iceberg commit failed in the last completed checkpoint. - * - * @param newPrefix prefix for Flink sink operator uid and name - * @return {@link Builder} to connect the iceberg table. - */ - public Builder uidPrefix(String newPrefix) { - this.uidPrefix = newPrefix; - return this; - } - - public Builder setSnapshotProperties(Map properties) { - snapshotProperties.putAll(properties); - return this; - } - - public Builder setSnapshotProperty(String property, String value) { - snapshotProperties.put(property, value); - return this; - } - - public Builder toBranch(String branch) { - writeOptions.put(FlinkWriteOptions.BRANCH.key(), branch); - return this; - } - - private DataStreamSink chainIcebergOperators() { - Preconditions.checkArgument( - inputCreator != null, - "Please use forRowData() or forMapperOutputType() to initialize the input DataStream."); - Preconditions.checkNotNull(tableLoader, "Table loader shouldn't be null"); - - DataStream rowDataInput = inputCreator.apply(uidPrefix); - - if (table == null) { - if (!tableLoader.isOpen()) { - tableLoader.open(); - } - - try (TableLoader loader = tableLoader) { - this.table = loader.loadTable(); - } catch (IOException e) { - throw new UncheckedIOException( - "Failed to load iceberg table from table loader: " + tableLoader, e); - } - } - - flinkWriteConf = new FlinkWriteConf(table, writeOptions, readableConfig); - - // Find out the equality field id list based on the user-provided equality field column names. - List equalityFieldIds = checkAndGetEqualityFieldIds(); - - // Convert the requested flink table schema to flink row type. - RowType flinkRowType = toFlinkRowType(table.schema(), tableSchema); - - // Distribute the records from input data stream based on the write.distribution-mode and - // equality fields. - DataStream distributeStream = - distributeDataStream( - rowDataInput, equalityFieldIds, table.spec(), table.schema(), flinkRowType); - - // Add parallel writers that append rows to files - SingleOutputStreamOperator writerStream = - appendWriter(distributeStream, flinkRowType, equalityFieldIds); - - // Add single-parallelism committer that commits files - // after successful checkpoint or end of input - SingleOutputStreamOperator committerStream = appendCommitter(writerStream); - - // Add dummy discard sink - return appendDummySink(committerStream); - } - - /** - * Append the iceberg sink operators to write records to iceberg table. - * - * @return {@link DataStreamSink} for sink. - */ - public DataStreamSink append() { - return chainIcebergOperators(); - } - - private String operatorName(String suffix) { - return uidPrefix != null ? uidPrefix + "-" + suffix : suffix; - } - - @VisibleForTesting - List checkAndGetEqualityFieldIds() { - List equalityFieldIds = Lists.newArrayList(table.schema().identifierFieldIds()); - if (equalityFieldColumns != null && !equalityFieldColumns.isEmpty()) { - Set equalityFieldSet = - Sets.newHashSetWithExpectedSize(equalityFieldColumns.size()); - for (String column : equalityFieldColumns) { - org.apache.iceberg.types.Types.NestedField field = table.schema().findField(column); - Preconditions.checkNotNull( - field, - "Missing required equality field column '%s' in table schema %s", - column, - table.schema()); - equalityFieldSet.add(field.fieldId()); - } - - if (!equalityFieldSet.equals(table.schema().identifierFieldIds())) { - LOG.warn( - "The configured equality field column IDs {} are not matched with the schema identifier field IDs" - + " {}, use job specified equality field columns as the equality fields by default.", - equalityFieldSet, - table.schema().identifierFieldIds()); - } - equalityFieldIds = Lists.newArrayList(equalityFieldSet); - } - return equalityFieldIds; - } - - @SuppressWarnings("unchecked") - private DataStreamSink appendDummySink( - SingleOutputStreamOperator committerStream) { - DataStreamSink resultStream = - committerStream - .addSink(new DiscardingSink()) - .name(operatorName(String.format("IcebergSink %s", this.table.name()))) - .setParallelism(1); - if (uidPrefix != null) { - resultStream = resultStream.uid(uidPrefix + "-dummysink"); - } - return resultStream; - } - - private SingleOutputStreamOperator appendCommitter( - SingleOutputStreamOperator writerStream) { - IcebergFilesCommitter filesCommitter = - new IcebergFilesCommitter( - tableLoader, - flinkWriteConf.overwriteMode(), - snapshotProperties, - flinkWriteConf.workerPoolSize(), - flinkWriteConf.branch(), - table.spec()); - SingleOutputStreamOperator committerStream = - writerStream - .transform(operatorName(ICEBERG_FILES_COMMITTER_NAME), Types.VOID, filesCommitter) - .setParallelism(1) - .setMaxParallelism(1); - if (uidPrefix != null) { - committerStream = committerStream.uid(uidPrefix + "-committer"); - } - return committerStream; - } - - private SingleOutputStreamOperator appendWriter( - DataStream input, RowType flinkRowType, List equalityFieldIds) { - // Validate the equality fields and partition fields if we enable the upsert mode. - if (flinkWriteConf.upsertMode()) { - Preconditions.checkState( - !flinkWriteConf.overwriteMode(), - "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); - Preconditions.checkState( - !equalityFieldIds.isEmpty(), - "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); - if (!table.spec().isUnpartitioned()) { - for (PartitionField partitionField : table.spec().fields()) { - Preconditions.checkState( - equalityFieldIds.contains(partitionField.sourceId()), - "In UPSERT mode, partition field '%s' should be included in equality fields: '%s'", - partitionField, - equalityFieldColumns); - } - } - } - - SerializableTable serializableTable = (SerializableTable) SerializableTable.copyOf(table); - Duration tableRefreshInterval = flinkWriteConf.tableRefreshInterval(); - - SerializableSupplier
    tableSupplier; - if (tableRefreshInterval != null) { - tableSupplier = - new CachingTableSupplier(serializableTable, tableLoader, tableRefreshInterval); - } else { - tableSupplier = () -> serializableTable; - } - - IcebergStreamWriter streamWriter = - createStreamWriter(tableSupplier, flinkWriteConf, flinkRowType, equalityFieldIds); - - int parallelism = - flinkWriteConf.writeParallelism() == null - ? input.getParallelism() - : flinkWriteConf.writeParallelism(); - SingleOutputStreamOperator writerStream = - input - .transform( - operatorName(ICEBERG_STREAM_WRITER_NAME), - TypeInformation.of(WriteResult.class), - streamWriter) - .setParallelism(parallelism); - if (uidPrefix != null) { - writerStream = writerStream.uid(uidPrefix + "-writer"); - } - return writerStream; - } - - private DataStream distributeDataStream( - DataStream input, - List equalityFieldIds, - PartitionSpec partitionSpec, - Schema iSchema, - RowType flinkRowType) { - DistributionMode writeMode = flinkWriteConf.distributionMode(); - - LOG.info("Write distribution mode is '{}'", writeMode.modeName()); - switch (writeMode) { - case NONE: - if (equalityFieldIds.isEmpty()) { - return input; - } else { - LOG.info("Distribute rows by equality fields, because there are equality fields set"); - return input.keyBy( - new EqualityFieldKeySelector(iSchema, flinkRowType, equalityFieldIds)); - } - - case HASH: - if (equalityFieldIds.isEmpty()) { - if (partitionSpec.isUnpartitioned()) { - LOG.warn( - "Fallback to use 'none' distribution mode, because there are no equality fields set " - + "and table is unpartitioned"); - return input; - } else { - return input.keyBy(new PartitionKeySelector(partitionSpec, iSchema, flinkRowType)); - } - } else { - if (partitionSpec.isUnpartitioned()) { - LOG.info( - "Distribute rows by equality fields, because there are equality fields set " - + "and table is unpartitioned"); - return input.keyBy( - new EqualityFieldKeySelector(iSchema, flinkRowType, equalityFieldIds)); - } else { - for (PartitionField partitionField : partitionSpec.fields()) { - Preconditions.checkState( - equalityFieldIds.contains(partitionField.sourceId()), - "In 'hash' distribution mode with equality fields set, partition field '%s' " - + "should be included in equality fields: '%s'", - partitionField, - equalityFieldColumns); - } - return input.keyBy(new PartitionKeySelector(partitionSpec, iSchema, flinkRowType)); - } - } - - case RANGE: - if (equalityFieldIds.isEmpty()) { - LOG.warn( - "Fallback to use 'none' distribution mode, because there are no equality fields set " - + "and {}=range is not supported yet in flink", - WRITE_DISTRIBUTION_MODE); - return input; - } else { - LOG.info( - "Distribute rows by equality fields, because there are equality fields set " - + "and{}=range is not supported yet in flink", - WRITE_DISTRIBUTION_MODE); - return input.keyBy( - new EqualityFieldKeySelector(iSchema, flinkRowType, equalityFieldIds)); - } - - default: - throw new RuntimeException("Unrecognized " + WRITE_DISTRIBUTION_MODE + ": " + writeMode); - } - } - } - - static RowType toFlinkRowType(Schema schema, TableSchema requestedSchema) { - if (requestedSchema != null) { - // Convert the flink schema to iceberg schema firstly, then reassign ids to match the existing - // iceberg schema. - Schema writeSchema = TypeUtil.reassignIds(FlinkSchemaUtil.convert(requestedSchema), schema); - TypeUtil.validateWriteSchema(schema, writeSchema, true, true); - - // We use this flink schema to read values from RowData. The flink's TINYINT and SMALLINT will - // be promoted to - // iceberg INTEGER, that means if we use iceberg's table schema to read TINYINT (backend by 1 - // 'byte'), we will - // read 4 bytes rather than 1 byte, it will mess up the byte array in BinaryRowData. So here - // we must use flink - // schema. - return (RowType) requestedSchema.toRowDataType().getLogicalType(); - } else { - return FlinkSchemaUtil.convert(schema); - } - } - - static IcebergStreamWriter createStreamWriter( - SerializableSupplier
    tableSupplier, - FlinkWriteConf flinkWriteConf, - RowType flinkRowType, - List equalityFieldIds) { - Preconditions.checkArgument(tableSupplier != null, "Iceberg table supplier shouldn't be null"); - - Table initTable = tableSupplier.get(); - FileFormat format = flinkWriteConf.dataFileFormat(); - TaskWriterFactory taskWriterFactory = - new RowDataTaskWriterFactory( - tableSupplier, - flinkRowType, - flinkWriteConf.targetDataFileSize(), - format, - writeProperties(initTable, format, flinkWriteConf), - equalityFieldIds, - flinkWriteConf.upsertMode()); - - return new IcebergStreamWriter<>(initTable.name(), taskWriterFactory); - } - - /** - * Based on the {@link FileFormat} overwrites the table level compression properties for the table - * write. - * - * @param table The table to get the table level settings - * @param format The FileFormat to use - * @param conf The write configuration - * @return The properties to use for writing - */ - private static Map writeProperties( - Table table, FileFormat format, FlinkWriteConf conf) { - Map writeProperties = Maps.newHashMap(table.properties()); - - switch (format) { - case PARQUET: - writeProperties.put(PARQUET_COMPRESSION, conf.parquetCompressionCodec()); - String parquetCompressionLevel = conf.parquetCompressionLevel(); - if (parquetCompressionLevel != null) { - writeProperties.put(PARQUET_COMPRESSION_LEVEL, parquetCompressionLevel); - } - - break; - case AVRO: - writeProperties.put(AVRO_COMPRESSION, conf.avroCompressionCodec()); - String avroCompressionLevel = conf.avroCompressionLevel(); - if (avroCompressionLevel != null) { - writeProperties.put(AVRO_COMPRESSION_LEVEL, conf.avroCompressionLevel()); - } - - break; - case ORC: - writeProperties.put(ORC_COMPRESSION, conf.orcCompressionCodec()); - writeProperties.put(ORC_COMPRESSION_STRATEGY, conf.orcCompressionStrategy()); - break; - default: - throw new IllegalArgumentException(String.format("Unknown file format %s", format)); - } - - return writeProperties; - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java deleted file mode 100644 index b9bceaa9311d..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java +++ /dev/null @@ -1,516 +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.iceberg.flink.sink; - -import java.io.IOException; -import java.util.Arrays; -import java.util.Comparator; -import java.util.List; -import java.util.Map; -import java.util.NavigableMap; -import java.util.SortedMap; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.TimeUnit; -import org.apache.flink.api.common.state.ListState; -import org.apache.flink.api.common.state.ListStateDescriptor; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; -import org.apache.flink.core.io.SimpleVersionedSerialization; -import org.apache.flink.runtime.state.StateInitializationContext; -import org.apache.flink.runtime.state.StateSnapshotContext; -import org.apache.flink.streaming.api.operators.AbstractStreamOperator; -import org.apache.flink.streaming.api.operators.BoundedOneInput; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.table.runtime.typeutils.SortedMapTypeInfo; -import org.apache.iceberg.AppendFiles; -import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.ReplacePartitions; -import org.apache.iceberg.RowDelta; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.SnapshotUpdate; -import org.apache.iceberg.Table; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.io.WriteResult; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.base.Strings; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Comparators; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.PropertyUtil; -import org.apache.iceberg.util.ThreadPools; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -class IcebergFilesCommitter extends AbstractStreamOperator - implements OneInputStreamOperator, BoundedOneInput { - - private static final long serialVersionUID = 1L; - private static final long INITIAL_CHECKPOINT_ID = -1L; - private static final byte[] EMPTY_MANIFEST_DATA = new byte[0]; - - private static final Logger LOG = LoggerFactory.getLogger(IcebergFilesCommitter.class); - private static final String FLINK_JOB_ID = "flink.job-id"; - private static final String OPERATOR_ID = "flink.operator-id"; - - // The max checkpoint id we've committed to iceberg table. As the flink's checkpoint is always - // increasing, so we could correctly commit all the data files whose checkpoint id is greater than - // the max committed one to iceberg table, for avoiding committing the same data files twice. This - // id will be attached to iceberg's meta when committing the iceberg transaction. - private static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed-checkpoint-id"; - static final String MAX_CONTINUOUS_EMPTY_COMMITS = "flink.max-continuous-empty-commits"; - - // TableLoader to load iceberg table lazily. - private final TableLoader tableLoader; - private final boolean replacePartitions; - private final Map snapshotProperties; - - // A sorted map to maintain the completed data files for each pending checkpointId (which have not - // been committed to iceberg table). We need a sorted map here because there's possible that few - // checkpoints snapshot failed, for example: the 1st checkpoint have 2 data files <1, >, the 2st checkpoint have 1 data files <2, >. Snapshot for checkpoint#1 - // interrupted because of network/disk failure etc, while we don't expect any data loss in iceberg - // table. So we keep the finished files <1, > in memory and retry to commit iceberg - // table when the next checkpoint happen. - private final NavigableMap dataFilesPerCheckpoint = Maps.newTreeMap(); - - // The completed files cache for current checkpoint. Once the snapshot barrier received, it will - // be flushed to the 'dataFilesPerCheckpoint'. - private final List writeResultsOfCurrentCkpt = Lists.newArrayList(); - private final String branch; - - // It will have an unique identifier for one job. - private transient String flinkJobId; - private transient String operatorUniqueId; - private transient Table table; - private transient IcebergFilesCommitterMetrics committerMetrics; - private transient ManifestOutputFileFactory manifestOutputFileFactory; - private transient long maxCommittedCheckpointId; - private transient int continuousEmptyCheckpoints; - private transient int maxContinuousEmptyCommits; - // There're two cases that we restore from flink checkpoints: the first case is restoring from - // snapshot created by the same flink job; another case is restoring from snapshot created by - // another different job. For the second case, we need to maintain the old flink job's id in flink - // state backend to find the max-committed-checkpoint-id when traversing iceberg table's - // snapshots. - private static final ListStateDescriptor JOB_ID_DESCRIPTOR = - new ListStateDescriptor<>("iceberg-flink-job-id", BasicTypeInfo.STRING_TYPE_INFO); - private transient ListState jobIdState; - // All pending checkpoints states for this function. - private static final ListStateDescriptor> STATE_DESCRIPTOR = - buildStateDescriptor(); - private transient ListState> checkpointsState; - - private final Integer workerPoolSize; - private final PartitionSpec spec; - private transient ExecutorService workerPool; - - IcebergFilesCommitter( - TableLoader tableLoader, - boolean replacePartitions, - Map snapshotProperties, - Integer workerPoolSize, - String branch, - PartitionSpec spec) { - this.tableLoader = tableLoader; - this.replacePartitions = replacePartitions; - this.snapshotProperties = snapshotProperties; - this.workerPoolSize = workerPoolSize; - this.branch = branch; - this.spec = spec; - } - - @Override - public void initializeState(StateInitializationContext context) throws Exception { - super.initializeState(context); - this.flinkJobId = getContainingTask().getEnvironment().getJobID().toString(); - this.operatorUniqueId = getRuntimeContext().getOperatorUniqueID(); - - // Open the table loader and load the table. - this.tableLoader.open(); - this.table = tableLoader.loadTable(); - this.committerMetrics = new IcebergFilesCommitterMetrics(super.metrics, table.name()); - - maxContinuousEmptyCommits = - PropertyUtil.propertyAsInt(table.properties(), MAX_CONTINUOUS_EMPTY_COMMITS, 10); - Preconditions.checkArgument( - maxContinuousEmptyCommits > 0, MAX_CONTINUOUS_EMPTY_COMMITS + " must be positive"); - - int subTaskId = getRuntimeContext().getIndexOfThisSubtask(); - int attemptId = getRuntimeContext().getAttemptNumber(); - this.manifestOutputFileFactory = - FlinkManifestUtil.createOutputFileFactory( - () -> table, table.properties(), flinkJobId, operatorUniqueId, subTaskId, attemptId); - this.maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID; - - this.checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR); - this.jobIdState = context.getOperatorStateStore().getListState(JOB_ID_DESCRIPTOR); - if (context.isRestored()) { - Iterable jobIdIterable = jobIdState.get(); - if (jobIdIterable == null || !jobIdIterable.iterator().hasNext()) { - LOG.warn( - "Failed to restore committer state. This can happen when operator uid changed and Flink " - + "allowNonRestoredState is enabled. Best practice is to explicitly set the operator id " - + "via FlinkSink#Builder#uidPrefix() so that the committer operator uid is stable. " - + "Otherwise, Flink auto generate an operator uid based on job topology." - + "With that, operator uid is subjective to change upon topology change."); - return; - } - - String restoredFlinkJobId = jobIdIterable.iterator().next(); - Preconditions.checkState( - !Strings.isNullOrEmpty(restoredFlinkJobId), - "Flink job id parsed from checkpoint snapshot shouldn't be null or empty"); - - // Since flink's checkpoint id will start from the max-committed-checkpoint-id + 1 in the new - // flink job even if it's restored from a snapshot created by another different flink job, so - // it's safe to assign the max committed checkpoint id from restored flink job to the current - // flink job. - this.maxCommittedCheckpointId = - getMaxCommittedCheckpointId(table, restoredFlinkJobId, operatorUniqueId, branch); - - NavigableMap uncommittedDataFiles = - Maps.newTreeMap(checkpointsState.get().iterator().next()) - .tailMap(maxCommittedCheckpointId, false); - if (!uncommittedDataFiles.isEmpty()) { - // Committed all uncommitted data files from the old flink job to iceberg table. - long maxUncommittedCheckpointId = uncommittedDataFiles.lastKey(); - commitUpToCheckpoint( - uncommittedDataFiles, restoredFlinkJobId, operatorUniqueId, maxUncommittedCheckpointId); - } - } - } - - @Override - public void snapshotState(StateSnapshotContext context) throws Exception { - super.snapshotState(context); - long checkpointId = context.getCheckpointId(); - LOG.info( - "Start to flush snapshot state to state backend, table: {}, checkpointId: {}", - table, - checkpointId); - - // Update the checkpoint state. - long startNano = System.nanoTime(); - dataFilesPerCheckpoint.put(checkpointId, writeToManifest(checkpointId)); - // Reset the snapshot state to the latest state. - checkpointsState.clear(); - checkpointsState.add(dataFilesPerCheckpoint); - - jobIdState.clear(); - jobIdState.add(flinkJobId); - - // Clear the local buffer for current checkpoint. - writeResultsOfCurrentCkpt.clear(); - committerMetrics.checkpointDuration( - TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); - } - - @Override - public void notifyCheckpointComplete(long checkpointId) throws Exception { - super.notifyCheckpointComplete(checkpointId); - // It's possible that we have the following events: - // 1. snapshotState(ckpId); - // 2. snapshotState(ckpId+1); - // 3. notifyCheckpointComplete(ckpId+1); - // 4. notifyCheckpointComplete(ckpId); - // For step#4, we don't need to commit iceberg table again because in step#3 we've committed all - // the files, - // Besides, we need to maintain the max-committed-checkpoint-id to be increasing. - if (checkpointId > maxCommittedCheckpointId) { - LOG.info("Checkpoint {} completed. Attempting commit.", checkpointId); - commitUpToCheckpoint(dataFilesPerCheckpoint, flinkJobId, operatorUniqueId, checkpointId); - this.maxCommittedCheckpointId = checkpointId; - } else { - LOG.info( - "Skipping committing checkpoint {}. {} is already committed.", - checkpointId, - maxCommittedCheckpointId); - } - - // reload the table in case new configuration is needed - this.table = tableLoader.loadTable(); - } - - private void commitUpToCheckpoint( - NavigableMap deltaManifestsMap, - String newFlinkJobId, - String operatorId, - long checkpointId) - throws IOException { - NavigableMap pendingMap = deltaManifestsMap.headMap(checkpointId, true); - List manifests = Lists.newArrayList(); - NavigableMap pendingResults = Maps.newTreeMap(); - for (Map.Entry e : pendingMap.entrySet()) { - if (Arrays.equals(EMPTY_MANIFEST_DATA, e.getValue())) { - // Skip the empty flink manifest. - continue; - } - - DeltaManifests deltaManifests = - SimpleVersionedSerialization.readVersionAndDeSerialize( - DeltaManifestsSerializer.INSTANCE, e.getValue()); - pendingResults.put( - e.getKey(), - FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs())); - manifests.addAll(deltaManifests.manifests()); - } - - CommitSummary summary = new CommitSummary(pendingResults); - commitPendingResult(pendingResults, summary, newFlinkJobId, operatorId, checkpointId); - committerMetrics.updateCommitSummary(summary); - pendingMap.clear(); - deleteCommittedManifests(manifests, newFlinkJobId, checkpointId); - } - - private void commitPendingResult( - NavigableMap pendingResults, - CommitSummary summary, - String newFlinkJobId, - String operatorId, - long checkpointId) { - long totalFiles = summary.dataFilesCount() + summary.deleteFilesCount(); - continuousEmptyCheckpoints = totalFiles == 0 ? continuousEmptyCheckpoints + 1 : 0; - if (totalFiles != 0 || continuousEmptyCheckpoints % maxContinuousEmptyCommits == 0) { - if (replacePartitions) { - replacePartitions(pendingResults, summary, newFlinkJobId, operatorId, checkpointId); - } else { - commitDeltaTxn(pendingResults, summary, newFlinkJobId, operatorId, checkpointId); - } - continuousEmptyCheckpoints = 0; - } else { - LOG.info("Skip commit for checkpoint {} due to no data files or delete files.", checkpointId); - } - } - - private void deleteCommittedManifests( - List manifests, String newFlinkJobId, long checkpointId) { - for (ManifestFile manifest : manifests) { - try { - table.io().deleteFile(manifest.path()); - } catch (Exception e) { - // The flink manifests cleaning failure shouldn't abort the completed checkpoint. - String details = - MoreObjects.toStringHelper(this) - .add("flinkJobId", newFlinkJobId) - .add("checkpointId", checkpointId) - .add("manifestPath", manifest.path()) - .toString(); - LOG.warn( - "The iceberg transaction has been committed, but we failed to clean the temporary flink manifests: {}", - details, - e); - } - } - } - - private void replacePartitions( - NavigableMap pendingResults, - CommitSummary summary, - String newFlinkJobId, - String operatorId, - long checkpointId) { - Preconditions.checkState( - summary.deleteFilesCount() == 0, "Cannot overwrite partitions with delete files."); - // Commit the overwrite transaction. - ReplacePartitions dynamicOverwrite = table.newReplacePartitions().scanManifestsWith(workerPool); - for (WriteResult result : pendingResults.values()) { - Preconditions.checkState( - result.referencedDataFiles().length == 0, "Should have no referenced data files."); - Arrays.stream(result.dataFiles()).forEach(dynamicOverwrite::addFile); - } - - commitOperation( - dynamicOverwrite, - summary, - "dynamic partition overwrite", - newFlinkJobId, - operatorId, - checkpointId); - } - - private void commitDeltaTxn( - NavigableMap pendingResults, - CommitSummary summary, - String newFlinkJobId, - String operatorId, - long checkpointId) { - if (summary.deleteFilesCount() == 0) { - // To be compatible with iceberg format V1. - AppendFiles appendFiles = table.newAppend().scanManifestsWith(workerPool); - for (WriteResult result : pendingResults.values()) { - Preconditions.checkState( - result.referencedDataFiles().length == 0, - "Should have no referenced data files for append."); - Arrays.stream(result.dataFiles()).forEach(appendFiles::appendFile); - } - commitOperation(appendFiles, summary, "append", newFlinkJobId, operatorId, checkpointId); - } else { - // To be compatible with iceberg format V2. - for (Map.Entry e : pendingResults.entrySet()) { - // We don't commit the merged result into a single transaction because for the sequential - // transaction txn1 and txn2, the equality-delete files of txn2 are required to be applied - // to data files from txn1. Committing the merged one will lead to the incorrect delete - // semantic. - WriteResult result = e.getValue(); - - // Row delta validations are not needed for streaming changes that write equality deletes. - // Equality deletes are applied to data in all previous sequence numbers, so retries may - // push deletes further in the future, but do not affect correctness. Position deletes - // committed to the table in this path are used only to delete rows from data files that are - // being added in this commit. There is no way for data files added along with the delete - // files to be concurrently removed, so there is no need to validate the files referenced by - // the position delete files that are being committed. - RowDelta rowDelta = table.newRowDelta().scanManifestsWith(workerPool); - - Arrays.stream(result.dataFiles()).forEach(rowDelta::addRows); - Arrays.stream(result.deleteFiles()).forEach(rowDelta::addDeletes); - commitOperation(rowDelta, summary, "rowDelta", newFlinkJobId, operatorId, e.getKey()); - } - } - } - - private void commitOperation( - SnapshotUpdate operation, - CommitSummary summary, - String description, - String newFlinkJobId, - String operatorId, - long checkpointId) { - LOG.info( - "Committing {} for checkpoint {} to table {} branch {} with summary: {}", - description, - checkpointId, - table.name(), - branch, - summary); - snapshotProperties.forEach(operation::set); - // custom snapshot metadata properties will be overridden if they conflict with internal ones - // used by the sink. - operation.set(MAX_COMMITTED_CHECKPOINT_ID, Long.toString(checkpointId)); - operation.set(FLINK_JOB_ID, newFlinkJobId); - operation.set(OPERATOR_ID, operatorId); - operation.toBranch(branch); - - long startNano = System.nanoTime(); - operation.commit(); // abort is automatically called if this fails. - long durationMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano); - LOG.info( - "Committed {} to table: {}, branch: {}, checkpointId {} in {} ms", - description, - table.name(), - branch, - checkpointId, - durationMs); - committerMetrics.commitDuration(durationMs); - } - - @Override - public void processElement(StreamRecord element) { - this.writeResultsOfCurrentCkpt.add(element.getValue()); - } - - @Override - public void endInput() throws IOException { - // Flush the buffered data files into 'dataFilesPerCheckpoint' firstly. - long currentCheckpointId = Long.MAX_VALUE; - dataFilesPerCheckpoint.put(currentCheckpointId, writeToManifest(currentCheckpointId)); - writeResultsOfCurrentCkpt.clear(); - - commitUpToCheckpoint(dataFilesPerCheckpoint, flinkJobId, operatorUniqueId, currentCheckpointId); - } - - /** - * Write all the complete data files to a newly created manifest file and return the manifest's - * avro serialized bytes. - */ - private byte[] writeToManifest(long checkpointId) throws IOException { - if (writeResultsOfCurrentCkpt.isEmpty()) { - return EMPTY_MANIFEST_DATA; - } - - WriteResult result = WriteResult.builder().addAll(writeResultsOfCurrentCkpt).build(); - DeltaManifests deltaManifests = - FlinkManifestUtil.writeCompletedFiles( - result, () -> manifestOutputFileFactory.create(checkpointId), spec); - - return SimpleVersionedSerialization.writeVersionAndSerialize( - DeltaManifestsSerializer.INSTANCE, deltaManifests); - } - - @Override - public void open() throws Exception { - super.open(); - - final String operatorID = getRuntimeContext().getOperatorUniqueID(); - this.workerPool = - ThreadPools.newWorkerPool("iceberg-worker-pool-" + operatorID, workerPoolSize); - } - - @Override - public void close() throws Exception { - if (tableLoader != null) { - tableLoader.close(); - } - - if (workerPool != null) { - workerPool.shutdown(); - } - } - - @VisibleForTesting - static ListStateDescriptor> buildStateDescriptor() { - Comparator longComparator = Comparators.forType(Types.LongType.get()); - // Construct a SortedMapTypeInfo. - SortedMapTypeInfo sortedMapTypeInfo = - new SortedMapTypeInfo<>( - BasicTypeInfo.LONG_TYPE_INFO, - PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO, - longComparator); - return new ListStateDescriptor<>("iceberg-files-committer-state", sortedMapTypeInfo); - } - - static long getMaxCommittedCheckpointId( - Table table, String flinkJobId, String operatorId, String branch) { - Snapshot snapshot = table.snapshot(branch); - long lastCommittedCheckpointId = INITIAL_CHECKPOINT_ID; - - while (snapshot != null) { - Map summary = snapshot.summary(); - String snapshotFlinkJobId = summary.get(FLINK_JOB_ID); - String snapshotOperatorId = summary.get(OPERATOR_ID); - if (flinkJobId.equals(snapshotFlinkJobId) - && (snapshotOperatorId == null || snapshotOperatorId.equals(operatorId))) { - String value = summary.get(MAX_COMMITTED_CHECKPOINT_ID); - if (value != null) { - lastCommittedCheckpointId = Long.parseLong(value); - break; - } - } - Long parentSnapshotId = snapshot.parentId(); - snapshot = parentSnapshotId != null ? table.snapshot(parentSnapshotId) : null; - } - - return lastCommittedCheckpointId; - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java deleted file mode 100644 index 9de0d6aaa551..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java +++ /dev/null @@ -1,96 +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.iceberg.flink.sink; - -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; -import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Gauge; -import org.apache.flink.metrics.MetricGroup; - -class IcebergFilesCommitterMetrics { - private final AtomicLong lastCheckpointDurationMs = new AtomicLong(); - private final AtomicLong lastCommitDurationMs = new AtomicLong(); - private final ElapsedTimeGauge elapsedSecondsSinceLastSuccessfulCommit; - private final Counter committedDataFilesCount; - private final Counter committedDataFilesRecordCount; - private final Counter committedDataFilesByteCount; - private final Counter committedDeleteFilesCount; - private final Counter committedDeleteFilesRecordCount; - private final Counter committedDeleteFilesByteCount; - - IcebergFilesCommitterMetrics(MetricGroup metrics, String fullTableName) { - MetricGroup committerMetrics = - metrics.addGroup("IcebergFilesCommitter").addGroup("table", fullTableName); - committerMetrics.gauge("lastCheckpointDurationMs", lastCheckpointDurationMs::get); - committerMetrics.gauge("lastCommitDurationMs", lastCommitDurationMs::get); - this.elapsedSecondsSinceLastSuccessfulCommit = new ElapsedTimeGauge(TimeUnit.SECONDS); - committerMetrics.gauge( - "elapsedSecondsSinceLastSuccessfulCommit", elapsedSecondsSinceLastSuccessfulCommit); - this.committedDataFilesCount = committerMetrics.counter("committedDataFilesCount"); - this.committedDataFilesRecordCount = committerMetrics.counter("committedDataFilesRecordCount"); - this.committedDataFilesByteCount = committerMetrics.counter("committedDataFilesByteCount"); - this.committedDeleteFilesCount = committerMetrics.counter("committedDeleteFilesCount"); - this.committedDeleteFilesRecordCount = - committerMetrics.counter("committedDeleteFilesRecordCount"); - this.committedDeleteFilesByteCount = committerMetrics.counter("committedDeleteFilesByteCount"); - } - - void checkpointDuration(long checkpointDurationMs) { - lastCheckpointDurationMs.set(checkpointDurationMs); - } - - void commitDuration(long commitDurationMs) { - lastCommitDurationMs.set(commitDurationMs); - } - - /** This is called upon a successful commit. */ - void updateCommitSummary(CommitSummary stats) { - elapsedSecondsSinceLastSuccessfulCommit.refreshLastRecordedTime(); - committedDataFilesCount.inc(stats.dataFilesCount()); - committedDataFilesRecordCount.inc(stats.dataFilesRecordCount()); - committedDataFilesByteCount.inc(stats.dataFilesByteCount()); - committedDeleteFilesCount.inc(stats.deleteFilesCount()); - committedDeleteFilesRecordCount.inc(stats.deleteFilesRecordCount()); - committedDeleteFilesByteCount.inc(stats.deleteFilesByteCount()); - } - - /** - * This gauge measures the elapsed time between now and last recorded time set by {@link - * ElapsedTimeGauge#refreshLastRecordedTime()}. - */ - private static class ElapsedTimeGauge implements Gauge { - private final TimeUnit reportUnit; - private volatile long lastRecordedTimeNano; - - ElapsedTimeGauge(TimeUnit timeUnit) { - this.reportUnit = timeUnit; - this.lastRecordedTimeNano = System.nanoTime(); - } - - void refreshLastRecordedTime() { - this.lastRecordedTimeNano = System.nanoTime(); - } - - @Override - public Long getValue() { - return reportUnit.convert(System.nanoTime() - lastRecordedTimeNano, TimeUnit.NANOSECONDS); - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java deleted file mode 100644 index 9ea0349fb057..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java +++ /dev/null @@ -1,120 +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.iceberg.flink.sink; - -import java.io.IOException; -import java.util.concurrent.TimeUnit; -import org.apache.flink.streaming.api.operators.AbstractStreamOperator; -import org.apache.flink.streaming.api.operators.BoundedOneInput; -import org.apache.flink.streaming.api.operators.ChainingStrategy; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.WriteResult; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; - -class IcebergStreamWriter extends AbstractStreamOperator - implements OneInputStreamOperator, BoundedOneInput { - - private static final long serialVersionUID = 1L; - - private final String fullTableName; - private final TaskWriterFactory taskWriterFactory; - - private transient TaskWriter writer; - private transient int subTaskId; - private transient int attemptId; - private transient IcebergStreamWriterMetrics writerMetrics; - - IcebergStreamWriter(String fullTableName, TaskWriterFactory taskWriterFactory) { - this.fullTableName = fullTableName; - this.taskWriterFactory = taskWriterFactory; - setChainingStrategy(ChainingStrategy.ALWAYS); - } - - @Override - public void open() { - this.subTaskId = getRuntimeContext().getIndexOfThisSubtask(); - this.attemptId = getRuntimeContext().getAttemptNumber(); - this.writerMetrics = new IcebergStreamWriterMetrics(super.metrics, fullTableName); - - // Initialize the task writer factory. - this.taskWriterFactory.initialize(subTaskId, attemptId); - - // Initialize the task writer. - this.writer = taskWriterFactory.create(); - } - - @Override - public void prepareSnapshotPreBarrier(long checkpointId) throws Exception { - flush(); - this.writer = taskWriterFactory.create(); - } - - @Override - public void processElement(StreamRecord element) throws Exception { - writer.write(element.getValue()); - } - - @Override - public void close() throws Exception { - super.close(); - if (writer != null) { - writer.close(); - writer = null; - } - } - - @Override - public void endInput() throws IOException { - // For bounded stream, it may don't enable the checkpoint mechanism so we'd better to emit the - // remaining completed files to downstream before closing the writer so that we won't miss any - // of them. - // Note that if the task is not closed after calling endInput, checkpoint may be triggered again - // causing files to be sent repeatedly, the writer is marked as null after the last file is sent - // to guard against duplicated writes. - flush(); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("table_name", fullTableName) - .add("subtask_id", subTaskId) - .add("attempt_id", attemptId) - .toString(); - } - - /** close all open files and emit files to downstream committer operator */ - private void flush() throws IOException { - if (writer == null) { - return; - } - - long startNano = System.nanoTime(); - WriteResult result = writer.complete(); - writerMetrics.updateFlushResult(result); - output.collect(new StreamRecord<>(result)); - writerMetrics.flushDuration(TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); - - // Set writer to null to prevent duplicate flushes in the corner case of - // prepareSnapshotPreBarrier happening after endInput. - writer = null; - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java deleted file mode 100644 index ce2a6c583fdf..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java +++ /dev/null @@ -1,89 +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.iceberg.flink.sink; - -import com.codahale.metrics.SlidingWindowReservoir; -import java.util.Arrays; -import java.util.concurrent.atomic.AtomicLong; -import org.apache.flink.dropwizard.metrics.DropwizardHistogramWrapper; -import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Histogram; -import org.apache.flink.metrics.MetricGroup; -import org.apache.iceberg.io.WriteResult; - -class IcebergStreamWriterMetrics { - // 1,024 reservoir size should cost about 8KB, which is quite small. - // It should also produce good accuracy for histogram distribution (like percentiles). - private static final int HISTOGRAM_RESERVOIR_SIZE = 1024; - - private final Counter flushedDataFiles; - private final Counter flushedDeleteFiles; - private final Counter flushedReferencedDataFiles; - private final AtomicLong lastFlushDurationMs; - private final Histogram dataFilesSizeHistogram; - private final Histogram deleteFilesSizeHistogram; - - IcebergStreamWriterMetrics(MetricGroup metrics, String fullTableName) { - MetricGroup writerMetrics = - metrics.addGroup("IcebergStreamWriter").addGroup("table", fullTableName); - this.flushedDataFiles = writerMetrics.counter("flushedDataFiles"); - this.flushedDeleteFiles = writerMetrics.counter("flushedDeleteFiles"); - this.flushedReferencedDataFiles = writerMetrics.counter("flushedReferencedDataFiles"); - this.lastFlushDurationMs = new AtomicLong(); - writerMetrics.gauge("lastFlushDurationMs", lastFlushDurationMs::get); - - com.codahale.metrics.Histogram dropwizardDataFilesSizeHistogram = - new com.codahale.metrics.Histogram(new SlidingWindowReservoir(HISTOGRAM_RESERVOIR_SIZE)); - this.dataFilesSizeHistogram = - writerMetrics.histogram( - "dataFilesSizeHistogram", - new DropwizardHistogramWrapper(dropwizardDataFilesSizeHistogram)); - com.codahale.metrics.Histogram dropwizardDeleteFilesSizeHistogram = - new com.codahale.metrics.Histogram(new SlidingWindowReservoir(HISTOGRAM_RESERVOIR_SIZE)); - this.deleteFilesSizeHistogram = - writerMetrics.histogram( - "deleteFilesSizeHistogram", - new DropwizardHistogramWrapper(dropwizardDeleteFilesSizeHistogram)); - } - - void updateFlushResult(WriteResult result) { - flushedDataFiles.inc(result.dataFiles().length); - flushedDeleteFiles.inc(result.deleteFiles().length); - flushedReferencedDataFiles.inc(result.referencedDataFiles().length); - - // For file size distribution histogram, we don't have to update them after successful commits. - // This should works equally well and we avoided the overhead of tracking the list of file sizes - // in the {@link CommitSummary}, which currently stores simple stats for counters and gauges - // metrics. - Arrays.stream(result.dataFiles()) - .forEach( - dataFile -> { - dataFilesSizeHistogram.update(dataFile.fileSizeInBytes()); - }); - Arrays.stream(result.deleteFiles()) - .forEach( - deleteFile -> { - deleteFilesSizeHistogram.update(deleteFile.fileSizeInBytes()); - }); - } - - void flushDuration(long flushDurationMs) { - lastFlushDurationMs.set(flushDurationMs); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java deleted file mode 100644 index da5e6e7627ae..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java +++ /dev/null @@ -1,94 +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.iceberg.flink.sink; - -import java.util.Map; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Supplier; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.HasTableOperations; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableOperations; -import org.apache.iceberg.io.OutputFile; -import org.apache.iceberg.relocated.com.google.common.base.Strings; - -class ManifestOutputFileFactory { - // Users could define their own flink manifests directory by setting this value in table - // properties. - static final String FLINK_MANIFEST_LOCATION = "flink.manifests.location"; - - private final Supplier
    tableSupplier; - private final Map props; - private final String flinkJobId; - private final String operatorUniqueId; - private final int subTaskId; - private final long attemptNumber; - private final AtomicInteger fileCount = new AtomicInteger(0); - - ManifestOutputFileFactory( - Supplier
    tableSupplier, - Map props, - String flinkJobId, - String operatorUniqueId, - int subTaskId, - long attemptNumber) { - this.tableSupplier = tableSupplier; - this.props = props; - this.flinkJobId = flinkJobId; - this.operatorUniqueId = operatorUniqueId; - this.subTaskId = subTaskId; - this.attemptNumber = attemptNumber; - } - - private String generatePath(long checkpointId) { - return FileFormat.AVRO.addExtension( - String.format( - "%s-%s-%05d-%d-%d-%05d", - flinkJobId, - operatorUniqueId, - subTaskId, - attemptNumber, - checkpointId, - fileCount.incrementAndGet())); - } - - OutputFile create(long checkpointId) { - String flinkManifestDir = props.get(FLINK_MANIFEST_LOCATION); - TableOperations ops = ((HasTableOperations) tableSupplier.get()).operations(); - - String newManifestFullPath; - if (Strings.isNullOrEmpty(flinkManifestDir)) { - // User don't specify any flink manifest directory, so just use the default metadata path. - newManifestFullPath = ops.metadataFileLocation(generatePath(checkpointId)); - } else { - newManifestFullPath = - String.format("%s/%s", stripTrailingSlash(flinkManifestDir), generatePath(checkpointId)); - } - - return tableSupplier.get().io().newOutputFile(newManifestFullPath); - } - - private static String stripTrailingSlash(String path) { - String result = path; - while (result.endsWith("/")) { - result = result.substring(0, result.length() - 1); - } - return result; - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java deleted file mode 100644 index df951684b446..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java +++ /dev/null @@ -1,64 +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.iceberg.flink.sink; - -import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.PartitionKey; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.RowDataWrapper; - -/** - * Create a {@link KeySelector} to shuffle by partition key, then each partition/bucket will be - * wrote by only one task. That will reduce lots of small files in partitioned fanout write policy - * for {@link FlinkSink}. - */ -class PartitionKeySelector implements KeySelector { - - private final Schema schema; - private final PartitionKey partitionKey; - private final RowType flinkSchema; - - private transient RowDataWrapper rowDataWrapper; - - PartitionKeySelector(PartitionSpec spec, Schema schema, RowType flinkSchema) { - this.schema = schema; - this.partitionKey = new PartitionKey(spec, schema); - this.flinkSchema = flinkSchema; - } - - /** - * Construct the {@link RowDataWrapper} lazily here because few members in it are not - * serializable. In this way, we don't have to serialize them with forcing. - */ - private RowDataWrapper lazyRowDataWrapper() { - if (rowDataWrapper == null) { - rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); - } - return rowDataWrapper; - } - - @Override - public String getKey(RowData row) { - partitionKey.partition(lazyRowDataWrapper().wrap(row)); - return partitionKey.toPath(); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java deleted file mode 100644 index 38062dd1a2c4..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java +++ /dev/null @@ -1,97 +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.iceberg.flink.sink; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.List; -import java.util.Map; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionKey; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.io.OutputFileFactory; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.util.Tasks; - -class PartitionedDeltaWriter extends BaseDeltaTaskWriter { - - private final PartitionKey partitionKey; - - private final Map writers = Maps.newHashMap(); - - PartitionedDeltaWriter( - PartitionSpec spec, - FileFormat format, - FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, - FileIO io, - long targetFileSize, - Schema schema, - RowType flinkSchema, - List equalityFieldIds, - boolean upsert) { - super( - spec, - format, - appenderFactory, - fileFactory, - io, - targetFileSize, - schema, - flinkSchema, - equalityFieldIds, - upsert); - this.partitionKey = new PartitionKey(spec, schema); - } - - @Override - RowDataDeltaWriter route(RowData 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/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java deleted file mode 100644 index 67422a1afeb1..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java +++ /dev/null @@ -1,244 +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.iceberg.flink.sink; - -import java.util.List; -import java.util.Map; -import java.util.function.Supplier; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionKey; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.io.OutputFileFactory; -import org.apache.iceberg.io.PartitionedFanoutWriter; -import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.UnpartitionedWriter; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.util.ArrayUtil; -import org.apache.iceberg.util.SerializableSupplier; - -public class RowDataTaskWriterFactory implements TaskWriterFactory { - private final Supplier
    tableSupplier; - private final Schema schema; - private final RowType flinkSchema; - private final PartitionSpec spec; - private final long targetFileSizeBytes; - private final FileFormat format; - private final List equalityFieldIds; - private final boolean upsert; - private final FileAppenderFactory appenderFactory; - - private transient OutputFileFactory outputFileFactory; - - public RowDataTaskWriterFactory( - Table table, - RowType flinkSchema, - long targetFileSizeBytes, - FileFormat format, - Map writeProperties, - List equalityFieldIds, - boolean upsert) { - this( - () -> table, - flinkSchema, - targetFileSizeBytes, - format, - writeProperties, - equalityFieldIds, - upsert); - } - - public RowDataTaskWriterFactory( - SerializableSupplier
    tableSupplier, - RowType flinkSchema, - long targetFileSizeBytes, - FileFormat format, - Map writeProperties, - List equalityFieldIds, - boolean upsert) { - this.tableSupplier = tableSupplier; - - Table table; - if (tableSupplier instanceof CachingTableSupplier) { - // rely on the initial table metadata for schema, etc., until schema evolution is supported - table = ((CachingTableSupplier) tableSupplier).initialTable(); - } else { - table = tableSupplier.get(); - } - - this.schema = table.schema(); - this.flinkSchema = flinkSchema; - this.spec = table.spec(); - this.targetFileSizeBytes = targetFileSizeBytes; - this.format = format; - this.equalityFieldIds = equalityFieldIds; - this.upsert = upsert; - - if (equalityFieldIds == null || equalityFieldIds.isEmpty()) { - this.appenderFactory = - new FlinkAppenderFactory( - table, schema, flinkSchema, writeProperties, spec, null, null, null); - } else if (upsert) { - // In upsert mode, only the new row is emitted using INSERT row kind. Therefore, any column of - // the inserted row - // may differ from the deleted row other than the primary key fields, and the delete file must - // contain values - // that are correct for the deleted row. Therefore, only write the equality delete fields. - this.appenderFactory = - new FlinkAppenderFactory( - table, - schema, - flinkSchema, - writeProperties, - spec, - ArrayUtil.toIntArray(equalityFieldIds), - TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds)), - null); - } else { - this.appenderFactory = - new FlinkAppenderFactory( - table, - schema, - flinkSchema, - writeProperties, - spec, - ArrayUtil.toIntArray(equalityFieldIds), - schema, - null); - } - } - - @Override - public void initialize(int taskId, int attemptId) { - Table table; - if (tableSupplier instanceof CachingTableSupplier) { - // rely on the initial table metadata for schema, etc., until schema evolution is supported - table = ((CachingTableSupplier) tableSupplier).initialTable(); - } else { - table = tableSupplier.get(); - } - - refreshTable(); - - this.outputFileFactory = - OutputFileFactory.builderFor(table, taskId, attemptId) - .format(format) - .ioSupplier(() -> tableSupplier.get().io()) - .build(); - } - - @Override - public TaskWriter create() { - Preconditions.checkNotNull( - outputFileFactory, - "The outputFileFactory shouldn't be null if we have invoked the initialize()."); - - refreshTable(); - - if (equalityFieldIds == null || equalityFieldIds.isEmpty()) { - // Initialize a task writer to write INSERT only. - if (spec.isUnpartitioned()) { - return new UnpartitionedWriter<>( - spec, - format, - appenderFactory, - outputFileFactory, - tableSupplier.get().io(), - targetFileSizeBytes); - } else { - return new RowDataPartitionedFanoutWriter( - spec, - format, - appenderFactory, - outputFileFactory, - tableSupplier.get().io(), - targetFileSizeBytes, - schema, - flinkSchema); - } - } else { - // Initialize a task writer to write both INSERT and equality DELETE. - if (spec.isUnpartitioned()) { - return new UnpartitionedDeltaWriter( - spec, - format, - appenderFactory, - outputFileFactory, - tableSupplier.get().io(), - targetFileSizeBytes, - schema, - flinkSchema, - equalityFieldIds, - upsert); - } else { - return new PartitionedDeltaWriter( - spec, - format, - appenderFactory, - outputFileFactory, - tableSupplier.get().io(), - targetFileSizeBytes, - schema, - flinkSchema, - equalityFieldIds, - upsert); - } - } - } - - void refreshTable() { - if (tableSupplier instanceof CachingTableSupplier) { - ((CachingTableSupplier) tableSupplier).refreshTable(); - } - } - - private static class RowDataPartitionedFanoutWriter extends PartitionedFanoutWriter { - - private final PartitionKey partitionKey; - private final RowDataWrapper rowDataWrapper; - - RowDataPartitionedFanoutWriter( - PartitionSpec spec, - FileFormat format, - FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, - FileIO io, - long targetFileSize, - Schema schema, - RowType flinkSchema) { - super(spec, format, appenderFactory, fileFactory, io, targetFileSize); - this.partitionKey = new PartitionKey(spec, schema); - this.rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); - } - - @Override - protected PartitionKey partition(RowData row) { - partitionKey.partition(rowDataWrapper.wrap(row)); - return partitionKey; - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java deleted file mode 100644 index e3a1245e8cbd..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java +++ /dev/null @@ -1,45 +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.iceberg.flink.sink; - -import java.io.Serializable; -import org.apache.iceberg.io.TaskWriter; - -/** - * Factory to create {@link TaskWriter} - * - * @param data type of record. - */ -public interface TaskWriterFactory extends Serializable { - - /** - * Initialize the factory with a given taskId and attemptId. - * - * @param taskId the identifier of task. - * @param attemptId the attempt id of this task. - */ - void initialize(int taskId, int attemptId); - - /** - * Initialize a {@link TaskWriter} with given task id and attempt id. - * - * @return a newly created task writer. - */ - TaskWriter create(); -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java deleted file mode 100644 index 7680fb933b20..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java +++ /dev/null @@ -1,69 +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.iceberg.flink.sink; - -import java.io.IOException; -import java.util.List; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.io.OutputFileFactory; - -class UnpartitionedDeltaWriter extends BaseDeltaTaskWriter { - private final RowDataDeltaWriter writer; - - UnpartitionedDeltaWriter( - PartitionSpec spec, - FileFormat format, - FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, - FileIO io, - long targetFileSize, - Schema schema, - RowType flinkSchema, - List equalityFieldIds, - boolean upsert) { - super( - spec, - format, - appenderFactory, - fileFactory, - io, - targetFileSize, - schema, - flinkSchema, - equalityFieldIds, - upsert); - this.writer = new RowDataDeltaWriter(null); - } - - @Override - RowDataDeltaWriter route(RowData row) { - return writer; - } - - @Override - public void close() throws IOException { - writer.close(); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java deleted file mode 100644 index 157f04b8b0ed..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java +++ /dev/null @@ -1,71 +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.iceberg.flink.sink.shuffle; - -import java.io.Serializable; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** - * AggregatedStatistics is used by {@link DataStatisticsCoordinator} to collect {@link - * DataStatistics} from {@link DataStatisticsOperator} subtasks for specific checkpoint. It stores - * the merged {@link DataStatistics} result from all reported subtasks. - */ -class AggregatedStatistics, S> implements Serializable { - - private final long checkpointId; - private final DataStatistics dataStatistics; - - AggregatedStatistics(long checkpoint, TypeSerializer> statisticsSerializer) { - this.checkpointId = checkpoint; - this.dataStatistics = statisticsSerializer.createInstance(); - } - - AggregatedStatistics(long checkpoint, DataStatistics dataStatistics) { - this.checkpointId = checkpoint; - this.dataStatistics = dataStatistics; - } - - long checkpointId() { - return checkpointId; - } - - DataStatistics dataStatistics() { - return dataStatistics; - } - - void mergeDataStatistic(String operatorName, long eventCheckpointId, D eventDataStatistics) { - Preconditions.checkArgument( - checkpointId == eventCheckpointId, - "Received unexpected event from operator %s checkpoint %s. Expected checkpoint %s", - operatorName, - eventCheckpointId, - checkpointId); - dataStatistics.merge(eventDataStatistics); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("checkpointId", checkpointId) - .add("dataStatistics", dataStatistics) - .toString(); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java deleted file mode 100644 index e8ff61dbeb27..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java +++ /dev/null @@ -1,133 +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.iceberg.flink.sink.shuffle; - -import java.util.Set; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * AggregatedStatisticsTracker is used by {@link DataStatisticsCoordinator} to track the in progress - * {@link AggregatedStatistics} received from {@link DataStatisticsOperator} subtasks for specific - * checkpoint. - */ -class AggregatedStatisticsTracker, S> { - private static final Logger LOG = LoggerFactory.getLogger(AggregatedStatisticsTracker.class); - private static final double ACCEPT_PARTIAL_AGGR_THRESHOLD = 90; - private final String operatorName; - private final TypeSerializer> statisticsSerializer; - private final int parallelism; - private final Set inProgressSubtaskSet; - private volatile AggregatedStatistics inProgressStatistics; - - AggregatedStatisticsTracker( - String operatorName, - TypeSerializer> statisticsSerializer, - int parallelism) { - this.operatorName = operatorName; - this.statisticsSerializer = statisticsSerializer; - this.parallelism = parallelism; - this.inProgressSubtaskSet = Sets.newHashSet(); - } - - AggregatedStatistics updateAndCheckCompletion( - int subtask, DataStatisticsEvent event) { - long checkpointId = event.checkpointId(); - - if (inProgressStatistics != null && inProgressStatistics.checkpointId() > checkpointId) { - LOG.info( - "Expect data statistics for operator {} checkpoint {}, but receive event from older checkpoint {}. Ignore it.", - operatorName, - inProgressStatistics.checkpointId(), - checkpointId); - return null; - } - - AggregatedStatistics completedStatistics = null; - if (inProgressStatistics != null && inProgressStatistics.checkpointId() < checkpointId) { - if ((double) inProgressSubtaskSet.size() / parallelism * 100 - >= ACCEPT_PARTIAL_AGGR_THRESHOLD) { - completedStatistics = inProgressStatistics; - LOG.info( - "Received data statistics from {} subtasks out of total {} for operator {} at checkpoint {}. " - + "Complete data statistics aggregation at checkpoint {} as it is more than the threshold of {} percentage", - inProgressSubtaskSet.size(), - parallelism, - operatorName, - checkpointId, - inProgressStatistics.checkpointId(), - ACCEPT_PARTIAL_AGGR_THRESHOLD); - } else { - LOG.info( - "Received data statistics from {} subtasks out of total {} for operator {} at checkpoint {}. " - + "Aborting the incomplete aggregation for checkpoint {}", - inProgressSubtaskSet.size(), - parallelism, - operatorName, - checkpointId, - inProgressStatistics.checkpointId()); - } - - inProgressStatistics = null; - inProgressSubtaskSet.clear(); - } - - if (inProgressStatistics == null) { - LOG.info("Starting a new data statistics for checkpoint {}", checkpointId); - inProgressStatistics = new AggregatedStatistics<>(checkpointId, statisticsSerializer); - inProgressSubtaskSet.clear(); - } - - if (!inProgressSubtaskSet.add(subtask)) { - LOG.debug( - "Ignore duplicated data statistics from operator {} subtask {} for checkpoint {}.", - operatorName, - subtask, - checkpointId); - } else { - inProgressStatistics.mergeDataStatistic( - operatorName, - event.checkpointId(), - DataStatisticsUtil.deserializeDataStatistics( - event.statisticsBytes(), statisticsSerializer)); - } - - if (inProgressSubtaskSet.size() == parallelism) { - completedStatistics = inProgressStatistics; - LOG.info( - "Received data statistics from all {} operators {} for checkpoint {}. Return last completed aggregator {}.", - parallelism, - operatorName, - inProgressStatistics.checkpointId(), - completedStatistics.dataStatistics()); - inProgressStatistics = new AggregatedStatistics<>(checkpointId + 1, statisticsSerializer); - inProgressSubtaskSet.clear(); - } - - return completedStatistics; - } - - @VisibleForTesting - AggregatedStatistics inProgressStatistics() { - return inProgressStatistics; - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java deleted file mode 100644 index 28a05201c02f..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java +++ /dev/null @@ -1,61 +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.iceberg.flink.sink.shuffle; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.table.data.RowData; - -/** - * DataStatistics defines the interface to collect data distribution information. - * - *

    Data statistics tracks traffic volume distribution across data keys. For low-cardinality key, - * a simple map of (key, count) can be used. For high-cardinality key, probabilistic data structures - * (sketching) can be used. - */ -@Internal -interface DataStatistics { - - /** - * Check if data statistics contains any statistics information. - * - * @return true if data statistics doesn't contain any statistics information - */ - boolean isEmpty(); - - /** - * Add data key to data statistics. - * - * @param key generate from data by applying key selector - */ - void add(RowData key); - - /** - * Merge current statistics with other statistics. - * - * @param otherStatistics the statistics to be merged - */ - void merge(D otherStatistics); - - /** - * Get the underline statistics. - * - * @return the underline statistics - */ - S statistics(); -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java deleted file mode 100644 index 26fb3c328481..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java +++ /dev/null @@ -1,322 +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.iceberg.flink.sink.shuffle; - -import java.util.concurrent.Callable; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.ThreadFactory; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; -import org.apache.flink.runtime.operators.coordination.OperatorEvent; -import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.util.FatalExitExceptionHandler; -import org.apache.flink.util.FlinkRuntimeException; -import org.apache.flink.util.Preconditions; -import org.apache.flink.util.ThrowableCatchingRunnable; -import org.apache.flink.util.function.ThrowingRunnable; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * DataStatisticsCoordinator receives {@link DataStatisticsEvent} from {@link - * DataStatisticsOperator} every subtask and then merge them together. Once aggregation for all - * subtasks data statistics completes, DataStatisticsCoordinator will send the aggregated data - * statistics back to {@link DataStatisticsOperator}. In the end a custom partitioner will - * distribute traffic based on the aggregated data statistics to improve data clustering. - */ -@Internal -class DataStatisticsCoordinator, S> implements OperatorCoordinator { - private static final Logger LOG = LoggerFactory.getLogger(DataStatisticsCoordinator.class); - - private final String operatorName; - private final ExecutorService coordinatorExecutor; - private final OperatorCoordinator.Context operatorCoordinatorContext; - private final OperatorCoordinator.SubtaskGateway[] subtaskGateways; - private final CoordinatorExecutorThreadFactory coordinatorThreadFactory; - private final TypeSerializer> statisticsSerializer; - private final transient AggregatedStatisticsTracker aggregatedStatisticsTracker; - private volatile AggregatedStatistics completedStatistics; - private volatile boolean started; - - DataStatisticsCoordinator( - String operatorName, - OperatorCoordinator.Context context, - TypeSerializer> statisticsSerializer) { - this.operatorName = operatorName; - this.coordinatorThreadFactory = - new CoordinatorExecutorThreadFactory( - "DataStatisticsCoordinator-" + operatorName, context.getUserCodeClassloader()); - this.coordinatorExecutor = Executors.newSingleThreadExecutor(coordinatorThreadFactory); - this.operatorCoordinatorContext = context; - this.subtaskGateways = - new OperatorCoordinator.SubtaskGateway[operatorCoordinatorContext.currentParallelism()]; - this.statisticsSerializer = statisticsSerializer; - this.aggregatedStatisticsTracker = - new AggregatedStatisticsTracker<>(operatorName, statisticsSerializer, parallelism()); - } - - @Override - public void start() throws Exception { - LOG.info("Starting data statistics coordinator: {}.", operatorName); - started = true; - } - - @Override - public void close() throws Exception { - coordinatorExecutor.shutdown(); - LOG.info("Closed data statistics coordinator: {}.", operatorName); - } - - @VisibleForTesting - void callInCoordinatorThread(Callable callable, String errorMessage) { - ensureStarted(); - // Ensure the task is done by the coordinator executor. - if (!coordinatorThreadFactory.isCurrentThreadCoordinatorThread()) { - try { - Callable guardedCallable = - () -> { - try { - return callable.call(); - } catch (Throwable t) { - LOG.error( - "Uncaught Exception in data statistics coordinator: {} executor", - operatorName, - t); - ExceptionUtils.rethrowException(t); - return null; - } - }; - - coordinatorExecutor.submit(guardedCallable).get(); - } catch (InterruptedException | ExecutionException e) { - throw new FlinkRuntimeException(errorMessage, e); - } - } else { - try { - callable.call(); - } catch (Throwable t) { - LOG.error( - "Uncaught Exception in data statistics coordinator: {} executor", operatorName, t); - throw new FlinkRuntimeException(errorMessage, t); - } - } - } - - public void runInCoordinatorThread(Runnable runnable) { - this.coordinatorExecutor.execute( - new ThrowableCatchingRunnable( - throwable -> - this.coordinatorThreadFactory.uncaughtException(Thread.currentThread(), throwable), - runnable)); - } - - private void runInCoordinatorThread(ThrowingRunnable action, String actionString) { - ensureStarted(); - runInCoordinatorThread( - () -> { - try { - action.run(); - } catch (Throwable t) { - ExceptionUtils.rethrowIfFatalErrorOrOOM(t); - LOG.error( - "Uncaught exception in the data statistics coordinator: {} while {}. Triggering job failover", - operatorName, - actionString, - t); - operatorCoordinatorContext.failJob(t); - } - }); - } - - private void ensureStarted() { - Preconditions.checkState(started, "The coordinator of %s has not started yet.", operatorName); - } - - private int parallelism() { - return operatorCoordinatorContext.currentParallelism(); - } - - private void handleDataStatisticRequest(int subtask, DataStatisticsEvent event) { - AggregatedStatistics aggregatedStatistics = - aggregatedStatisticsTracker.updateAndCheckCompletion(subtask, event); - - if (aggregatedStatistics != null) { - completedStatistics = aggregatedStatistics; - sendDataStatisticsToSubtasks( - completedStatistics.checkpointId(), completedStatistics.dataStatistics()); - } - } - - private void sendDataStatisticsToSubtasks( - long checkpointId, DataStatistics globalDataStatistics) { - callInCoordinatorThread( - () -> { - DataStatisticsEvent dataStatisticsEvent = - DataStatisticsEvent.create(checkpointId, globalDataStatistics, statisticsSerializer); - int parallelism = parallelism(); - for (int i = 0; i < parallelism; ++i) { - subtaskGateways[i].sendEvent(dataStatisticsEvent); - } - - return null; - }, - String.format( - "Failed to send operator %s coordinator global data statistics for checkpoint %d", - operatorName, checkpointId)); - } - - @Override - public void handleEventFromOperator(int subtask, OperatorEvent event) throws Exception { - runInCoordinatorThread( - () -> { - LOG.debug("Handling event from subtask {} of {}: {}", subtask, operatorName, event); - Preconditions.checkArgument(event instanceof DataStatisticsEvent); - handleDataStatisticRequest(subtask, ((DataStatisticsEvent) event)); - }, - String.format("handling operator event %s from subtask %d", event.getClass(), subtask)); - } - - @Override - public void checkpointCoordinator(long checkpointId, CompletableFuture resultFuture) { - runInCoordinatorThread( - () -> { - LOG.debug( - "Snapshotting data statistics coordinator {} for checkpoint {}", - operatorName, - checkpointId); - resultFuture.complete( - DataStatisticsUtil.serializeAggregatedStatistics( - completedStatistics, statisticsSerializer)); - }, - String.format("taking checkpoint %d", checkpointId)); - } - - @Override - public void notifyCheckpointComplete(long checkpointId) {} - - @Override - public void resetToCheckpoint(long checkpointId, @Nullable byte[] checkpointData) - throws Exception { - Preconditions.checkState( - !started, "The coordinator %s can only be reset if it was not yet started", operatorName); - - if (checkpointData == null) { - LOG.info( - "Data statistic coordinator {} has nothing to restore from checkpoint {}", - operatorName, - checkpointId); - return; - } - - LOG.info( - "Restoring data statistic coordinator {} from checkpoint {}", operatorName, checkpointId); - completedStatistics = - DataStatisticsUtil.deserializeAggregatedStatistics(checkpointData, statisticsSerializer); - } - - @Override - public void subtaskFailed(int subtask, @Nullable Throwable reason) { - runInCoordinatorThread( - () -> { - LOG.info( - "Unregistering gateway after failure for subtask {} of data statistic {}", - subtask, - operatorName); - Preconditions.checkState( - this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); - subtaskGateways[subtask] = null; - }, - String.format("handling subtask %d failure", subtask)); - } - - @Override - public void subtaskReset(int subtask, long checkpointId) { - LOG.info( - "Data statistic coordinator {} subtask {} is reset to checkpoint {}", - operatorName, - subtask, - checkpointId); - } - - @Override - public void subtaskReady(int subtask, SubtaskGateway gateway) { - Preconditions.checkArgument(subtask == gateway.getSubtask()); - runInCoordinatorThread( - () -> { - Preconditions.checkState( - this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); - subtaskGateways[subtask] = gateway; - }, - String.format("making event gateway to subtask %d available", subtask)); - } - - @VisibleForTesting - AggregatedStatistics completedStatistics() { - return completedStatistics; - } - - private static class CoordinatorExecutorThreadFactory - implements ThreadFactory, Thread.UncaughtExceptionHandler { - - private final String coordinatorThreadName; - private final ClassLoader classLoader; - private final Thread.UncaughtExceptionHandler errorHandler; - - @javax.annotation.Nullable private Thread thread; - - CoordinatorExecutorThreadFactory( - final String coordinatorThreadName, final ClassLoader contextClassLoader) { - this(coordinatorThreadName, contextClassLoader, FatalExitExceptionHandler.INSTANCE); - } - - @org.apache.flink.annotation.VisibleForTesting - CoordinatorExecutorThreadFactory( - final String coordinatorThreadName, - final ClassLoader contextClassLoader, - final Thread.UncaughtExceptionHandler errorHandler) { - this.coordinatorThreadName = coordinatorThreadName; - this.classLoader = contextClassLoader; - this.errorHandler = errorHandler; - } - - @Override - public synchronized Thread newThread(@NotNull Runnable runnable) { - thread = new Thread(runnable, coordinatorThreadName); - thread.setContextClassLoader(classLoader); - thread.setUncaughtExceptionHandler(this); - return thread; - } - - @Override - public synchronized void uncaughtException(Thread t, Throwable e) { - errorHandler.uncaughtException(t, e); - } - - boolean isCurrentThreadCoordinatorThread() { - return Thread.currentThread() == thread; - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java deleted file mode 100644 index 47dbfc3cfbe1..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java +++ /dev/null @@ -1,51 +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.iceberg.flink.sink.shuffle; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.runtime.jobgraph.OperatorID; -import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; -import org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator; - -/** - * DataStatisticsCoordinatorProvider provides the method to create new {@link - * DataStatisticsCoordinator} - */ -@Internal -public class DataStatisticsCoordinatorProvider, S> - extends RecreateOnResetOperatorCoordinator.Provider { - - private final String operatorName; - private final TypeSerializer> statisticsSerializer; - - public DataStatisticsCoordinatorProvider( - String operatorName, - OperatorID operatorID, - TypeSerializer> statisticsSerializer) { - super(operatorID); - this.operatorName = operatorName; - this.statisticsSerializer = statisticsSerializer; - } - - @Override - public OperatorCoordinator getCoordinator(OperatorCoordinator.Context context) { - return new DataStatisticsCoordinator<>(operatorName, context, statisticsSerializer); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java deleted file mode 100644 index 852d2157b8cb..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java +++ /dev/null @@ -1,57 +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.iceberg.flink.sink.shuffle; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.runtime.operators.coordination.OperatorEvent; - -/** - * DataStatisticsEvent is sent between data statistics coordinator and operator to transmit data - * statistics in bytes - */ -@Internal -class DataStatisticsEvent, S> implements OperatorEvent { - - private static final long serialVersionUID = 1L; - private final long checkpointId; - private final byte[] statisticsBytes; - - private DataStatisticsEvent(long checkpointId, byte[] statisticsBytes) { - this.checkpointId = checkpointId; - this.statisticsBytes = statisticsBytes; - } - - static , S> DataStatisticsEvent create( - long checkpointId, - DataStatistics dataStatistics, - TypeSerializer> statisticsSerializer) { - return new DataStatisticsEvent<>( - checkpointId, - DataStatisticsUtil.serializeDataStatistics(dataStatistics, statisticsSerializer)); - } - - long checkpointId() { - return checkpointId; - } - - byte[] statisticsBytes() { - return statisticsBytes; - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java deleted file mode 100644 index d00d5d2e5aa9..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java +++ /dev/null @@ -1,188 +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.iceberg.flink.sink.shuffle; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.state.ListState; -import org.apache.flink.api.common.state.ListStateDescriptor; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.runtime.operators.coordination.OperatorEvent; -import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; -import org.apache.flink.runtime.operators.coordination.OperatorEventHandler; -import org.apache.flink.runtime.state.StateInitializationContext; -import org.apache.flink.runtime.state.StateSnapshotContext; -import org.apache.flink.streaming.api.operators.AbstractStreamOperator; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** - * DataStatisticsOperator collects traffic distribution statistics. A custom partitioner shall be - * attached to the DataStatisticsOperator output. The custom partitioner leverages the statistics to - * shuffle record to improve data clustering while maintaining relative balanced traffic - * distribution to downstream subtasks. - */ -@Internal -class DataStatisticsOperator, S> - extends AbstractStreamOperator> - implements OneInputStreamOperator>, OperatorEventHandler { - private static final long serialVersionUID = 1L; - - private final String operatorName; - // keySelector will be used to generate key from data for collecting data statistics - private final KeySelector keySelector; - private final OperatorEventGateway operatorEventGateway; - private final TypeSerializer> statisticsSerializer; - private transient volatile DataStatistics localStatistics; - private transient volatile DataStatistics globalStatistics; - private transient ListState> globalStatisticsState; - - DataStatisticsOperator( - String operatorName, - KeySelector keySelector, - OperatorEventGateway operatorEventGateway, - TypeSerializer> statisticsSerializer) { - this.operatorName = operatorName; - this.keySelector = keySelector; - this.operatorEventGateway = operatorEventGateway; - this.statisticsSerializer = statisticsSerializer; - } - - @Override - public void initializeState(StateInitializationContext context) throws Exception { - localStatistics = statisticsSerializer.createInstance(); - globalStatisticsState = - context - .getOperatorStateStore() - .getUnionListState( - new ListStateDescriptor<>("globalStatisticsState", statisticsSerializer)); - - if (context.isRestored()) { - int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask(); - if (globalStatisticsState.get() == null - || !globalStatisticsState.get().iterator().hasNext()) { - LOG.warn( - "Operator {} subtask {} doesn't have global statistics state to restore", - operatorName, - subtaskIndex); - globalStatistics = statisticsSerializer.createInstance(); - } else { - LOG.info( - "Restoring operator {} global statistics state for subtask {}", - operatorName, - subtaskIndex); - globalStatistics = globalStatisticsState.get().iterator().next(); - } - } else { - globalStatistics = statisticsSerializer.createInstance(); - } - } - - @Override - public void open() throws Exception { - if (!globalStatistics.isEmpty()) { - output.collect( - new StreamRecord<>(DataStatisticsOrRecord.fromDataStatistics(globalStatistics))); - } - } - - @Override - @SuppressWarnings("unchecked") - public void handleOperatorEvent(OperatorEvent event) { - int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask(); - Preconditions.checkArgument( - event instanceof DataStatisticsEvent, - String.format( - "Operator %s subtask %s received unexpected operator event %s", - operatorName, subtaskIndex, event.getClass())); - DataStatisticsEvent statisticsEvent = (DataStatisticsEvent) event; - LOG.info( - "Operator {} received global data event from coordinator checkpoint {}", - operatorName, - statisticsEvent.checkpointId()); - globalStatistics = - DataStatisticsUtil.deserializeDataStatistics( - statisticsEvent.statisticsBytes(), statisticsSerializer); - output.collect(new StreamRecord<>(DataStatisticsOrRecord.fromDataStatistics(globalStatistics))); - } - - @Override - public void processElement(StreamRecord streamRecord) throws Exception { - RowData record = streamRecord.getValue(); - RowData key = keySelector.getKey(record); - localStatistics.add(key); - output.collect(new StreamRecord<>(DataStatisticsOrRecord.fromRecord(record))); - } - - @Override - public void snapshotState(StateSnapshotContext context) throws Exception { - long checkpointId = context.getCheckpointId(); - int subTaskId = getRuntimeContext().getIndexOfThisSubtask(); - LOG.info( - "Snapshotting data statistics operator {} for checkpoint {} in subtask {}", - operatorName, - checkpointId, - subTaskId); - - // Pass global statistics to partitioners so that all the operators refresh statistics - // at same checkpoint barrier - if (!globalStatistics.isEmpty()) { - output.collect( - new StreamRecord<>(DataStatisticsOrRecord.fromDataStatistics(globalStatistics))); - } - - // Only subtask 0 saves the state so that globalStatisticsState(UnionListState) stores - // an exact copy of globalStatistics - if (!globalStatistics.isEmpty() && getRuntimeContext().getIndexOfThisSubtask() == 0) { - globalStatisticsState.clear(); - LOG.info( - "Saving operator {} global statistics {} to state in subtask {}", - operatorName, - globalStatistics, - subTaskId); - globalStatisticsState.add(globalStatistics); - } - - // For now, local statistics are sent to coordinator at checkpoint - operatorEventGateway.sendEventToCoordinator( - DataStatisticsEvent.create(checkpointId, localStatistics, statisticsSerializer)); - LOG.debug( - "Subtask {} of operator {} sent local statistics to coordinator at checkpoint{}: {}", - subTaskId, - operatorName, - checkpointId, - localStatistics); - - // Recreate the local statistics - localStatistics = statisticsSerializer.createInstance(); - } - - @VisibleForTesting - DataStatistics localDataStatistics() { - return localStatistics; - } - - @VisibleForTesting - DataStatistics globalDataStatistics() { - return globalStatistics; - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java deleted file mode 100644 index 889e85112e16..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java +++ /dev/null @@ -1,113 +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.iceberg.flink.sink.shuffle; - -import java.io.Serializable; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** - * The wrapper class for data statistics and record. It is the only way for data statistics operator - * to send global data statistics to custom partitioner to distribute data based on statistics - * - *

    DataStatisticsOrRecord contains either data statistics(globally aggregated) or a record. It is - * sent from {@link DataStatisticsOperator} to partitioner. Once partitioner receives the data - * statistics, it will use that to decide the coming record should send to which writer subtask. - * After shuffling, a filter and mapper are required to filter out the data distribution weight, - * unwrap the object and extract the original record type T. - */ -class DataStatisticsOrRecord, S> implements Serializable { - - private static final long serialVersionUID = 1L; - - private DataStatistics statistics; - private RowData record; - - private DataStatisticsOrRecord(DataStatistics statistics, RowData record) { - Preconditions.checkArgument( - record != null ^ statistics != null, "DataStatistics or record, not neither or both"); - this.statistics = statistics; - this.record = record; - } - - static , S> DataStatisticsOrRecord fromRecord( - RowData record) { - return new DataStatisticsOrRecord<>(null, record); - } - - static , S> DataStatisticsOrRecord fromDataStatistics( - DataStatistics statistics) { - return new DataStatisticsOrRecord<>(statistics, null); - } - - static , S> DataStatisticsOrRecord reuseRecord( - DataStatisticsOrRecord reuse, TypeSerializer recordSerializer) { - if (reuse.hasRecord()) { - return reuse; - } else { - // not reusable - return DataStatisticsOrRecord.fromRecord(recordSerializer.createInstance()); - } - } - - static , S> DataStatisticsOrRecord reuseStatistics( - DataStatisticsOrRecord reuse, - TypeSerializer> statisticsSerializer) { - if (reuse.hasDataStatistics()) { - return reuse; - } else { - // not reusable - return DataStatisticsOrRecord.fromDataStatistics(statisticsSerializer.createInstance()); - } - } - - boolean hasDataStatistics() { - return statistics != null; - } - - boolean hasRecord() { - return record != null; - } - - DataStatistics dataStatistics() { - return statistics; - } - - void dataStatistics(DataStatistics newStatistics) { - this.statistics = newStatistics; - } - - RowData record() { - return record; - } - - void record(RowData newRecord) { - this.record = newRecord; - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("statistics", statistics) - .add("record", record) - .toString(); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java deleted file mode 100644 index e9a6fa0cbfc5..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java +++ /dev/null @@ -1,219 +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.iceberg.flink.sink.shuffle; - -import java.io.IOException; -import java.util.Objects; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.table.data.RowData; - -@Internal -class DataStatisticsOrRecordSerializer, S> - extends TypeSerializer> { - private final TypeSerializer> statisticsSerializer; - private final TypeSerializer recordSerializer; - - DataStatisticsOrRecordSerializer( - TypeSerializer> statisticsSerializer, - TypeSerializer recordSerializer) { - this.statisticsSerializer = statisticsSerializer; - this.recordSerializer = recordSerializer; - } - - @Override - public boolean isImmutableType() { - return false; - } - - @SuppressWarnings("ReferenceEquality") - @Override - public TypeSerializer> duplicate() { - TypeSerializer> duplicateStatisticsSerializer = - statisticsSerializer.duplicate(); - TypeSerializer duplicateRowDataSerializer = recordSerializer.duplicate(); - if ((statisticsSerializer != duplicateStatisticsSerializer) - || (recordSerializer != duplicateRowDataSerializer)) { - return new DataStatisticsOrRecordSerializer<>( - duplicateStatisticsSerializer, duplicateRowDataSerializer); - } else { - return this; - } - } - - @Override - public DataStatisticsOrRecord createInstance() { - // arbitrarily always create RowData value instance - return DataStatisticsOrRecord.fromRecord(recordSerializer.createInstance()); - } - - @Override - public DataStatisticsOrRecord copy(DataStatisticsOrRecord from) { - if (from.hasRecord()) { - return DataStatisticsOrRecord.fromRecord(recordSerializer.copy(from.record())); - } else { - return DataStatisticsOrRecord.fromDataStatistics( - statisticsSerializer.copy(from.dataStatistics())); - } - } - - @Override - public DataStatisticsOrRecord copy( - DataStatisticsOrRecord from, DataStatisticsOrRecord reuse) { - DataStatisticsOrRecord to; - if (from.hasRecord()) { - to = DataStatisticsOrRecord.reuseRecord(reuse, recordSerializer); - RowData record = recordSerializer.copy(from.record(), to.record()); - to.record(record); - } else { - to = DataStatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); - DataStatistics statistics = - statisticsSerializer.copy(from.dataStatistics(), to.dataStatistics()); - to.dataStatistics(statistics); - } - - return to; - } - - @Override - public int getLength() { - return -1; - } - - @Override - public void serialize(DataStatisticsOrRecord statisticsOrRecord, DataOutputView target) - throws IOException { - if (statisticsOrRecord.hasRecord()) { - target.writeBoolean(true); - recordSerializer.serialize(statisticsOrRecord.record(), target); - } else { - target.writeBoolean(false); - statisticsSerializer.serialize(statisticsOrRecord.dataStatistics(), target); - } - } - - @Override - public DataStatisticsOrRecord deserialize(DataInputView source) throws IOException { - boolean isRecord = source.readBoolean(); - if (isRecord) { - return DataStatisticsOrRecord.fromRecord(recordSerializer.deserialize(source)); - } else { - return DataStatisticsOrRecord.fromDataStatistics(statisticsSerializer.deserialize(source)); - } - } - - @Override - public DataStatisticsOrRecord deserialize( - DataStatisticsOrRecord reuse, DataInputView source) throws IOException { - DataStatisticsOrRecord to; - boolean isRecord = source.readBoolean(); - if (isRecord) { - to = DataStatisticsOrRecord.reuseRecord(reuse, recordSerializer); - RowData record = recordSerializer.deserialize(to.record(), source); - to.record(record); - } else { - to = DataStatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); - DataStatistics statistics = - statisticsSerializer.deserialize(to.dataStatistics(), source); - to.dataStatistics(statistics); - } - - return to; - } - - @Override - public void copy(DataInputView source, DataOutputView target) throws IOException { - boolean hasRecord = source.readBoolean(); - target.writeBoolean(hasRecord); - if (hasRecord) { - recordSerializer.copy(source, target); - } else { - statisticsSerializer.copy(source, target); - } - } - - @Override - public boolean equals(Object obj) { - if (!(obj instanceof DataStatisticsOrRecordSerializer)) { - return false; - } - - @SuppressWarnings("unchecked") - DataStatisticsOrRecordSerializer other = (DataStatisticsOrRecordSerializer) obj; - return Objects.equals(statisticsSerializer, other.statisticsSerializer) - && Objects.equals(recordSerializer, other.recordSerializer); - } - - @Override - public int hashCode() { - return Objects.hash(statisticsSerializer, recordSerializer); - } - - @Override - public TypeSerializerSnapshot> snapshotConfiguration() { - return new DataStatisticsOrRecordSerializerSnapshot<>(this); - } - - public static class DataStatisticsOrRecordSerializerSnapshot, S> - extends CompositeTypeSerializerSnapshot< - DataStatisticsOrRecord, DataStatisticsOrRecordSerializer> { - private static final int CURRENT_VERSION = 1; - - // constructors need to public. Otherwise, Flink state restore would complain - // "The class has no (implicit) public nullary constructor". - @SuppressWarnings("checkstyle:RedundantModifier") - public DataStatisticsOrRecordSerializerSnapshot() { - super(DataStatisticsOrRecordSerializer.class); - } - - @SuppressWarnings("checkstyle:RedundantModifier") - public DataStatisticsOrRecordSerializerSnapshot( - DataStatisticsOrRecordSerializer serializer) { - super(serializer); - } - - @SuppressWarnings("checkstyle:RedundantModifier") - @Override - protected int getCurrentOuterSnapshotVersion() { - return CURRENT_VERSION; - } - - @Override - protected TypeSerializer[] getNestedSerializers( - DataStatisticsOrRecordSerializer outerSerializer) { - return new TypeSerializer[] { - outerSerializer.statisticsSerializer, outerSerializer.recordSerializer - }; - } - - @SuppressWarnings("unchecked") - @Override - protected DataStatisticsOrRecordSerializer createOuterSerializerWithNestedSerializers( - TypeSerializer[] nestedSerializers) { - TypeSerializer> statisticsSerializer = - (TypeSerializer>) nestedSerializers[0]; - TypeSerializer recordSerializer = (TypeSerializer) nestedSerializers[1]; - return new DataStatisticsOrRecordSerializer<>(statisticsSerializer, recordSerializer); - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java deleted file mode 100644 index 2737b1346f0f..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java +++ /dev/null @@ -1,97 +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.iceberg.flink.sink.shuffle; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.core.memory.DataInputDeserializer; -import org.apache.flink.core.memory.DataOutputSerializer; - -/** - * DataStatisticsUtil is the utility to serialize and deserialize {@link DataStatistics} and {@link - * AggregatedStatistics} - */ -class DataStatisticsUtil { - - private DataStatisticsUtil() {} - - static , S> byte[] serializeDataStatistics( - DataStatistics dataStatistics, - TypeSerializer> statisticsSerializer) { - DataOutputSerializer out = new DataOutputSerializer(64); - try { - statisticsSerializer.serialize(dataStatistics, out); - return out.getCopyOfBuffer(); - } catch (IOException e) { - throw new IllegalStateException("Fail to serialize data statistics", e); - } - } - - @SuppressWarnings("unchecked") - static , S> D deserializeDataStatistics( - byte[] bytes, TypeSerializer> statisticsSerializer) { - DataInputDeserializer input = new DataInputDeserializer(bytes, 0, bytes.length); - try { - return (D) statisticsSerializer.deserialize(input); - } catch (IOException e) { - throw new IllegalStateException("Fail to deserialize data statistics", e); - } - } - - static , S> byte[] serializeAggregatedStatistics( - AggregatedStatistics aggregatedStatistics, - TypeSerializer> statisticsSerializer) - throws IOException { - ByteArrayOutputStream bytes = new ByteArrayOutputStream(); - ObjectOutputStream out = new ObjectOutputStream(bytes); - - DataOutputSerializer outSerializer = new DataOutputSerializer(64); - out.writeLong(aggregatedStatistics.checkpointId()); - statisticsSerializer.serialize(aggregatedStatistics.dataStatistics(), outSerializer); - byte[] statisticsBytes = outSerializer.getCopyOfBuffer(); - out.writeInt(statisticsBytes.length); - out.write(statisticsBytes); - out.flush(); - - return bytes.toByteArray(); - } - - @SuppressWarnings("unchecked") - static , S> - AggregatedStatistics deserializeAggregatedStatistics( - byte[] bytes, TypeSerializer> statisticsSerializer) - throws IOException { - ByteArrayInputStream bytesIn = new ByteArrayInputStream(bytes); - ObjectInputStream in = new ObjectInputStream(bytesIn); - - long completedCheckpointId = in.readLong(); - int statisticsBytesLength = in.readInt(); - byte[] statisticsBytes = new byte[statisticsBytesLength]; - in.readFully(statisticsBytes); - DataInputDeserializer input = - new DataInputDeserializer(statisticsBytes, 0, statisticsBytesLength); - DataStatistics dataStatistics = statisticsSerializer.deserialize(input); - - return new AggregatedStatistics<>(completedCheckpointId, dataStatistics); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java deleted file mode 100644 index 246b56526fd6..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java +++ /dev/null @@ -1,65 +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.iceberg.flink.sink.shuffle; - -import java.util.Map; -import org.apache.flink.annotation.Internal; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; - -/** MapDataStatistics uses map to count key frequency */ -@Internal -class MapDataStatistics implements DataStatistics> { - private final Map statistics; - - MapDataStatistics() { - this.statistics = Maps.newHashMap(); - } - - MapDataStatistics(Map statistics) { - this.statistics = statistics; - } - - @Override - public boolean isEmpty() { - return statistics.size() == 0; - } - - @Override - public void add(RowData key) { - // increase count of occurrence by one in the dataStatistics map - statistics.merge(key, 1L, Long::sum); - } - - @Override - public void merge(MapDataStatistics otherStatistics) { - otherStatistics.statistics().forEach((key, count) -> statistics.merge(key, count, Long::sum)); - } - - @Override - public Map statistics() { - return statistics; - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this).add("statistics", statistics).toString(); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java deleted file mode 100644 index 6d07637b29b3..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java +++ /dev/null @@ -1,183 +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.iceberg.flink.sink.shuffle; - -import java.io.IOException; -import java.util.Map; -import java.util.Objects; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; -import org.apache.flink.api.common.typeutils.base.LongSerializer; -import org.apache.flink.api.common.typeutils.base.MapSerializer; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.table.data.RowData; -import org.apache.flink.util.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; - -@Internal -class MapDataStatisticsSerializer - extends TypeSerializer>> { - private final MapSerializer mapSerializer; - - static TypeSerializer>> fromKeySerializer( - TypeSerializer keySerializer) { - return new MapDataStatisticsSerializer( - new MapSerializer<>(keySerializer, LongSerializer.INSTANCE)); - } - - MapDataStatisticsSerializer(MapSerializer mapSerializer) { - this.mapSerializer = mapSerializer; - } - - @Override - public boolean isImmutableType() { - return false; - } - - @SuppressWarnings("ReferenceEquality") - @Override - public TypeSerializer>> duplicate() { - MapSerializer duplicateMapSerializer = - (MapSerializer) mapSerializer.duplicate(); - return (duplicateMapSerializer == mapSerializer) - ? this - : new MapDataStatisticsSerializer(duplicateMapSerializer); - } - - @Override - public DataStatistics> createInstance() { - return new MapDataStatistics(); - } - - @Override - public DataStatistics> copy(DataStatistics obj) { - Preconditions.checkArgument( - obj instanceof MapDataStatistics, "Invalid data statistics type: " + obj.getClass()); - MapDataStatistics from = (MapDataStatistics) obj; - TypeSerializer keySerializer = mapSerializer.getKeySerializer(); - Map newMap = Maps.newHashMapWithExpectedSize(from.statistics().size()); - for (Map.Entry entry : from.statistics().entrySet()) { - RowData newKey = keySerializer.copy(entry.getKey()); - // no need to copy value since it is just a Long - newMap.put(newKey, entry.getValue()); - } - - return new MapDataStatistics(newMap); - } - - @Override - public DataStatistics> copy( - DataStatistics from, DataStatistics reuse) { - // not much benefit to reuse - return copy(from); - } - - @Override - public int getLength() { - return -1; - } - - @Override - public void serialize(DataStatistics obj, DataOutputView target) throws IOException { - Preconditions.checkArgument( - obj instanceof MapDataStatistics, "Invalid data statistics type: " + obj.getClass()); - MapDataStatistics mapStatistics = (MapDataStatistics) obj; - mapSerializer.serialize(mapStatistics.statistics(), target); - } - - @Override - public DataStatistics> deserialize(DataInputView source) - throws IOException { - return new MapDataStatistics(mapSerializer.deserialize(source)); - } - - @Override - public DataStatistics> deserialize( - DataStatistics reuse, DataInputView source) throws IOException { - // not much benefit to reuse - return deserialize(source); - } - - @Override - public void copy(DataInputView source, DataOutputView target) throws IOException { - mapSerializer.copy(source, target); - } - - @Override - public boolean equals(Object obj) { - if (!(obj instanceof MapDataStatisticsSerializer)) { - return false; - } - - MapDataStatisticsSerializer other = (MapDataStatisticsSerializer) obj; - return Objects.equals(mapSerializer, other.mapSerializer); - } - - @Override - public int hashCode() { - return mapSerializer.hashCode(); - } - - @Override - public TypeSerializerSnapshot>> - snapshotConfiguration() { - return new MapDataStatisticsSerializerSnapshot(this); - } - - public static class MapDataStatisticsSerializerSnapshot - extends CompositeTypeSerializerSnapshot< - DataStatistics>, MapDataStatisticsSerializer> { - private static final int CURRENT_VERSION = 1; - - // constructors need to public. Otherwise, Flink state restore would complain - // "The class has no (implicit) public nullary constructor". - @SuppressWarnings("checkstyle:RedundantModifier") - public MapDataStatisticsSerializerSnapshot() { - super(MapDataStatisticsSerializer.class); - } - - @SuppressWarnings("checkstyle:RedundantModifier") - public MapDataStatisticsSerializerSnapshot(MapDataStatisticsSerializer serializer) { - super(serializer); - } - - @Override - protected int getCurrentOuterSnapshotVersion() { - return CURRENT_VERSION; - } - - @Override - protected TypeSerializer[] getNestedSerializers( - MapDataStatisticsSerializer outerSerializer) { - return new TypeSerializer[] {outerSerializer.mapSerializer}; - } - - @Override - protected MapDataStatisticsSerializer createOuterSerializerWithNestedSerializers( - TypeSerializer[] nestedSerializers) { - @SuppressWarnings("unchecked") - MapSerializer mapSerializer = - (MapSerializer) nestedSerializers[0]; - return new MapDataStatisticsSerializer(mapSerializer); - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java deleted file mode 100644 index 796434c45136..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java +++ /dev/null @@ -1,42 +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.iceberg.flink.source; - -import org.apache.avro.generic.GenericRecord; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.encryption.InputFilesDecryptor; -import org.apache.iceberg.io.CloseableIterator; - -public class AvroGenericRecordFileScanTaskReader implements FileScanTaskReader { - private final RowDataFileScanTaskReader rowDataReader; - private final RowDataToAvroGenericRecordConverter converter; - - public AvroGenericRecordFileScanTaskReader( - RowDataFileScanTaskReader rowDataReader, RowDataToAvroGenericRecordConverter converter) { - this.rowDataReader = rowDataReader; - this.converter = converter; - } - - @Override - public CloseableIterator open( - FileScanTask fileScanTask, InputFilesDecryptor inputFilesDecryptor) { - return CloseableIterator.transform( - rowDataReader.open(fileScanTask, inputFilesDecryptor), converter); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java deleted file mode 100644 index 91d975349b19..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java +++ /dev/null @@ -1,156 +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.iceberg.flink.source; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.Iterator; -import org.apache.flink.annotation.Internal; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.encryption.EncryptionManager; -import org.apache.iceberg.encryption.InputFilesDecryptor; -import org.apache.iceberg.io.CloseableIterator; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** - * Flink data iterator that reads {@link CombinedScanTask} into a {@link CloseableIterator} - * - * @param is the output data type returned by this iterator. - */ -@Internal -public class DataIterator implements CloseableIterator { - - private final FileScanTaskReader fileScanTaskReader; - - private final InputFilesDecryptor inputFilesDecryptor; - private final CombinedScanTask combinedTask; - - private Iterator tasks; - private CloseableIterator currentIterator; - private int fileOffset; - private long recordOffset; - - public DataIterator( - FileScanTaskReader fileScanTaskReader, - CombinedScanTask task, - FileIO io, - EncryptionManager encryption) { - this.fileScanTaskReader = fileScanTaskReader; - - this.inputFilesDecryptor = new InputFilesDecryptor(task, io, encryption); - this.combinedTask = task; - - this.tasks = task.files().iterator(); - this.currentIterator = CloseableIterator.empty(); - - // fileOffset starts at -1 because we started - // from an empty iterator that is not from the split files. - this.fileOffset = -1; - // record offset points to the record that next() should return when called - this.recordOffset = 0L; - } - - /** - * (startingFileOffset, startingRecordOffset) points to the next row that reader should resume - * from. E.g., if the seek position is (file=0, record=1), seek moves the iterator position to the - * 2nd row in file 0. When next() is called after seek, 2nd row from file 0 should be returned. - */ - public void seek(int startingFileOffset, long startingRecordOffset) { - Preconditions.checkState( - fileOffset == -1, "Seek should be called before any other iterator actions"); - // skip files - Preconditions.checkState( - startingFileOffset < combinedTask.files().size(), - "Invalid starting file offset %s for combined scan task with %s files: %s", - startingFileOffset, - combinedTask.files().size(), - combinedTask); - for (long i = 0L; i < startingFileOffset; ++i) { - tasks.next(); - } - - updateCurrentIterator(); - // skip records within the file - for (long i = 0; i < startingRecordOffset; ++i) { - if (currentFileHasNext() && hasNext()) { - next(); - } else { - throw new IllegalStateException( - String.format( - "Invalid starting record offset %d for file %d from CombinedScanTask: %s", - startingRecordOffset, startingFileOffset, combinedTask)); - } - } - - fileOffset = startingFileOffset; - recordOffset = startingRecordOffset; - } - - @Override - public boolean hasNext() { - updateCurrentIterator(); - return currentIterator.hasNext(); - } - - @Override - public T next() { - updateCurrentIterator(); - recordOffset += 1; - return currentIterator.next(); - } - - public boolean currentFileHasNext() { - return currentIterator.hasNext(); - } - - /** Updates the current iterator field to ensure that the current Iterator is not exhausted. */ - private void updateCurrentIterator() { - try { - while (!currentIterator.hasNext() && tasks.hasNext()) { - currentIterator.close(); - currentIterator = openTaskIterator(tasks.next()); - fileOffset += 1; - recordOffset = 0L; - } - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - private CloseableIterator openTaskIterator(FileScanTask scanTask) { - return fileScanTaskReader.open(scanTask, inputFilesDecryptor); - } - - @Override - public void close() throws IOException { - // close the current iterator - currentIterator.close(); - tasks = null; - } - - public int fileOffset() { - return fileOffset; - } - - public long recordOffset() { - return recordOffset; - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java deleted file mode 100644 index 4394dab4d4cc..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java +++ /dev/null @@ -1,47 +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.iceberg.flink.source; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.Schema; -import org.apache.iceberg.encryption.InputFilesDecryptor; -import org.apache.iceberg.flink.data.StructRowData; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.CloseableIterator; - -@Internal -public class DataTaskReader implements FileScanTaskReader { - - private final Schema readSchema; - - public DataTaskReader(Schema readSchema) { - this.readSchema = readSchema; - } - - @Override - public CloseableIterator open( - FileScanTask task, InputFilesDecryptor inputFilesDecryptor) { - StructRowData row = new StructRowData(readSchema.asStruct()); - CloseableIterable iterable = - CloseableIterable.transform(task.asDataTask().rows(), row::setStruct); - return iterable.iterator(); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java deleted file mode 100644 index 927a804a4792..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java +++ /dev/null @@ -1,35 +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.iceberg.flink.source; - -import java.io.Serializable; -import org.apache.flink.annotation.Internal; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.encryption.InputFilesDecryptor; -import org.apache.iceberg.io.CloseableIterator; - -/** - * Read a {@link FileScanTask} into a {@link CloseableIterator} - * - * @param is the output data type returned by this iterator. - */ -@Internal -public interface FileScanTaskReader extends Serializable { - CloseableIterator open(FileScanTask fileScanTask, InputFilesDecryptor inputFilesDecryptor); -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java deleted file mode 100644 index 9a5123dc489e..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java +++ /dev/null @@ -1,141 +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.iceberg.flink.source; - -import java.io.IOException; -import java.util.concurrent.ExecutorService; -import org.apache.flink.api.common.io.DefaultInputSplitAssigner; -import org.apache.flink.api.common.io.InputFormat; -import org.apache.flink.api.common.io.LocatableInputSplitAssigner; -import org.apache.flink.api.common.io.RichInputFormat; -import org.apache.flink.api.common.io.statistics.BaseStatistics; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.io.InputSplitAssigner; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.BaseMetadataTable; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.encryption.EncryptionManager; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.util.ThreadPools; - -/** Flink {@link InputFormat} for Iceberg. */ -public class FlinkInputFormat extends RichInputFormat { - - private static final long serialVersionUID = 1L; - - private final TableLoader tableLoader; - private final FileIO io; - private final EncryptionManager encryption; - private final ScanContext context; - private final FileScanTaskReader rowDataReader; - - private transient DataIterator iterator; - private transient long currentReadCount = 0L; - - FlinkInputFormat( - TableLoader tableLoader, - Schema tableSchema, - FileIO io, - EncryptionManager encryption, - ScanContext context) { - this.tableLoader = tableLoader; - this.io = io; - this.encryption = encryption; - this.context = context; - - tableLoader.open(); - Table table = tableLoader.loadTable(); - if (table instanceof BaseMetadataTable) { - this.rowDataReader = new DataTaskReader(context.project()); - } else { - this.rowDataReader = - new RowDataFileScanTaskReader( - tableSchema, - context.project(), - context.nameMapping(), - context.caseSensitive(), - context.filters()); - } - } - - @VisibleForTesting - Schema projectedSchema() { - return context.project(); - } - - @Override - public BaseStatistics getStatistics(BaseStatistics cachedStatistics) { - // Legacy method, not be used. - return null; - } - - @Override - public FlinkInputSplit[] createInputSplits(int minNumSplits) throws IOException { - // Called in Job manager, so it is OK to load table from catalog. - tableLoader.open(); - final ExecutorService workerPool = - ThreadPools.newWorkerPool("iceberg-plan-worker-pool", context.planParallelism()); - try (TableLoader loader = tableLoader) { - Table table = loader.loadTable(); - return FlinkSplitPlanner.planInputSplits(table, context, workerPool); - } finally { - workerPool.shutdown(); - } - } - - @Override - public InputSplitAssigner getInputSplitAssigner(FlinkInputSplit[] inputSplits) { - return context.exposeLocality() - ? new LocatableInputSplitAssigner(inputSplits) - : new DefaultInputSplitAssigner(inputSplits); - } - - @Override - public void configure(Configuration parameters) {} - - @Override - public void open(FlinkInputSplit split) { - this.iterator = new DataIterator<>(rowDataReader, split.getTask(), io, encryption); - } - - @Override - public boolean reachedEnd() { - if (context.limit() > 0 && currentReadCount >= context.limit()) { - return true; - } else { - return !iterator.hasNext(); - } - } - - @Override - public RowData nextRecord(RowData reuse) { - currentReadCount++; - return iterator.next(); - } - - @Override - public void close() throws IOException { - if (iterator != null) { - iterator.close(); - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java deleted file mode 100644 index 16fd4f39596c..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java +++ /dev/null @@ -1,48 +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.iceberg.flink.source; - -import java.util.Arrays; -import javax.annotation.Nullable; -import org.apache.flink.core.io.LocatableInputSplit; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; - -public class FlinkInputSplit extends LocatableInputSplit { - - private final CombinedScanTask task; - - FlinkInputSplit(int splitNumber, CombinedScanTask task, @Nullable String[] hostnames) { - super(splitNumber, hostnames); - this.task = task; - } - - CombinedScanTask getTask() { - return task; - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("splitNumber", getSplitNumber()) - .add("task", task) - .add("hosts", Arrays.toString(getHostnames())) - .toString(); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java deleted file mode 100644 index fa1656c55278..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java +++ /dev/null @@ -1,310 +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.iceberg.flink.source; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.time.Duration; -import java.util.List; -import java.util.Map; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.TableScan; -import org.apache.iceberg.encryption.EncryptionManager; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.FlinkReadOptions; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.util.PropertyUtil; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class FlinkSource { - private static final Logger LOG = LoggerFactory.getLogger(FlinkSource.class); - - private FlinkSource() {} - - /** - * Initialize a {@link Builder} to read the data from iceberg table. Equivalent to {@link - * TableScan}. See more options in {@link ScanContext}. - * - *

    The Source can be read static data in bounded mode. It can also continuously check the - * arrival of new data and read records incrementally. - * - *

      - *
    • Without startSnapshotId: Bounded - *
    • With startSnapshotId and with endSnapshotId: Bounded - *
    • With startSnapshotId (-1 means unbounded preceding) and Without endSnapshotId: Unbounded - *
    - * - *

    - * - * @return {@link Builder} to connect the iceberg table. - */ - public static Builder forRowData() { - return new Builder(); - } - - /** Source builder to build {@link DataStream}. */ - public static class Builder { - private StreamExecutionEnvironment env; - private Table table; - private TableLoader tableLoader; - private TableSchema projectedSchema; - private ReadableConfig readableConfig = new Configuration(); - private final ScanContext.Builder contextBuilder = ScanContext.builder(); - private Boolean exposeLocality; - - private final Map readOptions = Maps.newHashMap(); - - public Builder tableLoader(TableLoader newLoader) { - this.tableLoader = newLoader; - return this; - } - - public Builder table(Table newTable) { - this.table = newTable; - return this; - } - - public Builder env(StreamExecutionEnvironment newEnv) { - this.env = newEnv; - return this; - } - - public Builder filters(List filters) { - contextBuilder.filters(filters); - return this; - } - - public Builder project(TableSchema schema) { - this.projectedSchema = schema; - return this; - } - - public Builder limit(Long newLimit) { - if (newLimit != null) { - readOptions.put(FlinkReadOptions.LIMIT, Long.toString(newLimit)); - } - return this; - } - - public Builder set(String property, String value) { - readOptions.put(property, value); - return this; - } - - public Builder setAll(Map properties) { - readOptions.putAll(properties); - return this; - } - - /** @deprecated Use {@link #setAll} instead. */ - @Deprecated - public Builder properties(Map properties) { - readOptions.putAll(properties); - return this; - } - - public Builder caseSensitive(boolean caseSensitive) { - readOptions.put(FlinkReadOptions.CASE_SENSITIVE, Boolean.toString(caseSensitive)); - return this; - } - - public Builder snapshotId(Long snapshotId) { - readOptions.put(FlinkReadOptions.SNAPSHOT_ID.key(), Long.toString(snapshotId)); - return this; - } - - public Builder branch(String branch) { - readOptions.put(FlinkReadOptions.BRANCH.key(), branch); - return this; - } - - public Builder tag(String tag) { - readOptions.put(FlinkReadOptions.TAG.key(), tag); - return this; - } - - public Builder startSnapshotId(Long startSnapshotId) { - readOptions.put(FlinkReadOptions.START_SNAPSHOT_ID.key(), Long.toString(startSnapshotId)); - return this; - } - - public Builder endSnapshotId(Long endSnapshotId) { - readOptions.put(FlinkReadOptions.END_SNAPSHOT_ID.key(), Long.toString(endSnapshotId)); - return this; - } - - public Builder startTag(String startTag) { - readOptions.put(FlinkReadOptions.START_TAG.key(), startTag); - return this; - } - - public Builder endTag(String endTag) { - readOptions.put(FlinkReadOptions.END_TAG.key(), endTag); - return this; - } - - public Builder asOfTimestamp(Long asOfTimestamp) { - readOptions.put(FlinkReadOptions.AS_OF_TIMESTAMP.key(), Long.toString(asOfTimestamp)); - return this; - } - - public Builder splitSize(Long splitSize) { - readOptions.put(FlinkReadOptions.SPLIT_SIZE, Long.toString(splitSize)); - return this; - } - - public Builder splitLookback(Integer splitLookback) { - readOptions.put(FlinkReadOptions.SPLIT_LOOKBACK, Integer.toString(splitLookback)); - return this; - } - - public Builder splitOpenFileCost(Long splitOpenFileCost) { - readOptions.put(FlinkReadOptions.SPLIT_FILE_OPEN_COST, Long.toString(splitOpenFileCost)); - return this; - } - - public Builder streaming(boolean streaming) { - readOptions.put(FlinkReadOptions.STREAMING, Boolean.toString(streaming)); - return this; - } - - public Builder exposeLocality(boolean newExposeLocality) { - this.exposeLocality = newExposeLocality; - return this; - } - - public Builder nameMapping(String nameMapping) { - readOptions.put(TableProperties.DEFAULT_NAME_MAPPING, nameMapping); - return this; - } - - public Builder monitorInterval(Duration interval) { - readOptions.put(FlinkReadOptions.MONITOR_INTERVAL, interval.toNanos() + " ns"); - return this; - } - - public Builder maxPlanningSnapshotCount(int newMaxPlanningSnapshotCount) { - readOptions.put( - FlinkReadOptions.MAX_PLANNING_SNAPSHOT_COUNT, - Integer.toString(newMaxPlanningSnapshotCount)); - return this; - } - - public Builder flinkConf(ReadableConfig config) { - this.readableConfig = config; - return this; - } - - public FlinkInputFormat buildFormat() { - Preconditions.checkNotNull(tableLoader, "TableLoader should not be null"); - - Schema icebergSchema; - FileIO io; - EncryptionManager encryption; - if (table == null) { - // load required fields by table loader. - tableLoader.open(); - try (TableLoader loader = tableLoader) { - table = loader.loadTable(); - icebergSchema = table.schema(); - io = table.io(); - encryption = table.encryption(); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } else { - icebergSchema = table.schema(); - io = table.io(); - encryption = table.encryption(); - } - - if (projectedSchema == null) { - contextBuilder.project(icebergSchema); - } else { - contextBuilder.project(FlinkSchemaUtil.convert(icebergSchema, projectedSchema)); - } - - contextBuilder.exposeLocality( - SourceUtil.isLocalityEnabled(table, readableConfig, exposeLocality)); - contextBuilder.planParallelism( - readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE)); - - contextBuilder.resolveConfig(table, readOptions, readableConfig); - - return new FlinkInputFormat( - tableLoader, icebergSchema, io, encryption, contextBuilder.build()); - } - - public DataStream build() { - Preconditions.checkNotNull(env, "StreamExecutionEnvironment should not be null"); - FlinkInputFormat format = buildFormat(); - - ScanContext context = contextBuilder.build(); - TypeInformation typeInfo = - FlinkCompatibilityUtil.toTypeInfo(FlinkSchemaUtil.convert(context.project())); - - if (!context.isStreaming()) { - int parallelism = - SourceUtil.inferParallelism( - readableConfig, - context.limit(), - () -> { - try { - return format.createInputSplits(0).length; - } catch (IOException e) { - throw new UncheckedIOException( - "Failed to create iceberg input splits for table: " + table, e); - } - }); - if (env.getMaxParallelism() > 0) { - parallelism = Math.min(parallelism, env.getMaxParallelism()); - } - return env.createInput(format, typeInfo).setParallelism(parallelism); - } else { - StreamingMonitorFunction function = new StreamingMonitorFunction(tableLoader, context); - - String monitorFunctionName = String.format("Iceberg table (%s) monitor", table); - String readerOperatorName = String.format("Iceberg table (%s) reader", table); - - return env.addSource(function, monitorFunctionName) - .transform(readerOperatorName, typeInfo, StreamingReaderOperator.factory(format)); - } - } - } - - public static boolean isBounded(Map properties) { - return !PropertyUtil.propertyAsBoolean(properties, FlinkReadOptions.STREAMING, false); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java deleted file mode 100644 index 15078809714f..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java +++ /dev/null @@ -1,189 +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.iceberg.flink.source; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.List; -import java.util.concurrent.ExecutorService; -import org.apache.flink.annotation.Internal; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.IncrementalAppendScan; -import org.apache.iceberg.Scan; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.TableScan; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.hadoop.Util; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.util.Tasks; - -@Internal -public class FlinkSplitPlanner { - private FlinkSplitPlanner() {} - - static FlinkInputSplit[] planInputSplits( - Table table, ScanContext context, ExecutorService workerPool) { - try (CloseableIterable tasksIterable = - planTasks(table, context, workerPool)) { - List tasks = Lists.newArrayList(tasksIterable); - FlinkInputSplit[] splits = new FlinkInputSplit[tasks.size()]; - boolean exposeLocality = context.exposeLocality(); - - Tasks.range(tasks.size()) - .stopOnFailure() - .executeWith(exposeLocality ? workerPool : null) - .run( - index -> { - CombinedScanTask task = tasks.get(index); - String[] hostnames = null; - if (exposeLocality) { - hostnames = Util.blockLocations(table.io(), task); - } - splits[index] = new FlinkInputSplit(index, task, hostnames); - }); - return splits; - } catch (IOException e) { - throw new UncheckedIOException("Failed to process tasks iterable", e); - } - } - - /** This returns splits for the FLIP-27 source */ - public static List planIcebergSourceSplits( - Table table, ScanContext context, ExecutorService workerPool) { - try (CloseableIterable tasksIterable = - planTasks(table, context, workerPool)) { - return Lists.newArrayList( - CloseableIterable.transform(tasksIterable, IcebergSourceSplit::fromCombinedScanTask)); - } catch (IOException e) { - throw new UncheckedIOException("Failed to process task iterable: ", e); - } - } - - static CloseableIterable planTasks( - Table table, ScanContext context, ExecutorService workerPool) { - ScanMode scanMode = checkScanMode(context); - if (scanMode == ScanMode.INCREMENTAL_APPEND_SCAN) { - IncrementalAppendScan scan = table.newIncrementalAppendScan(); - scan = refineScanWithBaseConfigs(scan, context, workerPool); - - if (context.startTag() != null) { - Preconditions.checkArgument( - table.snapshot(context.startTag()) != null, - "Cannot find snapshot with tag %s", - context.startTag()); - scan = scan.fromSnapshotExclusive(table.snapshot(context.startTag()).snapshotId()); - } - - if (context.startSnapshotId() != null) { - Preconditions.checkArgument( - context.startTag() == null, "START_SNAPSHOT_ID and START_TAG cannot both be set"); - scan = scan.fromSnapshotExclusive(context.startSnapshotId()); - } - - if (context.endTag() != null) { - Preconditions.checkArgument( - table.snapshot(context.endTag()) != null, - "Cannot find snapshot with tag %s", - context.endTag()); - scan = scan.toSnapshot(table.snapshot(context.endTag()).snapshotId()); - } - - if (context.endSnapshotId() != null) { - Preconditions.checkArgument( - context.endTag() == null, "END_SNAPSHOT_ID and END_TAG cannot both be set"); - scan = scan.toSnapshot(context.endSnapshotId()); - } - - return scan.planTasks(); - } else { - TableScan scan = table.newScan(); - scan = refineScanWithBaseConfigs(scan, context, workerPool); - - if (context.snapshotId() != null) { - scan = scan.useSnapshot(context.snapshotId()); - } else if (context.tag() != null) { - scan = scan.useRef(context.tag()); - } else if (context.branch() != null) { - scan = scan.useRef(context.branch()); - } - - if (context.asOfTimestamp() != null) { - scan = scan.asOfTime(context.asOfTimestamp()); - } - - return scan.planTasks(); - } - } - - @VisibleForTesting - enum ScanMode { - BATCH, - INCREMENTAL_APPEND_SCAN - } - - @VisibleForTesting - static ScanMode checkScanMode(ScanContext context) { - if (context.startSnapshotId() != null - || context.endSnapshotId() != null - || context.startTag() != null - || context.endTag() != null) { - return ScanMode.INCREMENTAL_APPEND_SCAN; - } else { - return ScanMode.BATCH; - } - } - - /** refine scan with common configs */ - private static > T refineScanWithBaseConfigs( - T scan, ScanContext context, ExecutorService workerPool) { - T refinedScan = - scan.caseSensitive(context.caseSensitive()).project(context.project()).planWith(workerPool); - - if (context.includeColumnStats()) { - refinedScan = refinedScan.includeColumnStats(); - } - - if (context.includeStatsForColumns() != null) { - refinedScan = refinedScan.includeColumnStats(context.includeStatsForColumns()); - } - - refinedScan = refinedScan.option(TableProperties.SPLIT_SIZE, context.splitSize().toString()); - - refinedScan = - refinedScan.option(TableProperties.SPLIT_LOOKBACK, context.splitLookback().toString()); - - refinedScan = - refinedScan.option( - TableProperties.SPLIT_OPEN_FILE_COST, context.splitOpenFileCost().toString()); - - if (context.filters() != null) { - for (Expression filter : context.filters()) { - refinedScan = refinedScan.filter(filter); - } - } - - return refinedScan; - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java deleted file mode 100644 index 179253cb3a18..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ /dev/null @@ -1,558 +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.iceberg.flink.source; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.time.Duration; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.TimeUnit; -import javax.annotation.Nullable; -import org.apache.flink.annotation.Experimental; -import org.apache.flink.api.connector.source.Boundedness; -import org.apache.flink.api.connector.source.Source; -import org.apache.flink.api.connector.source.SourceReader; -import org.apache.flink.api.connector.source.SourceReaderContext; -import org.apache.flink.api.connector.source.SplitEnumerator; -import org.apache.flink.api.connector.source.SplitEnumeratorContext; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.core.io.SimpleVersionedSerializer; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.data.RowData; -import org.apache.flink.util.Preconditions; -import org.apache.iceberg.BaseMetadataTable; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.FlinkReadOptions; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.source.assigner.OrderedSplitAssignerFactory; -import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; -import org.apache.iceberg.flink.source.assigner.SplitAssigner; -import org.apache.iceberg.flink.source.assigner.SplitAssignerFactory; -import org.apache.iceberg.flink.source.enumerator.ContinuousIcebergEnumerator; -import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlanner; -import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlannerImpl; -import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorState; -import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer; -import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator; -import org.apache.iceberg.flink.source.reader.ColumnStatsWatermarkExtractor; -import org.apache.iceberg.flink.source.reader.IcebergSourceReader; -import org.apache.iceberg.flink.source.reader.IcebergSourceReaderMetrics; -import org.apache.iceberg.flink.source.reader.MetaDataReaderFunction; -import org.apache.iceberg.flink.source.reader.ReaderFunction; -import org.apache.iceberg.flink.source.reader.RowDataReaderFunction; -import org.apache.iceberg.flink.source.reader.SerializableRecordEmitter; -import org.apache.iceberg.flink.source.reader.SplitWatermarkExtractor; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer; -import org.apache.iceberg.flink.source.split.SerializableComparator; -import org.apache.iceberg.flink.source.split.SplitComparators; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.util.ThreadPools; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@Experimental -public class IcebergSource implements Source { - private static final Logger LOG = LoggerFactory.getLogger(IcebergSource.class); - - private final TableLoader tableLoader; - private final ScanContext scanContext; - private final ReaderFunction readerFunction; - private final SplitAssignerFactory assignerFactory; - private final SerializableComparator splitComparator; - private final SerializableRecordEmitter emitter; - - // Can't use SerializableTable as enumerator needs a regular table - // that can discover table changes - private transient Table table; - - IcebergSource( - TableLoader tableLoader, - ScanContext scanContext, - ReaderFunction readerFunction, - SplitAssignerFactory assignerFactory, - SerializableComparator splitComparator, - Table table, - SerializableRecordEmitter emitter) { - this.tableLoader = tableLoader; - this.scanContext = scanContext; - this.readerFunction = readerFunction; - this.assignerFactory = assignerFactory; - this.splitComparator = splitComparator; - this.table = table; - this.emitter = emitter; - } - - String name() { - return "IcebergSource-" + lazyTable().name(); - } - - private String planningThreadName() { - // Ideally, operatorId should be used as the threadPoolName as Flink guarantees its uniqueness - // within a job. SplitEnumeratorContext doesn't expose the OperatorCoordinator.Context, which - // would contain the OperatorID. Need to discuss with Flink community whether it is ok to expose - // a public API like the protected method "OperatorCoordinator.Context getCoordinatorContext()" - // from SourceCoordinatorContext implementation. For now,

    - is used as - // the unique thread pool name. - return lazyTable().name() + "-" + UUID.randomUUID(); - } - - private List planSplitsForBatch(String threadName) { - ExecutorService workerPool = - ThreadPools.newWorkerPool(threadName, scanContext.planParallelism()); - try { - List splits = - FlinkSplitPlanner.planIcebergSourceSplits(lazyTable(), scanContext, workerPool); - LOG.info( - "Discovered {} splits from table {} during job initialization", - splits.size(), - lazyTable().name()); - return splits; - } finally { - workerPool.shutdown(); - } - } - - private Table lazyTable() { - if (table == null) { - tableLoader.open(); - try (TableLoader loader = tableLoader) { - this.table = loader.loadTable(); - } catch (IOException e) { - throw new UncheckedIOException("Failed to close table loader", e); - } - } - - return table; - } - - @Override - public Boundedness getBoundedness() { - return scanContext.isStreaming() ? Boundedness.CONTINUOUS_UNBOUNDED : Boundedness.BOUNDED; - } - - @Override - public SourceReader createReader(SourceReaderContext readerContext) { - IcebergSourceReaderMetrics metrics = - new IcebergSourceReaderMetrics(readerContext.metricGroup(), lazyTable().name()); - return new IcebergSourceReader<>( - emitter, metrics, readerFunction, splitComparator, readerContext); - } - - @Override - public SplitEnumerator createEnumerator( - SplitEnumeratorContext enumContext) { - return createEnumerator(enumContext, null); - } - - @Override - public SplitEnumerator restoreEnumerator( - SplitEnumeratorContext enumContext, IcebergEnumeratorState enumState) { - return createEnumerator(enumContext, enumState); - } - - @Override - public SimpleVersionedSerializer getSplitSerializer() { - return new IcebergSourceSplitSerializer(scanContext.caseSensitive()); - } - - @Override - public SimpleVersionedSerializer getEnumeratorCheckpointSerializer() { - return new IcebergEnumeratorStateSerializer(scanContext.caseSensitive()); - } - - private SplitEnumerator createEnumerator( - SplitEnumeratorContext enumContext, - @Nullable IcebergEnumeratorState enumState) { - SplitAssigner assigner; - if (enumState == null) { - assigner = assignerFactory.createAssigner(); - } else { - LOG.info( - "Iceberg source restored {} splits from state for table {}", - enumState.pendingSplits().size(), - lazyTable().name()); - assigner = assignerFactory.createAssigner(enumState.pendingSplits()); - } - - if (scanContext.isStreaming()) { - ContinuousSplitPlanner splitPlanner = - new ContinuousSplitPlannerImpl(tableLoader.clone(), scanContext, planningThreadName()); - return new ContinuousIcebergEnumerator( - enumContext, assigner, scanContext, splitPlanner, enumState); - } else { - List splits = planSplitsForBatch(planningThreadName()); - assigner.onDiscoveredSplits(splits); - return new StaticIcebergEnumerator(enumContext, assigner); - } - } - - public static Builder builder() { - return new Builder<>(); - } - - public static Builder forRowData() { - return new Builder<>(); - } - - public static class Builder { - private TableLoader tableLoader; - private Table table; - private SplitAssignerFactory splitAssignerFactory; - private SerializableComparator splitComparator; - private String watermarkColumn; - private TimeUnit watermarkTimeUnit = TimeUnit.MICROSECONDS; - private ReaderFunction readerFunction; - private ReadableConfig flinkConfig = new Configuration(); - private final ScanContext.Builder contextBuilder = ScanContext.builder(); - private TableSchema projectedFlinkSchema; - private Boolean exposeLocality; - - private final Map readOptions = Maps.newHashMap(); - - Builder() {} - - public Builder tableLoader(TableLoader loader) { - this.tableLoader = loader; - return this; - } - - public Builder table(Table newTable) { - this.table = newTable; - return this; - } - - public Builder assignerFactory(SplitAssignerFactory assignerFactory) { - Preconditions.checkArgument( - watermarkColumn == null, - "Watermark column and SplitAssigner should not be set in the same source"); - this.splitAssignerFactory = assignerFactory; - return this; - } - - public Builder splitComparator( - SerializableComparator newSplitComparator) { - this.splitComparator = newSplitComparator; - return this; - } - - public Builder readerFunction(ReaderFunction newReaderFunction) { - this.readerFunction = newReaderFunction; - return this; - } - - public Builder flinkConfig(ReadableConfig config) { - this.flinkConfig = config; - return this; - } - - public Builder caseSensitive(boolean newCaseSensitive) { - readOptions.put(FlinkReadOptions.CASE_SENSITIVE, Boolean.toString(newCaseSensitive)); - return this; - } - - public Builder useSnapshotId(Long newSnapshotId) { - if (newSnapshotId != null) { - readOptions.put(FlinkReadOptions.SNAPSHOT_ID.key(), Long.toString(newSnapshotId)); - } - return this; - } - - public Builder streamingStartingStrategy(StreamingStartingStrategy newStartingStrategy) { - readOptions.put(FlinkReadOptions.STARTING_STRATEGY, newStartingStrategy.name()); - return this; - } - - public Builder startSnapshotTimestamp(Long newStartSnapshotTimestamp) { - if (newStartSnapshotTimestamp != null) { - readOptions.put( - FlinkReadOptions.START_SNAPSHOT_TIMESTAMP.key(), - Long.toString(newStartSnapshotTimestamp)); - } - return this; - } - - public Builder startSnapshotId(Long newStartSnapshotId) { - if (newStartSnapshotId != null) { - readOptions.put( - FlinkReadOptions.START_SNAPSHOT_ID.key(), Long.toString(newStartSnapshotId)); - } - return this; - } - - public Builder tag(String tag) { - readOptions.put(FlinkReadOptions.TAG.key(), tag); - return this; - } - - public Builder branch(String branch) { - readOptions.put(FlinkReadOptions.BRANCH.key(), branch); - return this; - } - - public Builder startTag(String startTag) { - readOptions.put(FlinkReadOptions.START_TAG.key(), startTag); - return this; - } - - public Builder endTag(String endTag) { - readOptions.put(FlinkReadOptions.END_TAG.key(), endTag); - return this; - } - - public Builder endSnapshotId(Long newEndSnapshotId) { - if (newEndSnapshotId != null) { - readOptions.put(FlinkReadOptions.END_SNAPSHOT_ID.key(), Long.toString(newEndSnapshotId)); - } - return this; - } - - public Builder asOfTimestamp(Long newAsOfTimestamp) { - if (newAsOfTimestamp != null) { - readOptions.put(FlinkReadOptions.AS_OF_TIMESTAMP.key(), Long.toString(newAsOfTimestamp)); - } - return this; - } - - public Builder splitSize(Long newSplitSize) { - if (newSplitSize != null) { - readOptions.put(FlinkReadOptions.SPLIT_SIZE, Long.toString(newSplitSize)); - } - return this; - } - - public Builder splitLookback(Integer newSplitLookback) { - if (newSplitLookback != null) { - readOptions.put(FlinkReadOptions.SPLIT_LOOKBACK, Integer.toString(newSplitLookback)); - } - return this; - } - - public Builder splitOpenFileCost(Long newSplitOpenFileCost) { - if (newSplitOpenFileCost != null) { - readOptions.put(FlinkReadOptions.SPLIT_FILE_OPEN_COST, Long.toString(newSplitOpenFileCost)); - } - - return this; - } - - public Builder streaming(boolean streaming) { - readOptions.put(FlinkReadOptions.STREAMING, Boolean.toString(streaming)); - return this; - } - - public Builder monitorInterval(Duration newMonitorInterval) { - if (newMonitorInterval != null) { - readOptions.put(FlinkReadOptions.MONITOR_INTERVAL, newMonitorInterval.toNanos() + " ns"); - } - return this; - } - - public Builder nameMapping(String newNameMapping) { - readOptions.put(TableProperties.DEFAULT_NAME_MAPPING, newNameMapping); - return this; - } - - public Builder project(Schema newProjectedSchema) { - this.contextBuilder.project(newProjectedSchema); - return this; - } - - public Builder project(TableSchema newProjectedFlinkSchema) { - this.projectedFlinkSchema = newProjectedFlinkSchema; - return this; - } - - public Builder filters(List newFilters) { - this.contextBuilder.filters(newFilters); - return this; - } - - public Builder limit(Long newLimit) { - if (newLimit != null) { - readOptions.put(FlinkReadOptions.LIMIT, Long.toString(newLimit)); - } - return this; - } - - public Builder includeColumnStats(boolean newIncludeColumnStats) { - readOptions.put( - FlinkReadOptions.INCLUDE_COLUMN_STATS, Boolean.toString(newIncludeColumnStats)); - return this; - } - - public Builder planParallelism(int planParallelism) { - readOptions.put( - FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.key(), - Integer.toString(planParallelism)); - return this; - } - - public Builder exposeLocality(boolean newExposeLocality) { - this.exposeLocality = newExposeLocality; - return this; - } - - public Builder maxAllowedPlanningFailures(int maxAllowedPlanningFailures) { - readOptions.put( - FlinkReadOptions.MAX_ALLOWED_PLANNING_FAILURES_OPTION.key(), - Integer.toString(maxAllowedPlanningFailures)); - return this; - } - - /** - * Set the read properties for Flink source. View the supported properties in {@link - * FlinkReadOptions} - */ - public Builder set(String property, String value) { - readOptions.put(property, value); - return this; - } - - /** - * Set the read properties for Flink source. View the supported properties in {@link - * FlinkReadOptions} - */ - public Builder setAll(Map properties) { - readOptions.putAll(properties); - return this; - } - - /** - * Emits watermarks once per split based on the min value of column statistics from files - * metadata in the given split. The generated watermarks are also used for ordering the splits - * for read. Accepted column types are timestamp/timestamptz/long. For long columns consider - * setting {@link #watermarkTimeUnit(TimeUnit)}. - * - *

    Consider setting `read.split.open-file-cost` to prevent combining small files to a single - * split when the watermark is used for watermark alignment. - */ - public Builder watermarkColumn(String columnName) { - Preconditions.checkArgument( - splitAssignerFactory == null, - "Watermark column and SplitAssigner should not be set in the same source"); - this.watermarkColumn = columnName; - return this; - } - - /** - * When the type of the {@link #watermarkColumn} is {@link - * org.apache.iceberg.types.Types.LongType}, then sets the {@link TimeUnit} to convert the - * value. The default value is {@link TimeUnit#MICROSECONDS}. - */ - public Builder watermarkTimeUnit(TimeUnit timeUnit) { - this.watermarkTimeUnit = timeUnit; - return this; - } - - /** @deprecated Use {@link #setAll} instead. */ - @Deprecated - public Builder properties(Map properties) { - readOptions.putAll(properties); - return this; - } - - public IcebergSource build() { - if (table == null) { - try (TableLoader loader = tableLoader) { - loader.open(); - this.table = tableLoader.loadTable(); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - contextBuilder.resolveConfig(table, readOptions, flinkConfig); - - Schema icebergSchema = table.schema(); - if (projectedFlinkSchema != null) { - contextBuilder.project(FlinkSchemaUtil.convert(icebergSchema, projectedFlinkSchema)); - } - - SerializableRecordEmitter emitter = SerializableRecordEmitter.defaultEmitter(); - if (watermarkColumn != null) { - // Column statistics is needed for watermark generation - contextBuilder.includeColumnStats(Sets.newHashSet(watermarkColumn)); - - SplitWatermarkExtractor watermarkExtractor = - new ColumnStatsWatermarkExtractor(icebergSchema, watermarkColumn, watermarkTimeUnit); - emitter = SerializableRecordEmitter.emitterWithWatermark(watermarkExtractor); - splitAssignerFactory = - new OrderedSplitAssignerFactory(SplitComparators.watermark(watermarkExtractor)); - } - - ScanContext context = contextBuilder.build(); - if (readerFunction == null) { - if (table instanceof BaseMetadataTable) { - MetaDataReaderFunction rowDataReaderFunction = - new MetaDataReaderFunction( - flinkConfig, table.schema(), context.project(), table.io(), table.encryption()); - this.readerFunction = (ReaderFunction) rowDataReaderFunction; - } else { - RowDataReaderFunction rowDataReaderFunction = - new RowDataReaderFunction( - flinkConfig, - table.schema(), - context.project(), - context.nameMapping(), - context.caseSensitive(), - table.io(), - table.encryption(), - context.filters()); - this.readerFunction = (ReaderFunction) rowDataReaderFunction; - } - } - - if (splitAssignerFactory == null) { - if (splitComparator == null) { - splitAssignerFactory = new SimpleSplitAssignerFactory(); - } else { - splitAssignerFactory = new OrderedSplitAssignerFactory(splitComparator); - } - } - - checkRequired(); - // Since builder already load the table, pass it to the source to avoid double loading - return new IcebergSource<>( - tableLoader, - context, - readerFunction, - splitAssignerFactory, - splitComparator, - table, - emitter); - } - - private void checkRequired() { - Preconditions.checkNotNull(tableLoader, "tableLoader is required."); - Preconditions.checkNotNull(splitAssignerFactory, "assignerFactory is required."); - Preconditions.checkNotNull(readerFunction, "readerFunction is required."); - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java deleted file mode 100644 index 610657e8d47b..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java +++ /dev/null @@ -1,229 +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.iceberg.flink.source; - -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSource; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.ProviderContext; -import org.apache.flink.table.connector.source.DataStreamScanProvider; -import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.connector.source.ScanTableSource; -import org.apache.flink.table.connector.source.abilities.SupportsFilterPushDown; -import org.apache.flink.table.connector.source.abilities.SupportsLimitPushDown; -import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.expressions.ResolvedExpression; -import org.apache.flink.table.types.DataType; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.FlinkFilters; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.source.assigner.SplitAssignerType; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -/** Flink Iceberg table source. */ -@Internal -public class IcebergTableSource - implements ScanTableSource, - SupportsProjectionPushDown, - SupportsFilterPushDown, - SupportsLimitPushDown { - - private int[] projectedFields; - private Long limit; - private List filters; - - private final TableLoader loader; - private final TableSchema schema; - private final Map properties; - private final boolean isLimitPushDown; - private final ReadableConfig readableConfig; - - private IcebergTableSource(IcebergTableSource toCopy) { - this.loader = toCopy.loader; - this.schema = toCopy.schema; - this.properties = toCopy.properties; - this.projectedFields = toCopy.projectedFields; - this.isLimitPushDown = toCopy.isLimitPushDown; - this.limit = toCopy.limit; - this.filters = toCopy.filters; - this.readableConfig = toCopy.readableConfig; - } - - public IcebergTableSource( - TableLoader loader, - TableSchema schema, - Map properties, - ReadableConfig readableConfig) { - this(loader, schema, properties, null, false, null, ImmutableList.of(), readableConfig); - } - - private IcebergTableSource( - TableLoader loader, - TableSchema schema, - Map properties, - int[] projectedFields, - boolean isLimitPushDown, - Long limit, - List filters, - ReadableConfig readableConfig) { - this.loader = loader; - this.schema = schema; - this.properties = properties; - this.projectedFields = projectedFields; - this.isLimitPushDown = isLimitPushDown; - this.limit = limit; - this.filters = filters; - this.readableConfig = readableConfig; - } - - @Override - public void applyProjection(int[][] projectFields) { - this.projectedFields = new int[projectFields.length]; - for (int i = 0; i < projectFields.length; i++) { - Preconditions.checkArgument( - projectFields[i].length == 1, "Don't support nested projection in iceberg source now."); - this.projectedFields[i] = projectFields[i][0]; - } - } - - private DataStream createDataStream(StreamExecutionEnvironment execEnv) { - return FlinkSource.forRowData() - .env(execEnv) - .tableLoader(loader) - .properties(properties) - .project(getProjectedSchema()) - .limit(limit) - .filters(filters) - .flinkConf(readableConfig) - .build(); - } - - private DataStreamSource createFLIP27Stream(StreamExecutionEnvironment env) { - SplitAssignerType assignerType = - readableConfig.get(FlinkConfigOptions.TABLE_EXEC_SPLIT_ASSIGNER_TYPE); - IcebergSource source = - IcebergSource.forRowData() - .tableLoader(loader) - .assignerFactory(assignerType.factory()) - .properties(properties) - .project(getProjectedSchema()) - .limit(limit) - .filters(filters) - .flinkConfig(readableConfig) - .build(); - DataStreamSource stream = - env.fromSource( - source, - WatermarkStrategy.noWatermarks(), - source.name(), - TypeInformation.of(RowData.class)); - return stream; - } - - private TableSchema getProjectedSchema() { - if (projectedFields == null) { - return schema; - } else { - String[] fullNames = schema.getFieldNames(); - DataType[] fullTypes = schema.getFieldDataTypes(); - return TableSchema.builder() - .fields( - Arrays.stream(projectedFields).mapToObj(i -> fullNames[i]).toArray(String[]::new), - Arrays.stream(projectedFields).mapToObj(i -> fullTypes[i]).toArray(DataType[]::new)) - .build(); - } - } - - @Override - public void applyLimit(long newLimit) { - this.limit = newLimit; - } - - @Override - public Result applyFilters(List flinkFilters) { - List acceptedFilters = Lists.newArrayList(); - List expressions = Lists.newArrayList(); - - for (ResolvedExpression resolvedExpression : flinkFilters) { - Optional icebergExpression = FlinkFilters.convert(resolvedExpression); - if (icebergExpression.isPresent()) { - expressions.add(icebergExpression.get()); - acceptedFilters.add(resolvedExpression); - } - } - - this.filters = expressions; - return Result.of(acceptedFilters, flinkFilters); - } - - @Override - public boolean supportsNestedProjection() { - // TODO: support nested projection - return false; - } - - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.insertOnly(); - } - - @Override - public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) { - return new DataStreamScanProvider() { - @Override - public DataStream produceDataStream( - ProviderContext providerContext, StreamExecutionEnvironment execEnv) { - if (readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE)) { - return createFLIP27Stream(execEnv); - } else { - return createDataStream(execEnv); - } - } - - @Override - public boolean isBounded() { - return FlinkSource.isBounded(properties); - } - }; - } - - @Override - public DynamicTableSource copy() { - return new IcebergTableSource(this); - } - - @Override - public String asSummaryString() { - return "Iceberg table source"; - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java deleted file mode 100644 index 88364f4e87b1..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java +++ /dev/null @@ -1,243 +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.iceberg.flink.source; - -import java.util.List; -import java.util.Map; -import org.apache.flink.annotation.Internal; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.MetadataColumns; -import org.apache.iceberg.Schema; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.data.DeleteFilter; -import org.apache.iceberg.encryption.InputFilesDecryptor; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.FlinkSourceFilter; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.data.FlinkAvroReader; -import org.apache.iceberg.flink.data.FlinkOrcReader; -import org.apache.iceberg.flink.data.FlinkParquetReaders; -import org.apache.iceberg.flink.data.RowDataProjection; -import org.apache.iceberg.flink.data.RowDataUtil; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.CloseableIterator; -import org.apache.iceberg.io.InputFile; -import org.apache.iceberg.mapping.NameMappingParser; -import org.apache.iceberg.orc.ORC; -import org.apache.iceberg.parquet.Parquet; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.util.PartitionUtil; - -@Internal -public class RowDataFileScanTaskReader implements FileScanTaskReader { - - private final Schema tableSchema; - private final Schema projectedSchema; - private final String nameMapping; - private final boolean caseSensitive; - private final FlinkSourceFilter rowFilter; - - public RowDataFileScanTaskReader( - Schema tableSchema, - Schema projectedSchema, - String nameMapping, - boolean caseSensitive, - List filters) { - this.tableSchema = tableSchema; - this.projectedSchema = projectedSchema; - this.nameMapping = nameMapping; - this.caseSensitive = caseSensitive; - - if (filters != null && !filters.isEmpty()) { - Expression combinedExpression = - filters.stream().reduce(Expressions.alwaysTrue(), Expressions::and); - this.rowFilter = - new FlinkSourceFilter(this.projectedSchema, combinedExpression, this.caseSensitive); - } else { - this.rowFilter = null; - } - } - - @Override - public CloseableIterator open( - FileScanTask task, InputFilesDecryptor inputFilesDecryptor) { - Schema partitionSchema = TypeUtil.select(projectedSchema, task.spec().identitySourceIds()); - - Map idToConstant = - partitionSchema.columns().isEmpty() - ? ImmutableMap.of() - : PartitionUtil.constantsMap(task, RowDataUtil::convertConstant); - - FlinkDeleteFilter deletes = - new FlinkDeleteFilter(task, tableSchema, projectedSchema, inputFilesDecryptor); - CloseableIterable iterable = - deletes.filter( - newIterable(task, deletes.requiredSchema(), idToConstant, inputFilesDecryptor)); - - // Project the RowData to remove the extra meta columns. - if (!projectedSchema.sameSchema(deletes.requiredSchema())) { - RowDataProjection rowDataProjection = - RowDataProjection.create( - deletes.requiredRowType(), - deletes.requiredSchema().asStruct(), - projectedSchema.asStruct()); - iterable = CloseableIterable.transform(iterable, rowDataProjection::wrap); - } - - return iterable.iterator(); - } - - private CloseableIterable newIterable( - FileScanTask task, - Schema schema, - Map idToConstant, - InputFilesDecryptor inputFilesDecryptor) { - CloseableIterable iter; - if (task.isDataTask()) { - throw new UnsupportedOperationException("Cannot read data task."); - } else { - switch (task.file().format()) { - case PARQUET: - iter = newParquetIterable(task, schema, idToConstant, inputFilesDecryptor); - break; - - case AVRO: - iter = newAvroIterable(task, schema, idToConstant, inputFilesDecryptor); - break; - - case ORC: - iter = newOrcIterable(task, schema, idToConstant, inputFilesDecryptor); - break; - - default: - throw new UnsupportedOperationException( - "Cannot read unknown format: " + task.file().format()); - } - } - - if (rowFilter != null) { - return CloseableIterable.filter(iter, rowFilter::filter); - } - return iter; - } - - private CloseableIterable newAvroIterable( - FileScanTask task, - Schema schema, - Map idToConstant, - InputFilesDecryptor inputFilesDecryptor) { - Avro.ReadBuilder builder = - Avro.read(inputFilesDecryptor.getInputFile(task)) - .reuseContainers() - .project(schema) - .split(task.start(), task.length()) - .createReaderFunc(readSchema -> new FlinkAvroReader(schema, readSchema, idToConstant)); - - if (nameMapping != null) { - builder.withNameMapping(NameMappingParser.fromJson(nameMapping)); - } - - return builder.build(); - } - - private CloseableIterable newParquetIterable( - FileScanTask task, - Schema schema, - Map idToConstant, - InputFilesDecryptor inputFilesDecryptor) { - Parquet.ReadBuilder builder = - Parquet.read(inputFilesDecryptor.getInputFile(task)) - .split(task.start(), task.length()) - .project(schema) - .createReaderFunc( - fileSchema -> FlinkParquetReaders.buildReader(schema, fileSchema, idToConstant)) - .filter(task.residual()) - .caseSensitive(caseSensitive) - .reuseContainers(); - - if (nameMapping != null) { - builder.withNameMapping(NameMappingParser.fromJson(nameMapping)); - } - - return builder.build(); - } - - private CloseableIterable newOrcIterable( - FileScanTask task, - Schema schema, - Map idToConstant, - InputFilesDecryptor inputFilesDecryptor) { - Schema readSchemaWithoutConstantAndMetadataFields = - TypeUtil.selectNot( - schema, Sets.union(idToConstant.keySet(), MetadataColumns.metadataFieldIds())); - - ORC.ReadBuilder builder = - ORC.read(inputFilesDecryptor.getInputFile(task)) - .project(readSchemaWithoutConstantAndMetadataFields) - .split(task.start(), task.length()) - .createReaderFunc( - readOrcSchema -> new FlinkOrcReader(schema, readOrcSchema, idToConstant)) - .filter(task.residual()) - .caseSensitive(caseSensitive); - - if (nameMapping != null) { - builder.withNameMapping(NameMappingParser.fromJson(nameMapping)); - } - - return builder.build(); - } - - private static class FlinkDeleteFilter extends DeleteFilter { - private final RowType requiredRowType; - private final RowDataWrapper asStructLike; - private final InputFilesDecryptor inputFilesDecryptor; - - FlinkDeleteFilter( - FileScanTask task, - Schema tableSchema, - Schema requestedSchema, - InputFilesDecryptor inputFilesDecryptor) { - super(task.file().path().toString(), task.deletes(), tableSchema, requestedSchema); - this.requiredRowType = FlinkSchemaUtil.convert(requiredSchema()); - this.asStructLike = new RowDataWrapper(requiredRowType, requiredSchema().asStruct()); - this.inputFilesDecryptor = inputFilesDecryptor; - } - - public RowType requiredRowType() { - return requiredRowType; - } - - @Override - protected StructLike asStructLike(RowData row) { - return asStructLike.wrap(row); - } - - @Override - protected InputFile getInputFile(String location) { - return inputFilesDecryptor.getInputFile(location); - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java deleted file mode 100644 index c958604c004a..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java +++ /dev/null @@ -1,172 +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.iceberg.flink.source; - -import static org.apache.iceberg.TableProperties.DEFAULT_NAME_MAPPING; - -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.stream.Collectors; -import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SerializableTable; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.encryption.EncryptionManager; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.sink.RowDataTaskWriterFactory; -import org.apache.iceberg.flink.sink.TaskWriterFactory; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.util.PropertyUtil; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class RowDataRewriter { - - private static final Logger LOG = LoggerFactory.getLogger(RowDataRewriter.class); - - private final Schema schema; - private final String nameMapping; - private final FileIO io; - private final boolean caseSensitive; - private final EncryptionManager encryptionManager; - private final TaskWriterFactory taskWriterFactory; - private final String tableName; - - public RowDataRewriter( - Table table, boolean caseSensitive, FileIO io, EncryptionManager encryptionManager) { - this.schema = table.schema(); - this.caseSensitive = caseSensitive; - this.io = io; - this.encryptionManager = encryptionManager; - this.nameMapping = - PropertyUtil.propertyAsString(table.properties(), DEFAULT_NAME_MAPPING, null); - this.tableName = table.name(); - - String formatString = - PropertyUtil.propertyAsString( - table.properties(), - TableProperties.DEFAULT_FILE_FORMAT, - TableProperties.DEFAULT_FILE_FORMAT_DEFAULT); - FileFormat format = FileFormat.fromString(formatString); - RowType flinkSchema = FlinkSchemaUtil.convert(table.schema()); - this.taskWriterFactory = - new RowDataTaskWriterFactory( - SerializableTable.copyOf(table), - flinkSchema, - Long.MAX_VALUE, - format, - table.properties(), - null, - false); - } - - public List rewriteDataForTasks( - DataStream dataStream, int parallelism) throws Exception { - RewriteMap map = - new RewriteMap( - schema, nameMapping, io, caseSensitive, encryptionManager, taskWriterFactory); - DataStream> ds = dataStream.map(map).setParallelism(parallelism); - return Lists.newArrayList(ds.executeAndCollect("Rewrite table :" + tableName)).stream() - .flatMap(Collection::stream) - .collect(Collectors.toList()); - } - - public static class RewriteMap extends RichMapFunction> { - - private TaskWriter writer; - private int subTaskId; - private int attemptId; - - private final Schema schema; - private final String nameMapping; - private final FileIO io; - private final boolean caseSensitive; - private final EncryptionManager encryptionManager; - private final TaskWriterFactory taskWriterFactory; - private final RowDataFileScanTaskReader rowDataReader; - - public RewriteMap( - Schema schema, - String nameMapping, - FileIO io, - boolean caseSensitive, - EncryptionManager encryptionManager, - TaskWriterFactory taskWriterFactory) { - this.schema = schema; - this.nameMapping = nameMapping; - this.io = io; - this.caseSensitive = caseSensitive; - this.encryptionManager = encryptionManager; - this.taskWriterFactory = taskWriterFactory; - this.rowDataReader = - new RowDataFileScanTaskReader( - schema, schema, nameMapping, caseSensitive, Collections.emptyList()); - } - - @Override - public void open(Configuration parameters) { - this.subTaskId = getRuntimeContext().getIndexOfThisSubtask(); - this.attemptId = getRuntimeContext().getAttemptNumber(); - // Initialize the task writer factory. - this.taskWriterFactory.initialize(subTaskId, attemptId); - } - - @Override - public List map(CombinedScanTask task) throws Exception { - // Initialize the task writer. - this.writer = taskWriterFactory.create(); - try (DataIterator iterator = - new DataIterator<>(rowDataReader, task, io, encryptionManager)) { - while (iterator.hasNext()) { - RowData rowData = iterator.next(); - writer.write(rowData); - } - return Lists.newArrayList(writer.dataFiles()); - } catch (Throwable originalThrowable) { - try { - LOG.error("Aborting commit for (subTaskId {}, attemptId {})", subTaskId, attemptId); - writer.abort(); - LOG.error("Aborted commit for (subTaskId {}, attemptId {})", subTaskId, attemptId); - } catch (Throwable inner) { - if (originalThrowable != inner) { - originalThrowable.addSuppressed(inner); - LOG.warn("Suppressing exception in catch: {}", inner.getMessage(), inner); - } - } - - if (originalThrowable instanceof Exception) { - throw originalThrowable; - } else { - throw new RuntimeException(originalThrowable); - } - } - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java deleted file mode 100644 index 8f95e3e554a0..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java +++ /dev/null @@ -1,70 +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.iceberg.flink.source; - -import java.io.Serializable; -import java.util.function.Function; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.flink.annotation.Internal; -import org.apache.flink.formats.avro.RowDataToAvroConverters; -import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.utils.TypeConversions; -import org.apache.iceberg.avro.AvroSchemaUtil; -import org.apache.iceberg.flink.FlinkSchemaUtil; - -/** - * This is not serializable because Avro {@link Schema} is not actually serializable, even though it - * implements {@link Serializable} interface. - */ -@Internal -public class RowDataToAvroGenericRecordConverter implements Function { - private final RowDataToAvroConverters.RowDataToAvroConverter converter; - private final Schema avroSchema; - - private RowDataToAvroGenericRecordConverter(RowType rowType, Schema avroSchema) { - this.converter = RowDataToAvroConverters.createConverter(rowType); - this.avroSchema = avroSchema; - } - - @Override - public GenericRecord apply(RowData rowData) { - return (GenericRecord) converter.convert(avroSchema, rowData); - } - - /** Create a converter based on Iceberg schema */ - public static RowDataToAvroGenericRecordConverter fromIcebergSchema( - String tableName, org.apache.iceberg.Schema icebergSchema) { - RowType rowType = FlinkSchemaUtil.convert(icebergSchema); - Schema avroSchema = AvroSchemaUtil.convert(icebergSchema, tableName); - return new RowDataToAvroGenericRecordConverter(rowType, avroSchema); - } - - /** Create a mapper based on Avro schema */ - public static RowDataToAvroGenericRecordConverter fromAvroSchema(Schema avroSchema) { - DataType dataType = AvroSchemaConverter.convertToDataType(avroSchema.toString()); - LogicalType logicalType = TypeConversions.fromDataToLogicalType(dataType); - RowType rowType = RowType.of(logicalType.getChildren().stream().toArray(LogicalType[]::new)); - return new RowDataToAvroGenericRecordConverter(rowType, avroSchema); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java deleted file mode 100644 index 4357b1f57df6..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java +++ /dev/null @@ -1,561 +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.iceberg.flink.source; - -import java.io.Serializable; -import java.time.Duration; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import org.apache.flink.annotation.Internal; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.util.Preconditions; -import org.apache.flink.util.TimeUtils; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.FlinkReadConf; -import org.apache.iceberg.flink.FlinkReadOptions; - -/** Context object with optional arguments for a Flink Scan. */ -@Internal -public class ScanContext implements Serializable { - - private static final long serialVersionUID = 1L; - - private final boolean caseSensitive; - private final boolean exposeLocality; - private final Long snapshotId; - private final String branch; - private final String tag; - private final StreamingStartingStrategy startingStrategy; - private final Long startSnapshotId; - private final Long startSnapshotTimestamp; - private final Long endSnapshotId; - private final Long asOfTimestamp; - private final String startTag; - private final String endTag; - private final Long splitSize; - private final Integer splitLookback; - private final Long splitOpenFileCost; - private final boolean isStreaming; - private final Duration monitorInterval; - - private final String nameMapping; - private final Schema schema; - private final List filters; - private final long limit; - private final boolean includeColumnStats; - private final Collection includeStatsForColumns; - private final Integer planParallelism; - private final int maxPlanningSnapshotCount; - private final int maxAllowedPlanningFailures; - - private ScanContext( - boolean caseSensitive, - Long snapshotId, - StreamingStartingStrategy startingStrategy, - Long startSnapshotTimestamp, - Long startSnapshotId, - Long endSnapshotId, - Long asOfTimestamp, - Long splitSize, - Integer splitLookback, - Long splitOpenFileCost, - boolean isStreaming, - Duration monitorInterval, - String nameMapping, - Schema schema, - List filters, - long limit, - boolean includeColumnStats, - Collection includeStatsForColumns, - boolean exposeLocality, - Integer planParallelism, - int maxPlanningSnapshotCount, - int maxAllowedPlanningFailures, - String branch, - String tag, - String startTag, - String endTag) { - this.caseSensitive = caseSensitive; - this.snapshotId = snapshotId; - this.tag = tag; - this.branch = branch; - this.startingStrategy = startingStrategy; - this.startSnapshotTimestamp = startSnapshotTimestamp; - this.startSnapshotId = startSnapshotId; - this.endSnapshotId = endSnapshotId; - this.asOfTimestamp = asOfTimestamp; - this.startTag = startTag; - this.endTag = endTag; - this.splitSize = splitSize; - this.splitLookback = splitLookback; - this.splitOpenFileCost = splitOpenFileCost; - this.isStreaming = isStreaming; - this.monitorInterval = monitorInterval; - - this.nameMapping = nameMapping; - this.schema = schema; - this.filters = filters; - this.limit = limit; - this.includeColumnStats = includeColumnStats; - this.includeStatsForColumns = includeStatsForColumns; - this.exposeLocality = exposeLocality; - this.planParallelism = planParallelism; - this.maxPlanningSnapshotCount = maxPlanningSnapshotCount; - this.maxAllowedPlanningFailures = maxAllowedPlanningFailures; - - validate(); - } - - private void validate() { - if (isStreaming) { - if (startingStrategy == StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) { - Preconditions.checkArgument( - startSnapshotId != null, - "Invalid starting snapshot id for SPECIFIC_START_SNAPSHOT_ID strategy: null"); - Preconditions.checkArgument( - startSnapshotTimestamp == null, - "Invalid starting snapshot timestamp for SPECIFIC_START_SNAPSHOT_ID strategy: not null"); - } - if (startingStrategy == StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) { - Preconditions.checkArgument( - startSnapshotTimestamp != null, - "Invalid starting snapshot timestamp for SPECIFIC_START_SNAPSHOT_TIMESTAMP strategy: null"); - Preconditions.checkArgument( - startSnapshotId == null, - "Invalid starting snapshot id for SPECIFIC_START_SNAPSHOT_ID strategy: not null"); - } - - Preconditions.checkArgument( - branch == null, - String.format( - "Cannot scan table using ref %s configured for streaming reader yet", branch)); - - Preconditions.checkArgument( - tag == null, - String.format("Cannot scan table using ref %s configured for streaming reader", tag)); - } - - Preconditions.checkArgument( - !(startTag != null && startSnapshotId() != null), - "START_SNAPSHOT_ID and START_TAG cannot both be set."); - - Preconditions.checkArgument( - !(endTag != null && endSnapshotId() != null), - "END_SNAPSHOT_ID and END_TAG cannot both be set."); - - Preconditions.checkArgument( - maxAllowedPlanningFailures >= -1, - "Cannot set maxAllowedPlanningFailures to a negative number other than -1."); - } - - public boolean caseSensitive() { - return caseSensitive; - } - - public Long snapshotId() { - return snapshotId; - } - - public String branch() { - return branch; - } - - public String tag() { - return tag; - } - - public String startTag() { - return startTag; - } - - public String endTag() { - return endTag; - } - - public StreamingStartingStrategy streamingStartingStrategy() { - return startingStrategy; - } - - public Long startSnapshotTimestamp() { - return startSnapshotTimestamp; - } - - public Long startSnapshotId() { - return startSnapshotId; - } - - public Long endSnapshotId() { - return endSnapshotId; - } - - public Long asOfTimestamp() { - return asOfTimestamp; - } - - public Long splitSize() { - return splitSize; - } - - public Integer splitLookback() { - return splitLookback; - } - - public Long splitOpenFileCost() { - return splitOpenFileCost; - } - - public boolean isStreaming() { - return isStreaming; - } - - public Duration monitorInterval() { - return monitorInterval; - } - - public String nameMapping() { - return nameMapping; - } - - public Schema project() { - return schema; - } - - public List filters() { - return filters; - } - - public long limit() { - return limit; - } - - public boolean includeColumnStats() { - return includeColumnStats; - } - - public Collection includeStatsForColumns() { - return includeStatsForColumns; - } - - public boolean exposeLocality() { - return exposeLocality; - } - - public Integer planParallelism() { - return planParallelism; - } - - public int maxPlanningSnapshotCount() { - return maxPlanningSnapshotCount; - } - - public int maxAllowedPlanningFailures() { - return maxAllowedPlanningFailures; - } - - public ScanContext copyWithAppendsBetween(Long newStartSnapshotId, long newEndSnapshotId) { - return ScanContext.builder() - .caseSensitive(caseSensitive) - .useSnapshotId(null) - .useBranch(branch) - .useTag(null) - .startSnapshotId(newStartSnapshotId) - .endSnapshotId(newEndSnapshotId) - .startTag(null) - .endTag(null) - .asOfTimestamp(null) - .splitSize(splitSize) - .splitLookback(splitLookback) - .splitOpenFileCost(splitOpenFileCost) - .streaming(isStreaming) - .monitorInterval(monitorInterval) - .nameMapping(nameMapping) - .project(schema) - .filters(filters) - .limit(limit) - .includeColumnStats(includeColumnStats) - .includeColumnStats(includeStatsForColumns) - .exposeLocality(exposeLocality) - .planParallelism(planParallelism) - .maxPlanningSnapshotCount(maxPlanningSnapshotCount) - .maxAllowedPlanningFailures(maxAllowedPlanningFailures) - .build(); - } - - public ScanContext copyWithSnapshotId(long newSnapshotId) { - return ScanContext.builder() - .caseSensitive(caseSensitive) - .useSnapshotId(newSnapshotId) - .useBranch(branch) - .useTag(tag) - .startSnapshotId(null) - .endSnapshotId(null) - .startTag(null) - .endTag(null) - .asOfTimestamp(null) - .splitSize(splitSize) - .splitLookback(splitLookback) - .splitOpenFileCost(splitOpenFileCost) - .streaming(isStreaming) - .monitorInterval(monitorInterval) - .nameMapping(nameMapping) - .project(schema) - .filters(filters) - .limit(limit) - .includeColumnStats(includeColumnStats) - .includeColumnStats(includeStatsForColumns) - .exposeLocality(exposeLocality) - .planParallelism(planParallelism) - .maxPlanningSnapshotCount(maxPlanningSnapshotCount) - .maxAllowedPlanningFailures(maxAllowedPlanningFailures) - .build(); - } - - public static Builder builder() { - return new Builder(); - } - - public static class Builder { - private boolean caseSensitive = FlinkReadOptions.CASE_SENSITIVE_OPTION.defaultValue(); - private Long snapshotId = FlinkReadOptions.SNAPSHOT_ID.defaultValue(); - private String branch = FlinkReadOptions.BRANCH.defaultValue(); - private String tag = FlinkReadOptions.TAG.defaultValue(); - private String startTag = FlinkReadOptions.START_TAG.defaultValue(); - private String endTag = FlinkReadOptions.END_TAG.defaultValue(); - private StreamingStartingStrategy startingStrategy = - FlinkReadOptions.STARTING_STRATEGY_OPTION.defaultValue(); - private Long startSnapshotTimestamp = FlinkReadOptions.START_SNAPSHOT_TIMESTAMP.defaultValue(); - private Long startSnapshotId = FlinkReadOptions.START_SNAPSHOT_ID.defaultValue(); - private Long endSnapshotId = FlinkReadOptions.END_SNAPSHOT_ID.defaultValue(); - private Long asOfTimestamp = FlinkReadOptions.AS_OF_TIMESTAMP.defaultValue(); - private Long splitSize = FlinkReadOptions.SPLIT_SIZE_OPTION.defaultValue(); - private Integer splitLookback = FlinkReadOptions.SPLIT_LOOKBACK_OPTION.defaultValue(); - private Long splitOpenFileCost = FlinkReadOptions.SPLIT_FILE_OPEN_COST_OPTION.defaultValue(); - private boolean isStreaming = FlinkReadOptions.STREAMING_OPTION.defaultValue(); - private Duration monitorInterval = - TimeUtils.parseDuration(FlinkReadOptions.MONITOR_INTERVAL_OPTION.defaultValue()); - private String nameMapping; - private Schema projectedSchema; - private List filters; - private long limit = FlinkReadOptions.LIMIT_OPTION.defaultValue(); - private boolean includeColumnStats = - FlinkReadOptions.INCLUDE_COLUMN_STATS_OPTION.defaultValue(); - private Collection includeStatsForColumns = null; - private boolean exposeLocality; - private Integer planParallelism = - FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.defaultValue(); - private int maxPlanningSnapshotCount = - FlinkReadOptions.MAX_PLANNING_SNAPSHOT_COUNT_OPTION.defaultValue(); - private int maxAllowedPlanningFailures = - FlinkReadOptions.MAX_ALLOWED_PLANNING_FAILURES_OPTION.defaultValue(); - - private Builder() {} - - public Builder caseSensitive(boolean newCaseSensitive) { - this.caseSensitive = newCaseSensitive; - return this; - } - - public Builder useSnapshotId(Long newSnapshotId) { - this.snapshotId = newSnapshotId; - return this; - } - - public Builder useTag(String newTag) { - this.tag = newTag; - return this; - } - - public Builder useBranch(String newBranch) { - this.branch = newBranch; - return this; - } - - public Builder startingStrategy(StreamingStartingStrategy newStartingStrategy) { - this.startingStrategy = newStartingStrategy; - return this; - } - - public Builder startSnapshotTimestamp(Long newStartSnapshotTimestamp) { - this.startSnapshotTimestamp = newStartSnapshotTimestamp; - return this; - } - - public Builder startSnapshotId(Long newStartSnapshotId) { - this.startSnapshotId = newStartSnapshotId; - return this; - } - - public Builder endSnapshotId(Long newEndSnapshotId) { - this.endSnapshotId = newEndSnapshotId; - return this; - } - - public Builder startTag(String newStartTag) { - this.startTag = newStartTag; - return this; - } - - public Builder endTag(String newEndTag) { - this.endTag = newEndTag; - return this; - } - - public Builder asOfTimestamp(Long newAsOfTimestamp) { - this.asOfTimestamp = newAsOfTimestamp; - return this; - } - - public Builder splitSize(Long newSplitSize) { - this.splitSize = newSplitSize; - return this; - } - - public Builder splitLookback(Integer newSplitLookback) { - this.splitLookback = newSplitLookback; - return this; - } - - public Builder splitOpenFileCost(Long newSplitOpenFileCost) { - this.splitOpenFileCost = newSplitOpenFileCost; - return this; - } - - public Builder streaming(boolean streaming) { - this.isStreaming = streaming; - return this; - } - - public Builder monitorInterval(Duration newMonitorInterval) { - this.monitorInterval = newMonitorInterval; - return this; - } - - public Builder nameMapping(String newNameMapping) { - this.nameMapping = newNameMapping; - return this; - } - - public Builder project(Schema newProjectedSchema) { - this.projectedSchema = newProjectedSchema; - return this; - } - - public Builder filters(List newFilters) { - this.filters = newFilters; - return this; - } - - public Builder limit(long newLimit) { - this.limit = newLimit; - return this; - } - - public Builder includeColumnStats(boolean newIncludeColumnStats) { - this.includeColumnStats = newIncludeColumnStats; - return this; - } - - public Builder includeColumnStats(Collection newIncludeStatsForColumns) { - this.includeStatsForColumns = newIncludeStatsForColumns; - return this; - } - - public Builder exposeLocality(boolean newExposeLocality) { - this.exposeLocality = newExposeLocality; - return this; - } - - public Builder planParallelism(Integer parallelism) { - this.planParallelism = parallelism; - return this; - } - - public Builder maxPlanningSnapshotCount(int newMaxPlanningSnapshotCount) { - this.maxPlanningSnapshotCount = newMaxPlanningSnapshotCount; - return this; - } - - public Builder maxAllowedPlanningFailures(int newMaxAllowedPlanningFailures) { - this.maxAllowedPlanningFailures = newMaxAllowedPlanningFailures; - return this; - } - - public Builder resolveConfig( - Table table, Map readOptions, ReadableConfig readableConfig) { - FlinkReadConf flinkReadConf = new FlinkReadConf(table, readOptions, readableConfig); - - return this.useSnapshotId(flinkReadConf.snapshotId()) - .useTag(flinkReadConf.tag()) - .useBranch(flinkReadConf.branch()) - .startTag(flinkReadConf.startTag()) - .endTag(flinkReadConf.endTag()) - .caseSensitive(flinkReadConf.caseSensitive()) - .asOfTimestamp(flinkReadConf.asOfTimestamp()) - .startingStrategy(flinkReadConf.startingStrategy()) - .startSnapshotTimestamp(flinkReadConf.startSnapshotTimestamp()) - .startSnapshotId(flinkReadConf.startSnapshotId()) - .endSnapshotId(flinkReadConf.endSnapshotId()) - .splitSize(flinkReadConf.splitSize()) - .splitLookback(flinkReadConf.splitLookback()) - .splitOpenFileCost(flinkReadConf.splitFileOpenCost()) - .streaming(flinkReadConf.streaming()) - .monitorInterval(flinkReadConf.monitorInterval()) - .nameMapping(flinkReadConf.nameMapping()) - .limit(flinkReadConf.limit()) - .planParallelism(flinkReadConf.workerPoolSize()) - .includeColumnStats(flinkReadConf.includeColumnStats()) - .maxPlanningSnapshotCount(flinkReadConf.maxPlanningSnapshotCount()) - .maxAllowedPlanningFailures(maxAllowedPlanningFailures); - } - - public ScanContext build() { - return new ScanContext( - caseSensitive, - snapshotId, - startingStrategy, - startSnapshotTimestamp, - startSnapshotId, - endSnapshotId, - asOfTimestamp, - splitSize, - splitLookback, - splitOpenFileCost, - isStreaming, - monitorInterval, - nameMapping, - projectedSchema, - filters, - limit, - includeColumnStats, - includeStatsForColumns, - exposeLocality, - planParallelism, - maxPlanningSnapshotCount, - maxAllowedPlanningFailures, - branch, - tag, - startTag, - endTag); - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java deleted file mode 100644 index 7c3a69dbc141..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java +++ /dev/null @@ -1,77 +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.iceberg.flink.source; - -import java.util.function.Supplier; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.table.api.config.ExecutionConfigOptions; -import org.apache.iceberg.Table; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.hadoop.Util; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -class SourceUtil { - private SourceUtil() {} - - static boolean isLocalityEnabled( - Table table, ReadableConfig readableConfig, Boolean exposeLocality) { - Boolean localityEnabled = - exposeLocality != null - ? exposeLocality - : readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_EXPOSE_SPLIT_LOCALITY_INFO); - - if (localityEnabled != null && !localityEnabled) { - return false; - } - - return Util.mayHaveBlockLocations(table.io(), table.location()); - } - - /** - * Infer source parallelism. - * - * @param readableConfig Flink config. - * @param splitCountProvider Split count supplier. As the computation may involve expensive split - * discover, lazy evaluation is performed if inferring parallelism is enabled. - * @param limitCount limited output count. - */ - static int inferParallelism( - ReadableConfig readableConfig, long limitCount, Supplier splitCountProvider) { - int parallelism = - readableConfig.get(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM); - if (readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM)) { - int maxInferParallelism = - readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX); - Preconditions.checkState( - maxInferParallelism >= 1, - FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX.key() - + " cannot be less than 1"); - parallelism = Math.min(splitCountProvider.get(), maxInferParallelism); - } - - if (limitCount > 0) { - int limit = limitCount >= Integer.MAX_VALUE ? Integer.MAX_VALUE : (int) limitCount; - parallelism = Math.min(parallelism, limit); - } - - // parallelism must be positive. - parallelism = Math.max(1, parallelism); - return parallelism; - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java deleted file mode 100644 index c27e29613fed..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java +++ /dev/null @@ -1,269 +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.iceberg.flink.source; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.List; -import java.util.concurrent.ExecutorService; -import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.api.common.state.ListState; -import org.apache.flink.api.common.state.ListStateDescriptor; -import org.apache.flink.api.common.typeutils.base.LongSerializer; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.state.FunctionInitializationContext; -import org.apache.flink.runtime.state.FunctionSnapshotContext; -import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; -import org.apache.flink.streaming.api.functions.source.RichSourceFunction; -import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.Table; -import org.apache.iceberg.exceptions.ValidationException; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.util.SnapshotUtil; -import org.apache.iceberg.util.ThreadPools; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * This is the single (non-parallel) monitoring task which takes a {@link FlinkInputFormat}, it is - * responsible for: - * - *

      - *
    1. Monitoring snapshots of the Iceberg table. - *
    2. Creating the {@link FlinkInputSplit splits} corresponding to the incremental files - *
    3. Assigning them to downstream tasks for further processing. - *
    - * - *

    The splits to be read are forwarded to the downstream {@link StreamingReaderOperator} which - * can have parallelism greater than one. - */ -public class StreamingMonitorFunction extends RichSourceFunction - implements CheckpointedFunction { - - private static final Logger LOG = LoggerFactory.getLogger(StreamingMonitorFunction.class); - - private static final long INIT_LAST_SNAPSHOT_ID = -1L; - - private final TableLoader tableLoader; - private final ScanContext scanContext; - - private volatile boolean isRunning = true; - - // The checkpoint thread is not the same thread that running the function for SourceStreamTask - // now. It's necessary to - // mark this as volatile. - private volatile long lastSnapshotId = INIT_LAST_SNAPSHOT_ID; - - private transient SourceContext sourceContext; - private transient Table table; - private transient ListState lastSnapshotIdState; - private transient ExecutorService workerPool; - - public StreamingMonitorFunction(TableLoader tableLoader, ScanContext scanContext) { - Preconditions.checkArgument( - scanContext.snapshotId() == null, "Cannot set snapshot-id option for streaming reader"); - Preconditions.checkArgument( - scanContext.asOfTimestamp() == null, - "Cannot set as-of-timestamp option for streaming reader"); - Preconditions.checkArgument( - scanContext.endSnapshotId() == null, - "Cannot set end-snapshot-id option for streaming reader"); - Preconditions.checkArgument( - scanContext.endTag() == null, "Cannot set end-tag option for streaming reader"); - Preconditions.checkArgument( - scanContext.maxPlanningSnapshotCount() > 0, - "The max-planning-snapshot-count must be greater than zero"); - this.tableLoader = tableLoader; - this.scanContext = scanContext; - } - - @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); - - final RuntimeContext runtimeContext = getRuntimeContext(); - ValidationException.check( - runtimeContext instanceof StreamingRuntimeContext, - "context should be instance of StreamingRuntimeContext"); - final String operatorID = ((StreamingRuntimeContext) runtimeContext).getOperatorUniqueID(); - this.workerPool = - ThreadPools.newWorkerPool( - "iceberg-worker-pool-" + operatorID, scanContext.planParallelism()); - } - - @Override - public void initializeState(FunctionInitializationContext context) throws Exception { - // Load iceberg table from table loader. - tableLoader.open(); - table = tableLoader.loadTable(); - - // Initialize the flink state for last snapshot id. - lastSnapshotIdState = - context - .getOperatorStateStore() - .getListState(new ListStateDescriptor<>("snapshot-id-state", LongSerializer.INSTANCE)); - - // Restore the last-snapshot-id from flink's state if possible. - if (context.isRestored()) { - LOG.info("Restoring state for the {}.", getClass().getSimpleName()); - lastSnapshotId = lastSnapshotIdState.get().iterator().next(); - } else if (scanContext.startTag() != null || scanContext.startSnapshotId() != null) { - Preconditions.checkArgument( - !(scanContext.startTag() != null && scanContext.startSnapshotId() != null), - "START_SNAPSHOT_ID and START_TAG cannot both be set."); - Preconditions.checkArgument( - scanContext.branch() == null, - "Cannot scan table using ref %s configured for streaming reader yet."); - Preconditions.checkNotNull( - table.currentSnapshot(), "Don't have any available snapshot in table."); - - long startSnapshotId; - if (scanContext.startTag() != null) { - Preconditions.checkArgument( - table.snapshot(scanContext.startTag()) != null, - "Cannot find snapshot with tag %s in table.", - scanContext.startTag()); - startSnapshotId = table.snapshot(scanContext.startTag()).snapshotId(); - } else { - startSnapshotId = scanContext.startSnapshotId(); - } - - long currentSnapshotId = table.currentSnapshot().snapshotId(); - Preconditions.checkState( - SnapshotUtil.isAncestorOf(table, currentSnapshotId, startSnapshotId), - "The option start-snapshot-id %s is not an ancestor of the current snapshot.", - startSnapshotId); - - lastSnapshotId = startSnapshotId; - } - } - - @Override - public void snapshotState(FunctionSnapshotContext context) throws Exception { - lastSnapshotIdState.clear(); - lastSnapshotIdState.add(lastSnapshotId); - } - - @Override - public void run(SourceContext ctx) throws Exception { - this.sourceContext = ctx; - while (isRunning) { - monitorAndForwardSplits(); - Thread.sleep(scanContext.monitorInterval().toMillis()); - } - } - - private long toSnapshotIdInclusive( - long lastConsumedSnapshotId, long currentSnapshotId, int maxPlanningSnapshotCount) { - List snapshotIds = - SnapshotUtil.snapshotIdsBetween(table, lastConsumedSnapshotId, currentSnapshotId); - if (snapshotIds.size() <= maxPlanningSnapshotCount) { - return currentSnapshotId; - } else { - // It uses reverted index since snapshotIdsBetween returns Ids that are ordered by committed - // time descending. - return snapshotIds.get(snapshotIds.size() - maxPlanningSnapshotCount); - } - } - - @VisibleForTesting - void sourceContext(SourceContext ctx) { - this.sourceContext = ctx; - } - - @VisibleForTesting - void monitorAndForwardSplits() { - // Refresh the table to get the latest committed snapshot. - table.refresh(); - - Snapshot snapshot = table.currentSnapshot(); - if (snapshot != null && snapshot.snapshotId() != lastSnapshotId) { - long snapshotId = snapshot.snapshotId(); - - ScanContext newScanContext; - if (lastSnapshotId == INIT_LAST_SNAPSHOT_ID) { - newScanContext = scanContext.copyWithSnapshotId(snapshotId); - } else { - snapshotId = - toSnapshotIdInclusive( - lastSnapshotId, snapshotId, scanContext.maxPlanningSnapshotCount()); - newScanContext = scanContext.copyWithAppendsBetween(lastSnapshotId, snapshotId); - } - - LOG.debug( - "Start discovering splits from {} (exclusive) to {} (inclusive)", - lastSnapshotId, - snapshotId); - long start = System.currentTimeMillis(); - FlinkInputSplit[] splits = - FlinkSplitPlanner.planInputSplits(table, newScanContext, workerPool); - LOG.debug( - "Discovered {} splits, time elapsed {}ms", - splits.length, - System.currentTimeMillis() - start); - - // only need to hold the checkpoint lock when emitting the splits and updating lastSnapshotId - start = System.currentTimeMillis(); - synchronized (sourceContext.getCheckpointLock()) { - for (FlinkInputSplit split : splits) { - sourceContext.collect(split); - } - - lastSnapshotId = snapshotId; - } - LOG.debug( - "Forwarded {} splits, time elapsed {}ms", - splits.length, - System.currentTimeMillis() - start); - } - } - - @Override - public void cancel() { - // this is to cover the case where cancel() is called before the run() - if (sourceContext != null) { - synchronized (sourceContext.getCheckpointLock()) { - isRunning = false; - } - } else { - isRunning = false; - } - - // Release all the resources here. - if (tableLoader != null) { - try { - tableLoader.close(); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } - } - - @Override - public void close() { - cancel(); - - if (workerPool != null) { - workerPool.shutdown(); - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java deleted file mode 100644 index ee6f7b63988d..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java +++ /dev/null @@ -1,246 +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.iceberg.flink.source; - -import java.io.IOException; -import java.util.Queue; -import org.apache.flink.api.common.operators.MailboxExecutor; -import org.apache.flink.api.common.state.ListState; -import org.apache.flink.api.common.state.ListStateDescriptor; -import org.apache.flink.runtime.state.JavaSerializer; -import org.apache.flink.runtime.state.StateInitializationContext; -import org.apache.flink.runtime.state.StateSnapshotContext; -import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.flink.streaming.api.operators.AbstractStreamOperator; -import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; -import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; -import org.apache.flink.streaming.api.operators.StreamOperator; -import org.apache.flink.streaming.api.operators.StreamOperatorParameters; -import org.apache.flink.streaming.api.operators.StreamSourceContexts; -import org.apache.flink.streaming.api.operators.YieldingOperatorFactory; -import org.apache.flink.streaming.api.watermark.Watermark; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * The operator that reads the {@link FlinkInputSplit splits} received from the preceding {@link - * StreamingMonitorFunction}. Contrary to the {@link StreamingMonitorFunction} which has a - * parallelism of 1, this operator can have multiple parallelism. - * - *

    As soon as a split descriptor is received, it is put in a queue, and use {@link - * MailboxExecutor} read the actual data of the split. This architecture allows the separation of - * the reading thread from the one split processing the checkpoint barriers, thus removing any - * potential back-pressure. - */ -public class StreamingReaderOperator extends AbstractStreamOperator - implements OneInputStreamOperator { - - private static final Logger LOG = LoggerFactory.getLogger(StreamingReaderOperator.class); - - // It's the same thread that is running this operator and checkpoint actions. we use this executor - // to schedule only - // one split for future reading, so that a new checkpoint could be triggered without blocking long - // time for exhausting - // all scheduled splits. - private final MailboxExecutor executor; - private FlinkInputFormat format; - - private transient SourceFunction.SourceContext sourceContext; - - private transient ListState inputSplitsState; - private transient Queue splits; - - // Splits are read by the same thread that calls processElement. Each read task is submitted to - // that thread by adding - // them to the executor. This state is used to ensure that only one read task is in that queue at - // a time, so that read - // tasks do not accumulate ahead of checkpoint tasks. When there is a read task in the queue, this - // is set to RUNNING. - // When there are no more files to read, this will be set to IDLE. - private transient SplitState currentSplitState; - - private StreamingReaderOperator( - FlinkInputFormat format, ProcessingTimeService timeService, MailboxExecutor mailboxExecutor) { - this.format = Preconditions.checkNotNull(format, "The InputFormat should not be null."); - this.processingTimeService = timeService; - this.executor = - Preconditions.checkNotNull(mailboxExecutor, "The mailboxExecutor should not be null."); - } - - @Override - public void initializeState(StateInitializationContext context) throws Exception { - super.initializeState(context); - - // TODO Replace Java serialization with Avro approach to keep state compatibility. - // See issue: https://github.com/apache/iceberg/issues/1698 - inputSplitsState = - context - .getOperatorStateStore() - .getListState(new ListStateDescriptor<>("splits", new JavaSerializer<>())); - - // Initialize the current split state to IDLE. - currentSplitState = SplitState.IDLE; - - // Recover splits state from flink state backend if possible. - splits = Lists.newLinkedList(); - if (context.isRestored()) { - int subtaskIdx = getRuntimeContext().getIndexOfThisSubtask(); - LOG.info("Restoring state for the {} (taskIdx: {}).", getClass().getSimpleName(), subtaskIdx); - - for (FlinkInputSplit split : inputSplitsState.get()) { - splits.add(split); - } - } - - this.sourceContext = - StreamSourceContexts.getSourceContext( - getOperatorConfig().getTimeCharacteristic(), - getProcessingTimeService(), - new Object(), // no actual locking needed - output, - getRuntimeContext().getExecutionConfig().getAutoWatermarkInterval(), - -1, - true); - - // Enqueue to process the recovered input splits. - enqueueProcessSplits(); - } - - @Override - public void snapshotState(StateSnapshotContext context) throws Exception { - super.snapshotState(context); - - inputSplitsState.clear(); - inputSplitsState.addAll(Lists.newArrayList(splits)); - } - - @Override - public void processElement(StreamRecord element) { - splits.add(element.getValue()); - enqueueProcessSplits(); - } - - private void enqueueProcessSplits() { - if (currentSplitState == SplitState.IDLE && !splits.isEmpty()) { - currentSplitState = SplitState.RUNNING; - executor.execute(this::processSplits, this.getClass().getSimpleName()); - } - } - - private void processSplits() throws IOException { - FlinkInputSplit split = splits.poll(); - if (split == null) { - currentSplitState = SplitState.IDLE; - return; - } - - format.open(split); - try { - RowData nextElement = null; - while (!format.reachedEnd()) { - nextElement = format.nextRecord(nextElement); - sourceContext.collect(nextElement); - } - } finally { - currentSplitState = SplitState.IDLE; - format.close(); - } - - // Re-schedule to process the next split. - enqueueProcessSplits(); - } - - @Override - public void processWatermark(Watermark mark) { - // we do nothing because we emit our own watermarks if needed. - } - - @Override - public void close() throws Exception { - super.close(); - - if (format != null) { - format.close(); - format.closeInputFormat(); - format = null; - } - - sourceContext = null; - } - - @Override - public void finish() throws Exception { - super.finish(); - output.close(); - if (sourceContext != null) { - sourceContext.emitWatermark(Watermark.MAX_WATERMARK); - sourceContext.close(); - sourceContext = null; - } - } - - static OneInputStreamOperatorFactory factory(FlinkInputFormat format) { - return new OperatorFactory(format); - } - - private enum SplitState { - IDLE, - RUNNING - } - - private static class OperatorFactory extends AbstractStreamOperatorFactory - implements YieldingOperatorFactory, - OneInputStreamOperatorFactory { - - private final FlinkInputFormat format; - - private transient MailboxExecutor mailboxExecutor; - - private OperatorFactory(FlinkInputFormat format) { - this.format = format; - } - - @Override - public void setMailboxExecutor(MailboxExecutor mailboxExecutor) { - this.mailboxExecutor = mailboxExecutor; - } - - @SuppressWarnings("unchecked") - @Override - public > O createStreamOperator( - StreamOperatorParameters parameters) { - StreamingReaderOperator operator = - new StreamingReaderOperator(format, processingTimeService, mailboxExecutor); - operator.setup( - parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); - return (O) operator; - } - - @Override - public Class getStreamOperatorClass(ClassLoader classLoader) { - return StreamingReaderOperator.class; - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java deleted file mode 100644 index 11707bf82a0f..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java +++ /dev/null @@ -1,54 +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.iceberg.flink.source; - -/** Starting strategy for streaming execution. */ -public enum StreamingStartingStrategy { - /** - * Do a regular table scan then switch to the incremental mode. - * - *

    The incremental mode starts from the current snapshot exclusive. - */ - TABLE_SCAN_THEN_INCREMENTAL, - - /** - * Start incremental mode from the latest snapshot inclusive. - * - *

    If it is an empty map, all future append snapshots should be discovered. - */ - INCREMENTAL_FROM_LATEST_SNAPSHOT, - - /** - * Start incremental mode from the earliest snapshot inclusive. - * - *

    If it is an empty map, all future append snapshots should be discovered. - */ - INCREMENTAL_FROM_EARLIEST_SNAPSHOT, - - /** Start incremental mode from a snapshot with a specific id inclusive. */ - INCREMENTAL_FROM_SNAPSHOT_ID, - - /** - * Start incremental mode from a snapshot with a specific timestamp inclusive. - * - *

    If the timestamp is between two snapshots, it should start from the snapshot after the - * timestamp. - */ - INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java deleted file mode 100644 index 37a0f1a6055f..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java +++ /dev/null @@ -1,112 +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.iceberg.flink.source.assigner; - -import java.util.ArrayDeque; -import java.util.Collection; -import java.util.PriorityQueue; -import java.util.Queue; -import java.util.concurrent.CompletableFuture; -import java.util.stream.Collectors; -import javax.annotation.Nullable; -import org.apache.flink.annotation.Internal; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; -import org.apache.iceberg.flink.source.split.SerializableComparator; - -/** - * Since all methods are called in the source coordinator thread by enumerator, there is no need for - * locking. - */ -@Internal -public class DefaultSplitAssigner implements SplitAssigner { - - private final Queue pendingSplits; - private CompletableFuture availableFuture; - - public DefaultSplitAssigner(SerializableComparator comparator) { - this.pendingSplits = comparator == null ? new ArrayDeque<>() : new PriorityQueue<>(comparator); - } - - public DefaultSplitAssigner( - SerializableComparator comparator, - Collection assignerState) { - this(comparator); - // Because default assigner only tracks unassigned splits, - // there is no need to filter splits based on status (unassigned) here. - assignerState.forEach(splitState -> pendingSplits.add(splitState.split())); - } - - @Override - public synchronized GetSplitResult getNext(@Nullable String hostname) { - if (pendingSplits.isEmpty()) { - return GetSplitResult.unavailable(); - } else { - IcebergSourceSplit split = pendingSplits.poll(); - return GetSplitResult.forSplit(split); - } - } - - @Override - public void onDiscoveredSplits(Collection splits) { - addSplits(splits); - } - - @Override - public void onUnassignedSplits(Collection splits) { - addSplits(splits); - } - - private synchronized void addSplits(Collection splits) { - if (!splits.isEmpty()) { - pendingSplits.addAll(splits); - // only complete pending future if new splits are discovered - completeAvailableFuturesIfNeeded(); - } - } - - /** Simple assigner only tracks unassigned splits */ - @Override - public synchronized Collection state() { - return pendingSplits.stream() - .map(split -> new IcebergSourceSplitState(split, IcebergSourceSplitStatus.UNASSIGNED)) - .collect(Collectors.toList()); - } - - @Override - public synchronized CompletableFuture isAvailable() { - if (availableFuture == null) { - availableFuture = new CompletableFuture<>(); - } - return availableFuture; - } - - @Override - public synchronized int pendingSplitCount() { - return pendingSplits.size(); - } - - private synchronized void completeAvailableFuturesIfNeeded() { - if (availableFuture != null && !pendingSplits.isEmpty()) { - availableFuture.complete(null); - } - availableFuture = null; - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java deleted file mode 100644 index 72deaeb890f3..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java +++ /dev/null @@ -1,77 +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.iceberg.flink.source.assigner; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.util.Preconditions; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; - -@Internal -public class GetSplitResult { - - public enum Status { - AVAILABLE, - - /** - * There are pending splits. But they can't be assigned due to constraints (like event time - * alignment) - */ - CONSTRAINED, - - /** Assigner doesn't have pending splits. */ - UNAVAILABLE - } - - private final Status status; - private final IcebergSourceSplit split; - - private GetSplitResult(Status status) { - this.status = status; - this.split = null; - } - - private GetSplitResult(IcebergSourceSplit split) { - Preconditions.checkNotNull(split, "Split cannot be null"); - this.status = Status.AVAILABLE; - this.split = split; - } - - public Status status() { - return status; - } - - public IcebergSourceSplit split() { - return split; - } - - private static final GetSplitResult UNAVAILABLE = new GetSplitResult(Status.UNAVAILABLE); - private static final GetSplitResult CONSTRAINED = new GetSplitResult(Status.CONSTRAINED); - - public static GetSplitResult unavailable() { - return UNAVAILABLE; - } - - public static GetSplitResult constrained() { - return CONSTRAINED; - } - - public static GetSplitResult forSplit(IcebergSourceSplit split) { - return new GetSplitResult(split); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java deleted file mode 100644 index e58478897aef..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java +++ /dev/null @@ -1,46 +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.iceberg.flink.source.assigner; - -import java.util.Collection; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; -import org.apache.iceberg.flink.source.split.SerializableComparator; - -/** - * Create default assigner with a comparator that hands out splits where the order of the splits - * will be defined by the {@link SerializableComparator}. - */ -public class OrderedSplitAssignerFactory implements SplitAssignerFactory { - private final SerializableComparator comparator; - - public OrderedSplitAssignerFactory(SerializableComparator comparator) { - this.comparator = comparator; - } - - @Override - public SplitAssigner createAssigner() { - return new DefaultSplitAssigner(comparator); - } - - @Override - public SplitAssigner createAssigner(Collection assignerState) { - return new DefaultSplitAssigner(comparator, assignerState); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java deleted file mode 100644 index a2e2ff364d46..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java +++ /dev/null @@ -1,37 +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.iceberg.flink.source.assigner; - -import java.util.Collection; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; - -/** Create simple assigner that hands out splits without any guarantee in order or locality. */ -public class SimpleSplitAssignerFactory implements SplitAssignerFactory { - public SimpleSplitAssignerFactory() {} - - @Override - public SplitAssigner createAssigner() { - return new DefaultSplitAssigner(null); - } - - @Override - public SplitAssigner createAssigner(Collection assignerState) { - return new DefaultSplitAssigner(null, assignerState); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java deleted file mode 100644 index ca60612f0ec9..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java +++ /dev/null @@ -1,118 +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.iceberg.flink.source.assigner; - -import java.io.Closeable; -import java.util.Collection; -import java.util.concurrent.CompletableFuture; -import javax.annotation.Nullable; -import org.apache.flink.api.connector.source.SplitEnumeratorContext; -import org.apache.iceberg.flink.source.ScanContext; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; - -/** - * SplitAssigner interface is extracted out as a separate component so that we can plug in different - * split assignment strategy for different requirements. E.g. - * - *

      - *
    • Simple assigner with no ordering guarantee or locality aware optimization. - *
    • Locality aware assigner that prefer splits that are local. - *
    • Snapshot aware assigner that assign splits based on the order they are committed. - *
    • Event time alignment assigner that assign splits satisfying certain time ordering within a - * single source or across sources. - *
    - * - *

    Assigner implementation needs to be thread safe. Enumerator call the assigner APIs mostly from - * the coordinator thread. But enumerator may call the {@link SplitAssigner#pendingSplitCount()} - * from the I/O threads. - */ -public interface SplitAssigner extends Closeable { - - /** - * Some assigners may need to start background threads or perform other activity such as - * registering as listeners to updates from other event sources e.g., watermark tracker. - */ - default void start() {} - - /** - * Some assigners may need to perform certain actions when their corresponding enumerators are - * closed - */ - @Override - default void close() {} - - /** - * Request a new split from the assigner when enumerator trying to assign splits to awaiting - * readers. - * - *

    If enumerator wasn't able to assign the split (e.g., reader disconnected), enumerator should - * call {@link SplitAssigner#onUnassignedSplits} to return the split. - */ - GetSplitResult getNext(@Nullable String hostname); - - /** Add new splits discovered by enumerator */ - void onDiscoveredSplits(Collection splits); - - /** Forward addSplitsBack event (for failed reader) to assigner */ - void onUnassignedSplits(Collection splits); - - /** - * Some assigner (like event time alignment) may rack in-progress splits to advance watermark upon - * completed splits - */ - default void onCompletedSplits(Collection completedSplitIds) {} - - /** - * Get assigner state for checkpointing. This is a super-set API that works for all currently - * imagined assigners. - */ - Collection state(); - - /** - * Enumerator can get a notification via CompletableFuture when the assigner has more splits - * available later. Enumerator should schedule assignment in the thenAccept action of the future. - * - *

    Assigner will return the same future if this method is called again before the previous - * future is completed. - * - *

    The future can be completed from other thread, e.g. the coordinator thread from another - * thread for event time alignment. - * - *

    If enumerator need to trigger action upon the future completion, it may want to run it in - * the coordinator thread using {@link SplitEnumeratorContext#runInCoordinatorThread(Runnable)}. - */ - CompletableFuture isAvailable(); - - /** - * Return the number of pending splits that haven't been assigned yet. - * - *

    The enumerator can poll this API to publish a metric on the number of pending splits. - * - *

    The enumerator can also use this information to throttle split discovery for streaming read. - * If there are already many pending splits tracked by the assigner, it is undesirable to discover - * more splits and track them in the assigner. That will increase the memory footprint and - * enumerator checkpoint size. - * - *

    Throttling works better together with {@link ScanContext#maxPlanningSnapshotCount()}. - * Otherwise, the next split discovery after throttling will just discover all non-enumerated - * snapshots and splits, which defeats the purpose of throttling. - */ - int pendingSplitCount(); -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java deleted file mode 100644 index 6e02a556ffcd..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java +++ /dev/null @@ -1,30 +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.iceberg.flink.source.assigner; - -import java.io.Serializable; -import java.util.Collection; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; - -public interface SplitAssignerFactory extends Serializable { - - SplitAssigner createAssigner(); - - SplitAssigner createAssigner(Collection assignerState); -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java deleted file mode 100644 index 03ba67a554f9..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java +++ /dev/null @@ -1,33 +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.iceberg.flink.source.assigner; - -import org.apache.flink.annotation.Internal; - -@Internal -public enum SplitAssignerType { - SIMPLE { - @Override - public SplitAssignerFactory factory() { - return new SimpleSplitAssignerFactory(); - } - }; - - public abstract SplitAssignerFactory factory(); -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java deleted file mode 100644 index 3aca390755ed..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java +++ /dev/null @@ -1,173 +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.iceberg.flink.source.enumerator; - -import java.io.IOException; -import java.util.Iterator; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.atomic.AtomicReference; -import javax.annotation.Nullable; -import org.apache.flink.api.connector.source.SourceEvent; -import org.apache.flink.api.connector.source.SplitEnumerator; -import org.apache.flink.api.connector.source.SplitEnumeratorContext; -import org.apache.iceberg.flink.source.assigner.GetSplitResult; -import org.apache.iceberg.flink.source.assigner.SplitAssigner; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.SplitRequestEvent; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * TODO: publish enumerator monitor metrics like number of pending metrics after FLINK-21000 is - * resolved - */ -abstract class AbstractIcebergEnumerator - implements SplitEnumerator { - private static final Logger LOG = LoggerFactory.getLogger(AbstractIcebergEnumerator.class); - - private final SplitEnumeratorContext enumeratorContext; - private final SplitAssigner assigner; - private final Map readersAwaitingSplit; - private final AtomicReference> availableFuture; - - AbstractIcebergEnumerator( - SplitEnumeratorContext enumeratorContext, SplitAssigner assigner) { - this.enumeratorContext = enumeratorContext; - this.assigner = assigner; - this.readersAwaitingSplit = new LinkedHashMap<>(); - this.availableFuture = new AtomicReference<>(); - } - - @Override - public void start() { - assigner.start(); - } - - @Override - public void close() throws IOException { - assigner.close(); - } - - @Override - public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) { - // Iceberg source uses custom split request event to piggyback finished split ids. - throw new UnsupportedOperationException( - String.format( - "Received invalid default split request event " - + "from subtask %d as Iceberg source uses custom split request event", - subtaskId)); - } - - @Override - public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { - if (sourceEvent instanceof SplitRequestEvent) { - SplitRequestEvent splitRequestEvent = (SplitRequestEvent) sourceEvent; - LOG.info("Received request split event from subtask {}", subtaskId); - assigner.onCompletedSplits(splitRequestEvent.finishedSplitIds()); - readersAwaitingSplit.put(subtaskId, splitRequestEvent.requesterHostname()); - assignSplits(); - } else { - throw new IllegalArgumentException( - String.format( - "Received unknown event from subtask %d: %s", - subtaskId, sourceEvent.getClass().getCanonicalName())); - } - } - - @Override - public void addSplitsBack(List splits, int subtaskId) { - LOG.info("Add {} splits back to the pool for failed subtask {}", splits.size(), subtaskId); - assigner.onUnassignedSplits(splits); - assignSplits(); - } - - @Override - public void addReader(int subtaskId) { - LOG.info("Added reader: {}", subtaskId); - } - - private void assignSplits() { - LOG.info("Assigning splits for {} awaiting readers", readersAwaitingSplit.size()); - Iterator> awaitingReader = - readersAwaitingSplit.entrySet().iterator(); - while (awaitingReader.hasNext()) { - Map.Entry nextAwaiting = awaitingReader.next(); - // if the reader that requested another split has failed in the meantime, remove - // it from the list of waiting readers - if (!enumeratorContext.registeredReaders().containsKey(nextAwaiting.getKey())) { - awaitingReader.remove(); - continue; - } - - int awaitingSubtask = nextAwaiting.getKey(); - String hostname = nextAwaiting.getValue(); - GetSplitResult getResult = assigner.getNext(hostname); - if (getResult.status() == GetSplitResult.Status.AVAILABLE) { - LOG.info("Assign split to subtask {}: {}", awaitingSubtask, getResult.split()); - enumeratorContext.assignSplit(getResult.split(), awaitingSubtask); - awaitingReader.remove(); - } else if (getResult.status() == GetSplitResult.Status.CONSTRAINED) { - getAvailableFutureIfNeeded(); - break; - } else if (getResult.status() == GetSplitResult.Status.UNAVAILABLE) { - if (shouldWaitForMoreSplits()) { - getAvailableFutureIfNeeded(); - break; - } else { - LOG.info("No more splits available for subtask {}", awaitingSubtask); - enumeratorContext.signalNoMoreSplits(awaitingSubtask); - awaitingReader.remove(); - } - } else { - throw new IllegalArgumentException("Unsupported status: " + getResult.status()); - } - } - } - - /** return true if enumerator should wait for splits like in the continuous enumerator case */ - protected abstract boolean shouldWaitForMoreSplits(); - - private synchronized void getAvailableFutureIfNeeded() { - if (availableFuture.get() != null) { - return; - } - - CompletableFuture future = - assigner - .isAvailable() - .thenAccept( - ignore -> - // Must run assignSplits in coordinator thread - // because the future may be completed from other threads. - // E.g., in event time alignment assigner, - // watermark advancement from another source may - // cause the available future to be completed - enumeratorContext.runInCoordinatorThread( - () -> { - LOG.debug("Executing callback of assignSplits"); - availableFuture.set(null); - assignSplits(); - })); - availableFuture.set(future); - LOG.debug("Registered callback for future available splits"); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java deleted file mode 100644 index 41863ffee60b..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java +++ /dev/null @@ -1,57 +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.iceberg.flink.source.enumerator; - -import java.util.Collection; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -class ContinuousEnumerationResult { - private final Collection splits; - private final IcebergEnumeratorPosition fromPosition; - private final IcebergEnumeratorPosition toPosition; - - /** - * @param splits should never be null. But it can be an empty collection - * @param fromPosition can be null - * @param toPosition should never be null. But it can have null snapshotId and snapshotTimestampMs - */ - ContinuousEnumerationResult( - Collection splits, - IcebergEnumeratorPosition fromPosition, - IcebergEnumeratorPosition toPosition) { - Preconditions.checkArgument(splits != null, "Invalid to splits collection: null"); - Preconditions.checkArgument(toPosition != null, "Invalid end position: null"); - this.splits = splits; - this.fromPosition = fromPosition; - this.toPosition = toPosition; - } - - public Collection splits() { - return splits; - } - - public IcebergEnumeratorPosition fromPosition() { - return fromPosition; - } - - public IcebergEnumeratorPosition toPosition() { - return toPosition; - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java deleted file mode 100644 index b1dadfb9a69a..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java +++ /dev/null @@ -1,177 +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.iceberg.flink.source.enumerator; - -import java.io.IOException; -import java.util.Collections; -import java.util.Objects; -import java.util.concurrent.atomic.AtomicReference; -import javax.annotation.Nullable; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.connector.source.SplitEnumeratorContext; -import org.apache.iceberg.flink.source.ScanContext; -import org.apache.iceberg.flink.source.assigner.SplitAssigner; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@Internal -public class ContinuousIcebergEnumerator extends AbstractIcebergEnumerator { - - private static final Logger LOG = LoggerFactory.getLogger(ContinuousIcebergEnumerator.class); - /** - * This is hardcoded, as {@link ScanContext#maxPlanningSnapshotCount()} could be the knob to - * control the total number of snapshots worth of splits tracked by assigner. - */ - private static final int ENUMERATION_SPLIT_COUNT_HISTORY_SIZE = 3; - - private final SplitEnumeratorContext enumeratorContext; - private final SplitAssigner assigner; - private final ScanContext scanContext; - private final ContinuousSplitPlanner splitPlanner; - - /** - * snapshotId for the last enumerated snapshot. next incremental enumeration should be based off - * this as the starting position. - */ - private final AtomicReference enumeratorPosition; - - /** Track enumeration result history for split discovery throttling. */ - private final EnumerationHistory enumerationHistory; - - /** Count the consecutive failures and throw exception if the max allowed failres are reached */ - private transient int consecutiveFailures = 0; - - public ContinuousIcebergEnumerator( - SplitEnumeratorContext enumeratorContext, - SplitAssigner assigner, - ScanContext scanContext, - ContinuousSplitPlanner splitPlanner, - @Nullable IcebergEnumeratorState enumState) { - super(enumeratorContext, assigner); - - this.enumeratorContext = enumeratorContext; - this.assigner = assigner; - this.scanContext = scanContext; - this.splitPlanner = splitPlanner; - this.enumeratorPosition = new AtomicReference<>(); - this.enumerationHistory = new EnumerationHistory(ENUMERATION_SPLIT_COUNT_HISTORY_SIZE); - - if (enumState != null) { - this.enumeratorPosition.set(enumState.lastEnumeratedPosition()); - } - } - - @Override - public void start() { - super.start(); - enumeratorContext.callAsync( - this::discoverSplits, - this::processDiscoveredSplits, - 0L, - scanContext.monitorInterval().toMillis()); - } - - @Override - public void close() throws IOException { - splitPlanner.close(); - super.close(); - } - - @Override - protected boolean shouldWaitForMoreSplits() { - return true; - } - - @Override - public IcebergEnumeratorState snapshotState(long checkpointId) { - return new IcebergEnumeratorState( - enumeratorPosition.get(), assigner.state(), enumerationHistory.snapshot()); - } - - /** This method is executed in an IO thread pool. */ - private ContinuousEnumerationResult discoverSplits() { - int pendingSplitCountFromAssigner = assigner.pendingSplitCount(); - if (enumerationHistory.shouldPauseSplitDiscovery(pendingSplitCountFromAssigner)) { - // If the assigner already has many pending splits, it is better to pause split discovery. - // Otherwise, eagerly discovering more splits will just increase assigner memory footprint - // and enumerator checkpoint state size. - LOG.info( - "Pause split discovery as the assigner already has too many pending splits: {}", - pendingSplitCountFromAssigner); - return new ContinuousEnumerationResult( - Collections.emptyList(), enumeratorPosition.get(), enumeratorPosition.get()); - } else { - return splitPlanner.planSplits(enumeratorPosition.get()); - } - } - - /** This method is executed in a single coordinator thread. */ - private void processDiscoveredSplits(ContinuousEnumerationResult result, Throwable error) { - if (error == null) { - consecutiveFailures = 0; - if (!Objects.equals(result.fromPosition(), enumeratorPosition.get())) { - // Multiple discoverSplits() may be triggered with the same starting snapshot to the I/O - // thread pool. E.g., the splitDiscoveryInterval is very short (like 10 ms in some unit - // tests) or the thread pool is busy and multiple discovery actions are executed - // concurrently. Discovery result should only be accepted if the starting position - // matches the enumerator position (like compare-and-swap). - LOG.info( - "Skip {} discovered splits because the scan starting position doesn't match " - + "the current enumerator position: enumerator position = {}, scan starting position = {}", - result.splits().size(), - enumeratorPosition.get(), - result.fromPosition()); - } else { - // Sometimes, enumeration may yield no splits for a few reasons. - // - upstream paused or delayed streaming writes to the Iceberg table. - // - enumeration frequency is higher than the upstream write frequency. - if (!result.splits().isEmpty()) { - assigner.onDiscoveredSplits(result.splits()); - // EnumerationHistory makes throttling decision on split discovery - // based on the total number of splits discovered in the last a few cycles. - // Only update enumeration history when there are some discovered splits. - enumerationHistory.add(result.splits().size()); - LOG.info( - "Added {} splits discovered between ({}, {}] to the assigner", - result.splits().size(), - result.fromPosition(), - result.toPosition()); - } else { - LOG.info( - "No new splits discovered between ({}, {}]", - result.fromPosition(), - result.toPosition()); - } - // update the enumerator position even if there is no split discovered - // or the toPosition is empty (e.g. for empty table). - enumeratorPosition.set(result.toPosition()); - LOG.info("Update enumerator position to {}", result.toPosition()); - } - } else { - consecutiveFailures++; - if (scanContext.maxAllowedPlanningFailures() < 0 - || consecutiveFailures <= scanContext.maxAllowedPlanningFailures()) { - LOG.error("Failed to discover new splits", error); - } else { - throw new RuntimeException("Failed to discover new splits", error); - } - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java deleted file mode 100644 index 2a1325178873..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java +++ /dev/null @@ -1,30 +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.iceberg.flink.source.enumerator; - -import java.io.Closeable; -import org.apache.flink.annotation.Internal; - -/** This interface is introduced so that we can plug in different split planner for unit test */ -@Internal -public interface ContinuousSplitPlanner extends Closeable { - - /** Discover the files appended between {@code lastPosition} and current table snapshot */ - ContinuousEnumerationResult planSplits(IcebergEnumeratorPosition lastPosition); -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java deleted file mode 100644 index f0d8ca8d7057..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java +++ /dev/null @@ -1,236 +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.iceberg.flink.source.enumerator; - -import java.io.IOException; -import java.util.Collections; -import java.util.List; -import java.util.Optional; -import java.util.concurrent.ExecutorService; -import org.apache.flink.annotation.Internal; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.util.Preconditions; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.Table; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.source.FlinkSplitPlanner; -import org.apache.iceberg.flink.source.ScanContext; -import org.apache.iceberg.flink.source.StreamingStartingStrategy; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.util.SnapshotUtil; -import org.apache.iceberg.util.ThreadPools; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@Internal -public class ContinuousSplitPlannerImpl implements ContinuousSplitPlanner { - private static final Logger LOG = LoggerFactory.getLogger(ContinuousSplitPlannerImpl.class); - - private final Table table; - private final ScanContext scanContext; - private final boolean isSharedPool; - private final ExecutorService workerPool; - private final TableLoader tableLoader; - - /** - * @param tableLoader A cloned tableLoader. - * @param threadName thread name prefix for worker pool to run the split planning. If null, a - * shared worker pool will be used. - */ - public ContinuousSplitPlannerImpl( - TableLoader tableLoader, ScanContext scanContext, String threadName) { - this.tableLoader = tableLoader; - this.tableLoader.open(); - this.table = tableLoader.loadTable(); - this.scanContext = scanContext; - this.isSharedPool = threadName == null; - this.workerPool = - isSharedPool - ? ThreadPools.getWorkerPool() - : ThreadPools.newWorkerPool( - "iceberg-plan-worker-pool-" + threadName, scanContext.planParallelism()); - } - - @Override - public void close() throws IOException { - if (!isSharedPool) { - workerPool.shutdown(); - } - tableLoader.close(); - } - - @Override - public ContinuousEnumerationResult planSplits(IcebergEnumeratorPosition lastPosition) { - table.refresh(); - if (lastPosition != null) { - return discoverIncrementalSplits(lastPosition); - } else { - return discoverInitialSplits(); - } - } - - private Snapshot toSnapshotInclusive( - Long lastConsumedSnapshotId, Snapshot currentSnapshot, int maxPlanningSnapshotCount) { - // snapshots are in reverse order (latest snapshot first) - List snapshots = - Lists.newArrayList( - SnapshotUtil.ancestorsBetween( - table, currentSnapshot.snapshotId(), lastConsumedSnapshotId)); - if (snapshots.size() <= maxPlanningSnapshotCount) { - return currentSnapshot; - } else { - // Because snapshots are in reverse order of commit history, this index returns - // the max allowed number of snapshots from the lastConsumedSnapshotId. - return snapshots.get(snapshots.size() - maxPlanningSnapshotCount); - } - } - - private ContinuousEnumerationResult discoverIncrementalSplits( - IcebergEnumeratorPosition lastPosition) { - Snapshot currentSnapshot = table.currentSnapshot(); - if (currentSnapshot == null) { - // empty table - Preconditions.checkArgument( - lastPosition.snapshotId() == null, - "Invalid last enumerated position for an empty table: not null"); - LOG.info("Skip incremental scan because table is empty"); - return new ContinuousEnumerationResult(Collections.emptyList(), lastPosition, lastPosition); - } else if (lastPosition.snapshotId() != null - && currentSnapshot.snapshotId() == lastPosition.snapshotId()) { - LOG.info("Current table snapshot is already enumerated: {}", currentSnapshot.snapshotId()); - return new ContinuousEnumerationResult(Collections.emptyList(), lastPosition, lastPosition); - } else { - Long lastConsumedSnapshotId = lastPosition != null ? lastPosition.snapshotId() : null; - Snapshot toSnapshotInclusive = - toSnapshotInclusive( - lastConsumedSnapshotId, currentSnapshot, scanContext.maxPlanningSnapshotCount()); - IcebergEnumeratorPosition newPosition = - IcebergEnumeratorPosition.of( - toSnapshotInclusive.snapshotId(), toSnapshotInclusive.timestampMillis()); - ScanContext incrementalScan = - scanContext.copyWithAppendsBetween( - lastPosition.snapshotId(), toSnapshotInclusive.snapshotId()); - List splits = - FlinkSplitPlanner.planIcebergSourceSplits(table, incrementalScan, workerPool); - LOG.info( - "Discovered {} splits from incremental scan: " - + "from snapshot (exclusive) is {}, to snapshot (inclusive) is {}", - splits.size(), - lastPosition, - newPosition); - return new ContinuousEnumerationResult(splits, lastPosition, newPosition); - } - } - - /** - * Discovery initial set of splits based on {@link StreamingStartingStrategy}. - *

  • {@link ContinuousEnumerationResult#splits()} should contain initial splits discovered from - * table scan for {@link StreamingStartingStrategy#TABLE_SCAN_THEN_INCREMENTAL}. For all other - * strategies, splits collection should be empty. - *
  • {@link ContinuousEnumerationResult#toPosition()} points to the starting position for the - * next incremental split discovery with exclusive behavior. Meaning files committed by the - * snapshot from the position in {@code ContinuousEnumerationResult} won't be included in the - * next incremental scan. - */ - private ContinuousEnumerationResult discoverInitialSplits() { - Optional startSnapshotOptional = startSnapshot(table, scanContext); - if (!startSnapshotOptional.isPresent()) { - return new ContinuousEnumerationResult( - Collections.emptyList(), null, IcebergEnumeratorPosition.empty()); - } - - Snapshot startSnapshot = startSnapshotOptional.get(); - LOG.info( - "Get starting snapshot id {} based on strategy {}", - startSnapshot.snapshotId(), - scanContext.streamingStartingStrategy()); - List splits; - IcebergEnumeratorPosition toPosition; - if (scanContext.streamingStartingStrategy() - == StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) { - // do a batch table scan first - splits = - FlinkSplitPlanner.planIcebergSourceSplits( - table, scanContext.copyWithSnapshotId(startSnapshot.snapshotId()), workerPool); - LOG.info( - "Discovered {} splits from initial batch table scan with snapshot Id {}", - splits.size(), - startSnapshot.snapshotId()); - // For TABLE_SCAN_THEN_INCREMENTAL, incremental mode starts exclusive from the startSnapshot - toPosition = - IcebergEnumeratorPosition.of(startSnapshot.snapshotId(), startSnapshot.timestampMillis()); - } else { - // For all other modes, starting snapshot should be consumed inclusively. - // Use parentId to achieve the inclusive behavior. It is fine if parentId is null. - splits = Collections.emptyList(); - Long parentSnapshotId = startSnapshot.parentId(); - if (parentSnapshotId != null) { - Snapshot parentSnapshot = table.snapshot(parentSnapshotId); - Long parentSnapshotTimestampMs = - parentSnapshot != null ? parentSnapshot.timestampMillis() : null; - toPosition = IcebergEnumeratorPosition.of(parentSnapshotId, parentSnapshotTimestampMs); - } else { - toPosition = IcebergEnumeratorPosition.empty(); - } - - LOG.info( - "Start incremental scan with start snapshot (inclusive): id = {}, timestamp = {}", - startSnapshot.snapshotId(), - startSnapshot.timestampMillis()); - } - - return new ContinuousEnumerationResult(splits, null, toPosition); - } - - /** - * Calculate the starting snapshot based on the {@link StreamingStartingStrategy} defined in - * {@code ScanContext}. - * - *

    If the {@link StreamingStartingStrategy} is not {@link - * StreamingStartingStrategy#TABLE_SCAN_THEN_INCREMENTAL}, the start snapshot should be consumed - * inclusively. - */ - @VisibleForTesting - static Optional startSnapshot(Table table, ScanContext scanContext) { - switch (scanContext.streamingStartingStrategy()) { - case TABLE_SCAN_THEN_INCREMENTAL: - case INCREMENTAL_FROM_LATEST_SNAPSHOT: - return Optional.ofNullable(table.currentSnapshot()); - case INCREMENTAL_FROM_EARLIEST_SNAPSHOT: - return Optional.ofNullable(SnapshotUtil.oldestAncestor(table)); - case INCREMENTAL_FROM_SNAPSHOT_ID: - Snapshot matchedSnapshotById = table.snapshot(scanContext.startSnapshotId()); - Preconditions.checkArgument( - matchedSnapshotById != null, - "Start snapshot id not found in history: " + scanContext.startSnapshotId()); - return Optional.of(matchedSnapshotById); - case INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP: - Snapshot matchedSnapshotByTimestamp = - SnapshotUtil.oldestAncestorAfter(table, scanContext.startSnapshotTimestamp()); - Preconditions.checkArgument( - matchedSnapshotByTimestamp != null, - "Cannot find a snapshot after: " + scanContext.startSnapshotTimestamp()); - return Optional.of(matchedSnapshotByTimestamp); - default: - throw new IllegalArgumentException( - "Unknown starting strategy: " + scanContext.streamingStartingStrategy()); - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java deleted file mode 100644 index ef21dad0199d..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java +++ /dev/null @@ -1,96 +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.iceberg.flink.source.enumerator; - -import java.util.Arrays; -import javax.annotation.concurrent.ThreadSafe; -import org.apache.flink.annotation.VisibleForTesting; - -/** - * This enumeration history is used for split discovery throttling. It tracks the discovered split - * count per every non-empty enumeration. - */ -@ThreadSafe -class EnumerationHistory { - - private final int[] history; - // int (2B) should be enough without overflow for enumeration history - private int count; - - EnumerationHistory(int maxHistorySize) { - this.history = new int[maxHistorySize]; - } - - synchronized void restore(int[] restoredHistory) { - int startingOffset = 0; - int restoreSize = restoredHistory.length; - - if (restoredHistory.length > history.length) { - // keep the newest history - startingOffset = restoredHistory.length - history.length; - // only restore the latest history up to maxHistorySize - restoreSize = history.length; - } - - System.arraycopy(restoredHistory, startingOffset, history, 0, restoreSize); - count = restoreSize; - } - - synchronized int[] snapshot() { - int len = history.length; - if (count > len) { - int[] copy = new int[len]; - // this is like a circular buffer - int indexForOldest = count % len; - System.arraycopy(history, indexForOldest, copy, 0, len - indexForOldest); - System.arraycopy(history, 0, copy, len - indexForOldest, indexForOldest); - return copy; - } else { - return Arrays.copyOfRange(history, 0, count); - } - } - - /** Add the split count from the last enumeration result. */ - synchronized void add(int splitCount) { - int pos = count % history.length; - history[pos] = splitCount; - count += 1; - } - - @VisibleForTesting - synchronized boolean hasFullHistory() { - return count >= history.length; - } - - /** @return true if split discovery should pause because assigner has too many splits already. */ - synchronized boolean shouldPauseSplitDiscovery(int pendingSplitCountFromAssigner) { - if (count < history.length) { - // only check throttling when full history is obtained. - return false; - } else { - // if ScanContext#maxPlanningSnapshotCount() is 10, each split enumeration can - // discovery splits up to 10 snapshots. if maxHistorySize is 3, the max number of - // splits tracked in assigner shouldn't be more than 10 * (3 + 1) snapshots - // worth of splits. +1 because there could be another enumeration when the - // pending splits fall just below the 10 * 3. - int totalSplitCountFromRecentDiscovery = Arrays.stream(history).reduce(0, Integer::sum); - return pendingSplitCountFromAssigner >= totalSplitCountFromRecentDiscovery; - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java deleted file mode 100644 index 96aba296f8cf..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java +++ /dev/null @@ -1,79 +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.iceberg.flink.source.enumerator; - -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -import org.apache.iceberg.relocated.com.google.common.base.Objects; - -class IcebergEnumeratorPosition { - private final Long snapshotId; - // Track snapshot timestamp mainly for info logging - private final Long snapshotTimestampMs; - - static IcebergEnumeratorPosition empty() { - return new IcebergEnumeratorPosition(null, null); - } - - static IcebergEnumeratorPosition of(long snapshotId, Long snapshotTimestampMs) { - return new IcebergEnumeratorPosition(snapshotId, snapshotTimestampMs); - } - - private IcebergEnumeratorPosition(Long snapshotId, Long snapshotTimestampMs) { - this.snapshotId = snapshotId; - this.snapshotTimestampMs = snapshotTimestampMs; - } - - boolean isEmpty() { - return snapshotId == null; - } - - Long snapshotId() { - return snapshotId; - } - - Long snapshotTimestampMs() { - return snapshotTimestampMs; - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("snapshotId", snapshotId) - .add("snapshotTimestampMs", snapshotTimestampMs) - .toString(); - } - - @Override - public int hashCode() { - return Objects.hashCode(snapshotId, snapshotTimestampMs); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - IcebergEnumeratorPosition other = (IcebergEnumeratorPosition) o; - return Objects.equal(snapshotId, other.snapshotId()) - && Objects.equal(snapshotTimestampMs, other.snapshotTimestampMs()); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java deleted file mode 100644 index 1c63807361c5..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java +++ /dev/null @@ -1,90 +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.iceberg.flink.source.enumerator; - -import java.io.IOException; -import org.apache.flink.core.io.SimpleVersionedSerializer; -import org.apache.flink.core.memory.DataInputDeserializer; -import org.apache.flink.core.memory.DataOutputSerializer; - -class IcebergEnumeratorPositionSerializer - implements SimpleVersionedSerializer { - - public static final IcebergEnumeratorPositionSerializer INSTANCE = - new IcebergEnumeratorPositionSerializer(); - - private static final int VERSION = 1; - - private static final ThreadLocal SERIALIZER_CACHE = - ThreadLocal.withInitial(() -> new DataOutputSerializer(128)); - - @Override - public int getVersion() { - return VERSION; - } - - @Override - public byte[] serialize(IcebergEnumeratorPosition position) throws IOException { - return serializeV1(position); - } - - @Override - public IcebergEnumeratorPosition deserialize(int version, byte[] serialized) throws IOException { - switch (version) { - case 1: - return deserializeV1(serialized); - default: - throw new IOException("Unknown version: " + version); - } - } - - private byte[] serializeV1(IcebergEnumeratorPosition position) throws IOException { - DataOutputSerializer out = SERIALIZER_CACHE.get(); - out.writeBoolean(position.snapshotId() != null); - if (position.snapshotId() != null) { - out.writeLong(position.snapshotId()); - } - out.writeBoolean(position.snapshotTimestampMs() != null); - if (position.snapshotTimestampMs() != null) { - out.writeLong(position.snapshotTimestampMs()); - } - byte[] result = out.getCopyOfBuffer(); - out.clear(); - return result; - } - - private IcebergEnumeratorPosition deserializeV1(byte[] serialized) throws IOException { - DataInputDeserializer in = new DataInputDeserializer(serialized); - Long snapshotId = null; - if (in.readBoolean()) { - snapshotId = in.readLong(); - } - - Long snapshotTimestampMs = null; - if (in.readBoolean()) { - snapshotTimestampMs = in.readLong(); - } - - if (snapshotId != null) { - return IcebergEnumeratorPosition.of(snapshotId, snapshotTimestampMs); - } else { - return IcebergEnumeratorPosition.empty(); - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java deleted file mode 100644 index 024d0b101165..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java +++ /dev/null @@ -1,65 +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.iceberg.flink.source.enumerator; - -import java.io.Serializable; -import java.util.Collection; -import javax.annotation.Nullable; -import org.apache.flink.annotation.Internal; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; - -/** Enumerator state for checkpointing */ -@Internal -public class IcebergEnumeratorState implements Serializable { - @Nullable private final IcebergEnumeratorPosition lastEnumeratedPosition; - private final Collection pendingSplits; - private int[] enumerationSplitCountHistory; - - public IcebergEnumeratorState(Collection pendingSplits) { - this(null, pendingSplits); - } - - public IcebergEnumeratorState( - @Nullable IcebergEnumeratorPosition lastEnumeratedPosition, - Collection pendingSplits) { - this(lastEnumeratedPosition, pendingSplits, new int[0]); - } - - public IcebergEnumeratorState( - @Nullable IcebergEnumeratorPosition lastEnumeratedPosition, - Collection pendingSplits, - int[] enumerationSplitCountHistory) { - this.lastEnumeratedPosition = lastEnumeratedPosition; - this.pendingSplits = pendingSplits; - this.enumerationSplitCountHistory = enumerationSplitCountHistory; - } - - @Nullable - public IcebergEnumeratorPosition lastEnumeratedPosition() { - return lastEnumeratedPosition; - } - - public Collection pendingSplits() { - return pendingSplits; - } - - public int[] enumerationSplitCountHistory() { - return enumerationSplitCountHistory; - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java deleted file mode 100644 index 95d6db2cfbc4..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java +++ /dev/null @@ -1,196 +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.iceberg.flink.source.enumerator; - -import java.io.IOException; -import java.util.Collection; -import org.apache.flink.annotation.Internal; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.core.io.SimpleVersionedSerializer; -import org.apache.flink.core.memory.DataInputDeserializer; -import org.apache.flink.core.memory.DataOutputSerializer; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -@Internal -public class IcebergEnumeratorStateSerializer - implements SimpleVersionedSerializer { - - private static final int VERSION = 2; - - private static final ThreadLocal SERIALIZER_CACHE = - ThreadLocal.withInitial(() -> new DataOutputSerializer(1024)); - - private final IcebergEnumeratorPositionSerializer positionSerializer = - IcebergEnumeratorPositionSerializer.INSTANCE; - private final IcebergSourceSplitSerializer splitSerializer; - - public IcebergEnumeratorStateSerializer(boolean caseSensitive) { - this.splitSerializer = new IcebergSourceSplitSerializer(caseSensitive); - } - - @Override - public int getVersion() { - return VERSION; - } - - @Override - public byte[] serialize(IcebergEnumeratorState enumState) throws IOException { - return serializeV2(enumState); - } - - @Override - public IcebergEnumeratorState deserialize(int version, byte[] serialized) throws IOException { - switch (version) { - case 1: - return deserializeV1(serialized); - case 2: - return deserializeV2(serialized); - default: - throw new IOException("Unknown version: " + version); - } - } - - @VisibleForTesting - byte[] serializeV1(IcebergEnumeratorState enumState) throws IOException { - DataOutputSerializer out = SERIALIZER_CACHE.get(); - serializeEnumeratorPosition(out, enumState.lastEnumeratedPosition(), positionSerializer); - serializePendingSplits(out, enumState.pendingSplits(), splitSerializer); - byte[] result = out.getCopyOfBuffer(); - out.clear(); - return result; - } - - @VisibleForTesting - IcebergEnumeratorState deserializeV1(byte[] serialized) throws IOException { - DataInputDeserializer in = new DataInputDeserializer(serialized); - IcebergEnumeratorPosition enumeratorPosition = - deserializeEnumeratorPosition(in, positionSerializer); - Collection pendingSplits = - deserializePendingSplits(in, splitSerializer); - return new IcebergEnumeratorState(enumeratorPosition, pendingSplits); - } - - @VisibleForTesting - byte[] serializeV2(IcebergEnumeratorState enumState) throws IOException { - DataOutputSerializer out = SERIALIZER_CACHE.get(); - serializeEnumeratorPosition(out, enumState.lastEnumeratedPosition(), positionSerializer); - serializePendingSplits(out, enumState.pendingSplits(), splitSerializer); - serializeEnumerationSplitCountHistory(out, enumState.enumerationSplitCountHistory()); - byte[] result = out.getCopyOfBuffer(); - out.clear(); - return result; - } - - @VisibleForTesting - IcebergEnumeratorState deserializeV2(byte[] serialized) throws IOException { - DataInputDeserializer in = new DataInputDeserializer(serialized); - IcebergEnumeratorPosition enumeratorPosition = - deserializeEnumeratorPosition(in, positionSerializer); - Collection pendingSplits = - deserializePendingSplits(in, splitSerializer); - int[] enumerationSplitCountHistory = deserializeEnumerationSplitCountHistory(in); - return new IcebergEnumeratorState( - enumeratorPosition, pendingSplits, enumerationSplitCountHistory); - } - - private static void serializeEnumeratorPosition( - DataOutputSerializer out, - IcebergEnumeratorPosition enumeratorPosition, - IcebergEnumeratorPositionSerializer positionSerializer) - throws IOException { - out.writeBoolean(enumeratorPosition != null); - if (enumeratorPosition != null) { - out.writeInt(positionSerializer.getVersion()); - byte[] positionBytes = positionSerializer.serialize(enumeratorPosition); - out.writeInt(positionBytes.length); - out.write(positionBytes); - } - } - - private static IcebergEnumeratorPosition deserializeEnumeratorPosition( - DataInputDeserializer in, IcebergEnumeratorPositionSerializer positionSerializer) - throws IOException { - IcebergEnumeratorPosition enumeratorPosition = null; - if (in.readBoolean()) { - int version = in.readInt(); - byte[] positionBytes = new byte[in.readInt()]; - in.read(positionBytes); - enumeratorPosition = positionSerializer.deserialize(version, positionBytes); - } - return enumeratorPosition; - } - - private static void serializePendingSplits( - DataOutputSerializer out, - Collection pendingSplits, - IcebergSourceSplitSerializer splitSerializer) - throws IOException { - out.writeInt(splitSerializer.getVersion()); - out.writeInt(pendingSplits.size()); - for (IcebergSourceSplitState splitState : pendingSplits) { - byte[] splitBytes = splitSerializer.serialize(splitState.split()); - out.writeInt(splitBytes.length); - out.write(splitBytes); - out.writeUTF(splitState.status().name()); - } - } - - private static Collection deserializePendingSplits( - DataInputDeserializer in, IcebergSourceSplitSerializer splitSerializer) throws IOException { - int splitSerializerVersion = in.readInt(); - int splitCount = in.readInt(); - Collection pendingSplits = Lists.newArrayListWithCapacity(splitCount); - for (int i = 0; i < splitCount; ++i) { - byte[] splitBytes = new byte[in.readInt()]; - in.read(splitBytes); - IcebergSourceSplit split = splitSerializer.deserialize(splitSerializerVersion, splitBytes); - String statusName = in.readUTF(); - pendingSplits.add( - new IcebergSourceSplitState(split, IcebergSourceSplitStatus.valueOf(statusName))); - } - return pendingSplits; - } - - private static void serializeEnumerationSplitCountHistory( - DataOutputSerializer out, int[] enumerationSplitCountHistory) throws IOException { - out.writeInt(enumerationSplitCountHistory.length); - if (enumerationSplitCountHistory.length > 0) { - for (int enumerationSplitCount : enumerationSplitCountHistory) { - out.writeInt(enumerationSplitCount); - } - } - } - - private static int[] deserializeEnumerationSplitCountHistory(DataInputDeserializer in) - throws IOException { - int historySize = in.readInt(); - int[] history = new int[historySize]; - if (historySize > 0) { - for (int i = 0; i < historySize; ++i) { - history[i] = in.readInt(); - } - } - - return history; - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java deleted file mode 100644 index 4e55ea5d5fd6..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java +++ /dev/null @@ -1,51 +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.iceberg.flink.source.enumerator; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.connector.source.SplitEnumeratorContext; -import org.apache.iceberg.flink.source.assigner.SplitAssigner; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; - -/** One-time split enumeration at the start-up for batch execution */ -@Internal -public class StaticIcebergEnumerator extends AbstractIcebergEnumerator { - private final SplitAssigner assigner; - - public StaticIcebergEnumerator( - SplitEnumeratorContext enumeratorContext, SplitAssigner assigner) { - super(enumeratorContext, assigner); - this.assigner = assigner; - } - - @Override - public void start() { - super.start(); - } - - @Override - protected boolean shouldWaitForMoreSplits() { - return false; - } - - @Override - public IcebergEnumeratorState snapshotState(long checkpointId) { - return new IcebergEnumeratorState(null, assigner.state(), new int[0]); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java deleted file mode 100644 index 7b94c364c976..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java +++ /dev/null @@ -1,171 +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.iceberg.flink.source.reader; - -import java.util.Collections; -import java.util.Set; -import javax.annotation.Nullable; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.flink.connector.file.src.util.Pool; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.flink.source.DataIterator; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** - * {@link RecordsWithSplitIds} is used to pass a batch of records from fetcher to source reader. - * Batching is to improve the efficiency for records handover. - * - *

    {@link RecordsWithSplitIds} interface can encapsulate batches from multiple splits. This is - * the case for Kafka source where fetchers can retrieve records from multiple Kafka partitions at - * the same time. - * - *

    For file-based sources like Iceberg, readers always read one split/file at a time. Hence, we - * will only have a batch of records for one split here. - * - *

    This class uses array to store a batch of records from the same file (with the same - * fileOffset). - */ -class ArrayBatchRecords implements RecordsWithSplitIds> { - @Nullable private String splitId; - @Nullable private final Pool.Recycler recycler; - @Nullable private final T[] records; - private final int numberOfRecords; - private final Set finishedSplits; - private final RecordAndPosition recordAndPosition; - - // point to current read position within the records array - private int position; - - private ArrayBatchRecords( - @Nullable String splitId, - @Nullable Pool.Recycler recycler, - @Nullable T[] records, - int numberOfRecords, - int fileOffset, - long startingRecordOffset, - Set finishedSplits) { - Preconditions.checkArgument(numberOfRecords >= 0, "numberOfRecords can't be negative"); - Preconditions.checkArgument(fileOffset >= 0, "fileOffset can't be negative"); - Preconditions.checkArgument(startingRecordOffset >= 0, "numberOfRecords can't be negative"); - - this.splitId = splitId; - this.recycler = recycler; - this.records = records; - this.numberOfRecords = numberOfRecords; - this.finishedSplits = - Preconditions.checkNotNull(finishedSplits, "finishedSplits can be empty but not null"); - this.recordAndPosition = new RecordAndPosition<>(); - - recordAndPosition.set(null, fileOffset, startingRecordOffset); - this.position = 0; - } - - @Nullable - @Override - public String nextSplit() { - String nextSplit = this.splitId; - // set the splitId to null to indicate no more splits - // this class only contains record for one split - this.splitId = null; - return nextSplit; - } - - @Nullable - @Override - public RecordAndPosition nextRecordFromSplit() { - if (position < numberOfRecords) { - recordAndPosition.record(records[position]); - position++; - return recordAndPosition; - } else { - return null; - } - } - - /** - * This method is called when all records from this batch has been emitted. If recycler is set, it - * should be called to return the records array back to pool. - */ - @Override - public void recycle() { - if (recycler != null) { - recycler.recycle(records); - } - } - - @Override - public Set finishedSplits() { - return finishedSplits; - } - - @VisibleForTesting - T[] records() { - return records; - } - - @VisibleForTesting - int numberOfRecords() { - return numberOfRecords; - } - - /** - * Create a ArrayBatchRecords backed up an array with records from the same file - * - * @param splitId Iceberg source only read from one split a time. We never have multiple records - * from multiple splits. - * @param recycler Because {@link DataIterator} with {@link RowData} returns an iterator of reused - * RowData object, we need to clone RowData eagerly when constructing a batch of records. We - * can use object pool to reuse the RowData array object which can be expensive to create. - * This recycler can be provided to recycle the array object back to pool after read is - * exhausted. If the {@link DataIterator} returns an iterator of non-reused objects, we don't - * need to clone objects. It is cheap to just create the batch array. Hence, we don't need - * object pool and recycler can be set to null. - * @param records an array (maybe reused) holding a batch of records - * @param numberOfRecords actual number of records in the array - * @param fileOffset fileOffset for all records in this batch - * @param startingRecordOffset starting recordOffset - * @param record type - */ - public static ArrayBatchRecords forRecords( - String splitId, - Pool.Recycler recycler, - T[] records, - int numberOfRecords, - int fileOffset, - long startingRecordOffset) { - return new ArrayBatchRecords<>( - splitId, - recycler, - records, - numberOfRecords, - fileOffset, - startingRecordOffset, - Collections.emptySet()); - } - - /** - * Create ab ArrayBatchRecords with only finished split id - * - * @param splitId for the split that is just exhausted - */ - public static ArrayBatchRecords finishedSplit(String splitId) { - return new ArrayBatchRecords<>(null, null, null, 0, 0, 0, Collections.singleton(splitId)); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java deleted file mode 100644 index 306afd1811be..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java +++ /dev/null @@ -1,130 +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.iceberg.flink.source.reader; - -import java.io.IOException; -import java.util.NoSuchElementException; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.flink.connector.base.source.reader.SourceReaderOptions; -import org.apache.flink.connector.file.src.util.Pool; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.source.DataIterator; -import org.apache.iceberg.io.CloseableIterator; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** This implementation stores record batch in array from recyclable pool */ -class ArrayPoolDataIteratorBatcher implements DataIteratorBatcher { - private final int batchSize; - private final int handoverQueueSize; - private final RecordFactory recordFactory; - - private transient Pool pool; - - ArrayPoolDataIteratorBatcher(ReadableConfig config, RecordFactory recordFactory) { - this.batchSize = config.get(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT); - this.handoverQueueSize = config.get(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY); - this.recordFactory = recordFactory; - } - - @Override - public CloseableIterator>> batch( - String splitId, DataIterator inputIterator) { - Preconditions.checkArgument(inputIterator != null, "Input data iterator can't be null"); - // lazily create pool as it is not serializable - if (pool == null) { - this.pool = createPoolOfBatches(handoverQueueSize); - } - return new ArrayPoolBatchIterator(splitId, inputIterator, pool); - } - - private Pool createPoolOfBatches(int numBatches) { - Pool poolOfBatches = new Pool<>(numBatches); - for (int batchId = 0; batchId < numBatches; batchId++) { - T[] batch = recordFactory.createBatch(batchSize); - poolOfBatches.add(batch); - } - - return poolOfBatches; - } - - private class ArrayPoolBatchIterator - implements CloseableIterator>> { - - private final String splitId; - private final DataIterator inputIterator; - private final Pool pool; - - ArrayPoolBatchIterator(String splitId, DataIterator inputIterator, Pool pool) { - this.splitId = splitId; - this.inputIterator = inputIterator; - this.pool = pool; - } - - @Override - public boolean hasNext() { - return inputIterator.hasNext(); - } - - @Override - public RecordsWithSplitIds> next() { - if (!inputIterator.hasNext()) { - throw new NoSuchElementException(); - } - - T[] batch = getCachedEntry(); - int recordCount = 0; - while (inputIterator.hasNext() && recordCount < batchSize) { - // The record produced by inputIterator can be reused like for the RowData case. - // inputIterator.next() can't be called again until the copy is made - // since the record is not consumed immediately. - T nextRecord = inputIterator.next(); - recordFactory.clone(nextRecord, batch, recordCount); - recordCount++; - if (!inputIterator.currentFileHasNext()) { - // break early so that records in the ArrayResultIterator - // have the same fileOffset. - break; - } - } - - return ArrayBatchRecords.forRecords( - splitId, - pool.recycler(), - batch, - recordCount, - inputIterator.fileOffset(), - inputIterator.recordOffset() - recordCount); - } - - @Override - public void close() throws IOException { - inputIterator.close(); - } - - private T[] getCachedEntry() { - try { - return pool.pollEntry(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException("Interrupted while waiting for array pool entry", e); - } - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java deleted file mode 100644 index 66e59633fff2..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java +++ /dev/null @@ -1,102 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.source.reader; - -import java.util.List; -import org.apache.avro.generic.GenericRecord; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.encryption.EncryptionManager; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.flink.source.AvroGenericRecordFileScanTaskReader; -import org.apache.iceberg.flink.source.DataIterator; -import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; -import org.apache.iceberg.flink.source.RowDataToAvroGenericRecordConverter; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** Read Iceberg rows as {@link GenericRecord}. */ -public class AvroGenericRecordReaderFunction extends DataIteratorReaderFunction { - private final String tableName; - private final Schema readSchema; - private final FileIO io; - private final EncryptionManager encryption; - private final RowDataFileScanTaskReader rowDataReader; - - private transient RowDataToAvroGenericRecordConverter converter; - - /** - * Create a reader function without projection and name mapping. Column name is case-insensitive. - */ - public static AvroGenericRecordReaderFunction fromTable(Table table) { - return new AvroGenericRecordReaderFunction( - table.name(), - new Configuration(), - table.schema(), - null, - null, - false, - table.io(), - table.encryption(), - null); - } - - public AvroGenericRecordReaderFunction( - String tableName, - ReadableConfig config, - Schema tableSchema, - Schema projectedSchema, - String nameMapping, - boolean caseSensitive, - FileIO io, - EncryptionManager encryption, - List filters) { - super(new ListDataIteratorBatcher<>(config)); - this.tableName = tableName; - this.readSchema = readSchema(tableSchema, projectedSchema); - this.io = io; - this.encryption = encryption; - this.rowDataReader = - new RowDataFileScanTaskReader(tableSchema, readSchema, nameMapping, caseSensitive, filters); - } - - @Override - protected DataIterator createDataIterator(IcebergSourceSplit split) { - return new DataIterator<>( - new AvroGenericRecordFileScanTaskReader(rowDataReader, lazyConverter()), - split.task(), - io, - encryption); - } - - private RowDataToAvroGenericRecordConverter lazyConverter() { - if (converter == null) { - this.converter = RowDataToAvroGenericRecordConverter.fromIcebergSchema(tableName, readSchema); - } - return converter; - } - - private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { - Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); - return projectedSchema == null ? tableSchema : projectedSchema; - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java deleted file mode 100644 index 4bb6f0a98c4c..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java +++ /dev/null @@ -1,98 +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.iceberg.flink.source.reader; - -import java.io.Serializable; -import java.util.Comparator; -import java.util.concurrent.TimeUnit; -import org.apache.flink.annotation.Internal; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.types.Conversions; -import org.apache.iceberg.types.Type.TypeID; -import org.apache.iceberg.types.Types; - -/** - * {@link SplitWatermarkExtractor} implementation which uses an Iceberg timestamp column statistics - * to get the watermarks for the {@link IcebergSourceSplit}. This watermark is emitted by the {@link - * WatermarkExtractorRecordEmitter} along with the actual records. - */ -@Internal -public class ColumnStatsWatermarkExtractor implements SplitWatermarkExtractor, Serializable { - private final int eventTimeFieldId; - private final String eventTimeFieldName; - private final TimeUnit timeUnit; - - /** - * Creates the extractor. - * - * @param schema The schema of the Table - * @param eventTimeFieldName The column which should be used as an event time - * @param timeUnit Used for converting the long value to epoch milliseconds - */ - public ColumnStatsWatermarkExtractor( - Schema schema, String eventTimeFieldName, TimeUnit timeUnit) { - Types.NestedField field = schema.findField(eventTimeFieldName); - TypeID typeID = field.type().typeId(); - Preconditions.checkArgument( - typeID.equals(TypeID.LONG) || typeID.equals(TypeID.TIMESTAMP), - "Found %s, expected a LONG or TIMESTAMP column for watermark generation.", - typeID); - this.eventTimeFieldId = field.fieldId(); - this.eventTimeFieldName = eventTimeFieldName; - // Use the timeUnit only for Long columns. - this.timeUnit = typeID.equals(TypeID.LONG) ? timeUnit : TimeUnit.MICROSECONDS; - } - - @VisibleForTesting - ColumnStatsWatermarkExtractor(int eventTimeFieldId, String eventTimeFieldName) { - this.eventTimeFieldId = eventTimeFieldId; - this.eventTimeFieldName = eventTimeFieldName; - this.timeUnit = TimeUnit.MICROSECONDS; - } - - /** - * Get the watermark for a split using column statistics. - * - * @param split The split - * @return The watermark - * @throws IllegalArgumentException if there is no statistics for the column - */ - @Override - public long extractWatermark(IcebergSourceSplit split) { - return split.task().files().stream() - .map( - scanTask -> { - Preconditions.checkArgument( - scanTask.file().lowerBounds() != null - && scanTask.file().lowerBounds().get(eventTimeFieldId) != null, - "Missing statistics for column name = %s in file = %s", - eventTimeFieldName, - eventTimeFieldId, - scanTask.file()); - return timeUnit.toMillis( - Conversions.fromByteBuffer( - Types.LongType.get(), scanTask.file().lowerBounds().get(eventTimeFieldId))); - }) - .min(Comparator.comparingLong(l -> l)) - .get(); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java deleted file mode 100644 index c376e359c600..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java +++ /dev/null @@ -1,36 +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.iceberg.flink.source.reader; - -import java.io.Serializable; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; -import org.apache.iceberg.flink.source.DataIterator; -import org.apache.iceberg.io.CloseableIterator; - -/** - * Batcher converts iterator of T into iterator of batched {@code - * RecordsWithSplitIds>}, as FLIP-27's {@link SplitReader#fetch()} returns - * batched records. - */ -@FunctionalInterface -public interface DataIteratorBatcher extends Serializable { - CloseableIterator>> batch( - String splitId, DataIterator inputIterator); -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java deleted file mode 100644 index bbf797ef4aa8..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java +++ /dev/null @@ -1,43 +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.iceberg.flink.source.reader; - -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.iceberg.flink.source.DataIterator; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.io.CloseableIterator; - -/** A {@link ReaderFunction} implementation that uses {@link DataIterator}. */ -public abstract class DataIteratorReaderFunction implements ReaderFunction { - private final DataIteratorBatcher batcher; - - public DataIteratorReaderFunction(DataIteratorBatcher batcher) { - this.batcher = batcher; - } - - protected abstract DataIterator createDataIterator(IcebergSourceSplit split); - - @Override - public CloseableIterator>> apply( - IcebergSourceSplit split) { - DataIterator inputIterator = createDataIterator(split); - inputIterator.seek(split.fileOffset(), split.recordOffset()); - return batcher.batch(split.splitId(), inputIterator); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java deleted file mode 100644 index f143b8d2df2e..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java +++ /dev/null @@ -1,77 +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.iceberg.flink.source.reader; - -import java.util.Collection; -import java.util.Collections; -import java.util.Map; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.connector.source.SourceReaderContext; -import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.SerializableComparator; -import org.apache.iceberg.flink.source.split.SplitRequestEvent; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -@Internal -public class IcebergSourceReader - extends SingleThreadMultiplexSourceReaderBase< - RecordAndPosition, T, IcebergSourceSplit, IcebergSourceSplit> { - - public IcebergSourceReader( - SerializableRecordEmitter emitter, - IcebergSourceReaderMetrics metrics, - ReaderFunction readerFunction, - SerializableComparator splitComparator, - SourceReaderContext context) { - super( - () -> new IcebergSourceSplitReader<>(metrics, readerFunction, splitComparator, context), - emitter, - context.getConfiguration(), - context); - } - - @Override - public void start() { - // We request a split only if we did not get splits during the checkpoint restore. - // Otherwise, reader restarts will keep requesting more and more splits. - if (getNumberOfCurrentlyAssignedSplits() == 0) { - requestSplit(Collections.emptyList()); - } - } - - @Override - protected void onSplitFinished(Map finishedSplitIds) { - requestSplit(Lists.newArrayList(finishedSplitIds.keySet())); - } - - @Override - protected IcebergSourceSplit initializedState(IcebergSourceSplit split) { - return split; - } - - @Override - protected IcebergSourceSplit toSplitType(String splitId, IcebergSourceSplit splitState) { - return splitState; - } - - private void requestSplit(Collection finishedSplitIds) { - context.sendSourceEventToCoordinator(new SplitRequestEvent(finishedSplitIds)); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java deleted file mode 100644 index 2a3e1dd86b95..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java +++ /dev/null @@ -1,61 +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.iceberg.flink.source.reader; - -import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.MetricGroup; - -public class IcebergSourceReaderMetrics { - private final Counter assignedSplits; - private final Counter assignedBytes; - private final Counter finishedSplits; - private final Counter finishedBytes; - private final Counter splitReaderFetchCalls; - - public IcebergSourceReaderMetrics(MetricGroup metrics, String fullTableName) { - MetricGroup readerMetrics = - metrics.addGroup("IcebergSourceReader").addGroup("table", fullTableName); - - this.assignedSplits = readerMetrics.counter("assignedSplits"); - this.assignedBytes = readerMetrics.counter("assignedBytes"); - this.finishedSplits = readerMetrics.counter("finishedSplits"); - this.finishedBytes = readerMetrics.counter("finishedBytes"); - this.splitReaderFetchCalls = readerMetrics.counter("splitReaderFetchCalls"); - } - - public void incrementAssignedSplits(long count) { - assignedSplits.inc(count); - } - - public void incrementAssignedBytes(long count) { - assignedBytes.inc(count); - } - - public void incrementFinishedSplits(long count) { - finishedSplits.inc(count); - } - - public void incrementFinishedBytes(long count) { - finishedBytes.inc(count); - } - - public void incrementSplitReaderFetchCalls(long count) { - splitReaderFetchCalls.inc(count); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java deleted file mode 100644 index 4e270dfa3d13..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java +++ /dev/null @@ -1,147 +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.iceberg.flink.source.reader; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.ArrayDeque; -import java.util.Collections; -import java.util.List; -import java.util.Queue; -import org.apache.flink.api.connector.source.SourceReaderContext; -import org.apache.flink.connector.base.source.reader.RecordsBySplits; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; -import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition; -import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.SerializableComparator; -import org.apache.iceberg.io.CloseableIterator; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -class IcebergSourceSplitReader implements SplitReader, IcebergSourceSplit> { - private static final Logger LOG = LoggerFactory.getLogger(IcebergSourceSplitReader.class); - - private final IcebergSourceReaderMetrics metrics; - private final ReaderFunction openSplitFunction; - private final SerializableComparator splitComparator; - private final int indexOfSubtask; - private final Queue splits; - - private CloseableIterator>> currentReader; - private IcebergSourceSplit currentSplit; - private String currentSplitId; - - IcebergSourceSplitReader( - IcebergSourceReaderMetrics metrics, - ReaderFunction openSplitFunction, - SerializableComparator splitComparator, - SourceReaderContext context) { - this.metrics = metrics; - this.openSplitFunction = openSplitFunction; - this.splitComparator = splitComparator; - this.indexOfSubtask = context.getIndexOfSubtask(); - this.splits = new ArrayDeque<>(); - } - - @Override - public RecordsWithSplitIds> fetch() throws IOException { - metrics.incrementSplitReaderFetchCalls(1); - if (currentReader == null) { - IcebergSourceSplit nextSplit = splits.poll(); - if (nextSplit != null) { - currentSplit = nextSplit; - currentSplitId = nextSplit.splitId(); - currentReader = openSplitFunction.apply(currentSplit); - } else { - // return an empty result, which will lead to split fetch to be idle. - // SplitFetcherManager will then close idle fetcher. - return new RecordsBySplits(Collections.emptyMap(), Collections.emptySet()); - } - } - - if (currentReader.hasNext()) { - // Because Iterator#next() doesn't support checked exception, - // we need to wrap and unwrap the checked IOException with UncheckedIOException - try { - return currentReader.next(); - } catch (UncheckedIOException e) { - throw e.getCause(); - } - } else { - return finishSplit(); - } - } - - @Override - public void handleSplitsChanges(SplitsChange splitsChange) { - if (!(splitsChange instanceof SplitsAddition)) { - throw new UnsupportedOperationException( - String.format("Unsupported split change: %s", splitsChange.getClass())); - } - - if (splitComparator != null) { - List newSplits = Lists.newArrayList(splitsChange.splits()); - newSplits.sort(splitComparator); - LOG.info("Add {} splits to reader: {}", newSplits.size(), newSplits); - splits.addAll(newSplits); - } else { - LOG.info("Add {} splits to reader", splitsChange.splits().size()); - splits.addAll(splitsChange.splits()); - } - metrics.incrementAssignedSplits(splitsChange.splits().size()); - metrics.incrementAssignedBytes(calculateBytes(splitsChange)); - } - - @Override - public void wakeUp() {} - - @Override - public void close() throws Exception { - currentSplitId = null; - if (currentReader != null) { - currentReader.close(); - } - } - - private long calculateBytes(IcebergSourceSplit split) { - return split.task().files().stream().map(FileScanTask::length).reduce(0L, Long::sum); - } - - private long calculateBytes(SplitsChange splitsChanges) { - return splitsChanges.splits().stream().map(this::calculateBytes).reduce(0L, Long::sum); - } - - private ArrayBatchRecords finishSplit() throws IOException { - if (currentReader != null) { - currentReader.close(); - currentReader = null; - } - - ArrayBatchRecords finishRecords = ArrayBatchRecords.finishedSplit(currentSplitId); - LOG.info("Split reader {} finished split: {}", indexOfSubtask, currentSplitId); - metrics.incrementFinishedSplits(1); - metrics.incrementFinishedBytes(calculateBytes(currentSplit)); - currentSplitId = null; - return finishRecords; - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java deleted file mode 100644 index 1acb3df76102..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java +++ /dev/null @@ -1,85 +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.iceberg.flink.source.reader; - -import java.util.Collections; -import java.util.List; -import java.util.Set; -import javax.annotation.Nullable; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -class ListBatchRecords implements RecordsWithSplitIds> { - private String splitId; - private final List records; - private final Set finishedSplits; - private final RecordAndPosition recordAndPosition; - - // point to current read position within the records list - private int position; - - ListBatchRecords( - String splitId, - List records, - int fileOffset, - long startingRecordOffset, - Set finishedSplits) { - this.splitId = splitId; - this.records = records; - this.finishedSplits = - Preconditions.checkNotNull(finishedSplits, "finishedSplits can be empty but not null"); - this.recordAndPosition = new RecordAndPosition<>(); - this.recordAndPosition.set(null, fileOffset, startingRecordOffset); - - this.position = 0; - } - - @Nullable - @Override - public String nextSplit() { - String nextSplit = this.splitId; - // set the splitId to null to indicate no more splits - // this class only contains record for one split - this.splitId = null; - return nextSplit; - } - - @Nullable - @Override - public RecordAndPosition nextRecordFromSplit() { - if (position < records.size()) { - recordAndPosition.record(records.get(position)); - position++; - return recordAndPosition; - } else { - return null; - } - } - - @Override - public Set finishedSplits() { - return finishedSplits; - } - - public static ListBatchRecords forRecords( - String splitId, List records, int fileOffset, long startingRecordOffset) { - return new ListBatchRecords<>( - splitId, records, fileOffset, startingRecordOffset, Collections.emptySet()); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java deleted file mode 100644 index 365416239d37..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java +++ /dev/null @@ -1,94 +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.iceberg.flink.source.reader; - -import java.io.IOException; -import java.util.List; -import java.util.NoSuchElementException; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.source.DataIterator; -import org.apache.iceberg.io.CloseableIterator; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -/** - * FlinkRecordReaderFunction essentially cloned objects already. So there is no need to use array - * pool to clone objects. Simply create a new ArrayList for each batch. - */ -class ListDataIteratorBatcher implements DataIteratorBatcher { - - private final int batchSize; - - ListDataIteratorBatcher(ReadableConfig config) { - this.batchSize = config.get(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT); - } - - @Override - public CloseableIterator>> batch( - String splitId, DataIterator dataIterator) { - return new ListBatchIterator(splitId, dataIterator); - } - - private class ListBatchIterator - implements CloseableIterator>> { - - private final String splitId; - private final DataIterator inputIterator; - - ListBatchIterator(String splitId, DataIterator inputIterator) { - this.splitId = splitId; - this.inputIterator = inputIterator; - } - - @Override - public boolean hasNext() { - return inputIterator.hasNext(); - } - - @Override - public RecordsWithSplitIds> next() { - if (!inputIterator.hasNext()) { - throw new NoSuchElementException(); - } - - final List batch = Lists.newArrayListWithCapacity(batchSize); - int recordCount = 0; - while (inputIterator.hasNext() && recordCount < batchSize) { - T nextRecord = inputIterator.next(); - batch.add(nextRecord); - recordCount++; - if (!inputIterator.currentFileHasNext()) { - // break early so that records have the same fileOffset. - break; - } - } - - return ListBatchRecords.forRecords( - splitId, batch, inputIterator.fileOffset(), inputIterator.recordOffset() - recordCount); - } - - @Override - public void close() throws IOException { - if (inputIterator != null) { - inputIterator.close(); - } - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java deleted file mode 100644 index fb4466913b90..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java +++ /dev/null @@ -1,65 +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.iceberg.flink.source.reader; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.encryption.EncryptionManager; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.source.DataIterator; -import org.apache.iceberg.flink.source.DataTaskReader; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** Reading metadata tables (like snapshots, manifests, etc.) */ -@Internal -public class MetaDataReaderFunction extends DataIteratorReaderFunction { - private final Schema readSchema; - private final FileIO io; - private final EncryptionManager encryption; - - public MetaDataReaderFunction( - ReadableConfig config, - Schema tableSchema, - Schema projectedSchema, - FileIO io, - EncryptionManager encryption) { - super( - new ArrayPoolDataIteratorBatcher<>( - config, - new RowDataRecordFactory( - FlinkSchemaUtil.convert(readSchema(tableSchema, projectedSchema))))); - this.readSchema = readSchema(tableSchema, projectedSchema); - this.io = io; - this.encryption = encryption; - } - - @Override - public DataIterator createDataIterator(IcebergSourceSplit split) { - return new DataIterator<>(new DataTaskReader(readSchema), split.task(), io, encryption); - } - - private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { - Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); - return projectedSchema == null ? tableSchema : projectedSchema; - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java deleted file mode 100644 index 1ea91f10b4e7..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java +++ /dev/null @@ -1,31 +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.iceberg.flink.source.reader; - -import java.io.Serializable; -import java.util.function.Function; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.io.CloseableIterator; - -@FunctionalInterface -public interface ReaderFunction - extends Serializable, - Function< - IcebergSourceSplit, CloseableIterator>>> {} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java deleted file mode 100644 index 6ac92592b6aa..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java +++ /dev/null @@ -1,78 +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.iceberg.flink.source.reader; - -import org.apache.flink.annotation.Internal; - -/** - * A record along with the reader position to be stored in the checkpoint. - * - *

    The position defines the point in the reader AFTER the record. Record processing and updating - * checkpointed state happens atomically. The position points to where the reader should resume - * after this record is processed. - * - *

    This mutable object is useful in cases where only one instance of a {@code RecordAndPosition} - * is needed at a time. Then the same instance of RecordAndPosition can be reused. - */ -@Internal -public class RecordAndPosition { - private T record; - private int fileOffset; - private long recordOffset; - - public RecordAndPosition(T record, int fileOffset, long recordOffset) { - this.record = record; - this.fileOffset = fileOffset; - this.recordOffset = recordOffset; - } - - public RecordAndPosition() {} - - // ------------------------------------------------------------------------ - - public T record() { - return record; - } - - public int fileOffset() { - return fileOffset; - } - - public long recordOffset() { - return recordOffset; - } - - /** Updates the record and position in this object. */ - public void set(T newRecord, int newFileOffset, long newRecordOffset) { - this.record = newRecord; - this.fileOffset = newFileOffset; - this.recordOffset = newRecordOffset; - } - - /** Sets the next record of a sequence. This increments the {@code recordOffset} by one. */ - public void record(T nextRecord) { - this.record = nextRecord; - this.recordOffset++; - } - - @Override - public String toString() { - return String.format("%s @ %d + %d", record, fileOffset, recordOffset); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java deleted file mode 100644 index ef92e2e6b81f..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java +++ /dev/null @@ -1,34 +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.iceberg.flink.source.reader; - -import java.io.Serializable; - -/** - * In FLIP-27 source, SplitReader#fetch() returns a batch of records. Since DataIterator for RowData - * returns an iterator of reused RowData objects, RecordFactory is needed to (1) create object array - * that is recyclable via pool. (2) clone RowData element from DataIterator to the batch array. - */ -interface RecordFactory extends Serializable { - /** Create a batch of records */ - T[] createBatch(int batchSize); - - /** Clone record into the specified position of the batch array */ - void clone(T from, T[] batch, int position); -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java deleted file mode 100644 index 5d0a00954e7a..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java +++ /dev/null @@ -1,79 +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.iceberg.flink.source.reader; - -import java.util.List; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.encryption.EncryptionManager; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.source.DataIterator; -import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -public class RowDataReaderFunction extends DataIteratorReaderFunction { - private final Schema tableSchema; - private final Schema readSchema; - private final String nameMapping; - private final boolean caseSensitive; - private final FileIO io; - private final EncryptionManager encryption; - private final List filters; - - public RowDataReaderFunction( - ReadableConfig config, - Schema tableSchema, - Schema projectedSchema, - String nameMapping, - boolean caseSensitive, - FileIO io, - EncryptionManager encryption, - List filters) { - super( - new ArrayPoolDataIteratorBatcher<>( - config, - new RowDataRecordFactory( - FlinkSchemaUtil.convert(readSchema(tableSchema, projectedSchema))))); - this.tableSchema = tableSchema; - this.readSchema = readSchema(tableSchema, projectedSchema); - this.nameMapping = nameMapping; - this.caseSensitive = caseSensitive; - this.io = io; - this.encryption = encryption; - this.filters = filters; - } - - @Override - public DataIterator createDataIterator(IcebergSourceSplit split) { - return new DataIterator<>( - new RowDataFileScanTaskReader(tableSchema, readSchema, nameMapping, caseSensitive, filters), - split.task(), - io, - encryption); - } - - private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { - Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); - return projectedSchema == null ? tableSchema : projectedSchema; - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java deleted file mode 100644 index 1e265b2663ce..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java +++ /dev/null @@ -1,61 +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.iceberg.flink.source.reader; - -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.runtime.typeutils.InternalSerializers; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.flink.data.RowDataUtil; - -class RowDataRecordFactory implements RecordFactory { - private final RowType rowType; - private final TypeSerializer[] fieldSerializers; - - RowDataRecordFactory(RowType rowType) { - this.rowType = rowType; - this.fieldSerializers = createFieldSerializers(rowType); - } - - static TypeSerializer[] createFieldSerializers(RowType rowType) { - return rowType.getChildren().stream() - .map(InternalSerializers::create) - .toArray(TypeSerializer[]::new); - } - - @Override - public RowData[] createBatch(int batchSize) { - RowData[] arr = new RowData[batchSize]; - for (int i = 0; i < batchSize; ++i) { - arr[i] = new GenericRowData(rowType.getFieldCount()); - } - return arr; - } - - @Override - public void clone(RowData from, RowData[] batch, int position) { - // Set the return value from RowDataUtil.clone back to the array. - // Clone method returns same clone target object (reused) if it is a GenericRowData. - // Clone method will allocate a new GenericRowData object - // if the target object is NOT a GenericRowData. - // So we should always set the clone return value back to the array. - batch[position] = RowDataUtil.clone(from, batch[position], rowType, fieldSerializers); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java deleted file mode 100644 index a6e2c1dae243..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java +++ /dev/null @@ -1,40 +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.iceberg.flink.source.reader; - -import java.io.Serializable; -import org.apache.flink.annotation.Internal; -import org.apache.flink.connector.base.source.reader.RecordEmitter; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; - -@Internal -@FunctionalInterface -public interface SerializableRecordEmitter - extends RecordEmitter, T, IcebergSourceSplit>, Serializable { - static SerializableRecordEmitter defaultEmitter() { - return (element, output, split) -> { - output.collect(element.record()); - split.updatePosition(element.fileOffset(), element.recordOffset()); - }; - } - - static SerializableRecordEmitter emitterWithWatermark(SplitWatermarkExtractor extractor) { - return new WatermarkExtractorRecordEmitter<>(extractor); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java deleted file mode 100644 index d1c50ac8ca52..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java +++ /dev/null @@ -1,28 +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.iceberg.flink.source.reader; - -import java.io.Serializable; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; - -/** The interface used to extract watermarks from splits. */ -public interface SplitWatermarkExtractor extends Serializable { - /** Get the watermark for a split. */ - long extractWatermark(IcebergSourceSplit split); -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java deleted file mode 100644 index 02ef57d344b1..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java +++ /dev/null @@ -1,67 +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.iceberg.flink.source.reader; - -import org.apache.flink.api.common.eventtime.Watermark; -import org.apache.flink.api.connector.source.SourceOutput; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Emitter which emits the watermarks, records and updates the split position. - * - *

    The Emitter emits watermarks at the beginning of every split provided by the {@link - * SplitWatermarkExtractor}. - */ -class WatermarkExtractorRecordEmitter implements SerializableRecordEmitter { - private static final Logger LOG = LoggerFactory.getLogger(WatermarkExtractorRecordEmitter.class); - private final SplitWatermarkExtractor timeExtractor; - private String lastSplitId = null; - private long watermark; - - WatermarkExtractorRecordEmitter(SplitWatermarkExtractor timeExtractor) { - this.timeExtractor = timeExtractor; - } - - @Override - public void emitRecord( - RecordAndPosition element, SourceOutput output, IcebergSourceSplit split) { - if (!split.splitId().equals(lastSplitId)) { - long newWatermark = timeExtractor.extractWatermark(split); - if (newWatermark < watermark) { - LOG.info( - "Received a new split with lower watermark. Previous watermark = {}, current watermark = {}, previous split = {}, current split = {}", - watermark, - newWatermark, - lastSplitId, - split.splitId()); - } else { - watermark = newWatermark; - output.emitWatermark(new Watermark(watermark)); - LOG.debug("Watermark = {} emitted based on split = {}", watermark, lastSplitId); - } - - lastSplitId = split.splitId(); - } - - output.collect(element.record()); - split.updatePosition(element.fileOffset(), element.recordOffset()); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java deleted file mode 100644 index e4bfbf1452e2..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java +++ /dev/null @@ -1,177 +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.iceberg.flink.source.split; - -import java.io.IOException; -import java.io.Serializable; -import java.util.Collection; -import java.util.List; -import java.util.stream.Collectors; -import javax.annotation.Nullable; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.connector.source.SourceSplit; -import org.apache.flink.core.memory.DataInputDeserializer; -import org.apache.flink.core.memory.DataOutputSerializer; -import org.apache.flink.util.InstantiationUtil; -import org.apache.iceberg.BaseCombinedScanTask; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.FileScanTaskParser; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -@Internal -public class IcebergSourceSplit implements SourceSplit, Serializable { - private static final long serialVersionUID = 1L; - private static final ThreadLocal SERIALIZER_CACHE = - ThreadLocal.withInitial(() -> new DataOutputSerializer(1024)); - - private final CombinedScanTask task; - - private int fileOffset; - private long recordOffset; - - // The splits are frequently serialized into checkpoints. - // Caching the byte representation makes repeated serialization cheap. - @Nullable private transient byte[] serializedBytesCache; - - private IcebergSourceSplit(CombinedScanTask task, int fileOffset, long recordOffset) { - this.task = task; - this.fileOffset = fileOffset; - this.recordOffset = recordOffset; - } - - public static IcebergSourceSplit fromCombinedScanTask(CombinedScanTask combinedScanTask) { - return fromCombinedScanTask(combinedScanTask, 0, 0L); - } - - public static IcebergSourceSplit fromCombinedScanTask( - CombinedScanTask combinedScanTask, int fileOffset, long recordOffset) { - return new IcebergSourceSplit(combinedScanTask, fileOffset, recordOffset); - } - - public CombinedScanTask task() { - return task; - } - - public int fileOffset() { - return fileOffset; - } - - public long recordOffset() { - return recordOffset; - } - - @Override - public String splitId() { - return MoreObjects.toStringHelper(this).add("files", toString(task.files())).toString(); - } - - public void updatePosition(int newFileOffset, long newRecordOffset) { - // invalidate the cache after position change - serializedBytesCache = null; - fileOffset = newFileOffset; - recordOffset = newRecordOffset; - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("files", toString(task.files())) - .add("fileOffset", fileOffset) - .add("recordOffset", recordOffset) - .toString(); - } - - private String toString(Collection files) { - return Iterables.toString( - files.stream() - .map( - fileScanTask -> - MoreObjects.toStringHelper(fileScanTask) - .add("file", fileScanTask.file().path().toString()) - .add("start", fileScanTask.start()) - .add("length", fileScanTask.length()) - .toString()) - .collect(Collectors.toList())); - } - - byte[] serializeV1() throws IOException { - if (serializedBytesCache == null) { - serializedBytesCache = InstantiationUtil.serializeObject(this); - } - - return serializedBytesCache; - } - - static IcebergSourceSplit deserializeV1(byte[] serialized) throws IOException { - try { - return InstantiationUtil.deserializeObject( - serialized, IcebergSourceSplit.class.getClassLoader()); - } catch (ClassNotFoundException e) { - throw new RuntimeException("Failed to deserialize the split.", e); - } - } - - byte[] serializeV2() throws IOException { - if (serializedBytesCache == null) { - DataOutputSerializer out = SERIALIZER_CACHE.get(); - Collection fileScanTasks = task.tasks(); - Preconditions.checkArgument( - fileOffset >= 0 && fileOffset < fileScanTasks.size(), - "Invalid file offset: %s. Should be within the range of [0, %s)", - fileOffset, - fileScanTasks.size()); - - out.writeInt(fileOffset); - out.writeLong(recordOffset); - out.writeInt(fileScanTasks.size()); - - for (FileScanTask fileScanTask : fileScanTasks) { - String taskJson = FileScanTaskParser.toJson(fileScanTask); - out.writeUTF(taskJson); - } - - serializedBytesCache = out.getCopyOfBuffer(); - out.clear(); - } - - return serializedBytesCache; - } - - static IcebergSourceSplit deserializeV2(byte[] serialized, boolean caseSensitive) - throws IOException { - DataInputDeserializer in = new DataInputDeserializer(serialized); - int fileOffset = in.readInt(); - long recordOffset = in.readLong(); - int taskCount = in.readInt(); - - List tasks = Lists.newArrayListWithCapacity(taskCount); - for (int i = 0; i < taskCount; ++i) { - String taskJson = in.readUTF(); - FileScanTask task = FileScanTaskParser.fromJson(taskJson, caseSensitive); - tasks.add(task); - } - - CombinedScanTask combinedScanTask = new BaseCombinedScanTask(tasks); - return IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, fileOffset, recordOffset); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java deleted file mode 100644 index 8c089819e731..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java +++ /dev/null @@ -1,60 +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.iceberg.flink.source.split; - -import java.io.IOException; -import org.apache.flink.annotation.Internal; -import org.apache.flink.core.io.SimpleVersionedSerializer; - -@Internal -public class IcebergSourceSplitSerializer implements SimpleVersionedSerializer { - private static final int VERSION = 2; - - private final boolean caseSensitive; - - public IcebergSourceSplitSerializer(boolean caseSensitive) { - this.caseSensitive = caseSensitive; - } - - @Override - public int getVersion() { - return VERSION; - } - - @Override - public byte[] serialize(IcebergSourceSplit split) throws IOException { - return split.serializeV2(); - } - - @Override - public IcebergSourceSplit deserialize(int version, byte[] serialized) throws IOException { - switch (version) { - case 1: - return IcebergSourceSplit.deserializeV1(serialized); - case 2: - return IcebergSourceSplit.deserializeV2(serialized, caseSensitive); - default: - throw new IOException( - String.format( - "Failed to deserialize IcebergSourceSplit. " - + "Encountered unsupported version: %d. Supported version are [1]", - version)); - } - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java deleted file mode 100644 index d9061e049e00..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java +++ /dev/null @@ -1,37 +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.iceberg.flink.source.split; - -public class IcebergSourceSplitState { - private final IcebergSourceSplit split; - private final IcebergSourceSplitStatus status; - - public IcebergSourceSplitState(IcebergSourceSplit split, IcebergSourceSplitStatus status) { - this.split = split; - this.status = status; - } - - public IcebergSourceSplit split() { - return split; - } - - public IcebergSourceSplitStatus status() { - return status; - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java deleted file mode 100644 index d4a84a165e1a..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java +++ /dev/null @@ -1,25 +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.iceberg.flink.source.split; - -public enum IcebergSourceSplitStatus { - UNASSIGNED, - ASSIGNED, - COMPLETED -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java deleted file mode 100644 index 319648ca275c..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java +++ /dev/null @@ -1,24 +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.iceberg.flink.source.split; - -import java.io.Serializable; -import java.util.Comparator; - -public interface SerializableComparator extends Comparator, Serializable {} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java deleted file mode 100644 index 56ee92014d12..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java +++ /dev/null @@ -1,76 +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.iceberg.flink.source.split; - -import org.apache.iceberg.flink.source.reader.SplitWatermarkExtractor; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** - * Provides implementations of {@link org.apache.iceberg.flink.source.split.SerializableComparator} - * which could be used for ordering splits. These are used by the {@link - * org.apache.iceberg.flink.source.assigner.OrderedSplitAssignerFactory} and the {@link - * org.apache.iceberg.flink.source.reader.IcebergSourceReader} - */ -public class SplitComparators { - private SplitComparators() {} - - /** Comparator which orders the splits based on the file sequence number of the data files */ - public static SerializableComparator fileSequenceNumber() { - return (IcebergSourceSplit o1, IcebergSourceSplit o2) -> { - Preconditions.checkArgument( - o1.task().files().size() == 1 && o2.task().files().size() == 1, - "Could not compare combined task. Please use 'split-open-file-cost' to prevent combining multiple files to a split"); - - Long seq1 = o1.task().files().iterator().next().file().fileSequenceNumber(); - Long seq2 = o2.task().files().iterator().next().file().fileSequenceNumber(); - - Preconditions.checkNotNull( - seq1, - "Invalid file sequence number: null. Doesn't support splits written with V1 format: %s", - o1); - Preconditions.checkNotNull( - seq2, - "Invalid file sequence number: null. Doesn't support splits written with V1 format: %s", - o2); - - int temp = Long.compare(seq1, seq2); - if (temp != 0) { - return temp; - } else { - return o1.splitId().compareTo(o2.splitId()); - } - }; - } - - /** Comparator which orders the splits based on watermark of the splits */ - public static SerializableComparator watermark( - SplitWatermarkExtractor watermarkExtractor) { - return (IcebergSourceSplit o1, IcebergSourceSplit o2) -> { - long watermark1 = watermarkExtractor.extractWatermark(o1); - long watermark2 = watermarkExtractor.extractWatermark(o2); - - int temp = Long.compare(watermark1, watermark2); - if (temp != 0) { - return temp; - } else { - return o1.splitId().compareTo(o2.splitId()); - } - }; - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java deleted file mode 100644 index eabd757aa638..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java +++ /dev/null @@ -1,54 +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.iceberg.flink.source.split; - -import java.util.Collection; -import java.util.Collections; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.connector.source.SourceEvent; - -/** We can remove this class once FLINK-21364 is resolved. */ -@Internal -public class SplitRequestEvent implements SourceEvent { - private static final long serialVersionUID = 1L; - - private final Collection finishedSplitIds; - private final String requesterHostname; - - public SplitRequestEvent() { - this(Collections.emptyList()); - } - - public SplitRequestEvent(Collection finishedSplitIds) { - this(finishedSplitIds, null); - } - - public SplitRequestEvent(Collection finishedSplitIds, String requesterHostname) { - this.finishedSplitIds = finishedSplitIds; - this.requesterHostname = requesterHostname; - } - - public Collection finishedSplitIds() { - return finishedSplitIds; - } - - public String requesterHostname() { - return requesterHostname; - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java deleted file mode 100644 index 2c5c587f4ebf..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java +++ /dev/null @@ -1,42 +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.iceberg.flink.util; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.table.api.TableColumn; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; -import org.apache.flink.table.types.logical.RowType; - -/** - * This is a small util class that try to hide calls to Flink Internal or PublicEvolve interfaces as - * Flink can change those APIs during minor version release. - */ -public class FlinkCompatibilityUtil { - - private FlinkCompatibilityUtil() {} - - public static TypeInformation toTypeInfo(RowType rowType) { - return InternalTypeInfo.of(rowType); - } - - public static boolean isPhysicalColumn(TableColumn column) { - return column.isPhysical(); - } -} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java deleted file mode 100644 index 00d74d8d345c..000000000000 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java +++ /dev/null @@ -1,33 +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.iceberg.flink.util; - -import org.apache.flink.streaming.api.datastream.DataStream; - -public class FlinkPackage { - /** Choose {@link DataStream} class because it is one of the core Flink API. */ - private static final String VERSION = DataStream.class.getPackage().getImplementationVersion(); - - private FlinkPackage() {} - - /** Returns Flink version string like x.y.z */ - public static String version() { - return VERSION; - } -} diff --git a/flink/v1.15/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink/v1.15/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory deleted file mode 100644 index 29a9955a7e20..000000000000 --- a/flink/v1.15/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ /dev/null @@ -1,16 +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. - -org.apache.iceberg.flink.FlinkDynamicTableFactory diff --git a/flink/v1.15/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink/v1.15/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory deleted file mode 100644 index 2b6bfa3cd579..000000000000 --- a/flink/v1.15/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory +++ /dev/null @@ -1,16 +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. - -org.apache.iceberg.flink.FlinkCatalogFactory diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java deleted file mode 100644 index 47319ec9bc60..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java +++ /dev/null @@ -1,90 +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.iceberg.flink; - -import org.junit.Test; - -public abstract class AvroGenericRecordConverterBase { - protected abstract void testConverter(DataGenerator dataGenerator) throws Exception; - - @Test - public void testPrimitiveTypes() throws Exception { - testConverter(new DataGenerators.Primitives()); - } - - @Test - public void testStructOfPrimitive() throws Exception { - testConverter(new DataGenerators.StructOfPrimitive()); - } - - @Test - public void testStructOfArray() throws Exception { - testConverter(new DataGenerators.StructOfArray()); - } - - @Test - public void testStructOfMap() throws Exception { - testConverter(new DataGenerators.StructOfMap()); - } - - @Test - public void testStructOfStruct() throws Exception { - testConverter(new DataGenerators.StructOfStruct()); - } - - @Test - public void testArrayOfPrimitive() throws Exception { - testConverter(new DataGenerators.ArrayOfPrimitive()); - } - - @Test - public void testArrayOfArray() throws Exception { - testConverter(new DataGenerators.ArrayOfArray()); - } - - @Test - public void testArrayOfMap() throws Exception { - testConverter(new DataGenerators.ArrayOfMap()); - } - - @Test - public void testArrayOfStruct() throws Exception { - testConverter(new DataGenerators.ArrayOfStruct()); - } - - @Test - public void testMapOfPrimitives() throws Exception { - testConverter(new DataGenerators.MapOfPrimitives()); - } - - @Test - public void testMapOfArray() throws Exception { - testConverter(new DataGenerators.MapOfArray()); - } - - @Test - public void testMapOfMap() throws Exception { - testConverter(new DataGenerators.MapOfMap()); - } - - @Test - public void testMapOfStruct() throws Exception { - testConverter(new DataGenerators.MapOfStruct()); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java deleted file mode 100644 index b1e3b20ff7ac..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java +++ /dev/null @@ -1,42 +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.iceberg.flink; - -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.GenericRecord; - -/** - * This interface defines test data generator. Different implementations for primitive and complex - * nested fields are defined in {@link DataGenerators}. - */ -public interface DataGenerator { - Schema icebergSchema(); - - RowType flinkRowType(); - - org.apache.avro.Schema avroSchema(); - - GenericRecord generateIcebergGenericRecord(); - - GenericRowData generateFlinkRowData(); - - org.apache.avro.generic.GenericRecord generateAvroGenericRecord(); -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java deleted file mode 100644 index e2cd411d7069..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java +++ /dev/null @@ -1,1172 +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.iceberg.flink; - -import static org.apache.iceberg.types.Types.NestedField.required; - -import com.fasterxml.jackson.databind.node.IntNode; -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.time.OffsetDateTime; -import java.time.ZoneOffset; -import java.util.Arrays; -import java.util.List; -import java.util.UUID; -import java.util.concurrent.TimeUnit; -import java.util.stream.Collectors; -import org.apache.avro.LogicalTypes; -import org.apache.avro.SchemaBuilder; -import org.apache.avro.generic.GenericData; -import org.apache.avro.util.Utf8; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.GenericArrayData; -import org.apache.flink.table.data.GenericMapData; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.Schema; -import org.apache.iceberg.avro.AvroSchemaUtil; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.types.Types; -import org.joda.time.DateTime; -import org.joda.time.DateTimeZone; -import org.joda.time.Days; - -/** - * Util class to generate test data with extensive coverage different field types: from primitives - * to complex nested types. - */ -public class DataGenerators { - - public static class Primitives implements DataGenerator { - private static final DateTime JODA_DATETIME_EPOC = - new DateTime(1970, 1, 1, 0, 0, 0, 0, DateTimeZone.UTC); - private static final DateTime JODA_DATETIME_20220110 = - new DateTime(2022, 1, 10, 0, 0, 0, 0, DateTimeZone.UTC); - private static final int DAYS_BTW_EPOC_AND_20220110 = - Days.daysBetween(JODA_DATETIME_EPOC, JODA_DATETIME_20220110).getDays(); - private static final int HOUR_8_IN_MILLI = (int) TimeUnit.HOURS.toMillis(8); - - private static final LocalDate JAVA_LOCAL_DATE_20220110 = LocalDate.of(2022, 1, 10); - private static final LocalTime JAVA_LOCAL_TIME_HOUR8 = LocalTime.of(8, 0); - private static final OffsetDateTime JAVA_OFFSET_DATE_TIME_20220110 = - OffsetDateTime.of(2022, 1, 10, 0, 0, 0, 0, ZoneOffset.UTC); - private static final LocalDateTime JAVA_LOCAL_DATE_TIME_20220110 = - LocalDateTime.of(2022, 1, 10, 0, 0, 0); - private static final BigDecimal BIG_DECIMAL_NEGATIVE = new BigDecimal("-1.50"); - private static final byte[] FIXED_BYTES = "012345689012345".getBytes(StandardCharsets.UTF_8); - - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - // primitive types - Types.NestedField.optional(2, "boolean_field", Types.BooleanType.get()), - Types.NestedField.optional(3, "int_field", Types.IntegerType.get()), - Types.NestedField.optional(4, "long_field", Types.LongType.get()), - Types.NestedField.optional(5, "float_field", Types.FloatType.get()), - Types.NestedField.optional(6, "double_field", Types.DoubleType.get()), - Types.NestedField.required(7, "string_field", Types.StringType.get()), - Types.NestedField.required(8, "date_field", Types.DateType.get()), - Types.NestedField.required(9, "time_field", Types.TimeType.get()), - Types.NestedField.required(10, "ts_with_zone_field", Types.TimestampType.withZone()), - Types.NestedField.required( - 11, "ts_without_zone_field", Types.TimestampType.withoutZone()), - Types.NestedField.required(12, "uuid_field", Types.UUIDType.get()), - Types.NestedField.required(13, "binary_field", Types.BinaryType.get()), - Types.NestedField.required(14, "decimal_field", Types.DecimalType.of(9, 2)), - Types.NestedField.required(15, "fixed_field", Types.FixedType.ofLength(16))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - /** - * Fix up Avro Schema that is converted from Iceberg Schema. - * - * @param schemaConvertedFromIceberg Avro Schema converted from Iceberg schema via {@link - * AvroSchemaUtil#convert(Schema, String)} - */ - private org.apache.avro.Schema fixupAvroSchemaConvertedFromIcebergSchema( - org.apache.avro.Schema schemaConvertedFromIceberg) { - List fixedFields = - schemaConvertedFromIceberg.getFields().stream() - .map( - field -> { - org.apache.avro.Schema.Field updatedField = field; - if (field.name().equals("time_field")) { - // Iceberg's AvroSchemaUtil uses timestamp-micros with Long value for time - // field, while AvroToRowDataConverters#convertToTime() always looks for - // Integer value assuming millis. The root problem is that - // AvroToRowDataConverters#createConverter() uses LogicalTypeRoot to - // determine converter and LogicalTypeRoot lost the timestamp precision - // carried by LogicalType like Time(6). - org.apache.avro.Schema fieldSchema = - LogicalTypes.timeMillis() - .addToSchema( - org.apache.avro.Schema.create(org.apache.avro.Schema.Type.INT)); - updatedField = new org.apache.avro.Schema.Field("time_field", fieldSchema); - } - - return new org.apache.avro.Schema.Field(updatedField, updatedField.schema()); - }) - .collect(Collectors.toList()); - return org.apache.avro.Schema.createRecord( - schemaConvertedFromIceberg.getName(), - schemaConvertedFromIceberg.getDoc(), - schemaConvertedFromIceberg.getNamespace(), - schemaConvertedFromIceberg.isError(), - fixedFields); - } - - private final org.apache.avro.Schema avroSchema = - fixupAvroSchemaConvertedFromIcebergSchema(AvroSchemaUtil.convert(icebergSchema, "table")); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField("boolean_field", false); - genericRecord.setField("int_field", Integer.MAX_VALUE); - genericRecord.setField("long_field", Long.MAX_VALUE); - genericRecord.setField("float_field", Float.MAX_VALUE); - genericRecord.setField("double_field", Double.MAX_VALUE); - genericRecord.setField("string_field", "str"); - - genericRecord.setField("date_field", JAVA_LOCAL_DATE_20220110); - genericRecord.setField("time_field", JAVA_LOCAL_TIME_HOUR8); - genericRecord.setField("ts_with_zone_field", JAVA_OFFSET_DATE_TIME_20220110); - genericRecord.setField("ts_without_zone_field", JAVA_LOCAL_DATE_TIME_20220110); - - byte[] uuidBytes = new byte[16]; - for (int i = 0; i < 16; ++i) { - uuidBytes[i] = (byte) i; - } - - genericRecord.setField("uuid_field", UUID.nameUUIDFromBytes(uuidBytes)); - - byte[] binaryBytes = new byte[7]; - for (int i = 0; i < 7; ++i) { - binaryBytes[i] = (byte) i; - } - genericRecord.setField("binary_field", ByteBuffer.wrap(binaryBytes)); - - genericRecord.setField("decimal_field", BIG_DECIMAL_NEGATIVE); - genericRecord.setField("fixed_field", FIXED_BYTES); - - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - byte[] uuidBytes = new byte[16]; - for (int i = 0; i < 16; ++i) { - uuidBytes[i] = (byte) i; - } - - byte[] binaryBytes = new byte[7]; - for (int i = 0; i < 7; ++i) { - binaryBytes[i] = (byte) i; - } - - return GenericRowData.of( - StringData.fromString("row_id_value"), - false, - Integer.MAX_VALUE, - Long.MAX_VALUE, - Float.MAX_VALUE, - Double.MAX_VALUE, - StringData.fromString("str"), - DAYS_BTW_EPOC_AND_20220110, - HOUR_8_IN_MILLI, - // Although Avro logical type for timestamp fields are in micro seconds, - // AvroToRowDataConverters only looks for long value in milliseconds. - TimestampData.fromEpochMillis(JODA_DATETIME_20220110.getMillis()), - TimestampData.fromEpochMillis(JODA_DATETIME_20220110.getMillis()), - uuidBytes, - binaryBytes, - DecimalData.fromBigDecimal(BIG_DECIMAL_NEGATIVE, 9, 2), - FIXED_BYTES); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", new Utf8("row_id_value")); - genericRecord.put("boolean_field", false); - genericRecord.put("int_field", Integer.MAX_VALUE); - genericRecord.put("long_field", Long.MAX_VALUE); - genericRecord.put("float_field", Float.MAX_VALUE); - genericRecord.put("double_field", Double.MAX_VALUE); - genericRecord.put("string_field", new Utf8("str")); - - genericRecord.put("date_field", DAYS_BTW_EPOC_AND_20220110); - genericRecord.put("time_field", HOUR_8_IN_MILLI); - // Although Avro logical type for timestamp fields are in micro seconds, - // AvroToRowDataConverters only looks for long value in milliseconds. - genericRecord.put("ts_with_zone_field", JODA_DATETIME_20220110.getMillis()); - genericRecord.put("ts_without_zone_field", JODA_DATETIME_20220110.getMillis()); - - byte[] uuidBytes = new byte[16]; - for (int i = 0; i < 16; ++i) { - uuidBytes[i] = (byte) i; - } - genericRecord.put("uuid_field", ByteBuffer.wrap(uuidBytes)); - - byte[] binaryBytes = new byte[7]; - for (int i = 0; i < 7; ++i) { - binaryBytes[i] = (byte) i; - } - genericRecord.put("binary_field", ByteBuffer.wrap(binaryBytes)); - - BigDecimal bigDecimal = new BigDecimal("-1.50"); - // unscaledValue().toByteArray() is to match the behavior of RowDataToAvroConverters from - // Flink for decimal type - genericRecord.put("decimal_field", ByteBuffer.wrap(bigDecimal.unscaledValue().toByteArray())); - - genericRecord.put("fixed_field", ByteBuffer.wrap(FIXED_BYTES)); - - return genericRecord; - } - } - - public static class StructOfPrimitive implements DataGenerator { - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.required( - 2, - "struct_of_primitive", - Types.StructType.of( - required(101, "id", Types.IntegerType.get()), - required(102, "name", Types.StringType.get())))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - private final org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(icebergSchema, "table"); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - Schema structSchema = - new Schema(icebergSchema.findField("struct_of_primitive").type().asStructType().fields()); - GenericRecord struct = GenericRecord.create(structSchema); - struct.setField("id", 1); - struct.setField("name", "Jane"); - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField("struct_of_primitive", struct); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - return GenericRowData.of( - StringData.fromString("row_id_value"), - GenericRowData.of(1, StringData.fromString("Jane"))); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.Schema structSchema = avroSchema.getField("struct_of_primitive").schema(); - org.apache.avro.generic.GenericRecord struct = new GenericData.Record(structSchema); - struct.put("id", 1); - struct.put("name", "Jane"); - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", "row_id_value"); - genericRecord.put("struct_of_primitive", struct); - return genericRecord; - } - } - - public static class StructOfArray implements DataGenerator { - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.required( - 2, - "struct_of_array", - Types.StructType.of( - required(101, "id", Types.IntegerType.get()), - required( - 102, "names", Types.ListType.ofRequired(201, Types.StringType.get()))))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - private final org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(icebergSchema, "table"); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - Schema structSchema = - new Schema(icebergSchema.findField("struct_of_array").type().asStructType().fields()); - GenericRecord struct = GenericRecord.create(structSchema); - struct.setField("id", 1); - struct.setField("names", Arrays.asList("Jane", "Joe")); - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField("struct_of_array", struct); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - StringData[] names = {StringData.fromString("Jane"), StringData.fromString("Joe")}; - return GenericRowData.of( - StringData.fromString("row_id_value"), GenericRowData.of(1, new GenericArrayData(names))); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.Schema structSchema = avroSchema.getField("struct_of_array").schema(); - org.apache.avro.generic.GenericRecord struct = new GenericData.Record(structSchema); - struct.put("id", 1); - struct.put("names", Arrays.asList("Jane", "Joe")); - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", "row_id_value"); - genericRecord.put("struct_of_array", struct); - return genericRecord; - } - } - - public static class StructOfMap implements DataGenerator { - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.required( - 2, - "struct_of_map", - Types.StructType.of( - required(101, "id", Types.IntegerType.get()), - required( - 102, - "names", - Types.MapType.ofRequired( - 201, 202, Types.StringType.get(), Types.StringType.get()))))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - private final org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(icebergSchema, "table"); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - Schema structSchema = - new Schema(icebergSchema.findField("struct_of_map").type().asStructType().fields()); - GenericRecord struct = GenericRecord.create(structSchema); - struct.setField("id", 1); - struct.setField("names", ImmutableMap.of("Jane", "female", "Joe", "male")); - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField("struct_of_map", struct); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - return GenericRowData.of( - StringData.fromString("row_id_value"), - GenericRowData.of( - 1, - new GenericMapData( - ImmutableMap.of( - StringData.fromString("Jane"), - StringData.fromString("female"), - StringData.fromString("Joe"), - StringData.fromString("male"))))); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.Schema structSchema = avroSchema.getField("struct_of_map").schema(); - org.apache.avro.generic.GenericRecord struct = new GenericData.Record(structSchema); - struct.put("id", 1); - struct.put("names", ImmutableMap.of("Jane", new Utf8("female"), "Joe", new Utf8("male"))); - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", "row_id_value"); - genericRecord.put("struct_of_map", struct); - return genericRecord; - } - } - - public static class StructOfStruct implements DataGenerator { - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.required( - 2, - "struct_of_struct", - Types.StructType.of( - required(101, "id", Types.IntegerType.get()), - required( - 102, - "person_struct", - Types.StructType.of( - Types.NestedField.required(201, "name", Types.StringType.get()), - Types.NestedField.required(202, "address", Types.StringType.get())))))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - private final org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(icebergSchema, "table"); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - Schema structSchema = - new Schema(icebergSchema.findField("struct_of_struct").type().asStructType().fields()); - Schema personSchema = - new Schema(structSchema.findField("person_struct").type().asStructType().fields()); - GenericRecord person = GenericRecord.create(personSchema); - person.setField("name", "Jane"); - person.setField("address", "Apple Park"); - GenericRecord struct = GenericRecord.create(structSchema); - struct.setField("id", 1); - struct.setField("person_struct", person); - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField("struct_of_struct", struct); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - return GenericRowData.of( - StringData.fromString("row_id_value"), - GenericRowData.of( - 1, - GenericRowData.of( - StringData.fromString("Jane"), StringData.fromString("Apple Park")))); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.Schema structSchema = avroSchema.getField("struct_of_struct").schema(); - org.apache.avro.Schema personSchema = structSchema.getField("person_struct").schema(); - org.apache.avro.generic.GenericRecord person = new GenericData.Record(personSchema); - person.put("name", "Jane"); - person.put("address", "Apple Park"); - org.apache.avro.generic.GenericRecord struct = new GenericData.Record(structSchema); - struct.put("id", 1); - struct.put("person_struct", person); - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", "row_id_value"); - genericRecord.put("struct_of_struct", struct); - return genericRecord; - } - } - - public static class ArrayOfPrimitive implements DataGenerator { - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.required( - 2, "array_of_int", Types.ListType.ofOptional(101, Types.IntegerType.get()))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - private final org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(icebergSchema, "table"); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField("array_of_int", Arrays.asList(1, 2, 3)); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - Integer[] arr = {1, 2, 3}; - return GenericRowData.of(StringData.fromString("row_id_value"), new GenericArrayData(arr)); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", "row_id_value"); - genericRecord.put("array_of_int", Arrays.asList(1, 2, 3)); - return genericRecord; - } - } - - public static class ArrayOfArray implements DataGenerator { - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.required( - 2, - "array_of_array", - Types.ListType.ofRequired( - 101, Types.ListType.ofRequired(201, Types.IntegerType.get())))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - private final org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(icebergSchema, "table"); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField( - "array_of_array", Arrays.asList(Arrays.asList(1, 2, 3), Arrays.asList(4, 5, 6))); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - // non-primitive - Integer[] array1 = {1, 2, 3}; - Integer[] array2 = {4, 5, 6}; - GenericArrayData[] arrayOfArrays = { - new GenericArrayData(array1), new GenericArrayData(array2) - }; - return GenericRowData.of( - StringData.fromString("row_id_value"), new GenericArrayData(arrayOfArrays)); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", "row_id_value"); - genericRecord.put( - "array_of_array", Arrays.asList(Arrays.asList(1, 2, 3), Arrays.asList(4, 5, 6))); - return genericRecord; - } - } - - public static class ArrayOfMap implements DataGenerator { - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.required( - 2, - "array_of_map", - Types.ListType.ofRequired( - 101, - Types.MapType.ofRequired( - 201, 202, Types.StringType.get(), Types.IntegerType.get())))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - private final org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(icebergSchema, "table"); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField( - "array_of_map", - Arrays.asList( - ImmutableMap.of("Jane", 1, "Joe", 2), ImmutableMap.of("Alice", 3, "Bob", 4))); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - GenericMapData[] array = { - new GenericMapData( - ImmutableMap.of(StringData.fromString("Jane"), 1, StringData.fromString("Joe"), 2)), - new GenericMapData( - ImmutableMap.of(StringData.fromString("Alice"), 3, StringData.fromString("Bob"), 4)) - }; - return GenericRowData.of(StringData.fromString("row_id_value"), new GenericArrayData(array)); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", "row_id_value"); - genericRecord.put( - "array_of_map", - Arrays.asList( - ImmutableMap.of("Jane", 1, "Joe", 2), ImmutableMap.of("Alice", 3, "Bob", 4))); - return genericRecord; - } - } - - public static class ArrayOfStruct implements DataGenerator { - private final Types.StructType structType = - Types.StructType.of( - required(201, "id", Types.IntegerType.get()), - required(202, "name", Types.StringType.get())); - private final Schema structIcebergSchema = new Schema(structType.fields()); - private final org.apache.avro.Schema structAvroSchema = - AvroSchemaUtil.convert(structIcebergSchema, "struct"); - - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.optional( - 2, "array_of_struct", Types.ListType.ofRequired(101, structType))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - private final org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(icebergSchema, "table"); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - GenericRecord struct1 = GenericRecord.create(structIcebergSchema); - struct1.setField("id", 1); - struct1.setField("name", "Jane"); - GenericRecord struct2 = GenericRecord.create(structIcebergSchema); - struct2.setField("id", 2); - struct2.setField("name", "Joe"); - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField("array_of_struct", Arrays.asList(struct1, struct2)); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - GenericRowData[] structArray = { - GenericRowData.of(1, StringData.fromString("Jane")), - GenericRowData.of(2, StringData.fromString("Joe")) - }; - return GenericRowData.of( - StringData.fromString("row_id_value"), new GenericArrayData(structArray)); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.generic.GenericRecord struct1 = new GenericData.Record(structAvroSchema); - struct1.put("id", 1); - struct1.put("name", "Jane"); - org.apache.avro.generic.GenericRecord struct2 = new GenericData.Record(structAvroSchema); - struct2.put("id", 2); - struct2.put("name", "Joe"); - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", "row_id_value"); - genericRecord.put("array_of_struct", Arrays.asList(struct1, struct2)); - return genericRecord; - } - } - - public static class MapOfPrimitives implements DataGenerator { - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.optional( - 2, - "map_of_primitives", - Types.MapType.ofRequired( - 101, 102, Types.StringType.get(), Types.IntegerType.get()))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - private final org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(icebergSchema, "table"); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField("map_of_primitives", ImmutableMap.of("Jane", 1, "Joe", 2)); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - return GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericMapData( - ImmutableMap.of(StringData.fromString("Jane"), 1, StringData.fromString("Joe"), 2))); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", "row_id_value"); - genericRecord.put("map_of_primitives", ImmutableMap.of("Jane", 1, "Joe", 2)); - return genericRecord; - } - } - - public static class MapOfArray implements DataGenerator { - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.required( - 2, - "map_of_array", - Types.MapType.ofRequired( - 101, - 102, - Types.StringType.get(), - Types.ListType.ofRequired(201, Types.IntegerType.get())))); - - private final RowType rowType = FlinkSchemaUtil.convert(icebergSchema); - - private final org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(icebergSchema, "table"); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return rowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField( - "map_of_array", - ImmutableMap.of( - "Jane", Arrays.asList(1, 2, 3), - "Joe", Arrays.asList(4, 5, 6))); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - Integer[] janeArray = {1, 2, 3}; - Integer[] joeArray = {4, 5, 6}; - return GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericMapData( - ImmutableMap.of( - StringData.fromString("Jane"), - new GenericArrayData(janeArray), - StringData.fromString("Joe"), - new GenericArrayData(joeArray)))); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", "row_id_value"); - genericRecord.put( - "map_of_array", - ImmutableMap.of( - "Jane", Arrays.asList(1, 2, 3), - "Joe", Arrays.asList(4, 5, 6))); - return genericRecord; - } - } - - public static class MapOfMap implements DataGenerator { - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.required( - 2, - "map_of_map", - Types.MapType.ofRequired( - 101, - 102, - Types.StringType.get(), - Types.MapType.ofRequired( - 301, 302, Types.StringType.get(), Types.IntegerType.get())))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - private final org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(icebergSchema, "table"); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField( - "map_of_map", - ImmutableMap.of( - "female", ImmutableMap.of("Jane", 1, "Alice", 2), - "male", ImmutableMap.of("Joe", 3, "Bob", 4))); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - return GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericMapData( - ImmutableMap.of( - StringData.fromString("female"), - new GenericMapData( - ImmutableMap.of( - StringData.fromString("Jane"), 1, StringData.fromString("Alice"), 2)), - StringData.fromString("male"), - new GenericMapData( - ImmutableMap.of( - StringData.fromString("Joe"), 3, StringData.fromString("Bob"), 4))))); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", "row_id_value"); - genericRecord.put( - "map_of_map", - ImmutableMap.of( - "female", ImmutableMap.of("Jane", 1, "Alice", 2), - "male", ImmutableMap.of("Joe", 3, "Bob", 4))); - return genericRecord; - } - } - - public static class MapOfStruct implements DataGenerator { - private org.apache.avro.Schema createAvroSchemaIdField() { - org.apache.avro.Schema schema = SchemaBuilder.builder().intType(); - // this is needed to match the converter generated schema props - schema.addProp("field-id", IntNode.valueOf(201)); - return schema; - } - - private org.apache.avro.Schema createAvroSchemaNameField() { - org.apache.avro.Schema schema = SchemaBuilder.builder().stringType(); - // this is needed to match the converter generated schema props - schema.addProp("field-id", IntNode.valueOf(202)); - return schema; - } - - private final Types.StructType structType = - Types.StructType.of( - required(201, "id", Types.IntegerType.get()), - required(202, "name", Types.StringType.get())); - private final Schema structIcebergSchema = new Schema(structType.fields()); - - private final org.apache.avro.Schema structAvroSchema = - SchemaBuilder.builder() - .record("struct") - .fields() - .name("id") - .type(createAvroSchemaIdField()) - .noDefault() - .name("name") - .type(createAvroSchemaNameField()) - .noDefault() - .endRecord(); - - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.required( - 2, - "map_of_struct", - Types.MapType.ofRequired(101, 102, Types.StringType.get(), structType))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - // Can't use AvroSchemaUtil.convert otherwise the nested schema will have generated name like - // `r102` not the specified name like `struct`. - org.apache.avro.Schema avroSchema = - SchemaBuilder.builder() - .record("table") - .fields() - .requiredString("row_id") - .name("map_of_struct") - .type(SchemaBuilder.builder().map().values(structAvroSchema)) - .noDefault() - .endRecord(); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - GenericRecord struct1 = GenericRecord.create(structIcebergSchema); - struct1.setField("id", 1); - struct1.setField("name", "Jane"); - GenericRecord struct2 = GenericRecord.create(structIcebergSchema); - struct2.setField("id", 2); - struct2.setField("name", "Joe"); - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField( - "map_of_struct", ImmutableMap.of("struct1", struct1, "struct2", struct2)); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - return GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericMapData( - ImmutableMap.of( - StringData.fromString("struct1"), - GenericRowData.of(1, StringData.fromString("Jane")), - StringData.fromString("struct2"), - GenericRowData.of(2, StringData.fromString("Joe"))))); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.generic.GenericRecord struct1 = new GenericData.Record(structAvroSchema); - struct1.put("id", 1); - struct1.put("name", new Utf8("Jane")); - org.apache.avro.generic.GenericRecord struct2 = new GenericData.Record(structAvroSchema); - struct2.put("id", 2); - struct2.put("name", new Utf8("Joe")); - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", new Utf8("row_id_value")); - genericRecord.put("map_of_struct", ImmutableMap.of("struct1", struct1, "struct2", struct2)); - return genericRecord; - } - } - - public static class MapOfStructStruct implements DataGenerator { - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.optional( - 2, - "map", - Types.MapType.ofOptional( - 101, - 102, - Types.StructType.of( - Types.NestedField.required(201, "key", Types.LongType.get()), - Types.NestedField.optional(202, "keyData", Types.StringType.get())), - Types.StructType.of( - Types.NestedField.required(203, "value", Types.LongType.get()), - Types.NestedField.optional(204, "valueData", Types.StringType.get()))))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - throw new UnsupportedOperationException( - "Not applicable as Avro Map only support string key type"); - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - throw new UnsupportedOperationException("Not implemented yet"); - } - - @Override - public GenericRowData generateFlinkRowData() { - return GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericMapData( - ImmutableMap.of( - GenericRowData.of(1L, StringData.fromString("key_data")), - GenericRowData.of(1L, StringData.fromString("value_data"))))); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - throw new UnsupportedOperationException("Avro Map only support string key type"); - } - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/FlinkCatalogTestBase.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/FlinkCatalogTestBase.java deleted file mode 100644 index 74c5d343e996..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/FlinkCatalogTestBase.java +++ /dev/null @@ -1,155 +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.iceberg.flink; - -import java.io.IOException; -import java.util.List; -import java.util.Map; -import org.apache.flink.util.ArrayUtils; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.catalog.SupportsNamespaces; -import org.apache.iceberg.hadoop.HadoopCatalog; -import org.apache.iceberg.relocated.com.google.common.base.Joiner; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public abstract class FlinkCatalogTestBase extends FlinkTestBase { - - protected static final String DATABASE = "db"; - private static TemporaryFolder hiveWarehouse = new TemporaryFolder(); - private static TemporaryFolder hadoopWarehouse = new TemporaryFolder(); - - @BeforeClass - public static void createWarehouse() throws IOException { - hiveWarehouse.create(); - hadoopWarehouse.create(); - } - - @AfterClass - public static void dropWarehouse() { - hiveWarehouse.delete(); - hadoopWarehouse.delete(); - } - - @Before - public void before() { - sql("CREATE CATALOG %s WITH %s", catalogName, toWithClause(config)); - } - - @After - public void clean() { - dropCatalog(catalogName, true); - } - - @Parameterized.Parameters(name = "catalogName = {0} baseNamespace = {1}") - public static Iterable parameters() { - return Lists.newArrayList( - new Object[] {"testhive", Namespace.empty()}, - new Object[] {"testhadoop", Namespace.empty()}, - new Object[] {"testhadoop_basenamespace", Namespace.of("l0", "l1")}); - } - - protected final String catalogName; - protected final Namespace baseNamespace; - protected final Catalog validationCatalog; - protected final SupportsNamespaces validationNamespaceCatalog; - protected final Map config = Maps.newHashMap(); - - protected final String flinkDatabase; - protected final Namespace icebergNamespace; - protected final boolean isHadoopCatalog; - - public FlinkCatalogTestBase(String catalogName, Namespace baseNamespace) { - this.catalogName = catalogName; - this.baseNamespace = baseNamespace; - this.isHadoopCatalog = catalogName.startsWith("testhadoop"); - this.validationCatalog = - isHadoopCatalog - ? new HadoopCatalog(hiveConf, "file:" + hadoopWarehouse.getRoot()) - : catalog; - this.validationNamespaceCatalog = (SupportsNamespaces) validationCatalog; - - config.put("type", "iceberg"); - if (!baseNamespace.isEmpty()) { - config.put(FlinkCatalogFactory.BASE_NAMESPACE, baseNamespace.toString()); - } - if (isHadoopCatalog) { - config.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hadoop"); - } else { - config.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hive"); - config.put(CatalogProperties.URI, getURI(hiveConf)); - } - config.put(CatalogProperties.WAREHOUSE_LOCATION, String.format("file://%s", warehouseRoot())); - - this.flinkDatabase = catalogName + "." + DATABASE; - this.icebergNamespace = - Namespace.of(ArrayUtils.concat(baseNamespace.levels(), new String[] {DATABASE})); - } - - protected String warehouseRoot() { - if (isHadoopCatalog) { - return hadoopWarehouse.getRoot().getAbsolutePath(); - } else { - return hiveWarehouse.getRoot().getAbsolutePath(); - } - } - - protected String getFullQualifiedTableName(String tableName) { - final List levels = Lists.newArrayList(icebergNamespace.levels()); - levels.add(tableName); - return Joiner.on('.').join(levels); - } - - static String getURI(HiveConf conf) { - return conf.get(HiveConf.ConfVars.METASTOREURIS.varname); - } - - static String toWithClause(Map props) { - StringBuilder builder = new StringBuilder(); - builder.append("("); - int propCount = 0; - for (Map.Entry entry : props.entrySet()) { - if (propCount > 0) { - builder.append(","); - } - builder - .append("'") - .append(entry.getKey()) - .append("'") - .append("=") - .append("'") - .append(entry.getValue()) - .append("'"); - propCount++; - } - builder.append(")"); - return builder.toString(); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java deleted file mode 100644 index 8076e0ec76f8..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java +++ /dev/null @@ -1,129 +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.iceberg.flink; - -import java.util.List; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.TableResult; -import org.apache.flink.test.util.MiniClusterWithClientResource; -import org.apache.flink.test.util.TestBaseUtils; -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.iceberg.CatalogUtil; -import org.apache.iceberg.hive.HiveCatalog; -import org.apache.iceberg.hive.TestHiveMetastore; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.rules.TemporaryFolder; - -public abstract class FlinkTestBase extends TestBaseUtils { - - @ClassRule - public static MiniClusterWithClientResource miniClusterResource = - MiniClusterResource.createWithClassloaderCheckDisabled(); - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - private static TestHiveMetastore metastore = null; - protected static HiveConf hiveConf = null; - protected static HiveCatalog catalog = null; - - private volatile TableEnvironment tEnv = null; - - @BeforeClass - public static void startMetastore() { - FlinkTestBase.metastore = new TestHiveMetastore(); - metastore.start(); - FlinkTestBase.hiveConf = metastore.hiveConf(); - FlinkTestBase.catalog = - (HiveCatalog) - CatalogUtil.loadCatalog( - HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf); - } - - @AfterClass - public static void stopMetastore() throws Exception { - metastore.stop(); - FlinkTestBase.catalog = null; - } - - protected TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - if (tEnv == null) { - EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build(); - - TableEnvironment env = TableEnvironment.create(settings); - env.getConfig() - .getConfiguration() - .set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); - tEnv = env; - } - } - } - return tEnv; - } - - protected static TableResult exec(TableEnvironment env, String query, Object... args) { - return env.executeSql(String.format(query, args)); - } - - protected TableResult exec(String query, Object... args) { - return exec(getTableEnv(), query, args); - } - - protected List sql(String query, Object... args) { - TableResult tableResult = exec(query, args); - try (CloseableIterator iter = tableResult.collect()) { - return Lists.newArrayList(iter); - } catch (Exception e) { - throw new RuntimeException("Failed to collect table result", e); - } - } - - protected void assertSameElements(Iterable expected, Iterable actual) { - Assertions.assertThat(actual).isNotNull().containsExactlyInAnyOrderElementsOf(expected); - } - - protected void assertSameElements(String message, Iterable expected, Iterable actual) { - Assertions.assertThat(actual) - .isNotNull() - .as(message) - .containsExactlyInAnyOrderElementsOf(expected); - } - - /** - * We can not drop currently used catalog after FLINK-29677, so we have make sure that we do not - * use the current catalog before dropping it. This method switches to the 'default_catalog' and - * drops the one requested. - * - * @param catalogName The catalog to drop - * @param ifExists If we should use the 'IF EXISTS' when dropping the catalog - */ - protected void dropCatalog(String catalogName, boolean ifExists) { - sql("USE CATALOG default_catalog"); - sql("DROP CATALOG %s %s", ifExists ? "IF EXISTS" : "", catalogName); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java deleted file mode 100644 index d8e1325254d9..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java +++ /dev/null @@ -1,104 +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.iceberg.flink; - -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.UUID; -import org.apache.commons.io.FileUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.hadoop.HadoopCatalog; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.assertj.core.api.Assertions; -import org.junit.jupiter.api.extension.AfterAllCallback; -import org.junit.jupiter.api.extension.AfterEachCallback; -import org.junit.jupiter.api.extension.BeforeAllCallback; -import org.junit.jupiter.api.extension.BeforeEachCallback; -import org.junit.jupiter.api.extension.ExtensionContext; - -public class HadoopCatalogExtension - implements BeforeAllCallback, BeforeEachCallback, AfterAllCallback, AfterEachCallback { - protected final String database; - protected final String tableName; - - protected Path temporaryFolder; - protected Catalog catalog; - protected CatalogLoader catalogLoader; - protected String warehouse; - protected TableLoader tableLoader; - - public HadoopCatalogExtension(String database, String tableName) { - this.database = database; - this.tableName = tableName; - } - - @Override - public void beforeAll(ExtensionContext context) throws Exception { - this.temporaryFolder = Files.createTempDirectory("junit5_hadoop_catalog-"); - } - - @Override - public void afterAll(ExtensionContext context) throws Exception { - FileUtils.deleteDirectory(temporaryFolder.toFile()); - } - - @Override - public void beforeEach(ExtensionContext context) throws Exception { - Assertions.assertThat(temporaryFolder).exists().isDirectory(); - this.warehouse = "file:" + temporaryFolder + "/" + UUID.randomUUID(); - this.catalogLoader = - CatalogLoader.hadoop( - "hadoop", - new Configuration(), - ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouse)); - this.catalog = catalogLoader.loadCatalog(); - this.tableLoader = - TableLoader.fromCatalog(catalogLoader, TableIdentifier.of(database, tableName)); - } - - @Override - public void afterEach(ExtensionContext context) throws Exception { - try { - catalog.dropTable(TableIdentifier.of(database, tableName)); - ((HadoopCatalog) catalog).close(); - tableLoader.close(); - } catch (Exception e) { - throw new RuntimeException("Failed to close catalog resource"); - } - } - - public TableLoader tableLoader() { - return tableLoader; - } - - public Catalog catalog() { - return catalog; - } - - public CatalogLoader catalogLoader() { - return catalogLoader; - } - - public String warehouse() { - return warehouse; - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java deleted file mode 100644 index 2b4694d3c268..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java +++ /dev/null @@ -1,90 +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.iceberg.flink; - -import java.io.File; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.hadoop.HadoopCatalog; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.rules.ExternalResource; -import org.junit.rules.TemporaryFolder; - -public class HadoopCatalogResource extends ExternalResource { - protected final TemporaryFolder temporaryFolder; - protected final String database; - protected final String tableName; - - protected Catalog catalog; - protected CatalogLoader catalogLoader; - protected String warehouse; - protected TableLoader tableLoader; - - public HadoopCatalogResource(TemporaryFolder temporaryFolder, String database, String tableName) { - this.temporaryFolder = temporaryFolder; - this.database = database; - this.tableName = tableName; - } - - @Override - protected void before() throws Throwable { - File warehouseFile = temporaryFolder.newFolder(); - Assert.assertTrue(warehouseFile.delete()); - // before variables - this.warehouse = "file:" + warehouseFile; - this.catalogLoader = - CatalogLoader.hadoop( - "hadoop", - new Configuration(), - ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouse)); - this.catalog = catalogLoader.loadCatalog(); - this.tableLoader = - TableLoader.fromCatalog(catalogLoader, TableIdentifier.of(database, tableName)); - } - - @Override - protected void after() { - try { - catalog.dropTable(TableIdentifier.of(database, tableName)); - ((HadoopCatalog) catalog).close(); - tableLoader.close(); - } catch (Exception e) { - throw new RuntimeException("Failed to close catalog resource"); - } - } - - public TableLoader tableLoader() { - return tableLoader; - } - - public Catalog catalog() { - return catalog; - } - - public CatalogLoader catalogLoader() { - return catalogLoader; - } - - public String warehouse() { - return warehouse; - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java deleted file mode 100644 index 2da6f74b91d8..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java +++ /dev/null @@ -1,64 +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.iceberg.flink; - -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.TableIdentifier; -import org.junit.rules.TemporaryFolder; - -public class HadoopTableResource extends HadoopCatalogResource { - private final Schema schema; - private final PartitionSpec partitionSpec; - - private Table table; - - public HadoopTableResource( - TemporaryFolder temporaryFolder, String database, String tableName, Schema schema) { - this(temporaryFolder, database, tableName, schema, null); - } - - public HadoopTableResource( - TemporaryFolder temporaryFolder, - String database, - String tableName, - Schema schema, - PartitionSpec partitionSpec) { - super(temporaryFolder, database, tableName); - this.schema = schema; - this.partitionSpec = partitionSpec; - } - - @Override - protected void before() throws Throwable { - super.before(); - if (partitionSpec == null) { - this.table = catalog.createTable(TableIdentifier.of(database, tableName), schema); - } else { - this.table = - catalog.createTable(TableIdentifier.of(database, tableName), schema, partitionSpec); - } - tableLoader.open(); - } - - public Table table() { - return table; - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java deleted file mode 100644 index 45af9241b743..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java +++ /dev/null @@ -1,53 +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.iceberg.flink; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.test.util.MiniClusterWithClientResource; - -public class MiniClusterResource { - - private static final int DEFAULT_TM_NUM = 1; - private static final int DEFAULT_PARALLELISM = 4; - - public static final Configuration DISABLE_CLASSLOADER_CHECK_CONFIG = - new Configuration() - // disable classloader check as Avro may cache class/object in the serializers. - .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); - - private MiniClusterResource() {} - - /** - * It will start a mini cluster with classloader.check-leaked-classloader=false, so that we won't - * break the unit tests because of the class loader leak issue. In our iceberg integration tests, - * there're some that will assert the results after finished the flink jobs, so actually we may - * access the class loader that has been closed by the flink task managers if we enable the switch - * classloader.check-leaked-classloader by default. - */ - public static MiniClusterWithClientResource createWithClassloaderCheckDisabled() { - return new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(DEFAULT_TM_NUM) - .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) - .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) - .build()); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java deleted file mode 100644 index 9a73b80e077d..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java +++ /dev/null @@ -1,53 +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.iceberg.flink; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.test.junit5.MiniClusterExtension; - -public class MiniFlinkClusterExtension { - - private static final int DEFAULT_TM_NUM = 1; - private static final int DEFAULT_PARALLELISM = 4; - - public static final Configuration DISABLE_CLASSLOADER_CHECK_CONFIG = - new Configuration() - // disable classloader check as Avro may cache class/object in the serializers. - .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); - - private MiniFlinkClusterExtension() {} - - /** - * It will start a mini cluster with classloader.check-leaked-classloader=false, so that we won't - * break the unit tests because of the class loader leak issue. In our iceberg integration tests, - * there're some that will assert the results after finished the flink jobs, so actually we may - * access the class loader that has been closed by the flink task managers if we enable the switch - * classloader.check-leaked-classloader by default. - */ - public static MiniClusterExtension createWithClassloaderCheckDisabled() { - return new MiniClusterExtension( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(DEFAULT_TM_NUM) - .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) - .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) - .build()); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java deleted file mode 100644 index e532fb62615c..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java +++ /dev/null @@ -1,135 +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.iceberg.flink; - -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import java.time.Instant; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.time.OffsetDateTime; -import java.time.ZoneOffset; -import java.time.temporal.ChronoUnit; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import java.util.concurrent.TimeUnit; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.GenericArrayData; -import org.apache.flink.table.data.GenericMapData; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; - -public class RowDataConverter { - private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC); - private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate(); - - private RowDataConverter() {} - - public static RowData convert(Schema iSchema, Record record) { - return convert(iSchema.asStruct(), record); - } - - private static RowData convert(Types.StructType struct, Record record) { - GenericRowData rowData = new GenericRowData(struct.fields().size()); - List fields = struct.fields(); - for (int i = 0; i < fields.size(); i += 1) { - Types.NestedField field = fields.get(i); - - Type fieldType = field.type(); - rowData.setField(i, convert(fieldType, record.get(i))); - } - return rowData; - } - - private static Object convert(Type type, Object object) { - if (object == null) { - return null; - } - - switch (type.typeId()) { - case BOOLEAN: - case INTEGER: - case LONG: - case FLOAT: - case DOUBLE: - case FIXED: - return object; - case DATE: - return (int) ChronoUnit.DAYS.between(EPOCH_DAY, (LocalDate) object); - case TIME: - // Iceberg's time is in microseconds, while flink's time is in milliseconds. - LocalTime localTime = (LocalTime) object; - return (int) TimeUnit.NANOSECONDS.toMillis(localTime.toNanoOfDay()); - case TIMESTAMP: - if (((Types.TimestampType) type).shouldAdjustToUTC()) { - return TimestampData.fromInstant(((OffsetDateTime) object).toInstant()); - } else { - return TimestampData.fromLocalDateTime((LocalDateTime) object); - } - case STRING: - return StringData.fromString((String) object); - case UUID: - UUID uuid = (UUID) object; - ByteBuffer bb = ByteBuffer.allocate(16); - bb.putLong(uuid.getMostSignificantBits()); - bb.putLong(uuid.getLeastSignificantBits()); - return bb.array(); - case BINARY: - ByteBuffer buffer = (ByteBuffer) object; - return Arrays.copyOfRange( - buffer.array(), - buffer.arrayOffset() + buffer.position(), - buffer.arrayOffset() + buffer.remaining()); - case DECIMAL: - Types.DecimalType decimalType = (Types.DecimalType) type; - return DecimalData.fromBigDecimal( - (BigDecimal) object, decimalType.precision(), decimalType.scale()); - case STRUCT: - return convert(type.asStructType(), (Record) object); - case LIST: - List list = (List) object; - Object[] convertedArray = new Object[list.size()]; - for (int i = 0; i < convertedArray.length; i++) { - convertedArray[i] = convert(type.asListType().elementType(), list.get(i)); - } - return new GenericArrayData(convertedArray); - case MAP: - Map convertedMap = Maps.newLinkedHashMap(); - Map map = (Map) object; - for (Map.Entry entry : map.entrySet()) { - convertedMap.put( - convert(type.asMapType().keyType(), entry.getKey()), - convert(type.asMapType().valueType(), entry.getValue())); - } - return new GenericMapData(convertedMap); - default: - throw new UnsupportedOperationException("Not a supported type: " + type); - } - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java deleted file mode 100644 index f48764f772b4..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java +++ /dev/null @@ -1,443 +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.iceberg.flink; - -import static org.apache.iceberg.hadoop.HadoopOutputFile.fromPath; - -import java.io.IOException; -import java.time.Duration; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.types.RowKind; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFiles; -import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.SnapshotRef; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableScan; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.data.IcebergGenerics; -import org.apache.iceberg.data.InternalRecordWrapper; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.deletes.EqualityDeleteWriter; -import org.apache.iceberg.deletes.PositionDelete; -import org.apache.iceberg.deletes.PositionDeleteWriter; -import org.apache.iceberg.encryption.EncryptedOutputFile; -import org.apache.iceberg.flink.sink.FlinkAppenderFactory; -import org.apache.iceberg.hadoop.HadoopInputFile; -import org.apache.iceberg.hadoop.HadoopTables; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.Pair; -import org.apache.iceberg.util.StructLikeSet; -import org.apache.iceberg.util.StructLikeWrapper; -import org.awaitility.Awaitility; -import org.junit.Assert; - -public class SimpleDataUtil { - - private SimpleDataUtil() {} - - public static final Schema SCHEMA = - new Schema( - Types.NestedField.optional(1, "id", Types.IntegerType.get()), - Types.NestedField.optional(2, "data", Types.StringType.get())); - - public static final TableSchema FLINK_SCHEMA = - TableSchema.builder().field("id", DataTypes.INT()).field("data", DataTypes.STRING()).build(); - - public static final RowType ROW_TYPE = (RowType) FLINK_SCHEMA.toRowDataType().getLogicalType(); - - public static final Record RECORD = GenericRecord.create(SCHEMA); - - public static Table createTable( - String path, Map properties, boolean partitioned) { - PartitionSpec spec; - if (partitioned) { - spec = PartitionSpec.builderFor(SCHEMA).identity("data").build(); - } else { - spec = PartitionSpec.unpartitioned(); - } - return new HadoopTables().create(SCHEMA, spec, properties, path); - } - - public static Record createRecord(Integer id, String data) { - Record record = RECORD.copy(); - record.setField("id", id); - record.setField("data", data); - return record; - } - - public static RowData createRowData(Integer id, String data) { - return GenericRowData.of(id, StringData.fromString(data)); - } - - public static RowData createInsert(Integer id, String data) { - return GenericRowData.ofKind(RowKind.INSERT, id, StringData.fromString(data)); - } - - public static RowData createDelete(Integer id, String data) { - return GenericRowData.ofKind(RowKind.DELETE, id, StringData.fromString(data)); - } - - public static RowData createUpdateBefore(Integer id, String data) { - return GenericRowData.ofKind(RowKind.UPDATE_BEFORE, id, StringData.fromString(data)); - } - - public static RowData createUpdateAfter(Integer id, String data) { - return GenericRowData.ofKind(RowKind.UPDATE_AFTER, id, StringData.fromString(data)); - } - - public static DataFile writeFile( - Table table, - Schema schema, - PartitionSpec spec, - Configuration conf, - String location, - String filename, - List rows) - throws IOException { - return writeFile(table, schema, spec, conf, location, filename, rows, null); - } - - /** Write the list of {@link RowData} to the given path and with the given partition data */ - public static DataFile writeFile( - Table table, - Schema schema, - PartitionSpec spec, - Configuration conf, - String location, - String filename, - List rows, - StructLike partition) - throws IOException { - Path path = new Path(location, filename); - FileFormat fileFormat = FileFormat.fromFileName(filename); - Preconditions.checkNotNull(fileFormat, "Cannot determine format for file: %s", filename); - - RowType flinkSchema = FlinkSchemaUtil.convert(schema); - FileAppenderFactory appenderFactory = - new FlinkAppenderFactory( - table, schema, flinkSchema, ImmutableMap.of(), spec, null, null, null); - - FileAppender appender = appenderFactory.newAppender(fromPath(path, conf), fileFormat); - try (FileAppender closeableAppender = appender) { - closeableAppender.addAll(rows); - } - - DataFiles.Builder builder = - DataFiles.builder(spec) - .withInputFile(HadoopInputFile.fromPath(path, conf)) - .withMetrics(appender.metrics()); - - if (partition != null) { - builder = builder.withPartition(partition); - } - - return builder.build(); - } - - public static DeleteFile writeEqDeleteFile( - Table table, - FileFormat format, - String filename, - FileAppenderFactory appenderFactory, - List deletes) - throws IOException { - EncryptedOutputFile outputFile = - table - .encryption() - .encrypt(fromPath(new Path(table.location(), filename), new Configuration())); - - EqualityDeleteWriter eqWriter = - appenderFactory.newEqDeleteWriter(outputFile, format, null); - try (EqualityDeleteWriter writer = eqWriter) { - writer.write(deletes); - } - return eqWriter.toDeleteFile(); - } - - public static DeleteFile writePosDeleteFile( - Table table, - FileFormat format, - String filename, - FileAppenderFactory appenderFactory, - List> positions) - throws IOException { - EncryptedOutputFile outputFile = - table - .encryption() - .encrypt(fromPath(new Path(table.location(), filename), new Configuration())); - - PositionDeleteWriter posWriter = - appenderFactory.newPosDeleteWriter(outputFile, format, null); - PositionDelete posDelete = PositionDelete.create(); - try (PositionDeleteWriter writer = posWriter) { - for (Pair p : positions) { - writer.write(posDelete.set(p.first(), p.second(), null)); - } - } - return posWriter.toDeleteFile(); - } - - private static List convertToRecords(List rows) { - List records = Lists.newArrayList(); - for (RowData row : rows) { - Integer id = row.isNullAt(0) ? null : row.getInt(0); - String data = row.isNullAt(1) ? null : row.getString(1).toString(); - records.add(createRecord(id, data)); - } - return records; - } - - public static void assertTableRows(String tablePath, List expected, String branch) - throws IOException { - assertTableRecords(tablePath, convertToRecords(expected), branch); - } - - public static void assertTableRows(Table table, List expected) throws IOException { - assertTableRecords(table, convertToRecords(expected), SnapshotRef.MAIN_BRANCH); - } - - public static void assertTableRows(Table table, List expected, String branch) - throws IOException { - assertTableRecords(table, convertToRecords(expected), branch); - } - - /** Get all rows for a table */ - public static List tableRecords(Table table) throws IOException { - table.refresh(); - List records = Lists.newArrayList(); - try (CloseableIterable iterable = IcebergGenerics.read(table).build()) { - for (Record record : iterable) { - records.add(record); - } - } - return records; - } - - public static boolean equalsRecords(List expected, List actual, Schema schema) { - if (expected.size() != actual.size()) { - return false; - } - Types.StructType type = schema.asStruct(); - StructLikeSet expectedSet = StructLikeSet.create(type); - expectedSet.addAll(expected); - StructLikeSet actualSet = StructLikeSet.create(type); - actualSet.addAll(actual); - return expectedSet.equals(actualSet); - } - - public static void assertRecordsEqual(List expected, List actual, Schema schema) { - Assert.assertEquals(expected.size(), actual.size()); - Types.StructType type = schema.asStruct(); - StructLikeSet expectedSet = StructLikeSet.create(type); - expectedSet.addAll(expected); - StructLikeSet actualSet = StructLikeSet.create(type); - actualSet.addAll(actual); - Assert.assertEquals(expectedSet, actualSet); - } - - /** - * Assert table contains the expected list of records after waiting up to the configured {@code - * timeout} - */ - public static void assertTableRecords(Table table, List expected, Duration timeout) { - Awaitility.await("expected list of records should be produced") - .atMost(timeout) - .untilAsserted( - () -> { - equalsRecords(expected, tableRecords(table), table.schema()); - assertRecordsEqual(expected, tableRecords(table), table.schema()); - }); - } - - public static void assertTableRecords(Table table, List expected) throws IOException { - assertTableRecords(table, expected, SnapshotRef.MAIN_BRANCH); - } - - public static void assertTableRecords(Table table, List expected, String branch) - throws IOException { - table.refresh(); - Snapshot snapshot = latestSnapshot(table, branch); - - if (snapshot == null) { - Assert.assertEquals(expected, ImmutableList.of()); - return; - } - - Types.StructType type = table.schema().asStruct(); - StructLikeSet expectedSet = StructLikeSet.create(type); - expectedSet.addAll(expected); - - try (CloseableIterable iterable = - IcebergGenerics.read(table).useSnapshot(snapshot.snapshotId()).build()) { - StructLikeSet actualSet = StructLikeSet.create(type); - - for (Record record : iterable) { - actualSet.add(record); - } - - Assert.assertEquals("Should produce the expected record", expectedSet, actualSet); - } - } - - // Returns the latest snapshot of the given branch in the table - public static Snapshot latestSnapshot(Table table, String branch) { - // For the main branch, currentSnapshot() is used to validate that the API behavior has - // not changed since that was the API used for validation prior to addition of branches. - if (branch.equals(SnapshotRef.MAIN_BRANCH)) { - return table.currentSnapshot(); - } - - return table.snapshot(branch); - } - - public static void assertTableRecords(String tablePath, List expected) - throws IOException { - Preconditions.checkArgument(expected != null, "expected records shouldn't be null"); - assertTableRecords(new HadoopTables().load(tablePath), expected, SnapshotRef.MAIN_BRANCH); - } - - public static void assertTableRecords(String tablePath, List expected, String branch) - throws IOException { - Preconditions.checkArgument(expected != null, "expected records shouldn't be null"); - assertTableRecords(new HadoopTables().load(tablePath), expected, branch); - } - - public static StructLikeSet expectedRowSet(Table table, Record... records) { - StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); - InternalRecordWrapper wrapper = new InternalRecordWrapper(table.schema().asStruct()); - for (Record record : records) { - set.add(wrapper.copyFor(record)); - } - return set; - } - - public static StructLikeSet actualRowSet(Table table, String... columns) throws IOException { - return actualRowSet(table, null, columns); - } - - public static StructLikeSet actualRowSet(Table table, Long snapshotId, String... columns) - throws IOException { - table.refresh(); - StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); - InternalRecordWrapper wrapper = new InternalRecordWrapper(table.schema().asStruct()); - try (CloseableIterable reader = - IcebergGenerics.read(table) - .useSnapshot(snapshotId == null ? table.currentSnapshot().snapshotId() : snapshotId) - .select(columns) - .build()) { - reader.forEach(record -> set.add(wrapper.copyFor(record))); - } - return set; - } - - public static List partitionDataFiles(Table table, Map partitionValues) - throws IOException { - table.refresh(); - Types.StructType partitionType = table.spec().partitionType(); - - Record partitionRecord = GenericRecord.create(partitionType).copy(partitionValues); - StructLikeWrapper expectedWrapper = - StructLikeWrapper.forType(partitionType).set(partitionRecord); - - List dataFiles = Lists.newArrayList(); - try (CloseableIterable fileScanTasks = table.newScan().planFiles()) { - for (FileScanTask scanTask : fileScanTasks) { - StructLikeWrapper wrapper = - StructLikeWrapper.forType(partitionType).set(scanTask.file().partition()); - - if (expectedWrapper.equals(wrapper)) { - dataFiles.add(scanTask.file()); - } - } - } - - return dataFiles; - } - - public static Map> snapshotToDataFiles(Table table) throws IOException { - table.refresh(); - - Map> result = Maps.newHashMap(); - Snapshot current = table.currentSnapshot(); - while (current != null) { - TableScan tableScan = table.newScan(); - if (current.parentId() != null) { - // Collect the data files that was added only in current snapshot. - tableScan = tableScan.appendsBetween(current.parentId(), current.snapshotId()); - } else { - // Collect the data files that was added in the oldest snapshot. - tableScan = tableScan.useSnapshot(current.snapshotId()); - } - try (CloseableIterable scanTasks = tableScan.planFiles()) { - result.put( - current.snapshotId(), - ImmutableList.copyOf(Iterables.transform(scanTasks, FileScanTask::file))); - } - - // Continue to traverse the parent snapshot if exists. - if (current.parentId() == null) { - break; - } - // Iterate to the parent snapshot. - current = table.snapshot(current.parentId()); - } - return result; - } - - public static List matchingPartitions( - List dataFiles, PartitionSpec partitionSpec, Map partitionValues) { - Types.StructType partitionType = partitionSpec.partitionType(); - Record partitionRecord = GenericRecord.create(partitionType).copy(partitionValues); - StructLikeWrapper expected = StructLikeWrapper.forType(partitionType).set(partitionRecord); - return dataFiles.stream() - .filter( - df -> { - StructLikeWrapper wrapper = - StructLikeWrapper.forType(partitionType).set(df.partition()); - return wrapper.equals(expected); - }) - .collect(Collectors.toList()); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestBase.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestBase.java deleted file mode 100644 index 4fc0207f269e..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestBase.java +++ /dev/null @@ -1,130 +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.iceberg.flink; - -import java.nio.file.Path; -import java.util.List; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.TableResult; -import org.apache.flink.test.junit5.MiniClusterExtension; -import org.apache.flink.test.util.TestBaseUtils; -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.iceberg.CatalogUtil; -import org.apache.iceberg.hive.HiveCatalog; -import org.apache.iceberg.hive.TestHiveMetastore; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.extension.RegisterExtension; -import org.junit.jupiter.api.io.TempDir; - -public abstract class TestBase extends TestBaseUtils { - - @RegisterExtension - public static MiniClusterExtension miniClusterResource = - MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); - - @TempDir Path temporaryDirectory; - - private static TestHiveMetastore metastore = null; - protected static HiveConf hiveConf = null; - protected static HiveCatalog catalog = null; - - private volatile TableEnvironment tEnv = null; - - @BeforeAll - public static void startMetastore() { - TestBase.metastore = new TestHiveMetastore(); - metastore.start(); - TestBase.hiveConf = metastore.hiveConf(); - TestBase.catalog = - (HiveCatalog) - CatalogUtil.loadCatalog( - HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf); - } - - @AfterAll - public static void stopMetastore() throws Exception { - metastore.stop(); - TestBase.catalog = null; - } - - protected TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - if (tEnv == null) { - EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build(); - - TableEnvironment env = TableEnvironment.create(settings); - env.getConfig() - .getConfiguration() - .set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); - tEnv = env; - } - } - } - return tEnv; - } - - protected static TableResult exec(TableEnvironment env, String query, Object... args) { - return env.executeSql(String.format(query, args)); - } - - protected TableResult exec(String query, Object... args) { - return exec(getTableEnv(), query, args); - } - - protected List sql(String query, Object... args) { - TableResult tableResult = exec(query, args); - try (CloseableIterator iter = tableResult.collect()) { - return Lists.newArrayList(iter); - } catch (Exception e) { - throw new RuntimeException("Failed to collect table result", e); - } - } - - protected void assertSameElements(Iterable expected, Iterable actual) { - Assertions.assertThat(actual).isNotNull().containsExactlyInAnyOrderElementsOf(expected); - } - - protected void assertSameElements(String message, Iterable expected, Iterable actual) { - Assertions.assertThat(actual) - .isNotNull() - .as(message) - .containsExactlyInAnyOrderElementsOf(expected); - } - - /** - * We can not drop currently used catalog after FLINK-29677, so we have make sure that we do not - * use the current catalog before dropping it. This method switches to the 'default_catalog' and - * drops the one requested. - * - * @param catalogName The catalog to drop - * @param ifExists If we should use the 'IF EXISTS' when dropping the catalog - */ - protected void dropCatalog(String catalogName, boolean ifExists) { - sql("USE CATALOG default_catalog"); - sql("DROP CATALOG %s %s", ifExists ? "IF EXISTS" : "", catalogName); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java deleted file mode 100644 index 384ac5c52d00..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java +++ /dev/null @@ -1,118 +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.iceberg.flink; - -import static org.apache.iceberg.CatalogProperties.URI; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.File; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import java.util.Map; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.hadoop.HadoopFileIO; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; - -/** Test for {@link CatalogLoader}. */ -public class TestCatalogLoader extends FlinkTestBase { - - private static File warehouse = null; - private static final TableIdentifier IDENTIFIER = TableIdentifier.of("default", "my_table"); - private static final Schema SCHEMA = - new Schema(Types.NestedField.required(1, "f1", Types.StringType.get())); - - @BeforeClass - public static void createWarehouse() throws IOException { - warehouse = File.createTempFile("warehouse", null); - Assert.assertTrue(warehouse.delete()); - hiveConf.set("my_key", "my_value"); - } - - @AfterClass - public static void dropWarehouse() throws IOException { - if (warehouse != null && warehouse.exists()) { - Path warehousePath = new Path(warehouse.getAbsolutePath()); - FileSystem fs = warehousePath.getFileSystem(hiveConf); - Assert.assertTrue("Failed to delete " + warehousePath, fs.delete(warehousePath, true)); - } - } - - @Test - public void testHadoopCatalogLoader() throws IOException, ClassNotFoundException { - Map properties = Maps.newHashMap(); - properties.put(CatalogProperties.WAREHOUSE_LOCATION, "file:" + warehouse); - CatalogLoader loader = CatalogLoader.hadoop("my_catalog", hiveConf, properties); - validateCatalogLoader(loader); - } - - @Test - public void testHiveCatalogLoader() throws IOException, ClassNotFoundException { - CatalogLoader loader = CatalogLoader.hive("my_catalog", hiveConf, Maps.newHashMap()); - validateCatalogLoader(loader); - } - - @Test - public void testRESTCatalogLoader() { - Map properties = Maps.newHashMap(); - properties.put(URI, "http://localhost/"); - CatalogLoader.rest("my_catalog", hiveConf, Maps.newHashMap()); - } - - private static void validateCatalogLoader(CatalogLoader loader) - throws IOException, ClassNotFoundException { - Table table = javaSerAndDeSer(loader).loadCatalog().createTable(IDENTIFIER, SCHEMA); - validateHadoopConf(table); - } - - private static void validateHadoopConf(Table table) { - FileIO io = table.io(); - Assertions.assertThat(io) - .as("FileIO should be a HadoopFileIO") - .isInstanceOf(HadoopFileIO.class); - HadoopFileIO hadoopIO = (HadoopFileIO) io; - Assert.assertEquals("my_value", hadoopIO.conf().get("my_key")); - } - - @SuppressWarnings("unchecked") - private static T javaSerAndDeSer(T object) throws IOException, ClassNotFoundException { - ByteArrayOutputStream bytes = new ByteArrayOutputStream(); - try (ObjectOutputStream out = new ObjectOutputStream(bytes)) { - out.writeObject(object); - } - - try (ObjectInputStream in = - new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { - return (T) in.readObject(); - } - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java deleted file mode 100644 index 147d2a173ddc..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java +++ /dev/null @@ -1,116 +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.iceberg.flink; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.File; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.hadoop.HadoopFileIO; -import org.apache.iceberg.hadoop.HadoopTables; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; - -/** Test for {@link TableLoader}. */ -public class TestCatalogTableLoader extends TestBase { - - private static File warehouse = null; - private static final TableIdentifier IDENTIFIER = TableIdentifier.of("default", "my_table"); - private static final Schema SCHEMA = - new Schema(Types.NestedField.required(1, "f1", Types.StringType.get())); - - @BeforeAll - public static void createWarehouse() throws IOException { - warehouse = File.createTempFile("warehouse", null); - Assertions.assertThat(warehouse.delete()).isTrue(); - hiveConf.set("my_key", "my_value"); - } - - @AfterAll - public static void dropWarehouse() throws IOException { - if (warehouse != null && warehouse.exists()) { - Path warehousePath = new Path(warehouse.getAbsolutePath()); - FileSystem fs = warehousePath.getFileSystem(hiveConf); - Assertions.assertThat(fs.delete(warehousePath, true)) - .as("Failed to delete " + warehousePath) - .isTrue(); - } - } - - @Test - public void testHadoopTableLoader() throws IOException, ClassNotFoundException { - String location = "file:" + warehouse + "/my_table"; - new HadoopTables(hiveConf).create(SCHEMA, location); - validateTableLoader(TableLoader.fromHadoopTable(location, hiveConf)); - } - - @Test - public void testHiveCatalogTableLoader() throws IOException, ClassNotFoundException { - CatalogLoader loader = CatalogLoader.hive("my_catalog", hiveConf, Maps.newHashMap()); - javaSerdes(loader).loadCatalog().createTable(IDENTIFIER, SCHEMA); - - CatalogLoader catalogLoader = CatalogLoader.hive("my_catalog", hiveConf, Maps.newHashMap()); - validateTableLoader(TableLoader.fromCatalog(catalogLoader, IDENTIFIER)); - } - - private static void validateTableLoader(TableLoader loader) - throws IOException, ClassNotFoundException { - TableLoader copied = javaSerdes(loader); - copied.open(); - try { - validateHadoopConf(copied.loadTable()); - } finally { - copied.close(); - } - } - - private static void validateHadoopConf(Table table) { - FileIO io = table.io(); - Assertions.assertThat(io) - .as("FileIO should be a HadoopFileIO") - .isInstanceOf(HadoopFileIO.class); - HadoopFileIO hadoopIO = (HadoopFileIO) io; - Assertions.assertThat(hadoopIO.conf().get("my_key")).isEqualTo("my_value"); - } - - @SuppressWarnings("unchecked") - private static T javaSerdes(T object) throws IOException, ClassNotFoundException { - ByteArrayOutputStream bytes = new ByteArrayOutputStream(); - try (ObjectOutputStream out = new ObjectOutputStream(bytes)) { - out.writeObject(object); - } - - try (ObjectInputStream in = - new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { - return (T) in.readObject(); - } - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java deleted file mode 100644 index 8e9066e391c9..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java +++ /dev/null @@ -1,305 +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.iceberg.flink; - -import java.io.File; -import java.io.IOException; -import java.util.List; -import org.apache.flink.types.Row; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.BaseTable; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableMetadata; -import org.apache.iceberg.TableOperations; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.source.BoundedTableFactory; -import org.apache.iceberg.flink.source.ChangeLogTableTestBase; -import org.apache.iceberg.relocated.com.google.common.base.Joiner; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.util.StructLikeSet; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -/** - * In this test case, we mainly cover the impact of primary key selection, multiple operations - * within a single transaction, and multiple operations between different txn on the correctness of - * the data. - */ -@RunWith(Parameterized.class) -public class TestChangeLogTable extends ChangeLogTableTestBase { - private static final Configuration CONF = new Configuration(); - private static final String SOURCE_TABLE = "default_catalog.default_database.source_change_logs"; - - private static final String CATALOG_NAME = "test_catalog"; - private static final String DATABASE_NAME = "test_db"; - private static final String TABLE_NAME = "test_table"; - private static String warehouse; - - private final boolean partitioned; - - @Parameterized.Parameters(name = "PartitionedTable={0}") - public static Iterable parameters() { - return ImmutableList.of(new Object[] {true}, new Object[] {false}); - } - - public TestChangeLogTable(boolean partitioned) { - this.partitioned = partitioned; - } - - @BeforeClass - public static void createWarehouse() throws IOException { - File warehouseFile = TEMPORARY_FOLDER.newFolder(); - Assert.assertTrue("The warehouse should be deleted", warehouseFile.delete()); - warehouse = String.format("file:%s", warehouseFile); - } - - @Before - public void before() { - sql( - "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - CATALOG_NAME, warehouse); - sql("USE CATALOG %s", CATALOG_NAME); - sql("CREATE DATABASE %s", DATABASE_NAME); - sql("USE %s", DATABASE_NAME); - // Set the table.exec.sink.upsert-materialize=NONE, so that downstream operators will receive - // the - // records with the same order as the source operator, bypassing Flink's inferred shuffle. - getTableEnv().getConfig().set("table.exec.sink.upsert-materialize", "NONE"); - } - - @After - @Override - public void clean() { - sql("DROP TABLE IF EXISTS %s", TABLE_NAME); - sql("DROP DATABASE IF EXISTS %s", DATABASE_NAME); - dropCatalog(CATALOG_NAME, true); - BoundedTableFactory.clearDataSets(); - } - - @Test - public void testSqlChangeLogOnIdKey() throws Exception { - List> inputRowsPerCheckpoint = - ImmutableList.of( - ImmutableList.of( - insertRow(1, "aaa"), - deleteRow(1, "aaa"), - insertRow(1, "bbb"), - insertRow(2, "aaa"), - deleteRow(2, "aaa"), - insertRow(2, "bbb")), - ImmutableList.of( - updateBeforeRow(2, "bbb"), - updateAfterRow(2, "ccc"), - deleteRow(2, "ccc"), - insertRow(2, "ddd")), - ImmutableList.of( - deleteRow(1, "bbb"), - insertRow(1, "ccc"), - deleteRow(1, "ccc"), - insertRow(1, "ddd"))); - - List> expectedRecordsPerCheckpoint = - ImmutableList.of( - ImmutableList.of(insertRow(1, "bbb"), insertRow(2, "bbb")), - ImmutableList.of(insertRow(1, "bbb"), insertRow(2, "ddd")), - ImmutableList.of(insertRow(1, "ddd"), insertRow(2, "ddd"))); - - testSqlChangeLog( - TABLE_NAME, ImmutableList.of("id"), inputRowsPerCheckpoint, expectedRecordsPerCheckpoint); - } - - @Test - public void testChangeLogOnDataKey() throws Exception { - List> elementsPerCheckpoint = - ImmutableList.of( - ImmutableList.of( - insertRow(1, "aaa"), - deleteRow(1, "aaa"), - insertRow(2, "bbb"), - insertRow(1, "bbb"), - insertRow(2, "aaa")), - ImmutableList.of( - updateBeforeRow(2, "aaa"), updateAfterRow(1, "ccc"), insertRow(1, "aaa")), - ImmutableList.of(deleteRow(1, "bbb"), insertRow(2, "aaa"), insertRow(2, "ccc"))); - - List> expectedRecords = - ImmutableList.of( - ImmutableList.of(insertRow(1, "bbb"), insertRow(2, "aaa")), - ImmutableList.of(insertRow(1, "aaa"), insertRow(1, "bbb"), insertRow(1, "ccc")), - ImmutableList.of( - insertRow(1, "aaa"), - insertRow(1, "ccc"), - insertRow(2, "aaa"), - insertRow(2, "ccc"))); - - testSqlChangeLog(TABLE_NAME, ImmutableList.of("data"), elementsPerCheckpoint, expectedRecords); - } - - @Test - public void testChangeLogOnIdDataKey() throws Exception { - List> elementsPerCheckpoint = - ImmutableList.of( - ImmutableList.of( - insertRow(1, "aaa"), - deleteRow(1, "aaa"), - insertRow(2, "bbb"), - insertRow(1, "bbb"), - insertRow(2, "aaa")), - ImmutableList.of( - updateBeforeRow(2, "aaa"), updateAfterRow(1, "ccc"), insertRow(1, "aaa")), - ImmutableList.of(deleteRow(1, "bbb"), insertRow(2, "aaa"))); - - List> expectedRecords = - ImmutableList.of( - ImmutableList.of(insertRow(1, "bbb"), insertRow(2, "aaa"), insertRow(2, "bbb")), - ImmutableList.of( - insertRow(1, "aaa"), insertRow(1, "bbb"), insertRow(1, "ccc"), insertRow(2, "bbb")), - ImmutableList.of( - insertRow(1, "aaa"), - insertRow(1, "ccc"), - insertRow(2, "aaa"), - insertRow(2, "bbb"))); - - testSqlChangeLog( - TABLE_NAME, ImmutableList.of("data", "id"), elementsPerCheckpoint, expectedRecords); - } - - @Test - public void testPureInsertOnIdKey() throws Exception { - List> elementsPerCheckpoint = - ImmutableList.of( - ImmutableList.of(insertRow(1, "aaa"), insertRow(2, "bbb")), - ImmutableList.of(insertRow(3, "ccc"), insertRow(4, "ddd")), - ImmutableList.of(insertRow(5, "eee"), insertRow(6, "fff"))); - - List> expectedRecords = - ImmutableList.of( - ImmutableList.of(insertRow(1, "aaa"), insertRow(2, "bbb")), - ImmutableList.of( - insertRow(1, "aaa"), insertRow(2, "bbb"), insertRow(3, "ccc"), insertRow(4, "ddd")), - ImmutableList.of( - insertRow(1, "aaa"), - insertRow(2, "bbb"), - insertRow(3, "ccc"), - insertRow(4, "ddd"), - insertRow(5, "eee"), - insertRow(6, "fff"))); - - testSqlChangeLog(TABLE_NAME, ImmutableList.of("data"), elementsPerCheckpoint, expectedRecords); - } - - private static Record record(int id, String data) { - return SimpleDataUtil.createRecord(id, data); - } - - private Table createTable(String tableName, List key, boolean isPartitioned) { - String partitionByCause = isPartitioned ? "PARTITIONED BY (data)" : ""; - sql( - "CREATE TABLE %s(id INT, data VARCHAR, PRIMARY KEY(%s) NOT ENFORCED) %s", - tableName, Joiner.on(',').join(key), partitionByCause); - - // Upgrade the iceberg table to format v2. - CatalogLoader loader = - CatalogLoader.hadoop( - "my_catalog", CONF, ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouse)); - Table table = loader.loadCatalog().loadTable(TableIdentifier.of(DATABASE_NAME, TABLE_NAME)); - TableOperations ops = ((BaseTable) table).operations(); - TableMetadata meta = ops.current(); - ops.commit(meta, meta.upgradeToFormatVersion(2)); - - return table; - } - - private void testSqlChangeLog( - String tableName, - List key, - List> inputRowsPerCheckpoint, - List> expectedRecordsPerCheckpoint) - throws Exception { - String dataId = BoundedTableFactory.registerDataSet(inputRowsPerCheckpoint); - sql( - "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" - + " WITH ('connector'='BoundedSource', 'data-id'='%s')", - SOURCE_TABLE, dataId); - - Assert.assertEquals( - "Should have the expected rows", - listJoin(inputRowsPerCheckpoint), - sql("SELECT * FROM %s", SOURCE_TABLE)); - - Table table = createTable(tableName, key, partitioned); - sql("INSERT INTO %s SELECT * FROM %s", tableName, SOURCE_TABLE); - - table.refresh(); - List snapshots = findValidSnapshots(table); - int expectedSnapshotNum = expectedRecordsPerCheckpoint.size(); - Assert.assertEquals( - "Should have the expected snapshot number", expectedSnapshotNum, snapshots.size()); - - for (int i = 0; i < expectedSnapshotNum; i++) { - long snapshotId = snapshots.get(i).snapshotId(); - List expectedRows = expectedRecordsPerCheckpoint.get(i); - Assert.assertEquals( - "Should have the expected records for the checkpoint#" + i, - expectedRowSet(table, expectedRows), - actualRowSet(table, snapshotId)); - } - - if (expectedSnapshotNum > 0) { - Assert.assertEquals( - "Should have the expected rows in the final table", - Sets.newHashSet(expectedRecordsPerCheckpoint.get(expectedSnapshotNum - 1)), - Sets.newHashSet(sql("SELECT * FROM %s", tableName))); - } - } - - private List findValidSnapshots(Table table) { - List validSnapshots = Lists.newArrayList(); - for (Snapshot snapshot : table.snapshots()) { - if (snapshot.allManifests(table.io()).stream() - .anyMatch(m -> snapshot.snapshotId() == m.snapshotId())) { - validSnapshots.add(snapshot); - } - } - return validSnapshots; - } - - private static StructLikeSet expectedRowSet(Table table, List rows) { - Record[] records = new Record[rows.size()]; - for (int i = 0; i < records.length; i++) { - records[i] = record((int) rows.get(i).getField(0), (String) rows.get(i).getField(1)); - } - return SimpleDataUtil.expectedRowSet(table, records); - } - - private static StructLikeSet actualRowSet(Table table, long snapshotId) throws IOException { - return SimpleDataUtil.actualRowSet(table, snapshotId, "*"); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java deleted file mode 100644 index e9372adda4c1..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java +++ /dev/null @@ -1,207 +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.iceberg.flink; - -import static org.apache.iceberg.types.Types.NestedField.optional; -import static org.apache.iceberg.types.Types.NestedField.required; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import java.nio.ByteBuffer; -import java.nio.ByteOrder; -import java.util.Map; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; -import org.apache.flink.core.memory.DataInputDeserializer; -import org.apache.flink.core.memory.DataOutputSerializer; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFiles; -import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.FileMetadata; -import org.apache.iceberg.Metrics; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Test; - -public class TestDataFileSerialization { - - private static final Schema DATE_SCHEMA = - new Schema( - required(1, "id", Types.LongType.get()), - optional(2, "data", Types.StringType.get()), - required(3, "date", Types.StringType.get()), - optional(4, "double", Types.DoubleType.get())); - - private static final PartitionSpec PARTITION_SPEC = - PartitionSpec.builderFor(DATE_SCHEMA).identity("date").build(); - - private static final Map COLUMN_SIZES = Maps.newHashMap(); - private static final Map VALUE_COUNTS = Maps.newHashMap(); - private static final Map NULL_VALUE_COUNTS = Maps.newHashMap(); - private static final Map NAN_VALUE_COUNTS = Maps.newHashMap(); - private static final Map LOWER_BOUNDS = Maps.newHashMap(); - private static final Map UPPER_BOUNDS = Maps.newHashMap(); - - static { - COLUMN_SIZES.put(1, 2L); - COLUMN_SIZES.put(2, 3L); - VALUE_COUNTS.put(1, 5L); - VALUE_COUNTS.put(2, 3L); - VALUE_COUNTS.put(4, 2L); - NULL_VALUE_COUNTS.put(1, 0L); - NULL_VALUE_COUNTS.put(2, 2L); - NAN_VALUE_COUNTS.put(4, 1L); - LOWER_BOUNDS.put(1, longToBuffer(0L)); - UPPER_BOUNDS.put(1, longToBuffer(4L)); - } - - private static final Metrics METRICS = - new Metrics( - 5L, null, VALUE_COUNTS, NULL_VALUE_COUNTS, NAN_VALUE_COUNTS, LOWER_BOUNDS, UPPER_BOUNDS); - - private static final DataFile DATA_FILE = - DataFiles.builder(PARTITION_SPEC) - .withPath("/path/to/data-1.parquet") - .withFileSizeInBytes(1234) - .withPartitionPath("date=2018-06-08") - .withMetrics(METRICS) - .withSplitOffsets(ImmutableList.of(4L)) - .withEncryptionKeyMetadata(ByteBuffer.allocate(4).putInt(34)) - .withSortOrder(SortOrder.unsorted()) - .build(); - - private static final DeleteFile POS_DELETE_FILE = - FileMetadata.deleteFileBuilder(PARTITION_SPEC) - .ofPositionDeletes() - .withPath("/path/to/pos-delete.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("date=2018-06-08") - .withMetrics(METRICS) - .withEncryptionKeyMetadata(ByteBuffer.allocate(4).putInt(35)) - .withRecordCount(23) - .build(); - - private static final DeleteFile EQ_DELETE_FILE = - FileMetadata.deleteFileBuilder(PARTITION_SPEC) - .ofEqualityDeletes(2, 3) - .withPath("/path/to/equality-delete.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("date=2018-06-08") - .withMetrics(METRICS) - .withEncryptionKeyMetadata(ByteBuffer.allocate(4).putInt(35)) - .withRecordCount(23) - .withSortOrder(SortOrder.unsorted()) - .build(); - - @Test - public void testJavaSerialization() throws Exception { - ByteArrayOutputStream bytes = new ByteArrayOutputStream(); - try (ObjectOutputStream out = new ObjectOutputStream(bytes)) { - out.writeObject(DATA_FILE); - out.writeObject(DATA_FILE.copy()); - - out.writeObject(POS_DELETE_FILE); - out.writeObject(POS_DELETE_FILE.copy()); - - out.writeObject(EQ_DELETE_FILE); - out.writeObject(EQ_DELETE_FILE.copy()); - } - - try (ObjectInputStream in = - new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { - for (int i = 0; i < 2; i += 1) { - Object obj = in.readObject(); - Assertions.assertThat(obj).as("Should be a DataFile").isInstanceOf(DataFile.class); - TestHelpers.assertEquals(DATA_FILE, (DataFile) obj); - } - - for (int i = 0; i < 2; i += 1) { - Object obj = in.readObject(); - Assertions.assertThat(obj) - .as("Should be a position DeleteFile") - .isInstanceOf(DeleteFile.class); - TestHelpers.assertEquals(POS_DELETE_FILE, (DeleteFile) obj); - } - - for (int i = 0; i < 2; i += 1) { - Object obj = in.readObject(); - Assertions.assertThat(obj) - .as("Should be a equality DeleteFile") - .isInstanceOf(DeleteFile.class); - TestHelpers.assertEquals(EQ_DELETE_FILE, (DeleteFile) obj); - } - } - } - - @Test - public void testDataFileKryoSerialization() throws IOException { - KryoSerializer kryo = new KryoSerializer<>(DataFile.class, new ExecutionConfig()); - - DataOutputSerializer outputView = new DataOutputSerializer(1024); - - kryo.serialize(DATA_FILE, outputView); - kryo.serialize(DATA_FILE.copy(), outputView); - - DataInputDeserializer inputView = new DataInputDeserializer(outputView.getCopyOfBuffer()); - DataFile dataFile1 = kryo.deserialize(inputView); - DataFile dataFile2 = kryo.deserialize(inputView); - - TestHelpers.assertEquals(DATA_FILE, dataFile1); - TestHelpers.assertEquals(DATA_FILE, dataFile2); - } - - @Test - public void testDeleteFileKryoSerialization() throws IOException { - KryoSerializer kryo = new KryoSerializer<>(DeleteFile.class, new ExecutionConfig()); - - DataOutputSerializer outputView = new DataOutputSerializer(1024); - - kryo.serialize(POS_DELETE_FILE, outputView); - kryo.serialize(POS_DELETE_FILE.copy(), outputView); - - kryo.serialize(EQ_DELETE_FILE, outputView); - kryo.serialize(EQ_DELETE_FILE.copy(), outputView); - - DataInputDeserializer inputView = new DataInputDeserializer(outputView.getCopyOfBuffer()); - - DeleteFile posDeleteFile1 = kryo.deserialize(inputView); - DeleteFile posDeleteFile2 = kryo.deserialize(inputView); - - TestHelpers.assertEquals(POS_DELETE_FILE, posDeleteFile1); - TestHelpers.assertEquals(POS_DELETE_FILE, posDeleteFile2); - - DeleteFile eqDeleteFile1 = kryo.deserialize(inputView); - DeleteFile eqDeleteFile2 = kryo.deserialize(inputView); - - TestHelpers.assertEquals(EQ_DELETE_FILE, eqDeleteFile1); - TestHelpers.assertEquals(EQ_DELETE_FILE, eqDeleteFile2); - } - - private static ByteBuffer longToBuffer(long value) { - return ByteBuffer.allocate(8).order(ByteOrder.LITTLE_ENDIAN).putLong(0, value); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java deleted file mode 100644 index 884ea2d1d3b1..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java +++ /dev/null @@ -1,59 +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.iceberg.flink; - -import static org.apache.iceberg.types.Types.NestedField.required; - -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.types.Types; - -public class TestFixtures { - - private TestFixtures() {} - - public static final Schema SCHEMA = - new Schema( - required(1, "data", Types.StringType.get()), - required(2, "id", Types.LongType.get()), - required(3, "dt", Types.StringType.get())); - - public static final PartitionSpec SPEC = - PartitionSpec.builderFor(SCHEMA).identity("dt").bucket("id", 1).build(); - - public static final RowType ROW_TYPE = FlinkSchemaUtil.convert(SCHEMA); - - public static final String DATABASE = "default"; - public static final String TABLE = "t"; - public static final String SINK_TABLE = "t_sink"; - - public static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DATABASE, TABLE); - - public static final Schema TS_SCHEMA = - new Schema( - required(1, "ts", Types.TimestampType.withoutZone()), - required(2, "str", Types.StringType.get())); - - public static final PartitionSpec TS_SPEC = - PartitionSpec.builderFor(TS_SCHEMA).hour("ts").build(); - - public static final RowType TS_ROW_TYPE = FlinkSchemaUtil.convert(TS_SCHEMA); -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java deleted file mode 100644 index 2fbd7627efab..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java +++ /dev/null @@ -1,64 +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.iceberg.flink; - -import java.io.File; -import java.util.concurrent.TimeUnit; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.Schema; -import org.apache.flink.table.api.Table; -import org.apache.flink.table.api.TableDescriptor; -import org.apache.flink.table.api.TableEnvironment; -import org.assertj.core.api.Assertions; -import org.awaitility.Awaitility; -import org.junit.Test; - -public class TestFlinkAnonymousTable extends FlinkTestBase { - - @Test - public void testWriteAnonymousTable() throws Exception { - File warehouseDir = TEMPORARY_FOLDER.newFolder(); - TableEnvironment tEnv = getTableEnv(); - Table table = - tEnv.from( - TableDescriptor.forConnector("datagen") - .schema(Schema.newBuilder().column("f0", DataTypes.STRING()).build()) - .option("number-of-rows", "3") - .build()); - - TableDescriptor descriptor = - TableDescriptor.forConnector("iceberg") - .schema(Schema.newBuilder().column("f0", DataTypes.STRING()).build()) - .option("catalog-name", "hadoop_test") - .option("catalog-type", "hadoop") - .option("catalog-database", "test_db") - .option("catalog-table", "test") - .option("warehouse", warehouseDir.getAbsolutePath()) - .build(); - - table.insertInto(descriptor).execute(); - Awaitility.await() - .atMost(3, TimeUnit.SECONDS) - .untilAsserted( - () -> - Assertions.assertThat( - warehouseDir.toPath().resolve("test_db").resolve("test").toFile()) - .exists()); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java deleted file mode 100644 index 47b47cb6262d..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java +++ /dev/null @@ -1,293 +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.iceberg.flink; - -import java.io.File; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException; -import org.apache.flink.types.Row; -import org.apache.iceberg.Schema; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Test; - -public class TestFlinkCatalogDatabase extends FlinkCatalogTestBase { - - public TestFlinkCatalogDatabase(String catalogName, Namespace baseNamespace) { - super(catalogName, baseNamespace); - } - - @After - @Override - public void clean() { - sql("DROP TABLE IF EXISTS %s.tl", flinkDatabase); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); - super.clean(); - } - - @Test - public void testCreateNamespace() { - Assert.assertFalse( - "Database should not already exist", - validationNamespaceCatalog.namespaceExists(icebergNamespace)); - - sql("CREATE DATABASE %s", flinkDatabase); - - Assert.assertTrue( - "Database should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace)); - - sql("CREATE DATABASE IF NOT EXISTS %s", flinkDatabase); - Assert.assertTrue( - "Database should still exist", - validationNamespaceCatalog.namespaceExists(icebergNamespace)); - - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); - Assert.assertFalse( - "Database should be dropped", validationNamespaceCatalog.namespaceExists(icebergNamespace)); - - sql("CREATE DATABASE IF NOT EXISTS %s", flinkDatabase); - Assert.assertTrue( - "Database should be created", validationNamespaceCatalog.namespaceExists(icebergNamespace)); - } - - @Test - public void testDropEmptyDatabase() { - Assert.assertFalse( - "Namespace should not already exist", - validationNamespaceCatalog.namespaceExists(icebergNamespace)); - - sql("CREATE DATABASE %s", flinkDatabase); - - Assert.assertTrue( - "Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace)); - - sql("DROP DATABASE %s", flinkDatabase); - - Assert.assertFalse( - "Namespace should have been dropped", - validationNamespaceCatalog.namespaceExists(icebergNamespace)); - } - - @Test - public void testDropNonEmptyNamespace() { - Assume.assumeFalse( - "Hadoop catalog throws IOException: Directory is not empty.", isHadoopCatalog); - - Assert.assertFalse( - "Namespace should not already exist", - validationNamespaceCatalog.namespaceExists(icebergNamespace)); - - sql("CREATE DATABASE %s", flinkDatabase); - - validationCatalog.createTable( - TableIdentifier.of(icebergNamespace, "tl"), - new Schema(Types.NestedField.optional(0, "id", Types.LongType.get()))); - - Assert.assertTrue( - "Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace)); - Assert.assertTrue( - "Table should exist", - validationCatalog.tableExists(TableIdentifier.of(icebergNamespace, "tl"))); - - Assertions.assertThatThrownBy(() -> sql("DROP DATABASE %s", flinkDatabase)) - .cause() - .isInstanceOf(DatabaseNotEmptyException.class) - .hasMessage( - String.format("Database %s in catalog %s is not empty.", DATABASE, catalogName)); - - sql("DROP TABLE %s.tl", flinkDatabase); - } - - @Test - public void testListTables() { - Assert.assertFalse( - "Namespace should not already exist", - validationNamespaceCatalog.namespaceExists(icebergNamespace)); - - sql("CREATE DATABASE %s", flinkDatabase); - sql("USE CATALOG %s", catalogName); - sql("USE %s", DATABASE); - - Assert.assertTrue( - "Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace)); - - Assert.assertEquals("Should not list any tables", 0, sql("SHOW TABLES").size()); - - validationCatalog.createTable( - TableIdentifier.of(icebergNamespace, "tl"), - new Schema(Types.NestedField.optional(0, "id", Types.LongType.get()))); - - List tables = sql("SHOW TABLES"); - Assert.assertEquals("Only 1 table", 1, tables.size()); - Assert.assertEquals("Table name should match", "tl", tables.get(0).getField(0)); - } - - @Test - public void testListNamespace() { - Assert.assertFalse( - "Namespace should not already exist", - validationNamespaceCatalog.namespaceExists(icebergNamespace)); - - sql("CREATE DATABASE %s", flinkDatabase); - sql("USE CATALOG %s", catalogName); - - Assert.assertTrue( - "Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace)); - - List databases = sql("SHOW DATABASES"); - - if (isHadoopCatalog) { - Assert.assertEquals("Should have 1 database", 1, databases.size()); - Assert.assertEquals("Should have db database", "db", databases.get(0).getField(0)); - - if (!baseNamespace.isEmpty()) { - // test namespace not belongs to this catalog - validationNamespaceCatalog.createNamespace( - Namespace.of(baseNamespace.level(0), "UNKNOWN_NAMESPACE")); - databases = sql("SHOW DATABASES"); - Assert.assertEquals("Should have 1 database", 1, databases.size()); - Assert.assertEquals( - "Should have db and default database", "db", databases.get(0).getField(0)); - } - } else { - // If there are multiple classes extends FlinkTestBase, TestHiveMetastore may loose the - // creation for default - // database. See HiveMetaStore.HMSHandler.init. - Assert.assertTrue( - "Should have db database", - databases.stream().anyMatch(d -> Objects.equals(d.getField(0), "db"))); - } - } - - @Test - public void testCreateNamespaceWithMetadata() { - Assume.assumeFalse("HadoopCatalog does not support namespace metadata", isHadoopCatalog); - - Assert.assertFalse( - "Namespace should not already exist", - validationNamespaceCatalog.namespaceExists(icebergNamespace)); - - sql("CREATE DATABASE %s WITH ('prop'='value')", flinkDatabase); - - Assert.assertTrue( - "Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace)); - - Map nsMetadata = - validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace); - - Assert.assertEquals( - "Namespace should have expected prop value", "value", nsMetadata.get("prop")); - } - - @Test - public void testCreateNamespaceWithComment() { - Assume.assumeFalse("HadoopCatalog does not support namespace metadata", isHadoopCatalog); - - Assert.assertFalse( - "Namespace should not already exist", - validationNamespaceCatalog.namespaceExists(icebergNamespace)); - - sql("CREATE DATABASE %s COMMENT 'namespace doc'", flinkDatabase); - - Assert.assertTrue( - "Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace)); - - Map nsMetadata = - validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace); - - Assert.assertEquals( - "Namespace should have expected comment", "namespace doc", nsMetadata.get("comment")); - } - - @Test - public void testCreateNamespaceWithLocation() throws Exception { - Assume.assumeFalse("HadoopCatalog does not support namespace metadata", isHadoopCatalog); - - Assert.assertFalse( - "Namespace should not already exist", - validationNamespaceCatalog.namespaceExists(icebergNamespace)); - - File location = TEMPORARY_FOLDER.newFile(); - Assert.assertTrue(location.delete()); - - sql("CREATE DATABASE %s WITH ('location'='%s')", flinkDatabase, location); - - Assert.assertTrue( - "Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace)); - - Map nsMetadata = - validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace); - - Assert.assertEquals( - "Namespace should have expected location", - "file:" + location.getPath(), - nsMetadata.get("location")); - } - - @Test - public void testSetProperties() { - Assume.assumeFalse("HadoopCatalog does not support namespace metadata", isHadoopCatalog); - - Assert.assertFalse( - "Namespace should not already exist", - validationNamespaceCatalog.namespaceExists(icebergNamespace)); - - sql("CREATE DATABASE %s", flinkDatabase); - - Assert.assertTrue( - "Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace)); - - Map defaultMetadata = - validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace); - Assert.assertFalse( - "Default metadata should not have custom property", defaultMetadata.containsKey("prop")); - - sql("ALTER DATABASE %s SET ('prop'='value')", flinkDatabase); - - Map nsMetadata = - validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace); - - Assert.assertEquals( - "Namespace should have expected prop value", "value", nsMetadata.get("prop")); - } - - @Test - public void testHadoopNotSupportMeta() { - Assume.assumeTrue("HadoopCatalog does not support namespace metadata", isHadoopCatalog); - - Assert.assertFalse( - "Namespace should not already exist", - validationNamespaceCatalog.namespaceExists(icebergNamespace)); - - Assertions.assertThatThrownBy( - () -> sql("CREATE DATABASE %s WITH ('prop'='value')", flinkDatabase)) - .cause() - .isInstanceOf(UnsupportedOperationException.class) - .hasMessage( - String.format( - "Cannot create namespace %s: metadata is not supported", icebergNamespace)); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java deleted file mode 100644 index c3f8bf92e47a..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java +++ /dev/null @@ -1,115 +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.iceberg.flink; - -import java.util.Map; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.hadoop.HadoopCatalog; -import org.apache.iceberg.hive.HiveCatalog; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.assertj.core.api.Assertions; -import org.junit.Before; -import org.junit.Test; - -public class TestFlinkCatalogFactory { - - private Map props; - - @Before - public void before() { - props = Maps.newHashMap(); - props.put("type", "iceberg"); - props.put(CatalogProperties.WAREHOUSE_LOCATION, "/tmp/location"); - } - - @Test - public void testCreateCatalogHive() { - String catalogName = "hiveCatalog"; - props.put( - FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HIVE); - - Catalog catalog = - FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration()) - .loadCatalog(); - - Assertions.assertThat(catalog).isNotNull().isInstanceOf(HiveCatalog.class); - } - - @Test - public void testCreateCatalogHadoop() { - String catalogName = "hadoopCatalog"; - props.put( - FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HADOOP); - - Catalog catalog = - FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration()) - .loadCatalog(); - - Assertions.assertThat(catalog).isNotNull().isInstanceOf(HadoopCatalog.class); - } - - @Test - public void testCreateCatalogCustom() { - String catalogName = "customCatalog"; - props.put(CatalogProperties.CATALOG_IMPL, CustomHadoopCatalog.class.getName()); - - Catalog catalog = - FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration()) - .loadCatalog(); - - Assertions.assertThat(catalog).isNotNull().isInstanceOf(CustomHadoopCatalog.class); - } - - @Test - public void testCreateCatalogCustomWithHiveCatalogTypeSet() { - String catalogName = "customCatalog"; - props.put(CatalogProperties.CATALOG_IMPL, CustomHadoopCatalog.class.getName()); - props.put( - FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HIVE); - Assertions.assertThatThrownBy( - () -> FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration())) - .isInstanceOf(IllegalArgumentException.class) - .hasMessageStartingWith( - "Cannot create catalog customCatalog, both catalog-type and catalog-impl are set"); - } - - @Test - public void testLoadCatalogUnknown() { - String catalogName = "unknownCatalog"; - props.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "fooType"); - Assertions.assertThatThrownBy( - () -> FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration())) - .isInstanceOf(UnsupportedOperationException.class) - .hasMessageStartingWith("Unknown catalog-type: fooType"); - } - - public static class CustomHadoopCatalog extends HadoopCatalog { - - public CustomHadoopCatalog() {} - - public CustomHadoopCatalog(Configuration conf, String warehouseLocation) { - setConf(conf); - initialize( - "custom", ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation)); - } - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java deleted file mode 100644 index 472cf70b44ba..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java +++ /dev/null @@ -1,430 +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.iceberg.flink; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.util.Arrays; -import java.util.Collections; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.stream.Collectors; -import java.util.stream.StreamSupport; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.api.constraints.UniqueConstraint; -import org.apache.flink.table.catalog.CatalogTable; -import org.apache.flink.table.catalog.ObjectPath; -import org.apache.flink.table.catalog.exceptions.TableNotExistException; -import org.apache.iceberg.BaseTable; -import org.apache.iceberg.ContentFile; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFiles; -import org.apache.iceberg.DataOperations; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableOperations; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.exceptions.NoSuchTableException; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Before; -import org.junit.Test; - -public class TestFlinkCatalogTable extends FlinkCatalogTestBase { - - public TestFlinkCatalogTable(String catalogName, Namespace baseNamespace) { - super(catalogName, baseNamespace); - } - - @Override - @Before - public void before() { - super.before(); - sql("CREATE DATABASE %s", flinkDatabase); - sql("USE CATALOG %s", catalogName); - sql("USE %s", DATABASE); - } - - @After - public void cleanNamespaces() { - sql("DROP TABLE IF EXISTS %s.tl", flinkDatabase); - sql("DROP TABLE IF EXISTS %s.tl2", flinkDatabase); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); - super.clean(); - } - - @Test - public void testGetTable() { - sql("CREATE TABLE tl(id BIGINT, strV STRING)"); - - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, "tl")); - Schema iSchema = - new Schema( - Types.NestedField.optional(1, "id", Types.LongType.get()), - Types.NestedField.optional(2, "strV", Types.StringType.get())); - Assert.assertEquals( - "Should load the expected iceberg schema", iSchema.toString(), table.schema().toString()); - } - - @Test - public void testRenameTable() { - Assume.assumeFalse("HadoopCatalog does not support rename table", isHadoopCatalog); - - final Schema tableSchema = - new Schema(Types.NestedField.optional(0, "id", Types.LongType.get())); - validationCatalog.createTable(TableIdentifier.of(icebergNamespace, "tl"), tableSchema); - sql("ALTER TABLE tl RENAME TO tl2"); - Assertions.assertThatThrownBy(() -> getTableEnv().from("tl")) - .isInstanceOf(ValidationException.class) - .hasMessage("Table `tl` was not found."); - Schema actualSchema = FlinkSchemaUtil.convert(getTableEnv().from("tl2").getSchema()); - Assert.assertEquals(tableSchema.asStruct(), actualSchema.asStruct()); - } - - @Test - public void testCreateTable() throws TableNotExistException { - sql("CREATE TABLE tl(id BIGINT)"); - - Table table = table("tl"); - Assert.assertEquals( - new Schema(Types.NestedField.optional(1, "id", Types.LongType.get())).asStruct(), - table.schema().asStruct()); - - CatalogTable catalogTable = catalogTable("tl"); - Assert.assertEquals( - TableSchema.builder().field("id", DataTypes.BIGINT()).build(), catalogTable.getSchema()); - } - - @Test - public void testCreateTableWithPrimaryKey() throws Exception { - sql("CREATE TABLE tl(id BIGINT, data STRING, key STRING PRIMARY KEY NOT ENFORCED)"); - - Table table = table("tl"); - Assert.assertEquals( - "Should have the expected row key.", - Sets.newHashSet(table.schema().findField("key").fieldId()), - table.schema().identifierFieldIds()); - - CatalogTable catalogTable = catalogTable("tl"); - Optional uniqueConstraintOptional = catalogTable.getSchema().getPrimaryKey(); - Assert.assertTrue( - "Should have the expected unique constraint", uniqueConstraintOptional.isPresent()); - Assert.assertEquals( - "Should have the expected columns", - ImmutableList.of("key"), - uniqueConstraintOptional.get().getColumns()); - } - - @Test - public void testCreateTableWithMultiColumnsInPrimaryKey() throws Exception { - sql( - "CREATE TABLE tl(id BIGINT, data STRING, CONSTRAINT pk_constraint PRIMARY KEY(data, id) NOT ENFORCED)"); - - Table table = table("tl"); - Assert.assertEquals( - "Should have the expected RowKey", - Sets.newHashSet( - table.schema().findField("id").fieldId(), table.schema().findField("data").fieldId()), - table.schema().identifierFieldIds()); - - CatalogTable catalogTable = catalogTable("tl"); - Optional uniqueConstraintOptional = catalogTable.getSchema().getPrimaryKey(); - Assert.assertTrue( - "Should have the expected unique constraint", uniqueConstraintOptional.isPresent()); - Assert.assertEquals( - "Should have the expected columns", - ImmutableSet.of("data", "id"), - ImmutableSet.copyOf(uniqueConstraintOptional.get().getColumns())); - } - - @Test - public void testCreateTableIfNotExists() { - sql("CREATE TABLE tl(id BIGINT)"); - - // Assert that table does exist. - assertThat(table("tl")).isNotNull(); - - sql("DROP TABLE tl"); - Assertions.assertThatThrownBy(() -> table("tl")) - .isInstanceOf(NoSuchTableException.class) - .hasMessage("Table does not exist: " + getFullQualifiedTableName("tl")); - - sql("CREATE TABLE IF NOT EXISTS tl(id BIGINT)"); - assertThat(table("tl").properties()).doesNotContainKey("key"); - - table("tl").updateProperties().set("key", "value").commit(); - assertThat(table("tl").properties()).containsEntry("key", "value"); - - sql("CREATE TABLE IF NOT EXISTS tl(id BIGINT)"); - assertThat(table("tl").properties()).containsEntry("key", "value"); - } - - @Test - public void testCreateTableLike() throws TableNotExistException { - sql("CREATE TABLE tl(id BIGINT)"); - sql("CREATE TABLE tl2 LIKE tl"); - - Table table = table("tl2"); - Assert.assertEquals( - new Schema(Types.NestedField.optional(1, "id", Types.LongType.get())).asStruct(), - table.schema().asStruct()); - - CatalogTable catalogTable = catalogTable("tl2"); - Assert.assertEquals( - TableSchema.builder().field("id", DataTypes.BIGINT()).build(), catalogTable.getSchema()); - } - - @Test - public void testCreateTableLocation() { - Assume.assumeFalse( - "HadoopCatalog does not support creating table with location", isHadoopCatalog); - - sql("CREATE TABLE tl(id BIGINT) WITH ('location'='file:///tmp/location')"); - - Table table = table("tl"); - Assert.assertEquals( - new Schema(Types.NestedField.optional(1, "id", Types.LongType.get())).asStruct(), - table.schema().asStruct()); - Assert.assertEquals("file:///tmp/location", table.location()); - } - - @Test - public void testCreatePartitionTable() throws TableNotExistException { - sql("CREATE TABLE tl(id BIGINT, dt STRING) PARTITIONED BY(dt)"); - - Table table = table("tl"); - Assert.assertEquals( - new Schema( - Types.NestedField.optional(1, "id", Types.LongType.get()), - Types.NestedField.optional(2, "dt", Types.StringType.get())) - .asStruct(), - table.schema().asStruct()); - Assert.assertEquals( - PartitionSpec.builderFor(table.schema()).identity("dt").build(), table.spec()); - - CatalogTable catalogTable = catalogTable("tl"); - Assert.assertEquals( - TableSchema.builder() - .field("id", DataTypes.BIGINT()) - .field("dt", DataTypes.STRING()) - .build(), - catalogTable.getSchema()); - Assert.assertEquals(Collections.singletonList("dt"), catalogTable.getPartitionKeys()); - } - - @Test - public void testCreateTableWithFormatV2ThroughTableProperty() throws Exception { - sql("CREATE TABLE tl(id BIGINT) WITH ('format-version'='2')"); - - Table table = table("tl"); - Assert.assertEquals( - "should create table using format v2", - 2, - ((BaseTable) table).operations().current().formatVersion()); - } - - @Test - public void testUpgradeTableWithFormatV2ThroughTableProperty() throws Exception { - sql("CREATE TABLE tl(id BIGINT) WITH ('format-version'='1')"); - - Table table = table("tl"); - TableOperations ops = ((BaseTable) table).operations(); - Assert.assertEquals("should create table using format v1", 1, ops.refresh().formatVersion()); - - sql("ALTER TABLE tl SET('format-version'='2')"); - Assert.assertEquals("should update table to use format v2", 2, ops.refresh().formatVersion()); - } - - @Test - public void testDowngradeTableToFormatV1ThroughTablePropertyFails() throws Exception { - sql("CREATE TABLE tl(id BIGINT) WITH ('format-version'='2')"); - - Table table = table("tl"); - TableOperations ops = ((BaseTable) table).operations(); - Assert.assertEquals("should create table using format v2", 2, ops.refresh().formatVersion()); - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl SET('format-version'='1')")) - .rootCause() - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot downgrade v2 table to v1"); - } - - @Test - public void testLoadTransformPartitionTable() throws TableNotExistException { - Schema schema = new Schema(Types.NestedField.optional(0, "id", Types.LongType.get())); - validationCatalog.createTable( - TableIdentifier.of(icebergNamespace, "tl"), - schema, - PartitionSpec.builderFor(schema).bucket("id", 100).build()); - - CatalogTable catalogTable = catalogTable("tl"); - Assert.assertEquals( - TableSchema.builder().field("id", DataTypes.BIGINT()).build(), catalogTable.getSchema()); - Assert.assertEquals(Collections.emptyList(), catalogTable.getPartitionKeys()); - } - - @Test - public void testAlterTable() throws TableNotExistException { - sql("CREATE TABLE tl(id BIGINT) WITH ('oldK'='oldV')"); - Map properties = Maps.newHashMap(); - properties.put("oldK", "oldV"); - - // new - sql("ALTER TABLE tl SET('newK'='newV')"); - properties.put("newK", "newV"); - assertThat(table("tl").properties()).containsAllEntriesOf(properties); - - // update old - sql("ALTER TABLE tl SET('oldK'='oldV2')"); - properties.put("oldK", "oldV2"); - assertThat(table("tl").properties()).containsAllEntriesOf(properties); - - // remove property - CatalogTable catalogTable = catalogTable("tl"); - properties.remove("oldK"); - getTableEnv() - .getCatalog(getTableEnv().getCurrentCatalog()) - .get() - .alterTable(new ObjectPath(DATABASE, "tl"), catalogTable.copy(properties), false); - assertThat(table("tl").properties()).containsAllEntriesOf(properties); - } - - @Test - public void testAlterTableWithPrimaryKey() throws TableNotExistException { - sql("CREATE TABLE tl(id BIGINT, PRIMARY KEY(id) NOT ENFORCED) WITH ('oldK'='oldV')"); - Map properties = Maps.newHashMap(); - properties.put("oldK", "oldV"); - - // new - sql("ALTER TABLE tl SET('newK'='newV')"); - properties.put("newK", "newV"); - assertThat(table("tl").properties()).containsAllEntriesOf(properties); - - // update old - sql("ALTER TABLE tl SET('oldK'='oldV2')"); - properties.put("oldK", "oldV2"); - assertThat(table("tl").properties()).containsAllEntriesOf(properties); - - // remove property - CatalogTable catalogTable = catalogTable("tl"); - properties.remove("oldK"); - getTableEnv() - .getCatalog(getTableEnv().getCurrentCatalog()) - .get() - .alterTable(new ObjectPath(DATABASE, "tl"), catalogTable.copy(properties), false); - assertThat(table("tl").properties()).containsAllEntriesOf(properties); - } - - @Test - public void testRelocateTable() { - Assume.assumeFalse("HadoopCatalog does not support relocate table", isHadoopCatalog); - - sql("CREATE TABLE tl(id BIGINT)"); - sql("ALTER TABLE tl SET('location'='file:///tmp/location')"); - Assert.assertEquals("file:///tmp/location", table("tl").location()); - } - - @Test - public void testSetCurrentAndCherryPickSnapshotId() { - sql("CREATE TABLE tl(c1 INT, c2 STRING, c3 STRING) PARTITIONED BY (c1)"); - - Table table = table("tl"); - - DataFile fileA = - DataFiles.builder(table.spec()) - .withPath("/path/to/data-a.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("c1=0") // easy way to set partition data for now - .withRecordCount(1) - .build(); - DataFile fileB = - DataFiles.builder(table.spec()) - .withPath("/path/to/data-b.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("c1=1") // easy way to set partition data for now - .withRecordCount(1) - .build(); - DataFile replacementFile = - DataFiles.builder(table.spec()) - .withPath("/path/to/data-a-replacement.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("c1=0") // easy way to set partition data for now - .withRecordCount(1) - .build(); - - table.newAppend().appendFile(fileA).commit(); - long snapshotId = table.currentSnapshot().snapshotId(); - - // stage an overwrite that replaces FILE_A - table.newReplacePartitions().addFile(replacementFile).stageOnly().commit(); - - Snapshot staged = Iterables.getLast(table.snapshots()); - Assert.assertEquals( - "Should find the staged overwrite snapshot", DataOperations.OVERWRITE, staged.operation()); - - // add another append so that the original commit can't be fast-forwarded - table.newAppend().appendFile(fileB).commit(); - - // test cherry pick - sql("ALTER TABLE tl SET('cherry-pick-snapshot-id'='%s')", staged.snapshotId()); - validateTableFiles(table, fileB, replacementFile); - - // test set current snapshot - sql("ALTER TABLE tl SET('current-snapshot-id'='%s')", snapshotId); - validateTableFiles(table, fileA); - } - - private void validateTableFiles(Table tbl, DataFile... expectedFiles) { - tbl.refresh(); - Set expectedFilePaths = - Arrays.stream(expectedFiles).map(DataFile::path).collect(Collectors.toSet()); - Set actualFilePaths = - StreamSupport.stream(tbl.newScan().planFiles().spliterator(), false) - .map(FileScanTask::file) - .map(ContentFile::path) - .collect(Collectors.toSet()); - Assert.assertEquals("Files should match", expectedFilePaths, actualFilePaths); - } - - private Table table(String name) { - return validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, name)); - } - - private CatalogTable catalogTable(String name) throws TableNotExistException { - return (CatalogTable) - getTableEnv() - .getCatalog(getTableEnv().getCurrentCatalog()) - .get() - .getTable(new ObjectPath(DATABASE, name)); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java deleted file mode 100644 index fad65f4c63c8..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java +++ /dev/null @@ -1,120 +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.iceberg.flink; - -import java.util.List; -import org.apache.flink.table.catalog.CatalogPartitionSpec; -import org.apache.flink.table.catalog.ObjectPath; -import org.apache.flink.table.catalog.exceptions.TableNotExistException; -import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runners.Parameterized; - -public class TestFlinkCatalogTablePartitions extends FlinkCatalogTestBase { - - private String tableName = "test_table"; - - private final FileFormat format; - - @Parameterized.Parameters( - name = "catalogName={0}, baseNamespace={1}, format={2}, cacheEnabled={3}") - public static Iterable parameters() { - List parameters = Lists.newArrayList(); - for (FileFormat format : - new FileFormat[] {FileFormat.ORC, FileFormat.AVRO, FileFormat.PARQUET}) { - for (Boolean cacheEnabled : new Boolean[] {true, false}) { - for (Object[] catalogParams : FlinkCatalogTestBase.parameters()) { - String catalogName = (String) catalogParams[0]; - Namespace baseNamespace = (Namespace) catalogParams[1]; - parameters.add(new Object[] {catalogName, baseNamespace, format, cacheEnabled}); - } - } - } - return parameters; - } - - public TestFlinkCatalogTablePartitions( - String catalogName, Namespace baseNamespace, FileFormat format, boolean cacheEnabled) { - super(catalogName, baseNamespace); - this.format = format; - config.put(CatalogProperties.CACHE_ENABLED, String.valueOf(cacheEnabled)); - } - - @Override - @Before - public void before() { - super.before(); - sql("CREATE DATABASE %s", flinkDatabase); - sql("USE CATALOG %s", catalogName); - sql("USE %s", DATABASE); - } - - @After - public void cleanNamespaces() { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); - super.clean(); - } - - @Test - public void testListPartitionsWithUnpartitionedTable() { - sql( - "CREATE TABLE %s (id INT, data VARCHAR) with ('write.format.default'='%s')", - tableName, format.name()); - sql("INSERT INTO %s SELECT 1,'a'", tableName); - - ObjectPath objectPath = new ObjectPath(DATABASE, tableName); - FlinkCatalog flinkCatalog = (FlinkCatalog) getTableEnv().getCatalog(catalogName).get(); - Assertions.assertThatThrownBy(() -> flinkCatalog.listPartitions(objectPath)) - .isInstanceOf(TableNotPartitionedException.class) - .hasMessage("Table " + objectPath + " in catalog " + catalogName + " is not partitioned."); - } - - @Test - public void testListPartitionsWithPartitionedTable() - throws TableNotExistException, TableNotPartitionedException { - sql( - "CREATE TABLE %s (id INT, data VARCHAR) PARTITIONED BY (data) " - + "with ('write.format.default'='%s')", - tableName, format.name()); - sql("INSERT INTO %s SELECT 1,'a'", tableName); - sql("INSERT INTO %s SELECT 2,'b'", tableName); - - ObjectPath objectPath = new ObjectPath(DATABASE, tableName); - FlinkCatalog flinkCatalog = (FlinkCatalog) getTableEnv().getCatalog(catalogName).get(); - List list = flinkCatalog.listPartitions(objectPath); - Assert.assertEquals("Should have 2 partition", 2, list.size()); - - List expected = Lists.newArrayList(); - CatalogPartitionSpec partitionSpec1 = new CatalogPartitionSpec(ImmutableMap.of("data", "a")); - CatalogPartitionSpec partitionSpec2 = new CatalogPartitionSpec(ImmutableMap.of("data", "b")); - expected.add(partitionSpec1); - expected.add(partitionSpec2); - Assert.assertEquals("Should produce the expected catalog partition specs.", list, expected); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java deleted file mode 100644 index 4b6ac25ab8e3..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java +++ /dev/null @@ -1,61 +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.iceberg.flink; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -import java.time.Duration; -import java.util.Map; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; -import org.apache.flink.configuration.Configuration; -import org.apache.iceberg.Table; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.jupiter.api.Test; - -public class TestFlinkConfParser { - - @Test - public void testDurationConf() { - Map writeOptions = ImmutableMap.of("write-prop", "111s"); - - ConfigOption configOption = - ConfigOptions.key("conf-prop").durationType().noDefaultValue(); - Configuration flinkConf = new Configuration(); - flinkConf.setString(configOption.key(), "222s"); - - Table table = mock(Table.class); - when(table.properties()).thenReturn(ImmutableMap.of("table-prop", "333s")); - - FlinkConfParser confParser = new FlinkConfParser(table, writeOptions, flinkConf); - Duration defaultVal = Duration.ofMillis(999); - - Duration result = - confParser.durationConf().option("write-prop").defaultValue(defaultVal).parse(); - assertThat(result).isEqualTo(Duration.ofSeconds(111)); - - result = confParser.durationConf().flinkConfig(configOption).defaultValue(defaultVal).parse(); - assertThat(result).isEqualTo(Duration.ofSeconds(222)); - - result = confParser.durationConf().tableProperty("table-prop").defaultValue(defaultVal).parse(); - assertThat(result).isEqualTo(Duration.ofSeconds(333)); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java deleted file mode 100644 index c89ea4f53054..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java +++ /dev/null @@ -1,467 +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.iceberg.flink; - -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import java.time.Instant; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.util.List; -import java.util.Optional; -import java.util.stream.Collectors; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.Expressions; -import org.apache.flink.table.api.TableColumn; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.expressions.ApiExpressionUtils; -import org.apache.flink.table.expressions.CallExpression; -import org.apache.flink.table.expressions.Expression; -import org.apache.flink.table.expressions.FieldReferenceExpression; -import org.apache.flink.table.expressions.ResolvedExpression; -import org.apache.flink.table.expressions.UnresolvedCallExpression; -import org.apache.flink.table.expressions.UnresolvedReferenceExpression; -import org.apache.flink.table.expressions.ValueLiteralExpression; -import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor; -import org.apache.flink.table.functions.BuiltInFunctionDefinitions; -import org.apache.iceberg.expressions.And; -import org.apache.iceberg.expressions.BoundLiteralPredicate; -import org.apache.iceberg.expressions.Not; -import org.apache.iceberg.expressions.Or; -import org.apache.iceberg.expressions.UnboundPredicate; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.util.DateTimeUtil; -import org.apache.iceberg.util.Pair; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; - -public class TestFlinkFilters { - - private static final TableSchema TABLE_SCHEMA = - TableSchema.builder() - .field("field1", DataTypes.INT()) - .field("field2", DataTypes.BIGINT()) - .field("field3", DataTypes.FLOAT()) - .field("field4", DataTypes.DOUBLE()) - .field("field5", DataTypes.STRING()) - .field("field6", DataTypes.BOOLEAN()) - .field("field7", DataTypes.BINARY(2)) - .field("field8", DataTypes.DECIMAL(10, 2)) - .field("field9", DataTypes.DATE()) - .field("field10", DataTypes.TIME()) - .field("field11", DataTypes.TIMESTAMP()) - .field("field12", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()) - .build(); - - // A map list of fields and values used to verify the conversion of flink expression to iceberg - // expression - private static final List> FIELD_VALUE_LIST = - ImmutableList.of( - Pair.of("field1", 1), - Pair.of("field2", 2L), - Pair.of("field3", 3F), - Pair.of("field4", 4D), - Pair.of("field5", "iceberg"), - Pair.of("field6", true), - Pair.of("field7", new byte[] {'a', 'b'}), - Pair.of("field8", BigDecimal.valueOf(10.12)), - Pair.of("field9", DateTimeUtil.daysFromDate(LocalDate.now())), - Pair.of("field10", DateTimeUtil.microsFromTime(LocalTime.now())), - Pair.of("field11", DateTimeUtil.microsFromTimestamp(LocalDateTime.now())), - Pair.of("field12", DateTimeUtil.microsFromInstant(Instant.now()))); - - @Test - public void testFlinkDataTypeEqual() { - matchLiteral("field1", 1, 1); - matchLiteral("field2", 10L, 10L); - matchLiteral("field3", 1.2F, 1.2F); - matchLiteral("field4", 3.4D, 3.4D); - matchLiteral("field5", "abcd", "abcd"); - matchLiteral("field6", true, true); - matchLiteral("field7", new byte[] {'a', 'b'}, ByteBuffer.wrap(new byte[] {'a', 'b'})); - matchLiteral("field8", BigDecimal.valueOf(10.12), BigDecimal.valueOf(10.12)); - - LocalDate date = LocalDate.parse("2020-12-23"); - matchLiteral("field9", date, DateTimeUtil.daysFromDate(date)); - - LocalTime time = LocalTime.parse("12:13:14"); - matchLiteral("field10", time, DateTimeUtil.microsFromTime(time)); - - LocalDateTime dateTime = LocalDateTime.parse("2020-12-23T12:13:14"); - matchLiteral("field11", dateTime, DateTimeUtil.microsFromTimestamp(dateTime)); - - Instant instant = Instant.parse("2020-12-23T12:13:14.00Z"); - matchLiteral("field12", instant, DateTimeUtil.microsFromInstant(instant)); - } - - @Test - public void testEquals() { - for (Pair pair : FIELD_VALUE_LIST) { - UnboundPredicate expected = - org.apache.iceberg.expressions.Expressions.equal(pair.first(), pair.second()); - - Optional actual = - FlinkFilters.convert( - resolve(Expressions.$(pair.first()).isEqual(Expressions.lit(pair.second())))); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); - assertPredicatesMatch(expected, actual.get()); - - Optional actual1 = - FlinkFilters.convert( - resolve(Expressions.lit(pair.second()).isEqual(Expressions.$(pair.first())))); - Assert.assertTrue("Conversion should succeed", actual1.isPresent()); - assertPredicatesMatch(expected, actual1.get()); - } - } - - @Test - public void testEqualsNaN() { - UnboundPredicate expected = org.apache.iceberg.expressions.Expressions.isNaN("field3"); - - Optional actual = - FlinkFilters.convert(resolve(Expressions.$("field3").isEqual(Expressions.lit(Float.NaN)))); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); - assertPredicatesMatch(expected, actual.get()); - - Optional actual1 = - FlinkFilters.convert(resolve(Expressions.lit(Float.NaN).isEqual(Expressions.$("field3")))); - Assert.assertTrue("Conversion should succeed", actual1.isPresent()); - assertPredicatesMatch(expected, actual1.get()); - } - - @Test - public void testNotEquals() { - for (Pair pair : FIELD_VALUE_LIST) { - UnboundPredicate expected = - org.apache.iceberg.expressions.Expressions.notEqual(pair.first(), pair.second()); - - Optional actual = - FlinkFilters.convert( - resolve(Expressions.$(pair.first()).isNotEqual(Expressions.lit(pair.second())))); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); - assertPredicatesMatch(expected, actual.get()); - - Optional actual1 = - FlinkFilters.convert( - resolve(Expressions.lit(pair.second()).isNotEqual(Expressions.$(pair.first())))); - Assert.assertTrue("Conversion should succeed", actual1.isPresent()); - assertPredicatesMatch(expected, actual1.get()); - } - } - - @Test - public void testNotEqualsNaN() { - UnboundPredicate expected = org.apache.iceberg.expressions.Expressions.notNaN("field3"); - - Optional actual = - FlinkFilters.convert( - resolve(Expressions.$("field3").isNotEqual(Expressions.lit(Float.NaN)))); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); - assertPredicatesMatch(expected, actual.get()); - - Optional actual1 = - FlinkFilters.convert( - resolve(Expressions.lit(Float.NaN).isNotEqual(Expressions.$("field3")))); - Assert.assertTrue("Conversion should succeed", actual1.isPresent()); - assertPredicatesMatch(expected, actual1.get()); - } - - @Test - public void testGreaterThan() { - UnboundPredicate expected = - org.apache.iceberg.expressions.Expressions.greaterThan("field1", 1); - - Optional actual = - FlinkFilters.convert(resolve(Expressions.$("field1").isGreater(Expressions.lit(1)))); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); - assertPredicatesMatch(expected, actual.get()); - - Optional actual1 = - FlinkFilters.convert(resolve(Expressions.lit(1).isLess(Expressions.$("field1")))); - Assert.assertTrue("Conversion should succeed", actual1.isPresent()); - assertPredicatesMatch(expected, actual1.get()); - } - - @Test - public void testGreaterThanEquals() { - UnboundPredicate expected = - org.apache.iceberg.expressions.Expressions.greaterThanOrEqual("field1", 1); - - Optional actual = - FlinkFilters.convert(resolve(Expressions.$("field1").isGreaterOrEqual(Expressions.lit(1)))); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); - assertPredicatesMatch(expected, actual.get()); - - Optional actual1 = - FlinkFilters.convert(resolve(Expressions.lit(1).isLessOrEqual(Expressions.$("field1")))); - Assert.assertTrue("Conversion should succeed", actual1.isPresent()); - assertPredicatesMatch(expected, actual1.get()); - } - - @Test - public void testLessThan() { - UnboundPredicate expected = - org.apache.iceberg.expressions.Expressions.lessThan("field1", 1); - - Optional actual = - FlinkFilters.convert(resolve(Expressions.$("field1").isLess(Expressions.lit(1)))); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); - assertPredicatesMatch(expected, actual.get()); - - Optional actual1 = - FlinkFilters.convert(resolve(Expressions.lit(1).isGreater(Expressions.$("field1")))); - Assert.assertTrue("Conversion should succeed", actual1.isPresent()); - assertPredicatesMatch(expected, actual1.get()); - } - - @Test - public void testLessThanEquals() { - UnboundPredicate expected = - org.apache.iceberg.expressions.Expressions.lessThanOrEqual("field1", 1); - - Optional actual = - FlinkFilters.convert(resolve(Expressions.$("field1").isLessOrEqual(Expressions.lit(1)))); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); - assertPredicatesMatch(expected, actual.get()); - - Optional actual1 = - FlinkFilters.convert(resolve(Expressions.lit(1).isGreaterOrEqual(Expressions.$("field1")))); - Assert.assertTrue("Conversion should succeed", actual1.isPresent()); - assertPredicatesMatch(expected, actual1.get()); - } - - @Test - public void testIsNull() { - Expression expr = resolve(Expressions.$("field1").isNull()); - Optional actual = FlinkFilters.convert(expr); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); - UnboundPredicate expected = org.apache.iceberg.expressions.Expressions.isNull("field1"); - assertPredicatesMatch(expected, actual.get()); - } - - @Test - public void testIsNotNull() { - Expression expr = resolve(Expressions.$("field1").isNotNull()); - Optional actual = FlinkFilters.convert(expr); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); - UnboundPredicate expected = - org.apache.iceberg.expressions.Expressions.notNull("field1"); - assertPredicatesMatch(expected, actual.get()); - } - - @Test - public void testAnd() { - Expression expr = - resolve( - Expressions.$("field1") - .isEqual(Expressions.lit(1)) - .and(Expressions.$("field2").isEqual(Expressions.lit(2L)))); - Optional actual = FlinkFilters.convert(expr); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); - And and = (And) actual.get(); - And expected = - (And) - org.apache.iceberg.expressions.Expressions.and( - org.apache.iceberg.expressions.Expressions.equal("field1", 1), - org.apache.iceberg.expressions.Expressions.equal("field2", 2L)); - - assertPredicatesMatch(expected.left(), and.left()); - assertPredicatesMatch(expected.right(), and.right()); - } - - @Test - public void testOr() { - Expression expr = - resolve( - Expressions.$("field1") - .isEqual(Expressions.lit(1)) - .or(Expressions.$("field2").isEqual(Expressions.lit(2L)))); - Optional actual = FlinkFilters.convert(expr); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); - Or or = (Or) actual.get(); - Or expected = - (Or) - org.apache.iceberg.expressions.Expressions.or( - org.apache.iceberg.expressions.Expressions.equal("field1", 1), - org.apache.iceberg.expressions.Expressions.equal("field2", 2L)); - - assertPredicatesMatch(expected.left(), or.left()); - assertPredicatesMatch(expected.right(), or.right()); - } - - @Test - public void testNot() { - Expression expr = - resolve( - ApiExpressionUtils.unresolvedCall( - BuiltInFunctionDefinitions.NOT, - Expressions.$("field1").isEqual(Expressions.lit(1)))); - Optional actual = FlinkFilters.convert(expr); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); - Not not = (Not) actual.get(); - Not expected = - (Not) - org.apache.iceberg.expressions.Expressions.not( - org.apache.iceberg.expressions.Expressions.equal("field1", 1)); - - Assert.assertEquals("Predicate operation should match", expected.op(), not.op()); - assertPredicatesMatch(expected.child(), not.child()); - } - - @Test - public void testLike() { - UnboundPredicate expected = - org.apache.iceberg.expressions.Expressions.startsWith("field5", "abc"); - Expression expr = - resolve( - ApiExpressionUtils.unresolvedCall( - BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("abc%"))); - Optional actual = FlinkFilters.convert(expr); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); - assertPredicatesMatch(expected, actual.get()); - - expr = - resolve( - ApiExpressionUtils.unresolvedCall( - BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("%abc"))); - actual = FlinkFilters.convert(expr); - Assert.assertFalse("Conversion should failed", actual.isPresent()); - - expr = - resolve( - ApiExpressionUtils.unresolvedCall( - BuiltInFunctionDefinitions.LIKE, - Expressions.$("field5"), - Expressions.lit("%abc%"))); - actual = FlinkFilters.convert(expr); - Assert.assertFalse("Conversion should failed", actual.isPresent()); - - expr = - resolve( - ApiExpressionUtils.unresolvedCall( - BuiltInFunctionDefinitions.LIKE, - Expressions.$("field5"), - Expressions.lit("abc%d"))); - actual = FlinkFilters.convert(expr); - Assert.assertFalse("Conversion should failed", actual.isPresent()); - - expr = - resolve( - ApiExpressionUtils.unresolvedCall( - BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("%"))); - actual = FlinkFilters.convert(expr); - Assert.assertFalse("Conversion should failed", actual.isPresent()); - - expr = - resolve( - ApiExpressionUtils.unresolvedCall( - BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("a_"))); - actual = FlinkFilters.convert(expr); - Assert.assertFalse("Conversion should failed", actual.isPresent()); - - expr = - resolve( - ApiExpressionUtils.unresolvedCall( - BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("a%b"))); - actual = FlinkFilters.convert(expr); - Assert.assertFalse("Conversion should failed", actual.isPresent()); - } - - @SuppressWarnings("unchecked") - private void matchLiteral(String fieldName, Object flinkLiteral, T icebergLiteral) { - Expression expr = resolve(Expressions.$(fieldName).isEqual(Expressions.lit(flinkLiteral))); - Optional actual = FlinkFilters.convert(expr); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); - org.apache.iceberg.expressions.Expression expression = actual.get(); - Assertions.assertThat(expression) - .as("The expression should be a UnboundPredicate") - .isInstanceOf(UnboundPredicate.class); - UnboundPredicate unboundPredicate = (UnboundPredicate) expression; - - org.apache.iceberg.expressions.Expression expression1 = - unboundPredicate.bind(FlinkSchemaUtil.convert(TABLE_SCHEMA).asStruct(), false); - Assertions.assertThat(expression1) - .as("The expression should be a BoundLiteralPredicate") - .isInstanceOf(BoundLiteralPredicate.class); - - BoundLiteralPredicate predicate = (BoundLiteralPredicate) expression1; - Assert.assertTrue("Should match the literal", predicate.test(icebergLiteral)); - } - - private static Expression resolve(Expression originalExpression) { - return originalExpression.accept( - new ApiExpressionDefaultVisitor() { - @Override - public Expression visit(UnresolvedReferenceExpression unresolvedReference) { - String name = unresolvedReference.getName(); - Optional field = TABLE_SCHEMA.getTableColumn(name); - if (field.isPresent()) { - int index = TABLE_SCHEMA.getTableColumns().indexOf(field.get()); - return new FieldReferenceExpression(name, field.get().getType(), 0, index); - } else { - return null; - } - } - - @Override - public Expression visit(UnresolvedCallExpression unresolvedCall) { - List children = - unresolvedCall.getChildren().stream() - .map(e -> (ResolvedExpression) e.accept(this)) - .collect(Collectors.toList()); - return new CallExpression( - unresolvedCall.getFunctionDefinition(), children, DataTypes.STRING()); - } - - @Override - public Expression visit(ValueLiteralExpression valueLiteral) { - return valueLiteral; - } - - @Override - protected Expression defaultMethod(Expression expression) { - throw new UnsupportedOperationException( - String.format("unsupported expression: %s", expression)); - } - }); - } - - private void assertPredicatesMatch( - org.apache.iceberg.expressions.Expression expected, - org.apache.iceberg.expressions.Expression actual) { - Assertions.assertThat(expected) - .as("The expected expression should be a UnboundPredicate") - .isInstanceOf(UnboundPredicate.class); - Assertions.assertThat(actual) - .as("The actual expression should be a UnboundPredicate") - .isInstanceOf(UnboundPredicate.class); - UnboundPredicate predicateExpected = (UnboundPredicate) expected; - UnboundPredicate predicateActual = (UnboundPredicate) actual; - Assert.assertEquals( - "Predicate operation should match", predicateExpected.op(), predicateActual.op()); - Assert.assertEquals( - "Predicate literal should match", predicateExpected.literal(), predicateActual.literal()); - Assert.assertEquals( - "Predicate name should match", - predicateExpected.ref().name(), - predicateActual.ref().name()); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java deleted file mode 100644 index 8f238587d30d..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java +++ /dev/null @@ -1,105 +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.iceberg.flink; - -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.Map; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class TestFlinkHiveCatalog extends FlinkTestBase { - - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); - - @Test - public void testCreateCatalogWithWarehouseLocation() throws IOException { - Map props = Maps.newHashMap(); - props.put("type", "iceberg"); - props.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hive"); - props.put(CatalogProperties.URI, FlinkCatalogTestBase.getURI(hiveConf)); - - File warehouseDir = tempFolder.newFolder(); - props.put(CatalogProperties.WAREHOUSE_LOCATION, "file://" + warehouseDir.getAbsolutePath()); - - checkSQLQuery(props, warehouseDir); - } - - @Test - public void testCreateCatalogWithHiveConfDir() throws IOException { - // Dump the hive conf into a local file. - File hiveConfDir = tempFolder.newFolder(); - File hiveSiteXML = new File(hiveConfDir, "hive-site.xml"); - File warehouseDir = tempFolder.newFolder(); - try (FileOutputStream fos = new FileOutputStream(hiveSiteXML)) { - Configuration newConf = new Configuration(hiveConf); - // Set another new directory which is different with the hive metastore's warehouse path. - newConf.set( - HiveConf.ConfVars.METASTOREWAREHOUSE.varname, "file://" + warehouseDir.getAbsolutePath()); - newConf.writeXml(fos); - } - Assert.assertTrue("hive-site.xml should be created now.", Files.exists(hiveSiteXML.toPath())); - - // Construct the catalog attributions. - Map props = Maps.newHashMap(); - props.put("type", "iceberg"); - props.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hive"); - props.put(CatalogProperties.URI, FlinkCatalogTestBase.getURI(hiveConf)); - // Set the 'hive-conf-dir' instead of 'warehouse' - props.put(FlinkCatalogFactory.HIVE_CONF_DIR, hiveConfDir.getAbsolutePath()); - - checkSQLQuery(props, warehouseDir); - } - - private void checkSQLQuery(Map catalogProperties, File warehouseDir) - throws IOException { - sql( - "CREATE CATALOG test_catalog WITH %s", - FlinkCatalogTestBase.toWithClause(catalogProperties)); - sql("USE CATALOG test_catalog"); - sql("CREATE DATABASE test_db"); - sql("USE test_db"); - sql("CREATE TABLE test_table(c1 INT, c2 STRING)"); - sql("INSERT INTO test_table SELECT 1, 'a'"); - - Path databasePath = warehouseDir.toPath().resolve("test_db.db"); - Assert.assertTrue("Database path should exist", Files.exists(databasePath)); - - Path tablePath = databasePath.resolve("test_table"); - Assert.assertTrue("Table path should exist", Files.exists(tablePath)); - - Path dataPath = tablePath.resolve("data"); - Assert.assertTrue("Table data path should exist", Files.exists(dataPath)); - Assert.assertEquals( - "Should have a .crc file and a .parquet file", 2, Files.list(dataPath).count()); - - sql("DROP TABLE test_table"); - sql("DROP DATABASE test_db"); - dropCatalog("test_catalog", false); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java deleted file mode 100644 index 16b220ba6715..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java +++ /dev/null @@ -1,415 +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.iceberg.flink; - -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.types.logical.BinaryType; -import org.apache.flink.table.types.logical.CharType; -import org.apache.flink.table.types.logical.LocalZonedTimestampType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.TimeType; -import org.apache.flink.table.types.logical.TimestampType; -import org.apache.flink.table.types.logical.VarBinaryType; -import org.apache.flink.table.types.logical.VarCharType; -import org.apache.iceberg.Schema; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; - -public class TestFlinkSchemaUtil { - - @Test - public void testConvertFlinkSchemaToIcebergSchema() { - TableSchema flinkSchema = - TableSchema.builder() - .field("id", DataTypes.INT().notNull()) - .field("name", DataTypes.STRING()) /* optional by default */ - .field("salary", DataTypes.DOUBLE().notNull()) - .field( - "locations", - DataTypes.MAP( - DataTypes.STRING(), - DataTypes.ROW( - DataTypes.FIELD("posX", DataTypes.DOUBLE().notNull(), "X field"), - DataTypes.FIELD("posY", DataTypes.DOUBLE().notNull(), "Y field")))) - .field("strArray", DataTypes.ARRAY(DataTypes.STRING()).nullable()) - .field("intArray", DataTypes.ARRAY(DataTypes.INT()).nullable()) - .field("char", DataTypes.CHAR(10).notNull()) - .field("varchar", DataTypes.VARCHAR(10).notNull()) - .field("boolean", DataTypes.BOOLEAN().nullable()) - .field("tinyint", DataTypes.TINYINT()) - .field("smallint", DataTypes.SMALLINT()) - .field("bigint", DataTypes.BIGINT()) - .field("varbinary", DataTypes.VARBINARY(10)) - .field("binary", DataTypes.BINARY(10)) - .field("time", DataTypes.TIME()) - .field("timestampWithoutZone", DataTypes.TIMESTAMP()) - .field("timestampWithZone", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()) - .field("date", DataTypes.DATE()) - .field("decimal", DataTypes.DECIMAL(2, 2)) - .field("decimal2", DataTypes.DECIMAL(38, 2)) - .field("decimal3", DataTypes.DECIMAL(10, 1)) - .field("multiset", DataTypes.MULTISET(DataTypes.STRING().notNull())) - .build(); - - Schema icebergSchema = - new Schema( - Types.NestedField.required(0, "id", Types.IntegerType.get(), null), - Types.NestedField.optional(1, "name", Types.StringType.get(), null), - Types.NestedField.required(2, "salary", Types.DoubleType.get(), null), - Types.NestedField.optional( - 3, - "locations", - Types.MapType.ofOptional( - 24, - 25, - Types.StringType.get(), - Types.StructType.of( - Types.NestedField.required(22, "posX", Types.DoubleType.get(), "X field"), - Types.NestedField.required( - 23, "posY", Types.DoubleType.get(), "Y field")))), - Types.NestedField.optional( - 4, "strArray", Types.ListType.ofOptional(26, Types.StringType.get())), - Types.NestedField.optional( - 5, "intArray", Types.ListType.ofOptional(27, Types.IntegerType.get())), - Types.NestedField.required(6, "char", Types.StringType.get()), - Types.NestedField.required(7, "varchar", Types.StringType.get()), - Types.NestedField.optional(8, "boolean", Types.BooleanType.get()), - Types.NestedField.optional(9, "tinyint", Types.IntegerType.get()), - Types.NestedField.optional(10, "smallint", Types.IntegerType.get()), - Types.NestedField.optional(11, "bigint", Types.LongType.get()), - Types.NestedField.optional(12, "varbinary", Types.BinaryType.get()), - Types.NestedField.optional(13, "binary", Types.FixedType.ofLength(10)), - Types.NestedField.optional(14, "time", Types.TimeType.get()), - Types.NestedField.optional( - 15, "timestampWithoutZone", Types.TimestampType.withoutZone()), - Types.NestedField.optional(16, "timestampWithZone", Types.TimestampType.withZone()), - Types.NestedField.optional(17, "date", Types.DateType.get()), - Types.NestedField.optional(18, "decimal", Types.DecimalType.of(2, 2)), - Types.NestedField.optional(19, "decimal2", Types.DecimalType.of(38, 2)), - Types.NestedField.optional(20, "decimal3", Types.DecimalType.of(10, 1)), - Types.NestedField.optional( - 21, - "multiset", - Types.MapType.ofRequired(28, 29, Types.StringType.get(), Types.IntegerType.get()))); - - checkSchema(flinkSchema, icebergSchema); - } - - @Test - public void testMapField() { - TableSchema flinkSchema = - TableSchema.builder() - .field( - "map_int_long", - DataTypes.MAP(DataTypes.INT(), DataTypes.BIGINT()).notNull()) /* Required */ - .field( - "map_int_array_string", - DataTypes.MAP(DataTypes.ARRAY(DataTypes.INT()), DataTypes.STRING())) - .field( - "map_decimal_string", DataTypes.MAP(DataTypes.DECIMAL(10, 2), DataTypes.STRING())) - .field( - "map_fields_fields", - DataTypes.MAP( - DataTypes.ROW( - DataTypes.FIELD("field_int", DataTypes.INT(), "doc - int"), - DataTypes.FIELD("field_string", DataTypes.STRING(), "doc - string")) - .notNull(), /* Required */ - DataTypes.ROW( - DataTypes.FIELD( - "field_array", - DataTypes.ARRAY(DataTypes.STRING()), - "doc - array")) - .notNull() /* Required */) - .notNull() /* Required */) - .build(); - - Schema icebergSchema = - new Schema( - Types.NestedField.required( - 0, - "map_int_long", - Types.MapType.ofOptional(4, 5, Types.IntegerType.get(), Types.LongType.get()), - null), - Types.NestedField.optional( - 1, - "map_int_array_string", - Types.MapType.ofOptional( - 7, - 8, - Types.ListType.ofOptional(6, Types.IntegerType.get()), - Types.StringType.get()), - null), - Types.NestedField.optional( - 2, - "map_decimal_string", - Types.MapType.ofOptional( - 9, 10, Types.DecimalType.of(10, 2), Types.StringType.get())), - Types.NestedField.required( - 3, - "map_fields_fields", - Types.MapType.ofRequired( - 15, - 16, - Types.StructType.of( - Types.NestedField.optional( - 11, "field_int", Types.IntegerType.get(), "doc - int"), - Types.NestedField.optional( - 12, "field_string", Types.StringType.get(), "doc - string")), - Types.StructType.of( - Types.NestedField.optional( - 14, - "field_array", - Types.ListType.ofOptional(13, Types.StringType.get()), - "doc - array"))))); - - checkSchema(flinkSchema, icebergSchema); - } - - @Test - public void testStructField() { - TableSchema flinkSchema = - TableSchema.builder() - .field( - "struct_int_string_decimal", - DataTypes.ROW( - DataTypes.FIELD("field_int", DataTypes.INT()), - DataTypes.FIELD("field_string", DataTypes.STRING()), - DataTypes.FIELD("field_decimal", DataTypes.DECIMAL(19, 2)), - DataTypes.FIELD( - "field_struct", - DataTypes.ROW( - DataTypes.FIELD("inner_struct_int", DataTypes.INT()), - DataTypes.FIELD( - "inner_struct_float_array", - DataTypes.ARRAY(DataTypes.FLOAT()))) - .notNull()) /* Row is required */) - .notNull()) /* Required */ - .field( - "struct_map_int_int", - DataTypes.ROW( - DataTypes.FIELD( - "field_map", DataTypes.MAP(DataTypes.INT(), DataTypes.INT()))) - .nullable()) /* Optional */ - .build(); - - Schema icebergSchema = - new Schema( - Types.NestedField.required( - 0, - "struct_int_string_decimal", - Types.StructType.of( - Types.NestedField.optional(5, "field_int", Types.IntegerType.get()), - Types.NestedField.optional(6, "field_string", Types.StringType.get()), - Types.NestedField.optional(7, "field_decimal", Types.DecimalType.of(19, 2)), - Types.NestedField.required( - 8, - "field_struct", - Types.StructType.of( - Types.NestedField.optional( - 3, "inner_struct_int", Types.IntegerType.get()), - Types.NestedField.optional( - 4, - "inner_struct_float_array", - Types.ListType.ofOptional(2, Types.FloatType.get())))))), - Types.NestedField.optional( - 1, - "struct_map_int_int", - Types.StructType.of( - Types.NestedField.optional( - 11, - "field_map", - Types.MapType.ofOptional( - 9, 10, Types.IntegerType.get(), Types.IntegerType.get()))))); - - checkSchema(flinkSchema, icebergSchema); - } - - @Test - public void testListField() { - TableSchema flinkSchema = - TableSchema.builder() - .field( - "list_struct_fields", - DataTypes.ARRAY(DataTypes.ROW(DataTypes.FIELD("field_int", DataTypes.INT()))) - .notNull()) /* Required */ - .field( - "list_optional_struct_fields", - DataTypes.ARRAY( - DataTypes.ROW( - DataTypes.FIELD( - "field_timestamp_with_local_time_zone", - DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()))) - .nullable()) /* Optional */ - .field( - "list_map_fields", - DataTypes.ARRAY( - DataTypes.MAP( - DataTypes.ARRAY( - DataTypes.INT().notNull()), /* Key of map must be required */ - DataTypes.ROW( - DataTypes.FIELD("field_0", DataTypes.INT(), "doc - int"))) - .notNull()) - .notNull()) /* Required */ - .build(); - - Schema icebergSchema = - new Schema( - Types.NestedField.required( - 0, - "list_struct_fields", - Types.ListType.ofOptional( - 4, - Types.StructType.of( - Types.NestedField.optional(3, "field_int", Types.IntegerType.get())))), - Types.NestedField.optional( - 1, - "list_optional_struct_fields", - Types.ListType.ofOptional( - 6, - Types.StructType.of( - Types.NestedField.optional( - 5, - "field_timestamp_with_local_time_zone", - Types.TimestampType.withZone())))), - Types.NestedField.required( - 2, - "list_map_fields", - Types.ListType.ofRequired( - 11, - Types.MapType.ofOptional( - 9, - 10, - Types.ListType.ofRequired(7, Types.IntegerType.get()), - Types.StructType.of( - Types.NestedField.optional( - 8, "field_0", Types.IntegerType.get(), "doc - int")))))); - - checkSchema(flinkSchema, icebergSchema); - } - - private void checkSchema(TableSchema flinkSchema, Schema icebergSchema) { - Assert.assertEquals(icebergSchema.asStruct(), FlinkSchemaUtil.convert(flinkSchema).asStruct()); - // The conversion is not a 1:1 mapping, so we just check iceberg types. - Assert.assertEquals( - icebergSchema.asStruct(), - FlinkSchemaUtil.convert(FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(icebergSchema))) - .asStruct()); - } - - @Test - public void testInconsistentTypes() { - checkInconsistentType( - Types.UUIDType.get(), new BinaryType(16), new BinaryType(16), Types.FixedType.ofLength(16)); - checkInconsistentType( - Types.StringType.get(), - new VarCharType(VarCharType.MAX_LENGTH), - new CharType(100), - Types.StringType.get()); - checkInconsistentType( - Types.BinaryType.get(), - new VarBinaryType(VarBinaryType.MAX_LENGTH), - new VarBinaryType(100), - Types.BinaryType.get()); - checkInconsistentType( - Types.TimeType.get(), new TimeType(), new TimeType(3), Types.TimeType.get()); - checkInconsistentType( - Types.TimestampType.withoutZone(), - new TimestampType(6), - new TimestampType(3), - Types.TimestampType.withoutZone()); - checkInconsistentType( - Types.TimestampType.withZone(), - new LocalZonedTimestampType(6), - new LocalZonedTimestampType(3), - Types.TimestampType.withZone()); - } - - private void checkInconsistentType( - Type icebergType, - LogicalType flinkExpectedType, - LogicalType flinkType, - Type icebergExpectedType) { - Assert.assertEquals(flinkExpectedType, FlinkSchemaUtil.convert(icebergType)); - Assert.assertEquals( - Types.StructType.of(Types.NestedField.optional(0, "f0", icebergExpectedType)), - FlinkSchemaUtil.convert(FlinkSchemaUtil.toSchema(RowType.of(flinkType))).asStruct()); - } - - @Test - public void testConvertFlinkSchemaBaseOnIcebergSchema() { - Schema baseSchema = - new Schema( - Lists.newArrayList( - Types.NestedField.required(101, "int", Types.IntegerType.get()), - Types.NestedField.optional(102, "string", Types.StringType.get())), - Sets.newHashSet(101)); - - TableSchema flinkSchema = - TableSchema.builder() - .field("int", DataTypes.INT().notNull()) - .field("string", DataTypes.STRING().nullable()) - .primaryKey("int") - .build(); - Schema convertedSchema = FlinkSchemaUtil.convert(baseSchema, flinkSchema); - Assert.assertEquals(baseSchema.asStruct(), convertedSchema.asStruct()); - Assert.assertEquals(ImmutableSet.of(101), convertedSchema.identifierFieldIds()); - } - - @Test - public void testConvertFlinkSchemaWithPrimaryKeys() { - Schema icebergSchema = - new Schema( - Lists.newArrayList( - Types.NestedField.required(1, "int", Types.IntegerType.get()), - Types.NestedField.required(2, "string", Types.StringType.get())), - Sets.newHashSet(1, 2)); - - TableSchema tableSchema = FlinkSchemaUtil.toSchema(icebergSchema); - Assert.assertTrue(tableSchema.getPrimaryKey().isPresent()); - Assert.assertEquals( - ImmutableSet.of("int", "string"), - ImmutableSet.copyOf(tableSchema.getPrimaryKey().get().getColumns())); - } - - @Test - public void testConvertFlinkSchemaWithNestedColumnInPrimaryKeys() { - Schema icebergSchema = - new Schema( - Lists.newArrayList( - Types.NestedField.required( - 1, - "struct", - Types.StructType.of( - Types.NestedField.required(2, "inner", Types.IntegerType.get())))), - Sets.newHashSet(2)); - Assertions.assertThatThrownBy(() -> FlinkSchemaUtil.toSchema(icebergSchema)) - .isInstanceOf(ValidationException.class) - .hasMessageStartingWith("Could not create a PRIMARY KEY") - .hasMessageContaining("Column 'struct.inner' does not exist"); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java deleted file mode 100644 index 754062798928..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java +++ /dev/null @@ -1,380 +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.iceberg.flink; - -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import org.apache.flink.api.dag.Transformation; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.Expressions; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; -import org.apache.flink.table.api.internal.TableEnvironmentImpl; -import org.apache.flink.table.operations.ModifyOperation; -import org.apache.flink.table.planner.delegation.PlannerBase; -import org.apache.flink.test.util.MiniClusterWithClientResource; -import org.apache.flink.types.Row; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DistributionMode; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.flink.source.BoundedTableFactory; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.junit.After; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestFlinkTableSink extends FlinkCatalogTestBase { - - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - private static final String SOURCE_TABLE = "default_catalog.default_database.bounded_source"; - private static final String TABLE_NAME = "test_table"; - private TableEnvironment tEnv; - private Table icebergTable; - - private final FileFormat format; - private final boolean isStreamingJob; - - @Parameterized.Parameters( - name = "catalogName={0}, baseNamespace={1}, format={2}, isStreaming={3}") - public static Iterable parameters() { - List parameters = Lists.newArrayList(); - for (FileFormat format : - new FileFormat[] {FileFormat.ORC, FileFormat.AVRO, FileFormat.PARQUET}) { - for (Boolean isStreaming : new Boolean[] {true, false}) { - for (Object[] catalogParams : FlinkCatalogTestBase.parameters()) { - String catalogName = (String) catalogParams[0]; - Namespace baseNamespace = (Namespace) catalogParams[1]; - parameters.add(new Object[] {catalogName, baseNamespace, format, isStreaming}); - } - } - } - return parameters; - } - - public TestFlinkTableSink( - String catalogName, Namespace baseNamespace, FileFormat format, Boolean isStreamingJob) { - super(catalogName, baseNamespace); - this.format = format; - this.isStreamingJob = isStreamingJob; - } - - @Override - protected TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - EnvironmentSettings.Builder settingsBuilder = EnvironmentSettings.newInstance(); - if (isStreamingJob) { - settingsBuilder.inStreamingMode(); - StreamExecutionEnvironment env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); - env.enableCheckpointing(400); - env.setMaxParallelism(2); - env.setParallelism(2); - tEnv = StreamTableEnvironment.create(env, settingsBuilder.build()); - } else { - settingsBuilder.inBatchMode(); - tEnv = TableEnvironment.create(settingsBuilder.build()); - } - } - } - return tEnv; - } - - @Override - @Before - public void before() { - super.before(); - sql("CREATE DATABASE %s", flinkDatabase); - sql("USE CATALOG %s", catalogName); - sql("USE %s", DATABASE); - sql( - "CREATE TABLE %s (id int, data varchar) with ('write.format.default'='%s')", - TABLE_NAME, format.name()); - icebergTable = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); - } - - @Override - @After - public void clean() { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); - BoundedTableFactory.clearDataSets(); - super.clean(); - } - - @Test - public void testInsertFromSourceTable() throws Exception { - // Register the rows into a temporary table. - getTableEnv() - .createTemporaryView( - "sourceTable", - getTableEnv() - .fromValues( - SimpleDataUtil.FLINK_SCHEMA.toRowDataType(), - Expressions.row(1, "hello"), - Expressions.row(2, "world"), - Expressions.row(3, (String) null), - Expressions.row(null, "bar"))); - - // Redirect the records from source table to destination table. - sql("INSERT INTO %s SELECT id,data from sourceTable", TABLE_NAME); - - // Assert the table records as expected. - SimpleDataUtil.assertTableRecords( - icebergTable, - Lists.newArrayList( - SimpleDataUtil.createRecord(1, "hello"), - SimpleDataUtil.createRecord(2, "world"), - SimpleDataUtil.createRecord(3, null), - SimpleDataUtil.createRecord(null, "bar"))); - } - - @Test - public void testOverwriteTable() throws Exception { - Assume.assumeFalse( - "Flink unbounded streaming does not support overwrite operation", isStreamingJob); - - sql("INSERT INTO %s SELECT 1, 'a'", TABLE_NAME); - SimpleDataUtil.assertTableRecords( - icebergTable, Lists.newArrayList(SimpleDataUtil.createRecord(1, "a"))); - - sql("INSERT OVERWRITE %s SELECT 2, 'b'", TABLE_NAME); - SimpleDataUtil.assertTableRecords( - icebergTable, Lists.newArrayList(SimpleDataUtil.createRecord(2, "b"))); - } - - @Test - public void testWriteParallelism() throws Exception { - List dataSet = - IntStream.range(1, 1000) - .mapToObj(i -> ImmutableList.of(Row.of(i, "aaa"), Row.of(i, "bbb"), Row.of(i, "ccc"))) - .flatMap(List::stream) - .collect(Collectors.toList()); - String dataId = BoundedTableFactory.registerDataSet(ImmutableList.of(dataSet)); - sql( - "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" - + " WITH ('connector'='BoundedSource', 'data-id'='%s')", - SOURCE_TABLE, dataId); - - PlannerBase planner = (PlannerBase) ((TableEnvironmentImpl) getTableEnv()).getPlanner(); - String insertSQL = - String.format( - "INSERT INTO %s /*+ OPTIONS('write-parallelism'='1') */ SELECT * FROM %s", - TABLE_NAME, SOURCE_TABLE); - ModifyOperation operation = (ModifyOperation) planner.getParser().parse(insertSQL).get(0); - Transformation dummySink = planner.translate(Collections.singletonList(operation)).get(0); - Transformation committer = dummySink.getInputs().get(0); - Transformation writer = committer.getInputs().get(0); - - Assert.assertEquals("Should have the expected 1 parallelism.", 1, writer.getParallelism()); - - writer - .getInputs() - .forEach( - input -> - Assert.assertEquals( - "Should have the expected parallelism.", - isStreamingJob ? 2 : 4, - input.getParallelism())); - } - - @Test - public void testReplacePartitions() throws Exception { - Assume.assumeFalse( - "Flink unbounded streaming does not support overwrite operation", isStreamingJob); - String tableName = "test_partition"; - sql( - "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH ('write.format.default'='%s')", - tableName, format.name()); - - try { - Table partitionedTable = - validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, tableName)); - - sql("INSERT INTO %s SELECT 1, 'a'", tableName); - sql("INSERT INTO %s SELECT 2, 'b'", tableName); - sql("INSERT INTO %s SELECT 3, 'c'", tableName); - - SimpleDataUtil.assertTableRecords( - partitionedTable, - Lists.newArrayList( - SimpleDataUtil.createRecord(1, "a"), - SimpleDataUtil.createRecord(2, "b"), - SimpleDataUtil.createRecord(3, "c"))); - - sql("INSERT OVERWRITE %s SELECT 4, 'b'", tableName); - sql("INSERT OVERWRITE %s SELECT 5, 'a'", tableName); - - SimpleDataUtil.assertTableRecords( - partitionedTable, - Lists.newArrayList( - SimpleDataUtil.createRecord(5, "a"), - SimpleDataUtil.createRecord(4, "b"), - SimpleDataUtil.createRecord(3, "c"))); - - sql("INSERT OVERWRITE %s PARTITION (data='a') SELECT 6", tableName); - - SimpleDataUtil.assertTableRecords( - partitionedTable, - Lists.newArrayList( - SimpleDataUtil.createRecord(6, "a"), - SimpleDataUtil.createRecord(4, "b"), - SimpleDataUtil.createRecord(3, "c"))); - } finally { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - } - } - - @Test - public void testInsertIntoPartition() throws Exception { - String tableName = "test_insert_into_partition"; - sql( - "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH ('write.format.default'='%s')", - tableName, format.name()); - - try { - Table partitionedTable = - validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, tableName)); - - // Full partition. - sql("INSERT INTO %s PARTITION (data='a') SELECT 1", tableName); - sql("INSERT INTO %s PARTITION (data='a') SELECT 2", tableName); - sql("INSERT INTO %s PARTITION (data='b') SELECT 3", tableName); - - SimpleDataUtil.assertTableRecords( - partitionedTable, - Lists.newArrayList( - SimpleDataUtil.createRecord(1, "a"), - SimpleDataUtil.createRecord(2, "a"), - SimpleDataUtil.createRecord(3, "b"))); - - // Partial partition. - sql("INSERT INTO %s SELECT 4, 'c'", tableName); - sql("INSERT INTO %s SELECT 5, 'd'", tableName); - - SimpleDataUtil.assertTableRecords( - partitionedTable, - Lists.newArrayList( - SimpleDataUtil.createRecord(1, "a"), - SimpleDataUtil.createRecord(2, "a"), - SimpleDataUtil.createRecord(3, "b"), - SimpleDataUtil.createRecord(4, "c"), - SimpleDataUtil.createRecord(5, "d"))); - } finally { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - } - } - - @Test - public void testHashDistributeMode() throws Exception { - String tableName = "test_hash_distribution_mode"; - Map tableProps = - ImmutableMap.of( - "write.format.default", - format.name(), - TableProperties.WRITE_DISTRIBUTION_MODE, - DistributionMode.HASH.modeName()); - - // Initialize a BoundedSource table to precisely emit those rows in only one checkpoint. - List dataSet = - IntStream.range(1, 1000) - .mapToObj(i -> ImmutableList.of(Row.of(i, "aaa"), Row.of(i, "bbb"), Row.of(i, "ccc"))) - .flatMap(List::stream) - .collect(Collectors.toList()); - String dataId = BoundedTableFactory.registerDataSet(ImmutableList.of(dataSet)); - sql( - "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" - + " WITH ('connector'='BoundedSource', 'data-id'='%s')", - SOURCE_TABLE, dataId); - Assert.assertEquals( - "Should have the expected rows in source table.", - Sets.newHashSet(dataSet), - Sets.newHashSet(sql("SELECT * FROM %s", SOURCE_TABLE))); - - sql( - "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH %s", - tableName, toWithClause(tableProps)); - - try { - // Insert data set. - sql("INSERT INTO %s SELECT * FROM %s", tableName, SOURCE_TABLE); - - Assert.assertEquals( - "Should have the expected rows in sink table.", - Sets.newHashSet(dataSet), - Sets.newHashSet(sql("SELECT * FROM %s", tableName))); - - // Sometimes we will have more than one checkpoint if we pass the auto checkpoint interval, - // thus producing multiple snapshots. Here we assert that each snapshot has only 1 file per - // partition. - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, tableName)); - Map> snapshotToDataFiles = SimpleDataUtil.snapshotToDataFiles(table); - for (List dataFiles : snapshotToDataFiles.values()) { - if (dataFiles.isEmpty()) { - continue; - } - - Assert.assertEquals( - "There should be 1 data file in partition 'aaa'", - 1, - SimpleDataUtil.matchingPartitions( - dataFiles, table.spec(), ImmutableMap.of("data", "aaa")) - .size()); - Assert.assertEquals( - "There should be 1 data file in partition 'bbb'", - 1, - SimpleDataUtil.matchingPartitions( - dataFiles, table.spec(), ImmutableMap.of("data", "bbb")) - .size()); - Assert.assertEquals( - "There should be 1 data file in partition 'ccc'", - 1, - SimpleDataUtil.matchingPartitions( - dataFiles, table.spec(), ImmutableMap.of("data", "ccc")) - .size()); - } - } finally { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - } - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java deleted file mode 100644 index a25ebab6c4c2..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java +++ /dev/null @@ -1,346 +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.iceberg.flink; - -import java.time.LocalDate; -import java.util.List; -import java.util.Map; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; -import org.apache.flink.test.util.MiniClusterWithClientResource; -import org.apache.flink.types.Row; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.junit.After; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestFlinkUpsert extends FlinkCatalogTestBase { - - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - private final boolean isStreamingJob; - private final Map tableUpsertProps = Maps.newHashMap(); - private TableEnvironment tEnv; - - public TestFlinkUpsert( - String catalogName, Namespace baseNamespace, FileFormat format, Boolean isStreamingJob) { - super(catalogName, baseNamespace); - this.isStreamingJob = isStreamingJob; - tableUpsertProps.put(TableProperties.FORMAT_VERSION, "2"); - tableUpsertProps.put(TableProperties.UPSERT_ENABLED, "true"); - tableUpsertProps.put(TableProperties.DEFAULT_FILE_FORMAT, format.name()); - } - - @Parameterized.Parameters( - name = "catalogName={0}, baseNamespace={1}, format={2}, isStreaming={3}") - public static Iterable parameters() { - List parameters = Lists.newArrayList(); - for (FileFormat format : - new FileFormat[] {FileFormat.PARQUET, FileFormat.AVRO, FileFormat.ORC}) { - for (Boolean isStreaming : new Boolean[] {true, false}) { - // Only test with one catalog as this is a file operation concern. - // FlinkCatalogTestBase requires the catalog name start with testhadoop if using hadoop - // catalog. - String catalogName = "testhadoop"; - Namespace baseNamespace = Namespace.of("default"); - parameters.add(new Object[] {catalogName, baseNamespace, format, isStreaming}); - } - } - return parameters; - } - - @Override - protected TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - EnvironmentSettings.Builder settingsBuilder = EnvironmentSettings.newInstance(); - if (isStreamingJob) { - settingsBuilder.inStreamingMode(); - StreamExecutionEnvironment env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); - env.enableCheckpointing(400); - env.setMaxParallelism(2); - env.setParallelism(2); - tEnv = StreamTableEnvironment.create(env, settingsBuilder.build()); - } else { - settingsBuilder.inBatchMode(); - tEnv = TableEnvironment.create(settingsBuilder.build()); - } - } - } - return tEnv; - } - - @Override - @Before - public void before() { - super.before(); - sql("CREATE DATABASE IF NOT EXISTS %s", flinkDatabase); - sql("USE CATALOG %s", catalogName); - sql("USE %s", DATABASE); - } - - @Override - @After - public void clean() { - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); - super.clean(); - } - - @Test - public void testUpsertAndQuery() { - String tableName = "test_upsert_query"; - LocalDate dt20220301 = LocalDate.of(2022, 3, 1); - LocalDate dt20220302 = LocalDate.of(2022, 3, 2); - - sql( - "CREATE TABLE %s(id INT NOT NULL, name STRING NOT NULL, dt DATE, PRIMARY KEY(id,dt) NOT ENFORCED) " - + "PARTITIONED BY (dt) WITH %s", - tableName, toWithClause(tableUpsertProps)); - - try { - sql( - "INSERT INTO %s VALUES " - + "(1, 'Bill', DATE '2022-03-01')," - + "(1, 'Jane', DATE '2022-03-01')," - + "(2, 'Jane', DATE '2022-03-01')", - tableName); - - sql( - "INSERT INTO %s VALUES " - + "(2, 'Bill', DATE '2022-03-01')," - + "(1, 'Jane', DATE '2022-03-02')," - + "(2, 'Jane', DATE '2022-03-02')", - tableName); - - List rowsOn20220301 = - Lists.newArrayList(Row.of(1, "Jane", dt20220301), Row.of(2, "Bill", dt20220301)); - TestHelpers.assertRows( - sql("SELECT * FROM %s WHERE dt < '2022-03-02'", tableName), rowsOn20220301); - - List rowsOn20220302 = - Lists.newArrayList(Row.of(1, "Jane", dt20220302), Row.of(2, "Jane", dt20220302)); - TestHelpers.assertRows( - sql("SELECT * FROM %s WHERE dt = '2022-03-02'", tableName), rowsOn20220302); - - TestHelpers.assertRows( - sql("SELECT * FROM %s", tableName), - Lists.newArrayList(Iterables.concat(rowsOn20220301, rowsOn20220302))); - } finally { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - } - } - - @Test - public void testUpsertOptions() { - String tableName = "test_upsert_options"; - LocalDate dt20220301 = LocalDate.of(2022, 3, 1); - LocalDate dt20220302 = LocalDate.of(2022, 3, 2); - - Map optionsUpsertProps = Maps.newHashMap(tableUpsertProps); - optionsUpsertProps.remove(TableProperties.UPSERT_ENABLED); - sql( - "CREATE TABLE %s(id INT NOT NULL, name STRING NOT NULL, dt DATE, PRIMARY KEY(id,dt) NOT ENFORCED) " - + "PARTITIONED BY (dt) WITH %s", - tableName, toWithClause(optionsUpsertProps)); - - try { - sql( - "INSERT INTO %s /*+ OPTIONS('upsert-enabled'='true')*/ VALUES " - + "(1, 'Bill', DATE '2022-03-01')," - + "(1, 'Jane', DATE '2022-03-01')," - + "(2, 'Jane', DATE '2022-03-01')", - tableName); - - sql( - "INSERT INTO %s /*+ OPTIONS('upsert-enabled'='true')*/ VALUES " - + "(2, 'Bill', DATE '2022-03-01')," - + "(1, 'Jane', DATE '2022-03-02')," - + "(2, 'Jane', DATE '2022-03-02')", - tableName); - - List rowsOn20220301 = - Lists.newArrayList(Row.of(1, "Jane", dt20220301), Row.of(2, "Bill", dt20220301)); - TestHelpers.assertRows( - sql("SELECT * FROM %s WHERE dt < '2022-03-02'", tableName), rowsOn20220301); - - List rowsOn20220302 = - Lists.newArrayList(Row.of(1, "Jane", dt20220302), Row.of(2, "Jane", dt20220302)); - TestHelpers.assertRows( - sql("SELECT * FROM %s WHERE dt = '2022-03-02'", tableName), rowsOn20220302); - - TestHelpers.assertRows( - sql("SELECT * FROM %s", tableName), - Lists.newArrayList(Iterables.concat(rowsOn20220301, rowsOn20220302))); - } finally { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - } - } - - @Test - public void testPrimaryKeyEqualToPartitionKey() { - // This is an SQL based reproduction of TestFlinkIcebergSinkV2#testUpsertOnDataKey - String tableName = "upsert_on_id_key"; - try { - sql( - "CREATE TABLE %s(id INT NOT NULL, name STRING NOT NULL, PRIMARY KEY(id) NOT ENFORCED) " - + "PARTITIONED BY (id) WITH %s", - tableName, toWithClause(tableUpsertProps)); - - sql("INSERT INTO %s VALUES " + "(1, 'Bill')," + "(1, 'Jane')," + "(2, 'Bill')", tableName); - - TestHelpers.assertRows( - sql("SELECT * FROM %s", tableName), - Lists.newArrayList(Row.of(1, "Jane"), Row.of(2, "Bill"))); - - sql("INSERT INTO %s VALUES " + "(1, 'Bill')," + "(2, 'Jane')", tableName); - - TestHelpers.assertRows( - sql("SELECT * FROM %s", tableName), - Lists.newArrayList(Row.of(1, "Bill"), Row.of(2, "Jane"))); - - sql("INSERT INTO %s VALUES " + "(3, 'Bill')," + "(4, 'Jane')", tableName); - - TestHelpers.assertRows( - sql("SELECT * FROM %s", tableName), - Lists.newArrayList( - Row.of(1, "Bill"), Row.of(2, "Jane"), Row.of(3, "Bill"), Row.of(4, "Jane"))); - } finally { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - } - } - - @Test - public void testPrimaryKeyFieldsAtBeginningOfSchema() { - String tableName = "upsert_on_pk_at_schema_start"; - LocalDate dt = LocalDate.of(2022, 3, 1); - try { - sql( - "CREATE TABLE %s(id INT, dt DATE NOT NULL, name STRING NOT NULL, PRIMARY KEY(id,dt) NOT ENFORCED) " - + "PARTITIONED BY (dt) WITH %s", - tableName, toWithClause(tableUpsertProps)); - - sql( - "INSERT INTO %s VALUES " - + "(1, DATE '2022-03-01', 'Andy')," - + "(1, DATE '2022-03-01', 'Bill')," - + "(2, DATE '2022-03-01', 'Jane')", - tableName); - - TestHelpers.assertRows( - sql("SELECT * FROM %s", tableName), - Lists.newArrayList(Row.of(1, dt, "Bill"), Row.of(2, dt, "Jane"))); - - sql( - "INSERT INTO %s VALUES " - + "(1, DATE '2022-03-01', 'Jane')," - + "(2, DATE '2022-03-01', 'Bill')", - tableName); - - TestHelpers.assertRows( - sql("SELECT * FROM %s", tableName), - Lists.newArrayList(Row.of(1, dt, "Jane"), Row.of(2, dt, "Bill"))); - - sql( - "INSERT INTO %s VALUES " - + "(3, DATE '2022-03-01', 'Duke')," - + "(4, DATE '2022-03-01', 'Leon')", - tableName); - - TestHelpers.assertRows( - sql("SELECT * FROM %s", tableName), - Lists.newArrayList( - Row.of(1, dt, "Jane"), - Row.of(2, dt, "Bill"), - Row.of(3, dt, "Duke"), - Row.of(4, dt, "Leon"))); - } finally { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - } - } - - @Test - public void testPrimaryKeyFieldsAtEndOfTableSchema() { - // This is the same test case as testPrimaryKeyFieldsAtBeginningOfSchema, but the primary key - // fields - // are located at the end of the flink schema. - String tableName = "upsert_on_pk_at_schema_end"; - LocalDate dt = LocalDate.of(2022, 3, 1); - try { - sql( - "CREATE TABLE %s(name STRING NOT NULL, id INT, dt DATE NOT NULL, PRIMARY KEY(id,dt) NOT ENFORCED) " - + "PARTITIONED BY (dt) WITH %s", - tableName, toWithClause(tableUpsertProps)); - - sql( - "INSERT INTO %s VALUES " - + "('Andy', 1, DATE '2022-03-01')," - + "('Bill', 1, DATE '2022-03-01')," - + "('Jane', 2, DATE '2022-03-01')", - tableName); - - TestHelpers.assertRows( - sql("SELECT * FROM %s", tableName), - Lists.newArrayList(Row.of("Bill", 1, dt), Row.of("Jane", 2, dt))); - - sql( - "INSERT INTO %s VALUES " - + "('Jane', 1, DATE '2022-03-01')," - + "('Bill', 2, DATE '2022-03-01')", - tableName); - - TestHelpers.assertRows( - sql("SELECT * FROM %s", tableName), - Lists.newArrayList(Row.of("Jane", 1, dt), Row.of("Bill", 2, dt))); - - sql( - "INSERT INTO %s VALUES " - + "('Duke', 3, DATE '2022-03-01')," - + "('Leon', 4, DATE '2022-03-01')", - tableName); - - TestHelpers.assertRows( - sql("SELECT * FROM %s", tableName), - Lists.newArrayList( - Row.of("Jane", 1, dt), - Row.of("Bill", 2, dt), - Row.of("Duke", 3, dt), - Row.of("Leon", 4, dt))); - } finally { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - } - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java deleted file mode 100644 index 3e574b841c8d..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java +++ /dev/null @@ -1,611 +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.iceberg.flink; - -import java.io.IOException; -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.time.OffsetDateTime; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import java.util.function.Consumer; -import java.util.stream.Collectors; -import org.apache.avro.generic.GenericData; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; -import org.apache.flink.core.memory.DataInputDeserializer; -import org.apache.flink.core.memory.DataOutputSerializer; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.GenericArrayData; -import org.apache.flink.table.data.GenericMapData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.data.conversion.DataStructureConverter; -import org.apache.flink.table.data.conversion.DataStructureConverters; -import org.apache.flink.table.runtime.typeutils.InternalSerializers; -import org.apache.flink.table.types.logical.ArrayType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.MapType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.utils.TypeConversions; -import org.apache.flink.types.Row; -import org.apache.iceberg.ContentFile; -import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.Schema; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.data.RowDataUtil; -import org.apache.iceberg.flink.source.FlinkInputFormat; -import org.apache.iceberg.flink.source.FlinkInputSplit; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Streams; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.DateTimeUtil; -import org.assertj.core.api.Assertions; -import org.junit.Assert; - -public class TestHelpers { - private TestHelpers() {} - - public static T roundTripKryoSerialize(Class clazz, T table) throws IOException { - KryoSerializer kryo = new KryoSerializer<>(clazz, new ExecutionConfig()); - - DataOutputSerializer outputView = new DataOutputSerializer(1024); - kryo.serialize(table, outputView); - - DataInputDeserializer inputView = new DataInputDeserializer(outputView.getCopyOfBuffer()); - return kryo.deserialize(inputView); - } - - public static RowData copyRowData(RowData from, RowType rowType) { - TypeSerializer[] fieldSerializers = - rowType.getChildren().stream() - .map((LogicalType type) -> InternalSerializers.create(type)) - .toArray(TypeSerializer[]::new); - return RowDataUtil.clone(from, null, rowType, fieldSerializers); - } - - public static void readRowData(FlinkInputFormat input, Consumer visitor) - throws IOException { - for (FlinkInputSplit s : input.createInputSplits(0)) { - input.open(s); - try { - while (!input.reachedEnd()) { - RowData row = input.nextRecord(null); - visitor.accept(row); - } - } finally { - input.close(); - } - } - } - - public static List readRowData(FlinkInputFormat inputFormat, RowType rowType) - throws IOException { - List results = Lists.newArrayList(); - readRowData(inputFormat, row -> results.add(copyRowData(row, rowType))); - return results; - } - - public static List readRows(FlinkInputFormat inputFormat, RowType rowType) - throws IOException { - return convertRowDataToRow(readRowData(inputFormat, rowType), rowType); - } - - public static List convertRowDataToRow(List rowDataList, RowType rowType) { - DataStructureConverter converter = - DataStructureConverters.getConverter(TypeConversions.fromLogicalToDataType(rowType)); - return rowDataList.stream() - .map(converter::toExternal) - .map(Row.class::cast) - .collect(Collectors.toList()); - } - - public static void assertRecords(List results, List expectedRecords, Schema schema) { - List expected = Lists.newArrayList(); - @SuppressWarnings("unchecked") - DataStructureConverter converter = - (DataStructureConverter) - DataStructureConverters.getConverter( - TypeConversions.fromLogicalToDataType(FlinkSchemaUtil.convert(schema))); - expectedRecords.forEach( - r -> expected.add(converter.toExternal(RowDataConverter.convert(schema, r)))); - assertRows(results, expected); - } - - public static void assertRows(List results, List expected, RowType rowType) { - assertRows(convertRowDataToRow(results, rowType), convertRowDataToRow(expected, rowType)); - } - - public static void assertRows(List results, List expected) { - Assertions.assertThat(results).containsExactlyInAnyOrderElementsOf(expected); - } - - public static void assertRowData(Schema schema, StructLike expected, RowData actual) { - assertRowData(schema.asStruct(), FlinkSchemaUtil.convert(schema), expected, actual); - } - - public static void assertRowData( - Types.StructType structType, - LogicalType rowType, - StructLike expectedRecord, - RowData actualRowData) { - if (expectedRecord == null && actualRowData == null) { - return; - } - - Assert.assertTrue( - "expected Record and actual RowData should be both null or not null", - expectedRecord != null && actualRowData != null); - - List types = Lists.newArrayList(); - for (Types.NestedField field : structType.fields()) { - types.add(field.type()); - } - - for (int i = 0; i < types.size(); i += 1) { - LogicalType logicalType = ((RowType) rowType).getTypeAt(i); - Object expected = expectedRecord.get(i, Object.class); - // The RowData.createFieldGetter won't return null for the required field. But in the - // projection case, if we are - // projecting a nested required field from an optional struct, then we should give a null for - // the projected field - // if the outer struct value is null. So we need to check the nullable for actualRowData here. - // For more details - // please see issue #2738. - Object actual = - actualRowData.isNullAt(i) - ? null - : RowData.createFieldGetter(logicalType, i).getFieldOrNull(actualRowData); - assertEquals(types.get(i), logicalType, expected, actual); - } - } - - private static void assertEquals( - Type type, LogicalType logicalType, Object expected, Object actual) { - - if (expected == null && actual == null) { - return; - } - - Assert.assertTrue( - "expected and actual should be both null or not null", expected != null && actual != null); - - switch (type.typeId()) { - case BOOLEAN: - Assert.assertEquals("boolean value should be equal", expected, actual); - break; - case INTEGER: - Assert.assertEquals("int value should be equal", expected, actual); - break; - case LONG: - Assert.assertEquals("long value should be equal", expected, actual); - break; - case FLOAT: - Assert.assertEquals("float value should be equal", expected, actual); - break; - case DOUBLE: - Assert.assertEquals("double value should be equal", expected, actual); - break; - case STRING: - Assertions.assertThat(expected) - .as("Should expect a CharSequence") - .isInstanceOf(CharSequence.class); - Assert.assertEquals("string should be equal", String.valueOf(expected), actual.toString()); - break; - case DATE: - Assertions.assertThat(expected).as("Should expect a Date").isInstanceOf(LocalDate.class); - LocalDate date = DateTimeUtil.dateFromDays((int) actual); - Assert.assertEquals("date should be equal", expected, date); - break; - case TIME: - Assertions.assertThat(expected) - .as("Should expect a LocalTime") - .isInstanceOf(LocalTime.class); - int milliseconds = (int) (((LocalTime) expected).toNanoOfDay() / 1000_000); - Assert.assertEquals("time millis should be equal", milliseconds, actual); - break; - case TIMESTAMP: - if (((Types.TimestampType) type).shouldAdjustToUTC()) { - Assertions.assertThat(expected) - .as("Should expect a OffsetDataTime") - .isInstanceOf(OffsetDateTime.class); - OffsetDateTime ts = (OffsetDateTime) expected; - Assert.assertEquals( - "OffsetDataTime should be equal", - ts.toLocalDateTime(), - ((TimestampData) actual).toLocalDateTime()); - } else { - Assertions.assertThat(expected) - .as("Should expect a LocalDataTime") - .isInstanceOf(LocalDateTime.class); - LocalDateTime ts = (LocalDateTime) expected; - Assert.assertEquals( - "LocalDataTime should be equal", ts, ((TimestampData) actual).toLocalDateTime()); - } - break; - case BINARY: - Assertions.assertThat(expected) - .as("Should expect a ByteBuffer") - .isInstanceOf(ByteBuffer.class); - Assert.assertEquals("binary should be equal", expected, ByteBuffer.wrap((byte[]) actual)); - break; - case DECIMAL: - Assertions.assertThat(expected) - .as("Should expect a BigDecimal") - .isInstanceOf(BigDecimal.class); - BigDecimal bd = (BigDecimal) expected; - Assert.assertEquals( - "decimal value should be equal", bd, ((DecimalData) actual).toBigDecimal()); - break; - case LIST: - Assertions.assertThat(expected) - .as("Should expect a Collection") - .isInstanceOf(Collection.class); - Collection expectedArrayData = (Collection) expected; - ArrayData actualArrayData = (ArrayData) actual; - LogicalType elementType = ((ArrayType) logicalType).getElementType(); - Assert.assertEquals( - "array length should be equal", expectedArrayData.size(), actualArrayData.size()); - assertArrayValues( - type.asListType().elementType(), elementType, expectedArrayData, actualArrayData); - break; - case MAP: - Assertions.assertThat(expected).as("Should expect a Map").isInstanceOf(Map.class); - assertMapValues(type.asMapType(), logicalType, (Map) expected, (MapData) actual); - break; - case STRUCT: - Assertions.assertThat(expected).as("Should expect a Record").isInstanceOf(StructLike.class); - assertRowData(type.asStructType(), logicalType, (StructLike) expected, (RowData) actual); - break; - case UUID: - Assertions.assertThat(expected).as("Should expect a UUID").isInstanceOf(UUID.class); - ByteBuffer bb = ByteBuffer.wrap((byte[]) actual); - long firstLong = bb.getLong(); - long secondLong = bb.getLong(); - Assert.assertEquals( - "UUID should be equal", - expected.toString(), - new UUID(firstLong, secondLong).toString()); - break; - case FIXED: - Assertions.assertThat(expected).as("Should expect byte[]").isInstanceOf(byte[].class); - Assert.assertArrayEquals("binary should be equal", (byte[]) expected, (byte[]) actual); - break; - default: - throw new IllegalArgumentException("Not a supported type: " + type); - } - } - - public static void assertEquals(Schema schema, List records, List rows) { - Streams.forEachPair( - records.stream(), rows.stream(), (record, row) -> assertEquals(schema, record, row)); - } - - public static void assertEquals(Schema schema, GenericData.Record record, Row row) { - List fields = schema.asStruct().fields(); - Assert.assertEquals(fields.size(), record.getSchema().getFields().size()); - Assert.assertEquals(fields.size(), row.getArity()); - RowType rowType = FlinkSchemaUtil.convert(schema); - for (int i = 0; i < fields.size(); ++i) { - Type fieldType = fields.get(i).type(); - Object expectedValue = record.get(i); - Object actualValue = row.getField(i); - LogicalType logicalType = rowType.getTypeAt(i); - assertAvroEquals(fieldType, logicalType, expectedValue, actualValue); - } - } - - private static void assertEquals(Types.StructType struct, GenericData.Record record, Row row) { - List fields = struct.fields(); - for (int i = 0; i < fields.size(); i += 1) { - Type fieldType = fields.get(i).type(); - Object expectedValue = record.get(i); - Object actualValue = row.getField(i); - assertAvroEquals(fieldType, null, expectedValue, actualValue); - } - } - - private static void assertAvroEquals( - Type type, LogicalType logicalType, Object expected, Object actual) { - - if (expected == null && actual == null) { - return; - } - - Assert.assertTrue( - "expected and actual should be both null or not null", expected != null && actual != null); - - switch (type.typeId()) { - case BOOLEAN: - case INTEGER: - case LONG: - case FLOAT: - case DOUBLE: - Assertions.assertThat(expected) - .as("Should expect a " + type.typeId().javaClass()) - .isInstanceOf(type.typeId().javaClass()); - Assertions.assertThat(actual) - .as("Should expect a " + type.typeId().javaClass()) - .isInstanceOf(type.typeId().javaClass()); - Assert.assertEquals(type.typeId() + " value should be equal", expected, actual); - break; - case STRING: - Assertions.assertThat(expected) - .as("Should expect a CharSequence") - .isInstanceOf(CharSequence.class); - Assertions.assertThat(actual) - .as("Should expect a CharSequence") - .isInstanceOf(CharSequence.class); - Assert.assertEquals("string should be equal", expected.toString(), actual.toString()); - break; - case DATE: - Assertions.assertThat(expected).as("Should expect a Date").isInstanceOf(LocalDate.class); - LocalDate date = DateTimeUtil.dateFromDays((int) actual); - Assert.assertEquals("date should be equal", expected, date); - break; - case TIME: - Assertions.assertThat(expected) - .as("Should expect a LocalTime") - .isInstanceOf(LocalTime.class); - int milliseconds = (int) (((LocalTime) expected).toNanoOfDay() / 1000_000); - Assert.assertEquals("time millis should be equal", milliseconds, actual); - break; - case TIMESTAMP: - if (((Types.TimestampType) type).shouldAdjustToUTC()) { - Assertions.assertThat(expected) - .as("Should expect a OffsetDataTime") - .isInstanceOf(OffsetDateTime.class); - OffsetDateTime ts = (OffsetDateTime) expected; - Assert.assertEquals( - "OffsetDataTime should be equal", - ts.toLocalDateTime(), - ((TimestampData) actual).toLocalDateTime()); - } else { - Assertions.assertThat(expected) - .as("Should expect a LocalDataTime") - .isInstanceOf(LocalDateTime.class); - LocalDateTime ts = (LocalDateTime) expected; - Assert.assertEquals( - "LocalDataTime should be equal", ts, ((TimestampData) actual).toLocalDateTime()); - } - break; - case BINARY: - Assertions.assertThat(expected) - .as("Should expect a ByteBuffer") - .isInstanceOf(ByteBuffer.class); - Assert.assertEquals("binary should be equal", expected, ByteBuffer.wrap((byte[]) actual)); - break; - case DECIMAL: - Assertions.assertThat(expected) - .as("Should expect a BigDecimal") - .isInstanceOf(BigDecimal.class); - BigDecimal bd = (BigDecimal) expected; - Assert.assertEquals( - "decimal value should be equal", bd, ((DecimalData) actual).toBigDecimal()); - break; - case LIST: - Assertions.assertThat(expected) - .as("Should expect a Collection") - .isInstanceOf(Collection.class); - Collection expectedArrayData = (Collection) expected; - ArrayData actualArrayData; - try { - actualArrayData = (ArrayData) actual; - } catch (ClassCastException e) { - actualArrayData = new GenericArrayData((Object[]) actual); - } - LogicalType elementType = ((ArrayType) logicalType).getElementType(); - Assert.assertEquals( - "array length should be equal", expectedArrayData.size(), actualArrayData.size()); - assertArrayValues( - type.asListType().elementType(), elementType, expectedArrayData, actualArrayData); - break; - case MAP: - Assertions.assertThat(expected).as("Should expect a Map").isInstanceOf(Map.class); - MapData actualMap; - try { - actualMap = (MapData) actual; - } catch (ClassCastException e) { - actualMap = new GenericMapData((Map) actual); - } - assertMapValues(type.asMapType(), logicalType, (Map) expected, actualMap); - break; - case STRUCT: - Assertions.assertThat(expected) - .as("Should expect a Record") - .isInstanceOf(GenericData.Record.class); - assertEquals( - type.asNestedType().asStructType(), (GenericData.Record) expected, (Row) actual); - break; - case UUID: - Assertions.assertThat(expected).as("Should expect a UUID").isInstanceOf(UUID.class); - ByteBuffer bb = ByteBuffer.wrap((byte[]) actual); - long firstLong = bb.getLong(); - long secondLong = bb.getLong(); - Assert.assertEquals( - "UUID should be equal", - expected.toString(), - new UUID(firstLong, secondLong).toString()); - break; - case FIXED: - Assertions.assertThat(expected).as("Should expect byte[]").isInstanceOf(byte[].class); - Assert.assertArrayEquals("binary should be equal", (byte[]) expected, (byte[]) actual); - break; - default: - throw new IllegalArgumentException("Not a supported type: " + type); - } - } - - private static void assertArrayValues( - Type type, LogicalType logicalType, Collection expectedArray, ArrayData actualArray) { - List expectedElements = Lists.newArrayList(expectedArray); - for (int i = 0; i < expectedArray.size(); i += 1) { - if (expectedElements.get(i) == null) { - Assert.assertTrue(actualArray.isNullAt(i)); - continue; - } - - Object expected = expectedElements.get(i); - - assertEquals( - type, - logicalType, - expected, - ArrayData.createElementGetter(logicalType).getElementOrNull(actualArray, i)); - } - } - - private static void assertMapValues( - Types.MapType mapType, LogicalType type, Map expected, MapData actual) { - Assert.assertEquals("map size should be equal", expected.size(), actual.size()); - - ArrayData actualKeyArrayData = actual.keyArray(); - ArrayData actualValueArrayData = actual.valueArray(); - LogicalType actualKeyType = ((MapType) type).getKeyType(); - LogicalType actualValueType = ((MapType) type).getValueType(); - Type keyType = mapType.keyType(); - Type valueType = mapType.valueType(); - - ArrayData.ElementGetter keyGetter = ArrayData.createElementGetter(actualKeyType); - ArrayData.ElementGetter valueGetter = ArrayData.createElementGetter(actualValueType); - - for (Map.Entry entry : expected.entrySet()) { - Object matchedActualKey = null; - int matchedKeyIndex = 0; - for (int i = 0; i < actual.size(); i += 1) { - try { - Object key = keyGetter.getElementOrNull(actualKeyArrayData, i); - assertEquals(keyType, actualKeyType, entry.getKey(), key); - matchedActualKey = key; - matchedKeyIndex = i; - break; - } catch (AssertionError e) { - // not found - } - } - Assert.assertNotNull("Should have a matching key", matchedActualKey); - final int valueIndex = matchedKeyIndex; - assertEquals( - valueType, - actualValueType, - entry.getValue(), - valueGetter.getElementOrNull(actualValueArrayData, valueIndex)); - } - } - - public static void assertEquals(ManifestFile expected, ManifestFile actual) { - if (expected == actual) { - return; - } - Assert.assertTrue("Should not be null.", expected != null && actual != null); - Assert.assertEquals("Path must match", expected.path(), actual.path()); - Assert.assertEquals("Length must match", expected.length(), actual.length()); - Assert.assertEquals("Spec id must match", expected.partitionSpecId(), actual.partitionSpecId()); - Assert.assertEquals("ManifestContent must match", expected.content(), actual.content()); - Assert.assertEquals( - "SequenceNumber must match", expected.sequenceNumber(), actual.sequenceNumber()); - Assert.assertEquals( - "MinSequenceNumber must match", expected.minSequenceNumber(), actual.minSequenceNumber()); - Assert.assertEquals("Snapshot id must match", expected.snapshotId(), actual.snapshotId()); - Assert.assertEquals( - "Added files flag must match", expected.hasAddedFiles(), actual.hasAddedFiles()); - Assert.assertEquals( - "Added files count must match", expected.addedFilesCount(), actual.addedFilesCount()); - Assert.assertEquals( - "Added rows count must match", expected.addedRowsCount(), actual.addedRowsCount()); - Assert.assertEquals( - "Existing files flag must match", expected.hasExistingFiles(), actual.hasExistingFiles()); - Assert.assertEquals( - "Existing files count must match", - expected.existingFilesCount(), - actual.existingFilesCount()); - Assert.assertEquals( - "Existing rows count must match", expected.existingRowsCount(), actual.existingRowsCount()); - Assert.assertEquals( - "Deleted files flag must match", expected.hasDeletedFiles(), actual.hasDeletedFiles()); - Assert.assertEquals( - "Deleted files count must match", expected.deletedFilesCount(), actual.deletedFilesCount()); - Assert.assertEquals( - "Deleted rows count must match", expected.deletedRowsCount(), actual.deletedRowsCount()); - - List expectedSummaries = expected.partitions(); - List actualSummaries = actual.partitions(); - Assert.assertEquals( - "PartitionFieldSummary size does not match", - expectedSummaries.size(), - actualSummaries.size()); - for (int i = 0; i < expectedSummaries.size(); i++) { - Assert.assertEquals( - "Null flag in partition must match", - expectedSummaries.get(i).containsNull(), - actualSummaries.get(i).containsNull()); - Assert.assertEquals( - "NaN flag in partition must match", - expectedSummaries.get(i).containsNaN(), - actualSummaries.get(i).containsNaN()); - Assert.assertEquals( - "Lower bounds in partition must match", - expectedSummaries.get(i).lowerBound(), - actualSummaries.get(i).lowerBound()); - Assert.assertEquals( - "Upper bounds in partition must match", - expectedSummaries.get(i).upperBound(), - actualSummaries.get(i).upperBound()); - } - } - - public static void assertEquals(ContentFile expected, ContentFile actual) { - if (expected == actual) { - return; - } - Assert.assertTrue("Shouldn't be null.", expected != null && actual != null); - Assert.assertEquals("SpecId", expected.specId(), actual.specId()); - Assert.assertEquals("Content", expected.content(), actual.content()); - Assert.assertEquals("Path", expected.path(), actual.path()); - Assert.assertEquals("Format", expected.format(), actual.format()); - Assert.assertEquals("Partition size", expected.partition().size(), actual.partition().size()); - for (int i = 0; i < expected.partition().size(); i++) { - Assert.assertEquals( - "Partition data at index " + i, - expected.partition().get(i, Object.class), - actual.partition().get(i, Object.class)); - } - Assert.assertEquals("Record count", expected.recordCount(), actual.recordCount()); - Assert.assertEquals("File size in bytes", expected.fileSizeInBytes(), actual.fileSizeInBytes()); - Assert.assertEquals("Column sizes", expected.columnSizes(), actual.columnSizes()); - Assert.assertEquals("Value counts", expected.valueCounts(), actual.valueCounts()); - Assert.assertEquals("Null value counts", expected.nullValueCounts(), actual.nullValueCounts()); - Assert.assertEquals("Lower bounds", expected.lowerBounds(), actual.lowerBounds()); - Assert.assertEquals("Upper bounds", expected.upperBounds(), actual.upperBounds()); - Assert.assertEquals("Key metadata", expected.keyMetadata(), actual.keyMetadata()); - Assert.assertEquals("Split offsets", expected.splitOffsets(), actual.splitOffsets()); - Assert.assertEquals( - "Equality field id list", actual.equalityFieldIds(), expected.equalityFieldIds()); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java deleted file mode 100644 index 4f71b5fe8d7c..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java +++ /dev/null @@ -1,350 +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.iceberg.flink; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.Map; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; -import org.apache.flink.table.catalog.Catalog; -import org.apache.flink.table.catalog.ObjectPath; -import org.apache.flink.types.Row; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.thrift.TException; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestIcebergConnector extends FlinkTestBase { - - private static final String TABLE_NAME = "test_table"; - - @ClassRule public static final TemporaryFolder WAREHOUSE = new TemporaryFolder(); - - private final String catalogName; - private final Map properties; - private final boolean isStreaming; - private volatile TableEnvironment tEnv; - - @Parameterized.Parameters(name = "catalogName = {0}, properties = {1}, isStreaming={2}") - public static Iterable parameters() { - return Lists.newArrayList( - // Create iceberg table in the hadoop catalog and default database. - new Object[] { - "testhadoop", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hadoop"), - true - }, - new Object[] { - "testhadoop", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hadoop", - "catalog-table", "not_existing_table"), - true - }, - new Object[] { - "testhadoop", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hadoop"), - false - }, - // Create iceberg table in the hadoop catalog and not_existing_db. - new Object[] { - "testhadoop", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hadoop", - "catalog-database", "not_existing_db"), - true - }, - new Object[] { - "testhadoop", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hadoop", - "catalog-database", "not_existing_db", - "catalog-table", "not_existing_table"), - true - }, - new Object[] { - "testhadoop", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hadoop", - "catalog-database", "not_existing_db"), - false - }, - // Create iceberg table in the hive catalog and default database. - new Object[] { - "testhive", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hive"), - true - }, - new Object[] { - "testhive", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hive", - "catalog-table", "not_existing_table"), - true - }, - new Object[] { - "testhive", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hive"), - false - }, - // Create iceberg table in the hive catalog and not_existing_db. - new Object[] { - "testhive", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hive", - "catalog-database", "not_existing_db"), - true - }, - new Object[] { - "testhive", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hive", - "catalog-database", "not_existing_db", - "catalog-table", "not_existing_table"), - true - }, - new Object[] { - "testhive", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hive", - "catalog-database", "not_existing_db"), - false - }); - } - - public TestIcebergConnector( - String catalogName, Map properties, boolean isStreaming) { - this.catalogName = catalogName; - this.properties = properties; - this.isStreaming = isStreaming; - } - - @Override - protected TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - if (tEnv == null) { - EnvironmentSettings.Builder settingsBuilder = EnvironmentSettings.newInstance(); - if (isStreaming) { - settingsBuilder.inStreamingMode(); - StreamExecutionEnvironment env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); - env.enableCheckpointing(400); - env.setMaxParallelism(2); - env.setParallelism(2); - tEnv = StreamTableEnvironment.create(env, settingsBuilder.build()); - } else { - settingsBuilder.inBatchMode(); - tEnv = TableEnvironment.create(settingsBuilder.build()); - } - // Set only one parallelism. - tEnv.getConfig() - .getConfiguration() - .set(CoreOptions.DEFAULT_PARALLELISM, 1) - .set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); - } - } - } - return tEnv; - } - - @After - public void after() throws TException { - sql("DROP TABLE IF EXISTS %s", TABLE_NAME); - - // Clean the created orphan databases and tables from hive-metastore. - if (isHiveCatalog()) { - HiveMetaStoreClient metaStoreClient = new HiveMetaStoreClient(hiveConf); - try { - metaStoreClient.dropTable(databaseName(), tableName()); - if (!isDefaultDatabaseName()) { - try { - metaStoreClient.dropDatabase(databaseName()); - } catch (Exception ignored) { - // Ignore - } - } - } finally { - metaStoreClient.close(); - } - } - } - - private void testCreateConnectorTable() { - Map tableProps = createTableProps(); - - // Create table under the flink's current database. - sql("CREATE TABLE %s (id BIGINT, data STRING) WITH %s", TABLE_NAME, toWithClause(tableProps)); - sql("INSERT INTO %s VALUES (1, 'AAA'), (2, 'BBB'), (3, 'CCC')", TABLE_NAME); - Assert.assertEquals( - "Should have expected rows", - Sets.newHashSet(Row.of(1L, "AAA"), Row.of(2L, "BBB"), Row.of(3L, "CCC")), - Sets.newHashSet(sql("SELECT * FROM %s", TABLE_NAME))); - - FlinkCatalogFactory factory = new FlinkCatalogFactory(); - Catalog flinkCatalog = factory.createCatalog(catalogName, tableProps, new Configuration()); - Assert.assertTrue( - "Should have created the expected database", flinkCatalog.databaseExists(databaseName())); - Assert.assertTrue( - "Should have created the expected table", - flinkCatalog.tableExists(new ObjectPath(databaseName(), tableName()))); - - // Drop and create it again. - sql("DROP TABLE %s", TABLE_NAME); - sql("CREATE TABLE %s (id BIGINT, data STRING) WITH %s", TABLE_NAME, toWithClause(tableProps)); - Assert.assertEquals( - "Should have expected rows", - Sets.newHashSet(Row.of(1L, "AAA"), Row.of(2L, "BBB"), Row.of(3L, "CCC")), - Sets.newHashSet(sql("SELECT * FROM %s", TABLE_NAME))); - } - - @Test - public void testCreateTableUnderDefaultDatabase() { - testCreateConnectorTable(); - } - - @Test - public void testCatalogDatabaseConflictWithFlinkDatabase() { - sql("CREATE DATABASE IF NOT EXISTS `%s`", databaseName()); - sql("USE `%s`", databaseName()); - - try { - testCreateConnectorTable(); - // Ensure that the table was created under the specific database. - Assertions.assertThatThrownBy( - () -> sql("CREATE TABLE `default_catalog`.`%s`.`%s`", databaseName(), TABLE_NAME)) - .isInstanceOf(org.apache.flink.table.api.TableException.class) - .hasMessageStartingWith("Could not execute CreateTable in path"); - } finally { - sql("DROP TABLE IF EXISTS `%s`.`%s`", databaseName(), TABLE_NAME); - if (!isDefaultDatabaseName()) { - sql("DROP DATABASE `%s`", databaseName()); - } - } - } - - @Test - public void testConnectorTableInIcebergCatalog() { - // Create the catalog properties - Map catalogProps = Maps.newHashMap(); - catalogProps.put("type", "iceberg"); - if (isHiveCatalog()) { - catalogProps.put("catalog-type", "hive"); - catalogProps.put(CatalogProperties.URI, FlinkCatalogTestBase.getURI(hiveConf)); - } else { - catalogProps.put("catalog-type", "hadoop"); - } - catalogProps.put(CatalogProperties.WAREHOUSE_LOCATION, createWarehouse()); - - // Create the table properties - Map tableProps = createTableProps(); - - // Create a connector table in an iceberg catalog. - sql("CREATE CATALOG `test_catalog` WITH %s", toWithClause(catalogProps)); - try { - Assertions.assertThatThrownBy( - () -> - sql( - "CREATE TABLE `test_catalog`.`%s`.`%s` (id BIGINT, data STRING) WITH %s", - FlinkCatalogFactory.DEFAULT_DATABASE_NAME, - TABLE_NAME, - toWithClause(tableProps))) - .cause() - .isInstanceOf(IllegalArgumentException.class) - .hasMessage( - "Cannot create the table with 'connector'='iceberg' table property in an iceberg catalog, " - + "Please create table with 'connector'='iceberg' property in a non-iceberg catalog or " - + "create table without 'connector'='iceberg' related properties in an iceberg table."); - } finally { - sql("DROP CATALOG IF EXISTS `test_catalog`"); - } - } - - private Map createTableProps() { - Map tableProps = Maps.newHashMap(properties); - tableProps.put("catalog-name", catalogName); - tableProps.put(CatalogProperties.WAREHOUSE_LOCATION, createWarehouse()); - if (isHiveCatalog()) { - tableProps.put(CatalogProperties.URI, FlinkCatalogTestBase.getURI(hiveConf)); - } - return tableProps; - } - - private boolean isHiveCatalog() { - return "testhive".equalsIgnoreCase(catalogName); - } - - private boolean isDefaultDatabaseName() { - return FlinkCatalogFactory.DEFAULT_DATABASE_NAME.equalsIgnoreCase(databaseName()); - } - - private String tableName() { - return properties.getOrDefault("catalog-table", TABLE_NAME); - } - - private String databaseName() { - return properties.getOrDefault("catalog-database", "default_database"); - } - - private String toWithClause(Map props) { - return FlinkCatalogTestBase.toWithClause(props); - } - - private static String createWarehouse() { - try { - return String.format("file://%s", WAREHOUSE.newFolder().getAbsolutePath()); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java deleted file mode 100644 index 6bd94e9ca61c..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java +++ /dev/null @@ -1,174 +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.iceberg.flink; - -import static org.apache.iceberg.types.Types.NestedField.optional; -import static org.apache.iceberg.types.Types.NestedField.required; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.File; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import java.nio.ByteBuffer; -import java.nio.ByteOrder; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; -import org.apache.flink.core.memory.DataInputDeserializer; -import org.apache.flink.core.memory.DataOutputSerializer; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFiles; -import org.apache.iceberg.GenericManifestFile; -import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.ManifestFiles; -import org.apache.iceberg.ManifestWriter; -import org.apache.iceberg.Metrics; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.hadoop.HadoopFileIO; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.io.OutputFile; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class TestManifestFileSerialization { - - private static final Schema SCHEMA = - new Schema( - required(1, "id", Types.LongType.get()), - optional(2, "data", Types.StringType.get()), - required(3, "date", Types.StringType.get()), - required(4, "double", Types.DoubleType.get())); - - private static final PartitionSpec SPEC = - PartitionSpec.builderFor(SCHEMA).identity("double").build(); - - private static final DataFile FILE_A = - DataFiles.builder(SPEC) - .withPath("/path/to/data-1.parquet") - .withFileSizeInBytes(0) - .withPartition(org.apache.iceberg.TestHelpers.Row.of(1D)) - .withPartitionPath("double=1") - .withMetrics( - new Metrics( - 5L, - null, // no column sizes - ImmutableMap.of(1, 5L, 2, 3L), // value count - ImmutableMap.of(1, 0L, 2, 2L), // null count - ImmutableMap.of(), // nan count - ImmutableMap.of(1, longToBuffer(0L)), // lower bounds - ImmutableMap.of(1, longToBuffer(4L)) // upper bounds - )) - .build(); - - private static final DataFile FILE_B = - DataFiles.builder(SPEC) - .withPath("/path/to/data-2.parquet") - .withFileSizeInBytes(0) - .withPartition(org.apache.iceberg.TestHelpers.Row.of(Double.NaN)) - .withPartitionPath("double=NaN") - .withMetrics( - new Metrics( - 1L, - null, // no column sizes - ImmutableMap.of(1, 1L, 4, 1L), // value count - ImmutableMap.of(1, 0L, 2, 0L), // null count - ImmutableMap.of(4, 1L), // nan count - ImmutableMap.of(1, longToBuffer(0L)), // lower bounds - ImmutableMap.of(1, longToBuffer(1L)) // upper bounds - )) - .build(); - - private static final FileIO FILE_IO = new HadoopFileIO(new Configuration()); - - @Rule public TemporaryFolder temp = new TemporaryFolder(); - - @Test - public void testKryoSerialization() throws IOException { - KryoSerializer kryo = - new KryoSerializer<>(ManifestFile.class, new ExecutionConfig()); - - DataOutputSerializer outputView = new DataOutputSerializer(1024); - - ManifestFile manifest = writeManifest(FILE_A, FILE_B); - - kryo.serialize(manifest, outputView); - kryo.serialize(manifest.copy(), outputView); - kryo.serialize(GenericManifestFile.copyOf(manifest).build(), outputView); - - DataInputDeserializer inputView = new DataInputDeserializer(outputView.getCopyOfBuffer()); - ManifestFile m1 = kryo.deserialize(inputView); - ManifestFile m2 = kryo.deserialize(inputView); - ManifestFile m3 = kryo.deserialize(inputView); - - TestHelpers.assertEquals(manifest, m1); - TestHelpers.assertEquals(manifest, m2); - TestHelpers.assertEquals(manifest, m3); - } - - @Test - public void testJavaSerialization() throws Exception { - ByteArrayOutputStream bytes = new ByteArrayOutputStream(); - - ManifestFile manifest = writeManifest(FILE_A, FILE_B); - - try (ObjectOutputStream out = new ObjectOutputStream(bytes)) { - out.writeObject(manifest); - out.writeObject(manifest.copy()); - out.writeObject(GenericManifestFile.copyOf(manifest).build()); - } - - try (ObjectInputStream in = - new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { - for (int i = 0; i < 3; i += 1) { - Object obj = in.readObject(); - Assertions.assertThat(obj).as("Should be a ManifestFile").isInstanceOf(ManifestFile.class); - TestHelpers.assertEquals(manifest, (ManifestFile) obj); - } - } - } - - private ManifestFile writeManifest(DataFile... files) throws IOException { - File manifestFile = temp.newFile("input.m0.avro"); - Assert.assertTrue(manifestFile.delete()); - OutputFile outputFile = FILE_IO.newOutputFile(manifestFile.getCanonicalPath()); - - ManifestWriter writer = ManifestFiles.write(SPEC, outputFile); - try { - for (DataFile file : files) { - writer.add(file); - } - } finally { - writer.close(); - } - - return writer.toManifestFile(); - } - - private static ByteBuffer longToBuffer(long value) { - return ByteBuffer.allocate(8).order(ByteOrder.LITTLE_ENDIAN).putLong(0, value); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java deleted file mode 100644 index c78fa51215dd..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java +++ /dev/null @@ -1,93 +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.iceberg.flink; - -import java.util.Iterator; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.RecordWrapperTest; -import org.apache.iceberg.Schema; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.data.InternalRecordWrapper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.data.RandomRowData; -import org.apache.iceberg.util.StructLikeWrapper; -import org.assertj.core.api.Assertions; -import org.junit.Assert; - -public class TestRowDataWrapper extends RecordWrapperTest { - - /** - * Flink's time type has been truncated to millis seconds, so we need a customized assert method - * to check the values. - */ - @Override - public void testTime() { - generateAndValidate( - new Schema(TIME.fields()), - (message, expectedWrapper, actualWrapper) -> { - for (int pos = 0; pos < TIME.fields().size(); pos++) { - Object expected = expectedWrapper.get().get(pos, Object.class); - Object actual = actualWrapper.get().get(pos, Object.class); - if (expected == actual) { - return; - } - - Assertions.assertThat(actual).isNotNull(); - Assertions.assertThat(expected).isNotNull(); - - int expectedMilliseconds = (int) ((long) expected / 1000_000); - int actualMilliseconds = (int) ((long) actual / 1000_000); - Assert.assertEquals(message, expectedMilliseconds, actualMilliseconds); - } - }); - } - - @Override - protected void generateAndValidate(Schema schema, RecordWrapperTest.AssertMethod assertMethod) { - int numRecords = 100; - Iterable recordList = RandomGenericData.generate(schema, numRecords, 101L); - Iterable rowDataList = RandomRowData.generate(schema, numRecords, 101L); - - InternalRecordWrapper recordWrapper = new InternalRecordWrapper(schema.asStruct()); - RowDataWrapper rowDataWrapper = - new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); - - Iterator actual = recordList.iterator(); - Iterator expected = rowDataList.iterator(); - - StructLikeWrapper actualWrapper = StructLikeWrapper.forType(schema.asStruct()); - StructLikeWrapper expectedWrapper = StructLikeWrapper.forType(schema.asStruct()); - for (int i = 0; i < numRecords; i++) { - Assert.assertTrue("Should have more records", actual.hasNext()); - Assert.assertTrue("Should have more RowData", expected.hasNext()); - - StructLike recordStructLike = recordWrapper.wrap(actual.next()); - StructLike rowDataStructLike = rowDataWrapper.wrap(expected.next()); - - assertMethod.assertEquals( - "Should have expected StructLike values", - actualWrapper.set(recordStructLike), - expectedWrapper.set(rowDataStructLike)); - } - - Assert.assertFalse("Shouldn't have more record", actual.hasNext()); - Assert.assertFalse("Shouldn't have more RowData", expected.hasNext()); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java deleted file mode 100644 index 4ad302dde436..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java +++ /dev/null @@ -1,57 +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.iceberg.flink; - -import java.io.File; -import org.apache.iceberg.Table; -import org.apache.iceberg.TestTables; - -public class TestTableLoader implements TableLoader { - private File dir; - - public static TableLoader of(String dir) { - return new TestTableLoader(dir); - } - - public TestTableLoader(String dir) { - this.dir = new File(dir); - } - - @Override - public void open() {} - - @Override - public boolean isOpen() { - return true; - } - - @Override - public Table loadTable() { - return TestTables.load(dir, "test"); - } - - @Override - @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) - public TableLoader clone() { - return new TestTableLoader(dir.getAbsolutePath()); - } - - @Override - public void close() {} -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java deleted file mode 100644 index 27124d93fef4..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java +++ /dev/null @@ -1,110 +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.iceberg.flink; - -import static org.apache.iceberg.flink.TestHelpers.roundTripKryoSerialize; -import static org.apache.iceberg.types.Types.NestedField.optional; -import static org.apache.iceberg.types.Types.NestedField.required; - -import java.io.File; -import java.io.IOException; -import java.util.Map; -import org.apache.iceberg.HasTableOperations; -import org.apache.iceberg.MetadataTableType; -import org.apache.iceberg.MetadataTableUtils; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SerializableTable; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableOperations; -import org.apache.iceberg.TestHelpers; -import org.apache.iceberg.Transaction; -import org.apache.iceberg.hadoop.HadoopTables; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class TestTableSerialization { - private static final HadoopTables TABLES = new HadoopTables(); - - private static final Schema SCHEMA = - new Schema( - required(1, "id", Types.LongType.get()), - optional(2, "data", Types.StringType.get()), - required(3, "date", Types.StringType.get()), - optional(4, "double", Types.DoubleType.get())); - - private static final PartitionSpec SPEC = - PartitionSpec.builderFor(SCHEMA).identity("date").build(); - - private static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); - - @Rule public TemporaryFolder temp = new TemporaryFolder(); - private Table table; - - @Before - public void initTable() throws IOException { - Map props = ImmutableMap.of("k1", "v1", "k2", "v2"); - - File tableLocation = temp.newFolder(); - Assert.assertTrue(tableLocation.delete()); - - this.table = TABLES.create(SCHEMA, SPEC, SORT_ORDER, props, tableLocation.toString()); - } - - @Test - public void testSerializableTableKryoSerialization() throws IOException { - SerializableTable serializableTable = (SerializableTable) SerializableTable.copyOf(table); - TestHelpers.assertSerializedAndLoadedMetadata( - table, roundTripKryoSerialize(SerializableTable.class, serializableTable)); - } - - @Test - public void testSerializableMetadataTableKryoSerialization() throws IOException { - for (MetadataTableType type : MetadataTableType.values()) { - TableOperations ops = ((HasTableOperations) table).operations(); - Table metadataTable = - MetadataTableUtils.createMetadataTableInstance(ops, table.name(), "meta", type); - SerializableTable serializableMetadataTable = - (SerializableTable) SerializableTable.copyOf(metadataTable); - - TestHelpers.assertSerializedAndLoadedMetadata( - metadataTable, - roundTripKryoSerialize(SerializableTable.class, serializableMetadataTable)); - } - } - - @Test - public void testSerializableTransactionTableKryoSerialization() throws IOException { - Transaction txn = table.newTransaction(); - - txn.updateProperties().set("k1", "v1").commit(); - - Table txnTable = txn.table(); - SerializableTable serializableTxnTable = (SerializableTable) SerializableTable.copyOf(txnTable); - - TestHelpers.assertSerializedMetadata( - txnTable, roundTripKryoSerialize(SerializableTable.class, serializableTxnTable)); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java deleted file mode 100644 index 07e5ca051da5..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java +++ /dev/null @@ -1,499 +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.iceberg.flink.actions; - -import static org.apache.iceberg.flink.SimpleDataUtil.RECORD; - -import java.io.File; -import java.io.IOException; -import java.util.Collection; -import java.util.List; -import java.util.Set; -import java.util.UUID; -import java.util.stream.Collectors; -import org.apache.commons.lang3.StringUtils; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.types.Row; -import org.apache.iceberg.ContentFile; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFiles; -import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.FileContent; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.Files; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.actions.RewriteDataFilesActionResult; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.data.GenericAppenderFactory; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.exceptions.ValidationException; -import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.flink.FlinkCatalogTestBase; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.Pair; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestRewriteDataFilesAction extends FlinkCatalogTestBase { - - private static final String TABLE_NAME_UNPARTITIONED = "test_table_unpartitioned"; - private static final String TABLE_NAME_PARTITIONED = "test_table_partitioned"; - private static final String TABLE_NAME_WITH_PK = "test_table_with_pk"; - private final FileFormat format; - private Table icebergTableUnPartitioned; - private Table icebergTablePartitioned; - private Table icebergTableWithPk; - - public TestRewriteDataFilesAction( - String catalogName, Namespace baseNamespace, FileFormat format) { - super(catalogName, baseNamespace); - this.format = format; - } - - @Override - protected TableEnvironment getTableEnv() { - super.getTableEnv().getConfig().getConfiguration().set(CoreOptions.DEFAULT_PARALLELISM, 1); - return super.getTableEnv(); - } - - @Parameterized.Parameters(name = "catalogName={0}, baseNamespace={1}, format={2}") - public static Iterable parameters() { - List parameters = Lists.newArrayList(); - for (FileFormat format : - new FileFormat[] {FileFormat.AVRO, FileFormat.ORC, FileFormat.PARQUET}) { - for (Object[] catalogParams : FlinkCatalogTestBase.parameters()) { - String catalogName = (String) catalogParams[0]; - Namespace baseNamespace = (Namespace) catalogParams[1]; - parameters.add(new Object[] {catalogName, baseNamespace, format}); - } - } - return parameters; - } - - @Rule public TemporaryFolder temp = new TemporaryFolder(); - - @Override - @Before - public void before() { - super.before(); - sql("CREATE DATABASE %s", flinkDatabase); - sql("USE CATALOG %s", catalogName); - sql("USE %s", DATABASE); - sql( - "CREATE TABLE %s (id int, data varchar) with ('write.format.default'='%s')", - TABLE_NAME_UNPARTITIONED, format.name()); - icebergTableUnPartitioned = - validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME_UNPARTITIONED)); - - sql( - "CREATE TABLE %s (id int, data varchar,spec varchar) " - + " PARTITIONED BY (data,spec) with ('write.format.default'='%s')", - TABLE_NAME_PARTITIONED, format.name()); - icebergTablePartitioned = - validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME_PARTITIONED)); - - sql( - "CREATE TABLE %s (id int, data varchar, PRIMARY KEY(`id`) NOT ENFORCED) with ('write.format.default'='%s', 'format-version'='2')", - TABLE_NAME_WITH_PK, format.name()); - icebergTableWithPk = - validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME_WITH_PK)); - } - - @Override - @After - public void clean() { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME_UNPARTITIONED); - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME_PARTITIONED); - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME_WITH_PK); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); - super.clean(); - } - - @Test - public void testRewriteDataFilesEmptyTable() throws Exception { - Assert.assertNull("Table must be empty", icebergTableUnPartitioned.currentSnapshot()); - Actions.forTable(icebergTableUnPartitioned).rewriteDataFiles().execute(); - Assert.assertNull("Table must stay empty", icebergTableUnPartitioned.currentSnapshot()); - } - - @Test - public void testRewriteDataFilesUnpartitionedTable() throws Exception { - sql("INSERT INTO %s SELECT 1, 'hello'", TABLE_NAME_UNPARTITIONED); - sql("INSERT INTO %s SELECT 2, 'world'", TABLE_NAME_UNPARTITIONED); - - icebergTableUnPartitioned.refresh(); - - CloseableIterable tasks = icebergTableUnPartitioned.newScan().planFiles(); - List dataFiles = - Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); - Assert.assertEquals("Should have 2 data files before rewrite", 2, dataFiles.size()); - - RewriteDataFilesActionResult result = - Actions.forTable(icebergTableUnPartitioned).rewriteDataFiles().execute(); - - Assert.assertEquals("Action should rewrite 2 data files", 2, result.deletedDataFiles().size()); - Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFiles().size()); - - icebergTableUnPartitioned.refresh(); - - CloseableIterable tasks1 = icebergTableUnPartitioned.newScan().planFiles(); - List dataFiles1 = - Lists.newArrayList(CloseableIterable.transform(tasks1, FileScanTask::file)); - Assert.assertEquals("Should have 1 data files after rewrite", 1, dataFiles1.size()); - - // Assert the table records as expected. - SimpleDataUtil.assertTableRecords( - icebergTableUnPartitioned, - Lists.newArrayList( - SimpleDataUtil.createRecord(1, "hello"), SimpleDataUtil.createRecord(2, "world"))); - } - - @Test - public void testRewriteDataFilesPartitionedTable() throws Exception { - sql("INSERT INTO %s SELECT 1, 'hello' ,'a'", TABLE_NAME_PARTITIONED); - sql("INSERT INTO %s SELECT 2, 'hello' ,'a'", TABLE_NAME_PARTITIONED); - sql("INSERT INTO %s SELECT 3, 'world' ,'b'", TABLE_NAME_PARTITIONED); - sql("INSERT INTO %s SELECT 4, 'world' ,'b'", TABLE_NAME_PARTITIONED); - - icebergTablePartitioned.refresh(); - - CloseableIterable tasks = icebergTablePartitioned.newScan().planFiles(); - List dataFiles = - Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); - Assert.assertEquals("Should have 4 data files before rewrite", 4, dataFiles.size()); - - RewriteDataFilesActionResult result = - Actions.forTable(icebergTablePartitioned).rewriteDataFiles().execute(); - - Assert.assertEquals("Action should rewrite 4 data files", 4, result.deletedDataFiles().size()); - Assert.assertEquals("Action should add 2 data file", 2, result.addedDataFiles().size()); - - icebergTablePartitioned.refresh(); - - CloseableIterable tasks1 = icebergTablePartitioned.newScan().planFiles(); - List dataFiles1 = - Lists.newArrayList(CloseableIterable.transform(tasks1, FileScanTask::file)); - Assert.assertEquals("Should have 2 data files after rewrite", 2, dataFiles1.size()); - - // Assert the table records as expected. - Schema schema = - new Schema( - Types.NestedField.optional(1, "id", Types.IntegerType.get()), - Types.NestedField.optional(2, "data", Types.StringType.get()), - Types.NestedField.optional(3, "spec", Types.StringType.get())); - - Record record = GenericRecord.create(schema); - SimpleDataUtil.assertTableRecords( - icebergTablePartitioned, - Lists.newArrayList( - record.copy("id", 1, "data", "hello", "spec", "a"), - record.copy("id", 2, "data", "hello", "spec", "a"), - record.copy("id", 3, "data", "world", "spec", "b"), - record.copy("id", 4, "data", "world", "spec", "b"))); - } - - @Test - public void testRewriteDataFilesWithFilter() throws Exception { - sql("INSERT INTO %s SELECT 1, 'hello' ,'a'", TABLE_NAME_PARTITIONED); - sql("INSERT INTO %s SELECT 2, 'hello' ,'a'", TABLE_NAME_PARTITIONED); - sql("INSERT INTO %s SELECT 3, 'world' ,'a'", TABLE_NAME_PARTITIONED); - sql("INSERT INTO %s SELECT 4, 'world' ,'b'", TABLE_NAME_PARTITIONED); - sql("INSERT INTO %s SELECT 5, 'world' ,'b'", TABLE_NAME_PARTITIONED); - - icebergTablePartitioned.refresh(); - - CloseableIterable tasks = icebergTablePartitioned.newScan().planFiles(); - List dataFiles = - Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); - Assert.assertEquals("Should have 5 data files before rewrite", 5, dataFiles.size()); - - RewriteDataFilesActionResult result = - Actions.forTable(icebergTablePartitioned) - .rewriteDataFiles() - .filter(Expressions.equal("spec", "a")) - .filter(Expressions.startsWith("data", "he")) - .execute(); - - Assert.assertEquals("Action should rewrite 2 data files", 2, result.deletedDataFiles().size()); - Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFiles().size()); - - icebergTablePartitioned.refresh(); - - CloseableIterable tasks1 = icebergTablePartitioned.newScan().planFiles(); - List dataFiles1 = - Lists.newArrayList(CloseableIterable.transform(tasks1, FileScanTask::file)); - Assert.assertEquals("Should have 4 data files after rewrite", 4, dataFiles1.size()); - - // Assert the table records as expected. - Schema schema = - new Schema( - Types.NestedField.optional(1, "id", Types.IntegerType.get()), - Types.NestedField.optional(2, "data", Types.StringType.get()), - Types.NestedField.optional(3, "spec", Types.StringType.get())); - - Record record = GenericRecord.create(schema); - SimpleDataUtil.assertTableRecords( - icebergTablePartitioned, - Lists.newArrayList( - record.copy("id", 1, "data", "hello", "spec", "a"), - record.copy("id", 2, "data", "hello", "spec", "a"), - record.copy("id", 3, "data", "world", "spec", "a"), - record.copy("id", 4, "data", "world", "spec", "b"), - record.copy("id", 5, "data", "world", "spec", "b"))); - } - - @Test - public void testRewriteLargeTableHasResiduals() throws IOException { - // all records belong to the same partition - List records1 = Lists.newArrayList(); - List records2 = Lists.newArrayList(); - List expected = Lists.newArrayList(); - for (int i = 0; i < 100; i++) { - int id = i; - String data = String.valueOf(i % 3); - if (i % 2 == 0) { - records1.add("(" + id + ",'" + data + "')"); - } else { - records2.add("(" + id + ",'" + data + "')"); - } - Record record = RECORD.copy(); - record.setField("id", id); - record.setField("data", data); - expected.add(record); - } - - sql("INSERT INTO %s values " + StringUtils.join(records1, ","), TABLE_NAME_UNPARTITIONED); - sql("INSERT INTO %s values " + StringUtils.join(records2, ","), TABLE_NAME_UNPARTITIONED); - - icebergTableUnPartitioned.refresh(); - - CloseableIterable tasks = - icebergTableUnPartitioned - .newScan() - .ignoreResiduals() - .filter(Expressions.equal("data", "0")) - .planFiles(); - for (FileScanTask task : tasks) { - Assert.assertEquals("Residuals must be ignored", Expressions.alwaysTrue(), task.residual()); - } - List dataFiles = - Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); - Assert.assertEquals("Should have 2 data files before rewrite", 2, dataFiles.size()); - - Actions actions = Actions.forTable(icebergTableUnPartitioned); - - RewriteDataFilesActionResult result = - actions.rewriteDataFiles().filter(Expressions.equal("data", "0")).execute(); - Assert.assertEquals("Action should rewrite 2 data files", 2, result.deletedDataFiles().size()); - Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFiles().size()); - - // Assert the table records as expected. - SimpleDataUtil.assertTableRecords(icebergTableUnPartitioned, expected); - } - - /** - * a test case to test avoid repeate compress - * - *

    If datafile cannot be combined to CombinedScanTask with other DataFiles, the size of the - * CombinedScanTask list size is 1, so we remove these CombinedScanTasks to avoid compressed - * repeatedly. - * - *

    In this test case,we generated 3 data files and set targetSizeInBytes greater than the - * largest file size so that it cannot be combined a CombinedScanTask with other datafiles. The - * datafile with the largest file size will not be compressed. - * - * @throws IOException IOException - */ - @Test - public void testRewriteAvoidRepeateCompress() throws IOException { - List expected = Lists.newArrayList(); - Schema schema = icebergTableUnPartitioned.schema(); - GenericAppenderFactory genericAppenderFactory = new GenericAppenderFactory(schema); - File file = temp.newFile(); - int count = 0; - try (FileAppender fileAppender = - genericAppenderFactory.newAppender(Files.localOutput(file), format)) { - long filesize = 20000; - for (; fileAppender.length() < filesize; count++) { - Record record = SimpleDataUtil.createRecord(count, UUID.randomUUID().toString()); - fileAppender.add(record); - expected.add(record); - } - } - - DataFile dataFile = - DataFiles.builder(icebergTableUnPartitioned.spec()) - .withPath(file.getAbsolutePath()) - .withFileSizeInBytes(file.length()) - .withFormat(format) - .withRecordCount(count) - .build(); - - icebergTableUnPartitioned.newAppend().appendFile(dataFile).commit(); - - sql("INSERT INTO %s SELECT 1,'a' ", TABLE_NAME_UNPARTITIONED); - sql("INSERT INTO %s SELECT 2,'b' ", TABLE_NAME_UNPARTITIONED); - - icebergTableUnPartitioned.refresh(); - - CloseableIterable tasks = icebergTableUnPartitioned.newScan().planFiles(); - List dataFiles = - Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); - Assert.assertEquals("Should have 3 data files before rewrite", 3, dataFiles.size()); - - Actions actions = Actions.forTable(icebergTableUnPartitioned); - - long targetSizeInBytes = file.length() + 10; - RewriteDataFilesActionResult result = - actions - .rewriteDataFiles() - .targetSizeInBytes(targetSizeInBytes) - .splitOpenFileCost(1) - .execute(); - Assert.assertEquals("Action should rewrite 2 data files", 2, result.deletedDataFiles().size()); - Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFiles().size()); - - icebergTableUnPartitioned.refresh(); - - CloseableIterable tasks1 = icebergTableUnPartitioned.newScan().planFiles(); - List dataFilesRewrote = - Lists.newArrayList(CloseableIterable.transform(tasks1, FileScanTask::file)); - Assert.assertEquals("Should have 2 data files after rewrite", 2, dataFilesRewrote.size()); - - // the biggest file do not be rewrote - List rewroteDataFileNames = - dataFilesRewrote.stream().map(ContentFile::path).collect(Collectors.toList()); - Assert.assertTrue(rewroteDataFileNames.contains(file.getAbsolutePath())); - - // Assert the table records as expected. - expected.add(SimpleDataUtil.createRecord(1, "a")); - expected.add(SimpleDataUtil.createRecord(2, "b")); - SimpleDataUtil.assertTableRecords(icebergTableUnPartitioned, expected); - } - - @Test - public void testRewriteNoConflictWithEqualityDeletes() throws IOException { - // Add 2 data files - sql("INSERT INTO %s SELECT 1, 'hello'", TABLE_NAME_WITH_PK); - sql("INSERT INTO %s SELECT 2, 'world'", TABLE_NAME_WITH_PK); - - // Load 2 stale tables to pass to rewrite actions - // Since the first rewrite will refresh stale1, we need another stale2 for the second rewrite - Table stale1 = - validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME_WITH_PK)); - Table stale2 = - validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME_WITH_PK)); - - // Add 1 data file and 1 equality-delete file - sql("INSERT INTO %s /*+ OPTIONS('upsert-enabled'='true')*/ SELECT 1, 'hi'", TABLE_NAME_WITH_PK); - - icebergTableWithPk.refresh(); - Assert.assertEquals( - "The latest sequence number should be greater than that of the stale snapshot", - stale1.currentSnapshot().sequenceNumber() + 1, - icebergTableWithPk.currentSnapshot().sequenceNumber()); - - CloseableIterable tasks = icebergTableWithPk.newScan().planFiles(); - List dataFiles = - Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); - Set deleteFiles = - Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::deletes)).stream() - .flatMap(Collection::stream) - .collect(Collectors.toSet()); - Assert.assertEquals("Should have 3 data files before rewrite", 3, dataFiles.size()); - Assert.assertEquals("Should have 1 delete file before rewrite", 1, deleteFiles.size()); - Assert.assertSame( - "The 1 delete file should be an equality-delete file", - Iterables.getOnlyElement(deleteFiles).content(), - FileContent.EQUALITY_DELETES); - shouldHaveDataAndFileSequenceNumbers( - TABLE_NAME_WITH_PK, - ImmutableList.of(Pair.of(1L, 1L), Pair.of(2L, 2L), Pair.of(3L, 3L), Pair.of(3L, 3L))); - - Assertions.assertThatThrownBy( - () -> - Actions.forTable(stale1) - .rewriteDataFiles() - .useStartingSequenceNumber(false) - .execute(), - "Rewrite using new sequence number should fail") - .isInstanceOf(ValidationException.class); - - // Rewrite using the starting sequence number should succeed - RewriteDataFilesActionResult result = - Actions.forTable(stale2).rewriteDataFiles().useStartingSequenceNumber(true).execute(); - - // Should not rewrite files from the new commit - Assert.assertEquals("Action should rewrite 2 data files", 2, result.deletedDataFiles().size()); - Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFiles().size()); - // The 2 older files with file-sequence-number <= 2 should be rewritten into a new file. - // The new file is the one with file-sequence-number == 4. - // The new file should use rewrite's starting-sequence-number 2 as its data-sequence-number. - shouldHaveDataAndFileSequenceNumbers( - TABLE_NAME_WITH_PK, ImmutableList.of(Pair.of(3L, 3L), Pair.of(3L, 3L), Pair.of(2L, 4L))); - - // Assert the table records as expected. - SimpleDataUtil.assertTableRecords( - icebergTableWithPk, - Lists.newArrayList( - SimpleDataUtil.createRecord(1, "hi"), SimpleDataUtil.createRecord(2, "world"))); - } - - /** - * Assert that data files and delete files in the table should have expected data sequence numbers - * and file sequence numbers - * - * @param tableName table name - * @param expectedSequenceNumbers list of {@link Pair}'s. Each {@link Pair} contains - * (expectedDataSequenceNumber, expectedFileSequenceNumber) of a file. - */ - private void shouldHaveDataAndFileSequenceNumbers( - String tableName, List> expectedSequenceNumbers) { - // "status < 2" for added or existing entries - List liveEntries = sql("SELECT * FROM %s$entries WHERE status < 2", tableName); - - List> actualSequenceNumbers = - liveEntries.stream() - .map( - row -> - Pair.of( - row.getFieldAs("sequence_number"), row.getFieldAs("file_sequence_number"))) - .collect(Collectors.toList()); - Assertions.assertThat(actualSequenceNumbers).hasSameElementsAs(expectedSequenceNumbers); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java deleted file mode 100644 index cc58d9817ac6..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java +++ /dev/null @@ -1,38 +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.iceberg.flink.data; - -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.RowDataConverter; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; - -public class RandomRowData { - private RandomRowData() {} - - public static Iterable generate(Schema schema, int numRecords, long seed) { - return convert(schema, RandomGenericData.generate(schema, numRecords, seed)); - } - - public static Iterable convert(Schema schema, Iterable records) { - return Iterables.transform(records, record -> RowDataConverter.convert(schema, record)); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java deleted file mode 100644 index 74b1da6007e6..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java +++ /dev/null @@ -1,50 +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.iceberg.flink.data; - -import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.conversion.DataStructureConverter; -import org.apache.flink.table.data.conversion.DataStructureConverters; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.utils.TypeConversions; -import org.apache.flink.types.Row; - -public class RowDataToRowMapper extends RichMapFunction { - - private final RowType rowType; - - private transient DataStructureConverter converter; - - public RowDataToRowMapper(RowType rowType) { - this.rowType = rowType; - } - - @Override - public void open(Configuration parameters) throws Exception { - this.converter = - DataStructureConverters.getConverter(TypeConversions.fromLogicalToDataType(rowType)); - } - - @Override - public Row map(RowData value) throws Exception { - return (Row) converter.toExternal(value); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java deleted file mode 100644 index e8aab824ea2d..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java +++ /dev/null @@ -1,184 +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.iceberg.flink.data; - -import java.io.File; -import java.io.IOException; -import java.math.BigDecimal; -import java.sql.Date; -import java.sql.Time; -import java.util.Iterator; -import java.util.List; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.Files; -import org.apache.iceberg.Schema; -import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.data.DataTest; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.data.avro.DataReader; -import org.apache.iceberg.data.avro.DataWriter; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.DateTimeUtil; -import org.junit.Assert; -import org.junit.Test; - -public class TestFlinkAvroReaderWriter extends DataTest { - - private static final int NUM_RECORDS = 100; - - private static final Schema SCHEMA_NUM_TYPE = - new Schema( - Types.NestedField.optional(1, "id", Types.IntegerType.get()), - Types.NestedField.optional(2, "int", Types.IntegerType.get()), - Types.NestedField.optional(3, "float", Types.FloatType.get()), - Types.NestedField.optional(4, "double", Types.DoubleType.get()), - Types.NestedField.optional(5, "date", Types.DateType.get()), - Types.NestedField.optional(6, "time", Types.TimeType.get()), - Types.NestedField.optional(7, "timestamp", Types.TimestampType.withoutZone()), - Types.NestedField.optional(8, "bigint", Types.LongType.get()), - Types.NestedField.optional(9, "decimal", Types.DecimalType.of(4, 2))); - - @Override - protected void writeAndValidate(Schema schema) throws IOException { - List expectedRecords = RandomGenericData.generate(schema, NUM_RECORDS, 1991L); - writeAndValidate(schema, expectedRecords, NUM_RECORDS); - } - - private void writeAndValidate(Schema schema, List expectedRecords, int numRecord) - throws IOException { - RowType flinkSchema = FlinkSchemaUtil.convert(schema); - List expectedRows = Lists.newArrayList(RandomRowData.convert(schema, expectedRecords)); - - File recordsFile = temp.newFile(); - Assert.assertTrue("Delete should succeed", recordsFile.delete()); - - // Write the expected records into AVRO file, then read them into RowData and assert with the - // expected Record list. - try (FileAppender writer = - Avro.write(Files.localOutput(recordsFile)) - .schema(schema) - .createWriterFunc(DataWriter::create) - .build()) { - writer.addAll(expectedRecords); - } - - try (CloseableIterable reader = - Avro.read(Files.localInput(recordsFile)) - .project(schema) - .createReaderFunc(FlinkAvroReader::new) - .build()) { - Iterator expected = expectedRecords.iterator(); - Iterator rows = reader.iterator(); - for (int i = 0; i < numRecord; i++) { - Assert.assertTrue("Should have expected number of records", rows.hasNext()); - TestHelpers.assertRowData(schema.asStruct(), flinkSchema, expected.next(), rows.next()); - } - Assert.assertFalse("Should not have extra records", rows.hasNext()); - } - - File rowDataFile = temp.newFile(); - Assert.assertTrue("Delete should succeed", rowDataFile.delete()); - - // Write the expected RowData into AVRO file, then read them into Record and assert with the - // expected RowData list. - try (FileAppender writer = - Avro.write(Files.localOutput(rowDataFile)) - .schema(schema) - .createWriterFunc(ignore -> new FlinkAvroWriter(flinkSchema)) - .build()) { - writer.addAll(expectedRows); - } - - try (CloseableIterable reader = - Avro.read(Files.localInput(rowDataFile)) - .project(schema) - .createReaderFunc(DataReader::create) - .build()) { - Iterator expected = expectedRows.iterator(); - Iterator records = reader.iterator(); - for (int i = 0; i < numRecord; i += 1) { - Assert.assertTrue("Should have expected number of records", records.hasNext()); - TestHelpers.assertRowData(schema.asStruct(), flinkSchema, records.next(), expected.next()); - } - Assert.assertFalse("Should not have extra records", records.hasNext()); - } - } - - private Record recordNumType( - int id, - int intV, - float floatV, - double doubleV, - long date, - long time, - long timestamp, - long bigint, - double decimal) { - Record record = GenericRecord.create(SCHEMA_NUM_TYPE); - record.setField("id", id); - record.setField("int", intV); - record.setField("float", floatV); - record.setField("double", doubleV); - record.setField( - "date", DateTimeUtil.dateFromDays((int) new Date(date).toLocalDate().toEpochDay())); - record.setField("time", new Time(time).toLocalTime()); - record.setField("timestamp", DateTimeUtil.timestampFromMicros(timestamp * 1000)); - record.setField("bigint", bigint); - record.setField("decimal", BigDecimal.valueOf(decimal)); - return record; - } - - @Test - public void testNumericTypes() throws IOException { - - List expected = - ImmutableList.of( - recordNumType( - 2, - Integer.MAX_VALUE, - Float.MAX_VALUE, - Double.MAX_VALUE, - Long.MAX_VALUE, - 1643811742000L, - 1643811742000L, - 1643811742000L, - 10.24d), - recordNumType( - 2, - Integer.MIN_VALUE, - Float.MIN_VALUE, - Double.MIN_VALUE, - Long.MIN_VALUE, - 1643811742000L, - 1643811742000L, - 1643811742000L, - 10.24d)); - - writeAndValidate(SCHEMA_NUM_TYPE, expected, 2); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java deleted file mode 100644 index fdffc0e01c20..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java +++ /dev/null @@ -1,106 +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.iceberg.flink.data; - -import java.io.File; -import java.io.IOException; -import java.util.Iterator; -import java.util.List; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.Files; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.DataTest; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.data.orc.GenericOrcReader; -import org.apache.iceberg.data.orc.GenericOrcWriter; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.orc.ORC; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; - -public class TestFlinkOrcReaderWriter extends DataTest { - private static final int NUM_RECORDS = 100; - - @Override - protected void writeAndValidate(Schema schema) throws IOException { - RowType flinkSchema = FlinkSchemaUtil.convert(schema); - List expectedRecords = RandomGenericData.generate(schema, NUM_RECORDS, 1990L); - List expectedRows = Lists.newArrayList(RandomRowData.convert(schema, expectedRecords)); - - File recordsFile = temp.newFile(); - Assert.assertTrue("Delete should succeed", recordsFile.delete()); - - // Write the expected records into ORC file, then read them into RowData and assert with the - // expected Record list. - try (FileAppender writer = - ORC.write(Files.localOutput(recordsFile)) - .schema(schema) - .createWriterFunc(GenericOrcWriter::buildWriter) - .build()) { - writer.addAll(expectedRecords); - } - - try (CloseableIterable reader = - ORC.read(Files.localInput(recordsFile)) - .project(schema) - .createReaderFunc(type -> new FlinkOrcReader(schema, type)) - .build()) { - Iterator expected = expectedRecords.iterator(); - Iterator rows = reader.iterator(); - for (int i = 0; i < NUM_RECORDS; i++) { - Assert.assertTrue("Should have expected number of records", rows.hasNext()); - TestHelpers.assertRowData(schema.asStruct(), flinkSchema, expected.next(), rows.next()); - } - Assert.assertFalse("Should not have extra records", rows.hasNext()); - } - - File rowDataFile = temp.newFile(); - Assert.assertTrue("Delete should succeed", rowDataFile.delete()); - - // Write the expected RowData into ORC file, then read them into Record and assert with the - // expected RowData list. - RowType rowType = FlinkSchemaUtil.convert(schema); - try (FileAppender writer = - ORC.write(Files.localOutput(rowDataFile)) - .schema(schema) - .createWriterFunc((iSchema, typeDesc) -> FlinkOrcWriter.buildWriter(rowType, iSchema)) - .build()) { - writer.addAll(expectedRows); - } - - try (CloseableIterable reader = - ORC.read(Files.localInput(rowDataFile)) - .project(schema) - .createReaderFunc(type -> GenericOrcReader.buildReader(schema, type)) - .build()) { - Iterator expected = expectedRows.iterator(); - Iterator records = reader.iterator(); - for (int i = 0; i < NUM_RECORDS; i += 1) { - Assert.assertTrue("Should have expected number of records", records.hasNext()); - TestHelpers.assertRowData(schema.asStruct(), flinkSchema, records.next(), expected.next()); - } - Assert.assertFalse("Should not have extra records", records.hasNext()); - } - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java deleted file mode 100644 index 30a2a7bb51ce..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java +++ /dev/null @@ -1,138 +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.iceberg.flink.data; - -import static org.apache.iceberg.types.Types.NestedField.optional; - -import java.io.File; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.Iterator; -import java.util.List; -import org.apache.avro.generic.GenericData; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.GenericRecordBuilder; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.hadoop.fs.Path; -import org.apache.iceberg.Files; -import org.apache.iceberg.Schema; -import org.apache.iceberg.avro.AvroSchemaUtil; -import org.apache.iceberg.data.DataTest; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.data.parquet.GenericParquetWriter; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.parquet.Parquet; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Types; -import org.apache.parquet.avro.AvroParquetWriter; -import org.apache.parquet.hadoop.ParquetWriter; -import org.junit.Assert; -import org.junit.Test; - -public class TestFlinkParquetReader extends DataTest { - private static final int NUM_RECORDS = 100; - - @Test - public void testTwoLevelList() throws IOException { - Schema schema = - new Schema( - optional(1, "arraybytes", Types.ListType.ofRequired(3, Types.BinaryType.get())), - optional(2, "topbytes", Types.BinaryType.get())); - org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(schema.asStruct()); - - File testFile = temp.newFile(); - Assert.assertTrue(testFile.delete()); - - ParquetWriter writer = - AvroParquetWriter.builder(new Path(testFile.toURI())) - .withDataModel(GenericData.get()) - .withSchema(avroSchema) - .config("parquet.avro.add-list-element-records", "true") - .config("parquet.avro.write-old-list-structure", "true") - .build(); - - GenericRecordBuilder recordBuilder = new GenericRecordBuilder(avroSchema); - List expectedByteList = Lists.newArrayList(); - byte[] expectedByte = {0x00, 0x01}; - ByteBuffer expectedBinary = ByteBuffer.wrap(expectedByte); - expectedByteList.add(expectedBinary); - recordBuilder.set("arraybytes", expectedByteList); - recordBuilder.set("topbytes", expectedBinary); - GenericData.Record expectedRecord = recordBuilder.build(); - - writer.write(expectedRecord); - writer.close(); - - try (CloseableIterable reader = - Parquet.read(Files.localInput(testFile)) - .project(schema) - .createReaderFunc(type -> FlinkParquetReaders.buildReader(schema, type)) - .build()) { - Iterator rows = reader.iterator(); - Assert.assertTrue("Should have at least one row", rows.hasNext()); - RowData rowData = rows.next(); - Assert.assertArrayEquals(rowData.getArray(0).getBinary(0), expectedByte); - Assert.assertArrayEquals(rowData.getBinary(1), expectedByte); - Assert.assertFalse("Should not have more than one row", rows.hasNext()); - } - } - - private void writeAndValidate(Iterable iterable, Schema schema) throws IOException { - File testFile = temp.newFile(); - Assert.assertTrue("Delete should succeed", testFile.delete()); - - try (FileAppender writer = - Parquet.write(Files.localOutput(testFile)) - .schema(schema) - .createWriterFunc(GenericParquetWriter::buildWriter) - .build()) { - writer.addAll(iterable); - } - - try (CloseableIterable reader = - Parquet.read(Files.localInput(testFile)) - .project(schema) - .createReaderFunc(type -> FlinkParquetReaders.buildReader(schema, type)) - .build()) { - Iterator expected = iterable.iterator(); - Iterator rows = reader.iterator(); - LogicalType rowType = FlinkSchemaUtil.convert(schema); - for (int i = 0; i < NUM_RECORDS; i += 1) { - Assert.assertTrue("Should have expected number of rows", rows.hasNext()); - TestHelpers.assertRowData(schema.asStruct(), rowType, expected.next(), rows.next()); - } - Assert.assertFalse("Should not have extra rows", rows.hasNext()); - } - } - - @Override - protected void writeAndValidate(Schema schema) throws IOException { - writeAndValidate(RandomGenericData.generate(schema, NUM_RECORDS, 19981), schema); - writeAndValidate( - RandomGenericData.generateDictionaryEncodableRecords(schema, NUM_RECORDS, 21124), schema); - writeAndValidate( - RandomGenericData.generateFallbackRecords(schema, NUM_RECORDS, 21124, NUM_RECORDS / 20), - schema); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java deleted file mode 100644 index 7b868eafc311..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java +++ /dev/null @@ -1,93 +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.iceberg.flink.data; - -import java.io.File; -import java.io.IOException; -import java.util.Iterator; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.iceberg.Files; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.DataTest; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.data.parquet.GenericParquetReaders; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.parquet.Parquet; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.rules.TemporaryFolder; - -public class TestFlinkParquetWriter extends DataTest { - private static final int NUM_RECORDS = 100; - - @Rule public TemporaryFolder temp = new TemporaryFolder(); - - private void writeAndValidate(Iterable iterable, Schema schema) throws IOException { - File testFile = temp.newFile(); - Assert.assertTrue("Delete should succeed", testFile.delete()); - - LogicalType logicalType = FlinkSchemaUtil.convert(schema); - - try (FileAppender writer = - Parquet.write(Files.localOutput(testFile)) - .schema(schema) - .createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(logicalType, msgType)) - .build()) { - writer.addAll(iterable); - } - - try (CloseableIterable reader = - Parquet.read(Files.localInput(testFile)) - .project(schema) - .createReaderFunc(msgType -> GenericParquetReaders.buildReader(schema, msgType)) - .build()) { - Iterator expected = iterable.iterator(); - Iterator actual = reader.iterator(); - LogicalType rowType = FlinkSchemaUtil.convert(schema); - for (int i = 0; i < NUM_RECORDS; i += 1) { - Assert.assertTrue("Should have expected number of rows", actual.hasNext()); - TestHelpers.assertRowData(schema.asStruct(), rowType, actual.next(), expected.next()); - } - Assert.assertFalse("Should not have extra rows", actual.hasNext()); - } - } - - @Override - protected void writeAndValidate(Schema schema) throws IOException { - writeAndValidate(RandomRowData.generate(schema, NUM_RECORDS, 19981), schema); - - writeAndValidate( - RandomRowData.convert( - schema, - RandomGenericData.generateDictionaryEncodableRecords(schema, NUM_RECORDS, 21124)), - schema); - - writeAndValidate( - RandomRowData.convert( - schema, - RandomGenericData.generateFallbackRecords( - schema, NUM_RECORDS, 21124, NUM_RECORDS / 20)), - schema); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java deleted file mode 100644 index 3cd25c8fa983..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java +++ /dev/null @@ -1,593 +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.iceberg.flink.data; - -import java.util.List; -import org.apache.flink.table.data.GenericArrayData; -import org.apache.flink.table.data.GenericMapData; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.DataGenerator; -import org.apache.iceberg.flink.DataGenerators; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.StructProjection; -import org.assertj.core.api.Assertions; -import org.junit.jupiter.api.Test; - -public class TestRowDataProjection { - @Test - public void testNullRootRowData() { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - RowDataProjection projection = RowDataProjection.create(schema, schema.select("id")); - - Assertions.assertThatThrownBy(() -> projection.wrap(null)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid row data: null"); - } - - @Test - public void testFullProjection() { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - generateAndValidate(schema, schema); - - GenericRowData rowData = GenericRowData.of(1L, StringData.fromString("a")); - GenericRowData copyRowData = GenericRowData.of(1L, StringData.fromString("a")); - GenericRowData otherRowData = GenericRowData.of(2L, StringData.fromString("b")); - testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); - } - - @Test - public void testReorderedFullProjection() { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - Schema reordered = - new Schema( - Types.NestedField.optional(1, "data", Types.StringType.get()), - Types.NestedField.required(0, "id", Types.LongType.get())); - - generateAndValidate(schema, reordered); - - GenericRowData rowData = GenericRowData.of(1L, StringData.fromString("a")); - GenericRowData copyRowData = GenericRowData.of(1L, StringData.fromString("a")); - GenericRowData otherRowData = GenericRowData.of(2L, StringData.fromString("b")); - testEqualsAndHashCode(schema, reordered, rowData, copyRowData, otherRowData); - } - - @Test - public void testBasicProjection() { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); - Schema dataOnly = new Schema(Types.NestedField.optional(1, "data", Types.StringType.get())); - generateAndValidate(schema, idOnly); - generateAndValidate(schema, dataOnly); - - GenericRowData rowData = GenericRowData.of(1L, StringData.fromString("a")); - GenericRowData copyRowData = GenericRowData.of(1L, StringData.fromString("a")); - GenericRowData otherRowData = GenericRowData.of(2L, StringData.fromString("b")); - testEqualsAndHashCode(schema, idOnly, rowData, copyRowData, otherRowData); - testEqualsAndHashCode(schema, dataOnly, rowData, copyRowData, otherRowData); - } - - @Test - public void testEmptyProjection() { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - generateAndValidate(schema, schema.select()); - - GenericRowData rowData = GenericRowData.of(1L, StringData.fromString("a")); - GenericRowData copyRowData = GenericRowData.of(1L, StringData.fromString("a")); - GenericRowData otherRowData = GenericRowData.of(2L, StringData.fromString("b")); - testEqualsAndHashCode(schema, schema.select(), rowData, copyRowData, otherRowData, true); - } - - @Test - public void testRename() { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - Schema renamed = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "renamed", Types.StringType.get())); - generateAndValidate(schema, renamed); - - GenericRowData rowData = GenericRowData.of(1L, StringData.fromString("a")); - GenericRowData copyRowData = GenericRowData.of(1L, StringData.fromString("a")); - GenericRowData otherRowData = GenericRowData.of(2L, StringData.fromString("b")); - testEqualsAndHashCode(schema, renamed, rowData, copyRowData, otherRowData); - } - - @Test - public void testNestedProjection() { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional( - 3, - "location", - Types.StructType.of( - Types.NestedField.required(1, "lat", Types.FloatType.get()), - Types.NestedField.required(2, "long", Types.FloatType.get())))); - - GenericRowData rowData = GenericRowData.of(1L, GenericRowData.of(1.0f, 1.0f)); - GenericRowData copyRowData = GenericRowData.of(1L, GenericRowData.of(1.0f, 1.0f)); - GenericRowData otherRowData = GenericRowData.of(2L, GenericRowData.of(2.0f, 2.0f)); - - GenericRowData rowDataNullStruct = GenericRowData.of(1L, null); - GenericRowData copyRowDataNullStruct = GenericRowData.of(1L, null); - GenericRowData otherRowDataNullStruct = GenericRowData.of(2L, null); - - // Project id only. - Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); - Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, idOnly); - testEqualsAndHashCode(schema, idOnly, rowData, copyRowData, otherRowData); - testEqualsAndHashCode( - schema, idOnly, rowDataNullStruct, copyRowDataNullStruct, otherRowDataNullStruct); - - // Project lat only. - Schema latOnly = - new Schema( - Types.NestedField.optional( - 3, - "location", - Types.StructType.of(Types.NestedField.required(1, "lat", Types.FloatType.get())))); - Assertions.assertThat(latOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, latOnly); - testEqualsAndHashCode(schema, latOnly, rowData, copyRowData, otherRowData); - testEqualsAndHashCode( - schema, latOnly, rowDataNullStruct, copyRowDataNullStruct, otherRowDataNullStruct, true); - - // Project long only. - Schema longOnly = - new Schema( - Types.NestedField.optional( - 3, - "location", - Types.StructType.of(Types.NestedField.required(2, "long", Types.FloatType.get())))); - Assertions.assertThat(longOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, longOnly); - testEqualsAndHashCode(schema, longOnly, rowData, copyRowData, otherRowData); - testEqualsAndHashCode( - schema, longOnly, rowDataNullStruct, copyRowDataNullStruct, otherRowDataNullStruct, true); - - // Project location. - Schema locationOnly = schema.select("location"); - Assertions.assertThat(locationOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, locationOnly); - testEqualsAndHashCode(schema, locationOnly, rowData, copyRowData, otherRowData); - testEqualsAndHashCode( - schema, - locationOnly, - rowDataNullStruct, - copyRowDataNullStruct, - otherRowDataNullStruct, - true); - } - - @Test - public void testPrimitivesFullProjection() { - DataGenerator dataGenerator = new DataGenerators.Primitives(); - Schema schema = dataGenerator.icebergSchema(); - generateAndValidate(schema, schema); - - GenericRowData rowData = dataGenerator.generateFlinkRowData(); - GenericRowData copyRowData = dataGenerator.generateFlinkRowData(); - GenericRowData otherRowData = dataGenerator.generateFlinkRowData(); - // modify the string field value (position 6) - otherRowData.setField(6, StringData.fromString("foo_bar")); - testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); - - GenericRowData rowDataNullOptionalFields = dataGenerator.generateFlinkRowData(); - setOptionalFieldsNullForPrimitives(rowDataNullOptionalFields); - GenericRowData copyRowDataNullOptionalFields = dataGenerator.generateFlinkRowData(); - setOptionalFieldsNullForPrimitives(copyRowDataNullOptionalFields); - GenericRowData otherRowDataNullOptionalFields = dataGenerator.generateFlinkRowData(); - // modify the string field value (position 6) - otherRowDataNullOptionalFields.setField(6, StringData.fromString("foo_bar")); - setOptionalFieldsNullForPrimitives(otherRowData); - testEqualsAndHashCode( - schema, - schema, - rowDataNullOptionalFields, - copyRowDataNullOptionalFields, - otherRowDataNullOptionalFields); - } - - private void setOptionalFieldsNullForPrimitives(GenericRowData rowData) { - // fields from [1, 5] range are optional - for (int pos = 1; pos <= 5; ++pos) { - rowData.setField(pos, null); - } - } - - @Test - public void testMapOfPrimitivesProjection() { - DataGenerator dataGenerator = new DataGenerators.MapOfPrimitives(); - Schema schema = dataGenerator.icebergSchema(); - - // Project id only. - Schema idOnly = schema.select("row_id"); - Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, idOnly); - - // Project map only. - Schema mapOnly = schema.select("map_of_primitives"); - Assertions.assertThat(mapOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, mapOnly); - - // Project all. - generateAndValidate(schema, schema); - - GenericRowData rowData = dataGenerator.generateFlinkRowData(); - GenericRowData copyRowData = dataGenerator.generateFlinkRowData(); - // modify the map field value - GenericRowData otherRowData = - GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericMapData( - ImmutableMap.of(StringData.fromString("foo"), 1, StringData.fromString("bar"), 2))); - testEqualsAndHashCode(schema, idOnly, rowData, copyRowData, otherRowData, true); - testEqualsAndHashCode(schema, mapOnly, rowData, copyRowData, otherRowData); - testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); - - GenericRowData rowDataNullOptionalFields = - GenericRowData.of(StringData.fromString("row_id_value"), null); - GenericRowData copyRowDataNullOptionalFields = - GenericRowData.of(StringData.fromString("row_id_value"), null); - // modify the map field value - GenericRowData otherRowDataNullOptionalFields = - GenericRowData.of(StringData.fromString("other_row_id_value"), null); - testEqualsAndHashCode( - schema, - idOnly, - rowDataNullOptionalFields, - copyRowDataNullOptionalFields, - otherRowDataNullOptionalFields); - testEqualsAndHashCode( - schema, - mapOnly, - rowDataNullOptionalFields, - copyRowDataNullOptionalFields, - otherRowDataNullOptionalFields, - true); - testEqualsAndHashCode( - schema, - schema, - rowDataNullOptionalFields, - copyRowDataNullOptionalFields, - otherRowDataNullOptionalFields); - } - - @Test - public void testMapOfStructStructProjection() { - DataGenerator dataGenerator = new DataGenerators.MapOfStructStruct(); - Schema schema = dataGenerator.icebergSchema(); - - // Project id only. - Schema idOnly = schema.select("row_id"); - Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, idOnly); - - // Project map only. - Schema mapOnly = schema.select("map"); - Assertions.assertThat(mapOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, mapOnly); - - // Project all. - generateAndValidate(schema, schema); - - // Project partial map key. - Schema partialMapKey = - new Schema( - Types.NestedField.optional( - 2, - "map", - Types.MapType.ofOptional( - 101, - 102, - Types.StructType.of( - Types.NestedField.required(201, "key", Types.LongType.get())), - Types.StructType.of( - Types.NestedField.required(203, "value", Types.LongType.get()), - Types.NestedField.required(204, "valueData", Types.StringType.get()))))); - Assertions.assertThatThrownBy(() -> generateAndValidate(schema, partialMapKey)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining("Cannot project a partial map key or value struct."); - - // Project partial map key. - Schema partialMapValue = - new Schema( - Types.NestedField.optional( - 2, - "map", - Types.MapType.ofOptional( - 101, - 102, - Types.StructType.of( - Types.NestedField.required(201, "key", Types.LongType.get()), - Types.NestedField.required(202, "keyData", Types.StringType.get())), - Types.StructType.of( - Types.NestedField.required(203, "value", Types.LongType.get()))))); - Assertions.assertThatThrownBy(() -> generateAndValidate(schema, partialMapValue)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining("Cannot project a partial map key or value struct."); - - GenericRowData rowData = dataGenerator.generateFlinkRowData(); - GenericRowData copyRowData = dataGenerator.generateFlinkRowData(); - // modify the map field value - GenericRowData otherRowData = - GenericRowData.of( - StringData.fromString("other_row_id_value"), - new GenericMapData( - ImmutableMap.of( - GenericRowData.of(1L, StringData.fromString("other_key_data")), - GenericRowData.of(1L, StringData.fromString("other_value_data"))))); - testEqualsAndHashCode(schema, idOnly, rowData, copyRowData, otherRowData); - testEqualsAndHashCode(schema, mapOnly, rowData, copyRowData, otherRowData); - testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); - - GenericRowData rowDataNullOptionalFields = - GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericMapData( - ImmutableMap.of(GenericRowData.of(1L, null), GenericRowData.of(1L, null)))); - GenericRowData copyRowDataNullOptionalFields = - GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericMapData( - ImmutableMap.of(GenericRowData.of(1L, null), GenericRowData.of(1L, null)))); - // modify the map field value - GenericRowData otherRowDataNullOptionalFields = - GenericRowData.of( - StringData.fromString("other_row_id_value"), - new GenericMapData( - ImmutableMap.of(GenericRowData.of(2L, null), GenericRowData.of(2L, null)))); - testEqualsAndHashCode( - schema, - idOnly, - rowDataNullOptionalFields, - copyRowDataNullOptionalFields, - otherRowDataNullOptionalFields); - testEqualsAndHashCode( - schema, - mapOnly, - rowDataNullOptionalFields, - copyRowDataNullOptionalFields, - otherRowDataNullOptionalFields); - testEqualsAndHashCode( - schema, - schema, - rowDataNullOptionalFields, - copyRowDataNullOptionalFields, - otherRowDataNullOptionalFields); - } - - @Test - public void testArrayOfPrimitiveProjection() { - DataGenerator dataGenerator = new DataGenerators.ArrayOfPrimitive(); - Schema schema = dataGenerator.icebergSchema(); - - // Project id only. - Schema idOnly = schema.select("row_id"); - Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, idOnly); - - // Project list only. - Schema arrayOnly = schema.select("array_of_int"); - Assertions.assertThat(arrayOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, arrayOnly); - - // Project all. - generateAndValidate(schema, schema); - - GenericRowData rowData = dataGenerator.generateFlinkRowData(); - GenericRowData copyRowData = dataGenerator.generateFlinkRowData(); - // modify the map field value - GenericRowData otherRowData = - GenericRowData.of( - StringData.fromString("other_row_id_value"), - new GenericArrayData(new Integer[] {4, 5, 6})); - testEqualsAndHashCode(schema, idOnly, rowData, copyRowData, otherRowData); - testEqualsAndHashCode(schema, arrayOnly, rowData, copyRowData, otherRowData); - testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); - - GenericRowData rowDataNullOptionalFields = - GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericArrayData(new Integer[] {1, null, 3})); - GenericRowData copyRowDataNullOptionalFields = - GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericArrayData(new Integer[] {1, null, 3})); - // modify the map field value - GenericRowData otherRowDataNullOptionalFields = - GenericRowData.of( - StringData.fromString("other_row_id_value"), - new GenericArrayData(new Integer[] {4, null, 6})); - testEqualsAndHashCode( - schema, - idOnly, - rowDataNullOptionalFields, - copyRowDataNullOptionalFields, - otherRowDataNullOptionalFields); - testEqualsAndHashCode( - schema, - arrayOnly, - rowDataNullOptionalFields, - copyRowDataNullOptionalFields, - otherRowDataNullOptionalFields); - testEqualsAndHashCode( - schema, - schema, - rowDataNullOptionalFields, - copyRowDataNullOptionalFields, - otherRowDataNullOptionalFields); - } - - @Test - public void testArrayOfStructProjection() { - DataGenerator dataGenerator = new DataGenerators.ArrayOfStruct(); - Schema schema = dataGenerator.icebergSchema(); - - // Project id only. - Schema idOnly = schema.select("row_id"); - Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, idOnly); - - // Project list only. - Schema arrayOnly = schema.select("array_of_struct"); - Assertions.assertThat(arrayOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, arrayOnly); - - // Project all. - generateAndValidate(schema, schema); - - // Project partial list value. - Schema partialList = - new Schema( - Types.NestedField.optional( - 2, - "array_of_struct", - Types.ListType.ofOptional( - 101, - Types.StructType.of( - Types.NestedField.required(202, "name", Types.StringType.get()))))); - - Assertions.assertThatThrownBy(() -> generateAndValidate(schema, partialList)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining("Cannot project a partial list element struct."); - - GenericRowData rowData = dataGenerator.generateFlinkRowData(); - GenericRowData copyRowData = dataGenerator.generateFlinkRowData(); - // modify the map field value - GenericRowData otherRowData = - GenericRowData.of( - StringData.fromString("row_id_value"), new GenericArrayData(new Integer[] {4, 5, 6})); - testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); - - GenericRowData rowDataNullOptionalFields = - GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericArrayData(new Integer[] {1, null, 3})); - GenericRowData copyRowDataNullOptionalFields = - GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericArrayData(new Integer[] {1, null, 3})); - // modify the map field value - GenericRowData otherRowDataNullOptionalFields = - GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericArrayData(new Integer[] {4, null, 6})); - testEqualsAndHashCode( - schema, - schema, - rowDataNullOptionalFields, - copyRowDataNullOptionalFields, - otherRowDataNullOptionalFields); - } - - private void generateAndValidate(Schema schema, Schema projectSchema) { - int numRecords = 100; - List recordList = RandomGenericData.generate(schema, numRecords, 102L); - List rowDataList = - Lists.newArrayList(RandomRowData.generate(schema, numRecords, 102L).iterator()); - Assertions.assertThat(rowDataList).hasSize(recordList.size()); - - StructProjection structProjection = StructProjection.create(schema, projectSchema); - RowDataProjection rowDataProjection = RowDataProjection.create(schema, projectSchema); - - for (int i = 0; i < numRecords; i++) { - StructLike expected = structProjection.wrap(recordList.get(i)); - RowData projected = rowDataProjection.wrap(rowDataList.get(i)); - TestHelpers.assertRowData(projectSchema, expected, projected); - - Assertions.assertThat(projected).isEqualTo(projected); - Assertions.assertThat(projected).hasSameHashCodeAs(projected); - // make sure toString doesn't throw NPE for null values - Assertions.assertThatNoException().isThrownBy(projected::toString); - } - } - - private void testEqualsAndHashCode( - Schema schema, - Schema projectionSchema, - RowData rowData, - RowData copyRowData, - RowData otherRowData) { - testEqualsAndHashCode(schema, projectionSchema, rowData, copyRowData, otherRowData, false); - } - - /** - * @param isOtherRowDataSameAsRowData sometimes projection on otherRowData can result in the same - * RowData, e.g. due to empty projection or null struct - */ - private void testEqualsAndHashCode( - Schema schema, - Schema projectionSchema, - RowData rowData, - RowData copyRowData, - RowData otherRowData, - boolean isOtherRowDataSameAsRowData) { - RowDataProjection projection = RowDataProjection.create(schema, projectionSchema); - RowDataProjection copyProjection = RowDataProjection.create(schema, projectionSchema); - RowDataProjection otherProjection = RowDataProjection.create(schema, projectionSchema); - - Assertions.assertThat(projection.wrap(rowData)).isEqualTo(copyProjection.wrap(copyRowData)); - Assertions.assertThat(projection.wrap(rowData)) - .hasSameHashCodeAs(copyProjection.wrap(copyRowData)); - - if (isOtherRowDataSameAsRowData) { - Assertions.assertThat(projection.wrap(rowData)).isEqualTo(otherProjection.wrap(otherRowData)); - Assertions.assertThat(projection.wrap(rowData)) - .hasSameHashCodeAs(otherProjection.wrap(otherRowData)); - } else { - Assertions.assertThat(projection.wrap(rowData)) - .isNotEqualTo(otherProjection.wrap(otherRowData)); - Assertions.assertThat(projection.wrap(rowData)) - .doesNotHaveSameHashCodeAs(otherProjection.wrap(otherRowData)); - } - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java deleted file mode 100644 index df2e6ae21c7e..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java +++ /dev/null @@ -1,580 +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.iceberg.flink.data; - -import java.io.File; -import java.io.IOException; -import java.util.Map; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.GenericArrayData; -import org.apache.flink.table.data.GenericMapData; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.iceberg.Files; -import org.apache.iceberg.Schema; -import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Comparators; -import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class TestRowProjection { - - @Rule public TemporaryFolder temp = new TemporaryFolder(); - - private RowData writeAndRead(String desc, Schema writeSchema, Schema readSchema, RowData row) - throws IOException { - File file = temp.newFile(desc + ".avro"); - Assert.assertTrue(file.delete()); - - try (FileAppender appender = - Avro.write(Files.localOutput(file)) - .schema(writeSchema) - .createWriterFunc(ignore -> new FlinkAvroWriter(FlinkSchemaUtil.convert(writeSchema))) - .build()) { - appender.add(row); - } - - Iterable records = - Avro.read(Files.localInput(file)) - .project(readSchema) - .createReaderFunc(FlinkAvroReader::new) - .build(); - - return Iterables.getOnlyElement(records); - } - - @Test - public void testFullProjection() throws Exception { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - RowData row = GenericRowData.of(34L, StringData.fromString("test")); - - RowData projected = writeAndRead("full_projection", schema, schema, row); - - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); - - int cmp = Comparators.charSequences().compare("test", projected.getString(1).toString()); - Assert.assertEquals("Should contain the correct data value", cmp, 0); - } - - @Test - public void testSpecialCharacterProjection() throws Exception { - Schema schema = - new Schema( - Types.NestedField.required(0, "user id", Types.LongType.get()), - Types.NestedField.optional(1, "data%0", Types.StringType.get())); - - RowData row = GenericRowData.of(34L, StringData.fromString("test")); - - RowData full = writeAndRead("special_chars", schema, schema, row); - - Assert.assertEquals("Should contain the correct id value", 34L, full.getLong(0)); - Assert.assertEquals( - "Should contain the correct data value", - 0, - Comparators.charSequences().compare("test", full.getString(1).toString())); - - RowData projected = writeAndRead("special_characters", schema, schema.select("data%0"), full); - - Assert.assertEquals("Should not contain id value", 1, projected.getArity()); - Assert.assertEquals( - "Should contain the correct data value", - 0, - Comparators.charSequences().compare("test", projected.getString(0).toString())); - } - - @Test - public void testReorderedFullProjection() throws Exception { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - RowData row = GenericRowData.of(34L, StringData.fromString("test")); - - Schema reordered = - new Schema( - Types.NestedField.optional(1, "data", Types.StringType.get()), - Types.NestedField.required(0, "id", Types.LongType.get())); - - RowData projected = writeAndRead("full_projection", schema, reordered, row); - - Assert.assertEquals( - "Should contain the correct 0 value", "test", projected.getString(0).toString()); - Assert.assertEquals("Should contain the correct 1 value", 34L, projected.getLong(1)); - } - - @Test - public void testReorderedProjection() throws Exception { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - RowData row = GenericRowData.of(34L, StringData.fromString("test")); - - Schema reordered = - new Schema( - Types.NestedField.optional(2, "missing_1", Types.StringType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get()), - Types.NestedField.optional(3, "missing_2", Types.LongType.get())); - - RowData projected = writeAndRead("full_projection", schema, reordered, row); - - Assert.assertTrue("Should contain the correct 0 value", projected.isNullAt(0)); - Assert.assertEquals( - "Should contain the correct 1 value", "test", projected.getString(1).toString()); - Assert.assertTrue("Should contain the correct 2 value", projected.isNullAt(2)); - } - - @Test - public void testRenamedAddedField() throws Exception { - Schema schema = - new Schema( - Types.NestedField.required(1, "a", Types.LongType.get()), - Types.NestedField.required(2, "b", Types.LongType.get()), - Types.NestedField.required(3, "d", Types.LongType.get())); - - RowData row = GenericRowData.of(100L, 200L, 300L); - - Schema renamedAdded = - new Schema( - Types.NestedField.optional(1, "a", Types.LongType.get()), - Types.NestedField.optional(2, "b", Types.LongType.get()), - Types.NestedField.optional(3, "c", Types.LongType.get()), - Types.NestedField.optional(4, "d", Types.LongType.get())); - - RowData projected = writeAndRead("rename_and_add_column_projection", schema, renamedAdded, row); - Assert.assertEquals("Should contain the correct value in column 1", projected.getLong(0), 100L); - Assert.assertEquals("Should contain the correct value in column 2", projected.getLong(1), 200L); - Assert.assertEquals("Should contain the correct value in column 3", projected.getLong(2), 300L); - Assert.assertTrue("Should contain empty value on new column 4", projected.isNullAt(3)); - } - - @Test - public void testEmptyProjection() throws Exception { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - RowData row = GenericRowData.of(34L, StringData.fromString("test")); - - RowData projected = writeAndRead("empty_projection", schema, schema.select(), row); - - Assert.assertNotNull("Should read a non-null record", projected); - Assert.assertEquals(0, projected.getArity()); - } - - @Test - public void testBasicProjection() throws Exception { - Schema writeSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - RowData row = GenericRowData.of(34L, StringData.fromString("test")); - - Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); - - RowData projected = writeAndRead("basic_projection_id", writeSchema, idOnly, row); - Assert.assertEquals("Should not project data", 1, projected.getArity()); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); - - Schema dataOnly = new Schema(Types.NestedField.optional(1, "data", Types.StringType.get())); - - projected = writeAndRead("basic_projection_data", writeSchema, dataOnly, row); - - Assert.assertEquals("Should not project id", 1, projected.getArity()); - int cmp = Comparators.charSequences().compare("test", projected.getString(0).toString()); - Assert.assertEquals("Should contain the correct data value", 0, cmp); - } - - @Test - public void testRename() throws Exception { - Schema writeSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - RowData row = GenericRowData.of(34L, StringData.fromString("test")); - - Schema readSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "renamed", Types.StringType.get())); - - RowData projected = writeAndRead("project_and_rename", writeSchema, readSchema, row); - - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); - int cmp = Comparators.charSequences().compare("test", projected.getString(1).toString()); - Assert.assertEquals("Should contain the correct data/renamed value", 0, cmp); - } - - @Test - public void testNestedStructProjection() throws Exception { - Schema writeSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional( - 3, - "location", - Types.StructType.of( - Types.NestedField.required(1, "lat", Types.FloatType.get()), - Types.NestedField.required(2, "long", Types.FloatType.get())))); - - RowData location = GenericRowData.of(52.995143f, -1.539054f); - RowData record = GenericRowData.of(34L, location); - - Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); - - RowData projected = writeAndRead("id_only", writeSchema, idOnly, record); - Assert.assertEquals("Should not project location", 1, projected.getArity()); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); - - Schema latOnly = - new Schema( - Types.NestedField.optional( - 3, - "location", - Types.StructType.of(Types.NestedField.required(1, "lat", Types.FloatType.get())))); - - projected = writeAndRead("latitude_only", writeSchema, latOnly, record); - RowData projectedLocation = projected.getRow(0, 1); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertFalse("Should project location", projected.isNullAt(0)); - Assert.assertEquals("Should not project longitude", 1, projectedLocation.getArity()); - Assert.assertEquals( - "Should project latitude", 52.995143f, projectedLocation.getFloat(0), 0.000001f); - - Schema longOnly = - new Schema( - Types.NestedField.optional( - 3, - "location", - Types.StructType.of(Types.NestedField.required(2, "long", Types.FloatType.get())))); - - projected = writeAndRead("longitude_only", writeSchema, longOnly, record); - projectedLocation = projected.getRow(0, 1); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertFalse("Should project location", projected.isNullAt(0)); - Assert.assertEquals("Should not project latitutde", 1, projectedLocation.getArity()); - Assert.assertEquals( - "Should project longitude", -1.539054f, projectedLocation.getFloat(0), 0.000001f); - - Schema locationOnly = writeSchema.select("location"); - projected = writeAndRead("location_only", writeSchema, locationOnly, record); - projectedLocation = projected.getRow(0, 1); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertFalse("Should project location", projected.isNullAt(0)); - Assert.assertEquals( - "Should project latitude", 52.995143f, projectedLocation.getFloat(0), 0.000001f); - Assert.assertEquals( - "Should project longitude", -1.539054f, projectedLocation.getFloat(1), 0.000001f); - } - - @Test - public void testMapProjection() throws IOException { - Schema writeSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional( - 5, - "properties", - Types.MapType.ofOptional(6, 7, Types.StringType.get(), Types.StringType.get()))); - - GenericMapData properties = - new GenericMapData( - ImmutableMap.of( - StringData.fromString("a"), - StringData.fromString("A"), - StringData.fromString("b"), - StringData.fromString("B"))); - - RowData row = GenericRowData.of(34L, properties); - - Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); - - RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); - Assert.assertEquals("Should not project properties map", 1, projected.getArity()); - - Schema keyOnly = writeSchema.select("properties.key"); - projected = writeAndRead("key_only", writeSchema, keyOnly, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project entire map", properties, projected.getMap(0)); - - Schema valueOnly = writeSchema.select("properties.value"); - projected = writeAndRead("value_only", writeSchema, valueOnly, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project entire map", properties, projected.getMap(0)); - - Schema mapOnly = writeSchema.select("properties"); - projected = writeAndRead("map_only", writeSchema, mapOnly, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project entire map", properties, projected.getMap(0)); - } - - private Map toStringMap(Map map) { - Map stringMap = Maps.newHashMap(); - for (Map.Entry entry : map.entrySet()) { - if (entry.getValue() instanceof CharSequence) { - stringMap.put(entry.getKey().toString(), entry.getValue().toString()); - } else { - stringMap.put(entry.getKey().toString(), entry.getValue()); - } - } - return stringMap; - } - - @Test - public void testMapOfStructsProjection() throws IOException { - Schema writeSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional( - 5, - "locations", - Types.MapType.ofOptional( - 6, - 7, - Types.StringType.get(), - Types.StructType.of( - Types.NestedField.required(1, "lat", Types.FloatType.get()), - Types.NestedField.required(2, "long", Types.FloatType.get()))))); - - RowData l1 = GenericRowData.of(53.992811f, -1.542616f); - RowData l2 = GenericRowData.of(52.995143f, -1.539054f); - GenericMapData map = - new GenericMapData( - ImmutableMap.of(StringData.fromString("L1"), l1, StringData.fromString("L2"), l2)); - RowData row = GenericRowData.of(34L, map); - - Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); - - RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); - Assert.assertEquals("Should not project locations map", 1, projected.getArity()); - - projected = writeAndRead("all_locations", writeSchema, writeSchema.select("locations"), row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project locations map", row.getMap(1), projected.getMap(0)); - - projected = writeAndRead("lat_only", writeSchema, writeSchema.select("locations.lat"), row); - GenericMapData locations = (GenericMapData) projected.getMap(0); - Assert.assertNotNull("Should project locations map", locations); - GenericArrayData l1l2Array = - new GenericArrayData( - new Object[] {StringData.fromString("L2"), StringData.fromString("L1")}); - Assert.assertEquals("Should contain L1 and L2", l1l2Array, locations.keyArray()); - RowData projectedL1 = (RowData) locations.get(StringData.fromString("L1")); - Assert.assertNotNull("L1 should not be null", projectedL1); - Assert.assertEquals("L1 should contain lat", 53.992811f, projectedL1.getFloat(0), 0.000001); - Assert.assertEquals("L1 should not contain long", 1, projectedL1.getArity()); - RowData projectedL2 = (RowData) locations.get(StringData.fromString("L2")); - Assert.assertNotNull("L2 should not be null", projectedL2); - Assert.assertEquals("L2 should contain lat", 52.995143f, projectedL2.getFloat(0), 0.000001); - Assert.assertEquals("L2 should not contain long", 1, projectedL2.getArity()); - - projected = writeAndRead("long_only", writeSchema, writeSchema.select("locations.long"), row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - locations = (GenericMapData) projected.getMap(0); - Assert.assertNotNull("Should project locations map", locations); - Assert.assertEquals("Should contain L1 and L2", l1l2Array, locations.keyArray()); - projectedL1 = (RowData) locations.get(StringData.fromString("L1")); - Assert.assertNotNull("L1 should not be null", projectedL1); - Assert.assertEquals("L1 should not contain lat", 1, projectedL1.getArity()); - Assert.assertEquals("L1 should contain long", -1.542616f, projectedL1.getFloat(0), 0.000001); - projectedL2 = (RowData) locations.get(StringData.fromString("L2")); - Assert.assertNotNull("L2 should not be null", projectedL2); - Assert.assertEquals("L2 should not contain lat", 1, projectedL2.getArity()); - Assert.assertEquals("L2 should contain long", -1.539054f, projectedL2.getFloat(0), 0.000001); - - Schema latitiudeRenamed = - new Schema( - Types.NestedField.optional( - 5, - "locations", - Types.MapType.ofOptional( - 6, - 7, - Types.StringType.get(), - Types.StructType.of( - Types.NestedField.required(1, "latitude", Types.FloatType.get()))))); - - projected = writeAndRead("latitude_renamed", writeSchema, latitiudeRenamed, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - locations = (GenericMapData) projected.getMap(0); - Assert.assertNotNull("Should project locations map", locations); - Assert.assertEquals("Should contain L1 and L2", l1l2Array, locations.keyArray()); - projectedL1 = (RowData) locations.get(StringData.fromString("L1")); - Assert.assertNotNull("L1 should not be null", projectedL1); - Assert.assertEquals( - "L1 should contain latitude", 53.992811f, projectedL1.getFloat(0), 0.000001); - projectedL2 = (RowData) locations.get(StringData.fromString("L2")); - Assert.assertNotNull("L2 should not be null", projectedL2); - Assert.assertEquals( - "L2 should contain latitude", 52.995143f, projectedL2.getFloat(0), 0.000001); - } - - @Test - public void testListProjection() throws IOException { - Schema writeSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional( - 10, "values", Types.ListType.ofOptional(11, Types.LongType.get()))); - - GenericArrayData values = new GenericArrayData(new Long[] {56L, 57L, 58L}); - - RowData row = GenericRowData.of(34L, values); - - Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); - - RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); - Assert.assertEquals("Should not project values list", 1, projected.getArity()); - - Schema elementOnly = writeSchema.select("values.element"); - projected = writeAndRead("element_only", writeSchema, elementOnly, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project entire list", values, projected.getArray(0)); - - Schema listOnly = writeSchema.select("values"); - projected = writeAndRead("list_only", writeSchema, listOnly, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project entire list", values, projected.getArray(0)); - } - - @Test - @SuppressWarnings("unchecked") - public void testListOfStructsProjection() throws IOException { - Schema writeSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional( - 22, - "points", - Types.ListType.ofOptional( - 21, - Types.StructType.of( - Types.NestedField.required(19, "x", Types.IntegerType.get()), - Types.NestedField.optional(18, "y", Types.IntegerType.get()))))); - - RowData p1 = GenericRowData.of(1, 2); - RowData p2 = GenericRowData.of(3, null); - GenericArrayData arrayData = new GenericArrayData(new RowData[] {p1, p2}); - RowData row = GenericRowData.of(34L, arrayData); - - Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); - - RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); - Assert.assertEquals("Should not project points list", 1, projected.getArity()); - - projected = writeAndRead("all_points", writeSchema, writeSchema.select("points"), row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project points list", row.getArray(1), projected.getArray(0)); - - projected = writeAndRead("x_only", writeSchema, writeSchema.select("points.x"), row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertFalse("Should project points list", projected.isNullAt(0)); - ArrayData points = projected.getArray(0); - Assert.assertEquals("Should read 2 points", 2, points.size()); - RowData projectedP1 = points.getRow(0, 2); - Assert.assertEquals("Should project x", 1, projectedP1.getInt(0)); - Assert.assertEquals("Should not project y", 1, projectedP1.getArity()); - RowData projectedP2 = points.getRow(1, 2); - Assert.assertEquals("Should not project y", 1, projectedP2.getArity()); - Assert.assertEquals("Should project x", 3, projectedP2.getInt(0)); - - projected = writeAndRead("y_only", writeSchema, writeSchema.select("points.y"), row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertFalse("Should project points list", projected.isNullAt(0)); - points = projected.getArray(0); - Assert.assertEquals("Should read 2 points", 2, points.size()); - projectedP1 = points.getRow(0, 2); - Assert.assertEquals("Should not project x", 1, projectedP1.getArity()); - Assert.assertEquals("Should project y", 2, projectedP1.getInt(0)); - projectedP2 = points.getRow(1, 2); - Assert.assertEquals("Should not project x", 1, projectedP2.getArity()); - Assert.assertTrue("Should project null y", projectedP2.isNullAt(0)); - - Schema yRenamed = - new Schema( - Types.NestedField.optional( - 22, - "points", - Types.ListType.ofOptional( - 21, - Types.StructType.of( - Types.NestedField.optional(18, "z", Types.IntegerType.get()))))); - - projected = writeAndRead("y_renamed", writeSchema, yRenamed, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertFalse("Should project points list", projected.isNullAt(0)); - points = projected.getArray(0); - Assert.assertEquals("Should read 2 points", 2, points.size()); - projectedP1 = points.getRow(0, 2); - Assert.assertEquals("Should not project x and y", 1, projectedP1.getArity()); - Assert.assertEquals("Should project z", 2, projectedP1.getInt(0)); - projectedP2 = points.getRow(1, 2); - Assert.assertEquals("Should not project x and y", 1, projectedP2.getArity()); - Assert.assertTrue("Should project null z", projectedP2.isNullAt(0)); - } - - @Test - public void testAddedFieldsWithRequiredChildren() throws Exception { - Schema schema = new Schema(Types.NestedField.required(1, "a", Types.LongType.get())); - - RowData row = GenericRowData.of(100L); - - Schema addedFields = - new Schema( - Types.NestedField.optional(1, "a", Types.LongType.get()), - Types.NestedField.optional( - 2, - "b", - Types.StructType.of(Types.NestedField.required(3, "c", Types.LongType.get()))), - Types.NestedField.optional(4, "d", Types.ListType.ofRequired(5, Types.LongType.get())), - Types.NestedField.optional( - 6, - "e", - Types.MapType.ofRequired(7, 8, Types.LongType.get(), Types.LongType.get()))); - - RowData projected = - writeAndRead("add_fields_with_required_children_projection", schema, addedFields, row); - Assert.assertEquals("Should contain the correct value in column 1", projected.getLong(0), 100L); - Assert.assertTrue("Should contain empty value in new column 2", projected.isNullAt(1)); - Assert.assertTrue("Should contain empty value in new column 4", projected.isNullAt(2)); - Assert.assertTrue("Should contain empty value in new column 6", projected.isNullAt(3)); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java deleted file mode 100644 index e0340e0743b0..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java +++ /dev/null @@ -1,100 +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.iceberg.flink.data; - -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.flink.DataGenerator; -import org.apache.iceberg.flink.DataGenerators; -import org.apache.iceberg.flink.TestHelpers; -import org.junit.Test; - -public class TestStructRowData { - - protected void testConverter(DataGenerator dataGenerator) { - StructRowData converter = new StructRowData(dataGenerator.icebergSchema().asStruct()); - GenericRecord expected = dataGenerator.generateIcebergGenericRecord(); - StructRowData actual = converter.setStruct(expected); - TestHelpers.assertRowData(dataGenerator.icebergSchema(), expected, actual); - } - - @Test - public void testPrimitiveTypes() { - testConverter(new DataGenerators.Primitives()); - } - - @Test - public void testStructOfPrimitive() { - testConverter(new DataGenerators.StructOfPrimitive()); - } - - @Test - public void testStructOfArray() { - testConverter(new DataGenerators.StructOfArray()); - } - - @Test - public void testStructOfMap() { - testConverter(new DataGenerators.StructOfMap()); - } - - @Test - public void testStructOfStruct() { - testConverter(new DataGenerators.StructOfStruct()); - } - - @Test - public void testArrayOfPrimitive() { - testConverter(new DataGenerators.ArrayOfPrimitive()); - } - - @Test - public void testArrayOfArray() { - testConverter(new DataGenerators.ArrayOfArray()); - } - - @Test - public void testArrayOfMap() { - testConverter(new DataGenerators.ArrayOfMap()); - } - - @Test - public void testArrayOfStruct() { - testConverter(new DataGenerators.ArrayOfStruct()); - } - - @Test - public void testMapOfPrimitives() { - testConverter(new DataGenerators.MapOfPrimitives()); - } - - @Test - public void testMapOfArray() { - testConverter(new DataGenerators.MapOfArray()); - } - - @Test - public void testMapOfMap() { - testConverter(new DataGenerators.MapOfMap()); - } - - @Test - public void testMapOfStruct() { - testConverter(new DataGenerators.MapOfStruct()); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java deleted file mode 100644 index 6a493692c20d..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java +++ /dev/null @@ -1,37 +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.iceberg.flink.sink; - -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.flink.AvroGenericRecordConverterBase; -import org.apache.iceberg.flink.DataGenerator; -import org.junit.Assert; - -public class TestAvroGenericRecordToRowDataMapper extends AvroGenericRecordConverterBase { - @Override - protected void testConverter(DataGenerator dataGenerator) throws Exception { - // Need to use avroSchema from DataGenerator because some primitive types have special Avro - // type handling. Hence the Avro schema converted from Iceberg schema won't work. - AvroGenericRecordToRowDataMapper mapper = - AvroGenericRecordToRowDataMapper.forAvroSchema(dataGenerator.avroSchema()); - RowData expected = dataGenerator.generateFlinkRowData(); - RowData actual = mapper.map(dataGenerator.generateAvroGenericRecord()); - Assert.assertEquals(expected, actual); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java deleted file mode 100644 index 5ebcc6361c7b..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java +++ /dev/null @@ -1,65 +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.iceberg.flink.sink; - -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.sink.TestBucketPartitionerUtil.TableSchemaType; -import org.assertj.core.api.Assertions; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.EnumSource; - -public class TestBucketPartitionKeySelector { - - @ParameterizedTest - @EnumSource( - value = TableSchemaType.class, - names = {"ONE_BUCKET", "IDENTITY_AND_BUCKET"}) - public void testCorrectKeySelection(TableSchemaType tableSchemaType) { - int numBuckets = 60; - - PartitionSpec partitionSpec = tableSchemaType.getPartitionSpec(numBuckets); - BucketPartitionKeySelector keySelector = - new BucketPartitionKeySelector( - partitionSpec, SimpleDataUtil.SCHEMA, SimpleDataUtil.ROW_TYPE); - - TestBucketPartitionerUtil.generateRowsForBucketIdRange(2, numBuckets) - .forEach( - rowData -> { - int expectedBucketId = - TestBucketPartitionerUtil.computeBucketId( - numBuckets, rowData.getString(1).toString()); - Integer key = keySelector.getKey(rowData); - Assertions.assertThat(key).isEqualTo(expectedBucketId); - }); - } - - @Test - public void testKeySelectorMultipleBucketsFail() { - PartitionSpec partitionSpec = TableSchemaType.TWO_BUCKETS.getPartitionSpec(1); - - Assertions.assertThatExceptionOfType(RuntimeException.class) - .isThrownBy( - () -> - new BucketPartitionKeySelector( - partitionSpec, SimpleDataUtil.SCHEMA, SimpleDataUtil.ROW_TYPE)) - .withMessage(BucketPartitionerUtil.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE, 2); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java deleted file mode 100644 index 835713e6b417..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java +++ /dev/null @@ -1,107 +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.iceberg.flink.sink; - -import static org.apache.iceberg.flink.sink.BucketPartitioner.BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE; -import static org.apache.iceberg.flink.sink.BucketPartitioner.BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE; -import static org.apache.iceberg.flink.sink.BucketPartitioner.BUCKET_NULL_MESSAGE; - -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.flink.sink.TestBucketPartitionerUtil.TableSchemaType; -import org.assertj.core.api.Assertions; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.CsvSource; - -public class TestBucketPartitioner { - - static final int DEFAULT_NUM_BUCKETS = 60; - - @ParameterizedTest - @CsvSource({"ONE_BUCKET,50", "IDENTITY_AND_BUCKET,50", "ONE_BUCKET,60", "IDENTITY_AND_BUCKET,60"}) - public void testPartitioningParallelismGreaterThanBuckets( - String schemaTypeStr, String numBucketsStr) { - int numPartitions = 500; - TableSchemaType tableSchemaType = TableSchemaType.valueOf(schemaTypeStr); - int numBuckets = Integer.parseInt(numBucketsStr); - PartitionSpec partitionSpec = tableSchemaType.getPartitionSpec(numBuckets); - BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); - - int bucketId = 0; - for (int expectedIndex = 0; expectedIndex < numPartitions; expectedIndex++) { - int actualPartitionIndex = bucketPartitioner.partition(bucketId, numPartitions); - Assertions.assertThat(actualPartitionIndex).isEqualTo(expectedIndex); - bucketId++; - if (bucketId == numBuckets) { - bucketId = 0; - } - } - } - - @ParameterizedTest - @CsvSource({"ONE_BUCKET,50", "IDENTITY_AND_BUCKET,50", "ONE_BUCKET,60", "IDENTITY_AND_BUCKET,60"}) - public void testPartitioningParallelismEqualLessThanBuckets( - String schemaTypeStr, String numBucketsStr) { - int numPartitions = 30; - TableSchemaType tableSchemaType = TableSchemaType.valueOf(schemaTypeStr); - int numBuckets = Integer.parseInt(numBucketsStr); - PartitionSpec partitionSpec = tableSchemaType.getPartitionSpec(numBuckets); - BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); - - for (int bucketId = 0; bucketId < numBuckets; bucketId++) { - int actualPartitionIndex = bucketPartitioner.partition(bucketId, numPartitions); - Assertions.assertThat(actualPartitionIndex).isEqualTo(bucketId % numPartitions); - } - } - - @Test - public void testPartitionerBucketIdNullFail() { - PartitionSpec partitionSpec = TableSchemaType.ONE_BUCKET.getPartitionSpec(DEFAULT_NUM_BUCKETS); - BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); - - Assertions.assertThatExceptionOfType(RuntimeException.class) - .isThrownBy(() -> bucketPartitioner.partition(null, DEFAULT_NUM_BUCKETS)) - .withMessage(BUCKET_NULL_MESSAGE); - } - - @Test - public void testPartitionerMultipleBucketsFail() { - PartitionSpec partitionSpec = TableSchemaType.TWO_BUCKETS.getPartitionSpec(DEFAULT_NUM_BUCKETS); - - Assertions.assertThatExceptionOfType(RuntimeException.class) - .isThrownBy(() -> new BucketPartitioner(partitionSpec)) - .withMessage(BucketPartitionerUtil.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE, 2); - } - - @Test - public void testPartitionerBucketIdOutOfRangeFail() { - PartitionSpec partitionSpec = TableSchemaType.ONE_BUCKET.getPartitionSpec(DEFAULT_NUM_BUCKETS); - BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); - - int negativeBucketId = -1; - Assertions.assertThatExceptionOfType(IllegalArgumentException.class) - .isThrownBy(() -> bucketPartitioner.partition(negativeBucketId, 1)) - .withMessage(BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE, negativeBucketId); - - int tooBigBucketId = DEFAULT_NUM_BUCKETS; - Assertions.assertThatExceptionOfType(IllegalArgumentException.class) - .isThrownBy(() -> bucketPartitioner.partition(tooBigBucketId, 1)) - .withMessage(BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE, tooBigBucketId, DEFAULT_NUM_BUCKETS); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java deleted file mode 100644 index 9dae43ce5e58..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java +++ /dev/null @@ -1,227 +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.iceberg.flink.sink; - -import static org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG; -import static org.apache.iceberg.flink.TestFixtures.DATABASE; -import static org.apache.iceberg.flink.TestFixtures.TABLE_IDENTIFIER; - -import java.io.IOException; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.util.DataFormatConverters; -import org.apache.flink.test.junit5.MiniClusterExtension; -import org.apache.flink.types.Row; -import org.apache.iceberg.DistributionMode; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.sink.TestBucketPartitionerUtil.TableSchemaType; -import org.apache.iceberg.flink.source.BoundedTestSource; -import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.assertj.core.api.Assertions; -import org.junit.jupiter.api.extension.RegisterExtension; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.EnumSource; - -public class TestBucketPartitionerFlinkIcebergSink { - - private static final int NUMBER_TASK_MANAGERS = 1; - private static final int SLOTS_PER_TASK_MANAGER = 8; - - @RegisterExtension - private static final MiniClusterExtension MINI_CLUSTER_RESOURCE = - new MiniClusterExtension( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(NUMBER_TASK_MANAGERS) - .setNumberSlotsPerTaskManager(SLOTS_PER_TASK_MANAGER) - .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) - .build()); - - @RegisterExtension - private static final HadoopCatalogExtension catalogExtension = - new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); - - private static final TypeInformation ROW_TYPE_INFO = - new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); - - // Parallelism = 8 (parallelism > numBuckets) throughout the test suite - private final int parallelism = NUMBER_TASK_MANAGERS * SLOTS_PER_TASK_MANAGER; - private final FileFormat format = FileFormat.PARQUET; - private final int numBuckets = 4; - - private Table table; - private StreamExecutionEnvironment env; - private TableLoader tableLoader; - - private void setupEnvironment(TableSchemaType tableSchemaType) { - PartitionSpec partitionSpec = tableSchemaType.getPartitionSpec(numBuckets); - table = - catalogExtension - .catalog() - .createTable( - TABLE_IDENTIFIER, - SimpleDataUtil.SCHEMA, - partitionSpec, - ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); - env = - StreamExecutionEnvironment.getExecutionEnvironment(DISABLE_CLASSLOADER_CHECK_CONFIG) - .enableCheckpointing(100) - .setParallelism(parallelism) - .setMaxParallelism(parallelism * 2); - tableLoader = catalogExtension.tableLoader(); - } - - private void appendRowsToTable(List allRows) throws Exception { - DataFormatConverters.RowConverter converter = - new DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes()); - - DataStream dataStream = - env.addSource( - new BoundedTestSource<>( - allRows.stream().map(converter::toExternal).toArray(Row[]::new)), - ROW_TYPE_INFO) - .map(converter::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)) - .partitionCustom( - new BucketPartitioner(table.spec()), - new BucketPartitionKeySelector( - table.spec(), - table.schema(), - FlinkSink.toFlinkRowType(table.schema(), SimpleDataUtil.FLINK_SCHEMA))); - - FlinkSink.forRowData(dataStream) - .table(table) - .tableLoader(tableLoader) - .writeParallelism(parallelism) - .distributionMode(DistributionMode.NONE) - .append(); - - env.execute("Test Iceberg DataStream"); - - SimpleDataUtil.assertTableRows(table, allRows); - } - - @ParameterizedTest - @EnumSource( - value = TableSchemaType.class, - names = {"ONE_BUCKET", "IDENTITY_AND_BUCKET"}) - public void testSendRecordsToAllBucketsEvenly(TableSchemaType tableSchemaType) throws Exception { - setupEnvironment(tableSchemaType); - List rows = generateTestDataRows(); - - appendRowsToTable(rows); - TableTestStats stats = extractPartitionResults(tableSchemaType); - - Assertions.assertThat(stats.totalRowCount).isEqualTo(rows.size()); - // All 4 buckets should've been written to - Assertions.assertThat(stats.writersPerBucket.size()).isEqualTo(numBuckets); - Assertions.assertThat(stats.numFilesPerBucket.size()).isEqualTo(numBuckets); - // Writer expectation (2 writers per bucket): - // - Bucket0 -> Writers [0, 4] - // - Bucket1 -> Writers [1, 5] - // - Bucket2 -> Writers [2, 6] - // - Bucket3 -> Writers [3, 7] - for (int i = 0, j = numBuckets; i < numBuckets; i++, j++) { - Assertions.assertThat(stats.writersPerBucket.get(i)).hasSameElementsAs(Arrays.asList(i, j)); - // 2 files per bucket (one file is created by each writer) - Assertions.assertThat(stats.numFilesPerBucket.get(i)).isEqualTo(2); - // 2 rows per file (total of 16 rows across 8 files) - Assertions.assertThat(stats.rowsPerWriter.get(i)).isEqualTo(2); - } - } - - /** - * Generating 16 rows to be sent uniformly to all writers (round-robin across 8 writers -> 4 - * buckets) - */ - private List generateTestDataRows() { - int totalNumRows = parallelism * 2; - int numRowsPerBucket = totalNumRows / numBuckets; - return TestBucketPartitionerUtil.generateRowsForBucketIdRange(numRowsPerBucket, numBuckets); - } - - private TableTestStats extractPartitionResults(TableSchemaType tableSchemaType) - throws IOException { - int totalRecordCount = 0; - Map> writersPerBucket = Maps.newHashMap(); // > - Map filesPerBucket = Maps.newHashMap(); // - Map rowsPerWriter = Maps.newHashMap(); // - - try (CloseableIterable fileScanTasks = table.newScan().planFiles()) { - for (FileScanTask scanTask : fileScanTasks) { - long recordCountInFile = scanTask.file().recordCount(); - - String[] splitFilePath = scanTask.file().path().toString().split("/"); - // Filename example: 00007-0-a7d3a29a-33e9-4740-88f4-0f494397d60c-00001.parquet - // Writer ID: .......^^^^^ - String filename = splitFilePath[splitFilePath.length - 1]; - int writerId = Integer.parseInt(filename.split("-")[0]); - - totalRecordCount += recordCountInFile; - int bucketId = - scanTask - .file() - .partition() - .get(tableSchemaType.bucketPartitionColumnPosition(), Integer.class); - writersPerBucket.computeIfAbsent(bucketId, k -> Lists.newArrayList()); - writersPerBucket.get(bucketId).add(writerId); - filesPerBucket.put(bucketId, filesPerBucket.getOrDefault(bucketId, 0) + 1); - rowsPerWriter.put(writerId, rowsPerWriter.getOrDefault(writerId, 0L) + recordCountInFile); - } - } - - return new TableTestStats(totalRecordCount, writersPerBucket, filesPerBucket, rowsPerWriter); - } - - /** DTO to hold Test Stats */ - private static class TableTestStats { - final int totalRowCount; - final Map> writersPerBucket; - final Map numFilesPerBucket; - final Map rowsPerWriter; - - TableTestStats( - int totalRecordCount, - Map> writersPerBucket, - Map numFilesPerBucket, - Map rowsPerWriter) { - this.totalRowCount = totalRecordCount; - this.writersPerBucket = writersPerBucket; - this.numFilesPerBucket = numFilesPerBucket; - this.rowsPerWriter = rowsPerWriter; - } - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java deleted file mode 100644 index e1309bfac6d5..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java +++ /dev/null @@ -1,126 +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.iceberg.flink.sink; - -import java.util.List; -import java.util.UUID; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.util.BucketUtil; - -final class TestBucketPartitionerUtil { - - enum TableSchemaType { - ONE_BUCKET { - @Override - public int bucketPartitionColumnPosition() { - return 0; - } - - @Override - public PartitionSpec getPartitionSpec(int numBuckets) { - return PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).bucket("data", numBuckets).build(); - } - }, - IDENTITY_AND_BUCKET { - @Override - public int bucketPartitionColumnPosition() { - return 1; - } - - @Override - public PartitionSpec getPartitionSpec(int numBuckets) { - return PartitionSpec.builderFor(SimpleDataUtil.SCHEMA) - .identity("id") - .bucket("data", numBuckets) - .build(); - } - }, - TWO_BUCKETS { - @Override - public int bucketPartitionColumnPosition() { - return 1; - } - - @Override - public PartitionSpec getPartitionSpec(int numBuckets) { - return PartitionSpec.builderFor(SimpleDataUtil.SCHEMA) - .bucket("id", numBuckets) - .bucket("data", numBuckets) - .build(); - } - }; - - public abstract int bucketPartitionColumnPosition(); - - public abstract PartitionSpec getPartitionSpec(int numBuckets); - } - - private TestBucketPartitionerUtil() {} - - /** - * Utility method to generate rows whose values will "hash" to a range of bucketIds (from 0 to - * numBuckets - 1) - * - * @param numRowsPerBucket how many different rows should be generated per bucket - * @param numBuckets max number of buckets to consider - * @return the list of rows whose data "hashes" to the desired bucketId - */ - static List generateRowsForBucketIdRange(int numRowsPerBucket, int numBuckets) { - List rows = Lists.newArrayListWithCapacity(numBuckets * numRowsPerBucket); - // For some of our tests, this order of the generated rows matters - for (int i = 0; i < numRowsPerBucket; i++) { - for (int bucketId = 0; bucketId < numBuckets; bucketId++) { - String value = generateValueForBucketId(bucketId, numBuckets); - rows.add(GenericRowData.of(1, StringData.fromString(value))); - } - } - return rows; - } - - /** - * Utility method to generate a UUID string that will "hash" to a desired bucketId - * - * @param bucketId the desired bucketId - * @return the string data that "hashes" to the desired bucketId - */ - private static String generateValueForBucketId(int bucketId, int numBuckets) { - while (true) { - String uuid = UUID.randomUUID().toString(); - if (computeBucketId(numBuckets, uuid) == bucketId) { - return uuid; - } - } - } - - /** - * Utility that performs the same hashing/bucketing mechanism used by Bucket.java - * - * @param numBuckets max number of buckets to consider - * @param value the string to compute the bucketId from - * @return the computed bucketId - */ - static int computeBucketId(int numBuckets, String value) { - return (BucketUtil.hash(value) & Integer.MAX_VALUE) % numBuckets; - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java deleted file mode 100644 index 360db658cd2f..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java +++ /dev/null @@ -1,81 +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.iceberg.flink.sink; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -import java.time.Duration; -import java.util.concurrent.TimeUnit; -import org.apache.iceberg.SerializableTable; -import org.apache.iceberg.Table; -import org.apache.iceberg.flink.TableLoader; -import org.awaitility.Awaitility; -import org.junit.jupiter.api.Test; - -public class TestCachingTableSupplier { - - @Test - public void testCheckArguments() { - SerializableTable initialTable = mock(SerializableTable.class); - - Table loadedTable = mock(Table.class); - TableLoader tableLoader = mock(TableLoader.class); - when(tableLoader.loadTable()).thenReturn(loadedTable); - - new CachingTableSupplier(initialTable, tableLoader, Duration.ofMillis(100)); - - assertThatThrownBy(() -> new CachingTableSupplier(initialTable, tableLoader, null)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("tableRefreshInterval cannot be null"); - assertThatThrownBy(() -> new CachingTableSupplier(null, tableLoader, Duration.ofMillis(100))) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("initialTable cannot be null"); - assertThatThrownBy(() -> new CachingTableSupplier(initialTable, null, Duration.ofMillis(100))) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("tableLoader cannot be null"); - } - - @Test - public void testTableReload() { - SerializableTable initialTable = mock(SerializableTable.class); - - Table loadedTable = mock(Table.class); - TableLoader tableLoader = mock(TableLoader.class); - when(tableLoader.loadTable()).thenReturn(loadedTable); - - CachingTableSupplier cachingTableSupplier = - new CachingTableSupplier(initialTable, tableLoader, Duration.ofMillis(100)); - - // refresh shouldn't do anything as the min reload interval hasn't passed - cachingTableSupplier.refreshTable(); - assertThat(cachingTableSupplier.get()).isEqualTo(initialTable); - - // refresh after waiting past the min reload interval - Awaitility.await() - .atLeast(100, TimeUnit.MILLISECONDS) - .untilAsserted( - () -> { - cachingTableSupplier.refreshTable(); - assertThat(cachingTableSupplier.get()).isEqualTo(loadedTable); - }); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java deleted file mode 100644 index d9d57fb7107e..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java +++ /dev/null @@ -1,254 +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.iceberg.flink.sink; - -import java.io.File; -import java.io.IOException; -import java.util.Map; -import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.common.DynFields; -import org.apache.iceberg.flink.FlinkWriteConf; -import org.apache.iceberg.flink.FlinkWriteOptions; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.io.BaseTaskWriter; -import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.WriteResult; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestCompressionSettings { - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); - - private Table table; - - private final Map initProperties; - - @Parameterized.Parameters(name = "tableProperties = {0}") - public static Object[] parameters() { - return new Object[] { - ImmutableMap.of(), - ImmutableMap.of( - TableProperties.AVRO_COMPRESSION, - "zstd", - TableProperties.AVRO_COMPRESSION_LEVEL, - "3", - TableProperties.PARQUET_COMPRESSION, - "zstd", - TableProperties.PARQUET_COMPRESSION_LEVEL, - "3", - TableProperties.ORC_COMPRESSION, - "zstd", - TableProperties.ORC_COMPRESSION_STRATEGY, - "compression") - }; - } - - public TestCompressionSettings(Map initProperties) { - this.initProperties = initProperties; - } - - @Before - public void before() throws IOException { - File folder = tempFolder.newFolder(); - table = SimpleDataUtil.createTable(folder.getAbsolutePath(), initProperties, false); - } - - @Test - public void testCompressionAvro() throws Exception { - // No override provided - Map resultProperties = - appenderProperties( - table, - SimpleDataUtil.FLINK_SCHEMA, - ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "AVRO")); - - if (initProperties.get(TableProperties.AVRO_COMPRESSION) == null) { - Assert.assertEquals( - TableProperties.AVRO_COMPRESSION_DEFAULT, - resultProperties.get(TableProperties.AVRO_COMPRESSION)); - Assert.assertEquals( - TableProperties.AVRO_COMPRESSION_LEVEL_DEFAULT, - resultProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); - } else { - Assert.assertEquals( - initProperties.get(TableProperties.AVRO_COMPRESSION), - resultProperties.get(TableProperties.AVRO_COMPRESSION)); - Assert.assertEquals( - initProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL), - resultProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); - } - - // Override compression to snappy and some random level - resultProperties = - appenderProperties( - table, - SimpleDataUtil.FLINK_SCHEMA, - ImmutableMap.of( - FlinkWriteOptions.WRITE_FORMAT.key(), - "AVRO", - FlinkWriteOptions.COMPRESSION_CODEC.key(), - "snappy", - FlinkWriteOptions.COMPRESSION_LEVEL.key(), - "6")); - - Assert.assertEquals("snappy", resultProperties.get(TableProperties.AVRO_COMPRESSION)); - Assert.assertEquals("6", resultProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); - } - - @Test - public void testCompressionParquet() throws Exception { - // No override provided - Map resultProperties = - appenderProperties( - table, - SimpleDataUtil.FLINK_SCHEMA, - ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "PARQUET")); - - if (initProperties.get(TableProperties.PARQUET_COMPRESSION) == null) { - Assert.assertEquals( - TableProperties.PARQUET_COMPRESSION_DEFAULT_SINCE_1_4_0, - resultProperties.get(TableProperties.PARQUET_COMPRESSION)); - Assert.assertEquals( - TableProperties.PARQUET_COMPRESSION_LEVEL_DEFAULT, - resultProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL_DEFAULT)); - } else { - Assert.assertEquals( - initProperties.get(TableProperties.PARQUET_COMPRESSION), - resultProperties.get(TableProperties.PARQUET_COMPRESSION)); - Assert.assertEquals( - initProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL), - resultProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL)); - } - - // Override compression to snappy and some random level - resultProperties = - appenderProperties( - table, - SimpleDataUtil.FLINK_SCHEMA, - ImmutableMap.of( - FlinkWriteOptions.WRITE_FORMAT.key(), - "PARQUET", - FlinkWriteOptions.COMPRESSION_CODEC.key(), - "snappy", - FlinkWriteOptions.COMPRESSION_LEVEL.key(), - "6")); - - Assert.assertEquals("snappy", resultProperties.get(TableProperties.PARQUET_COMPRESSION)); - Assert.assertEquals("6", resultProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL)); - } - - @Test - public void testCompressionOrc() throws Exception { - // No override provided - Map resultProperties = - appenderProperties( - table, - SimpleDataUtil.FLINK_SCHEMA, - ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "ORC")); - - if (initProperties.get(TableProperties.ORC_COMPRESSION) == null) { - Assert.assertEquals( - TableProperties.ORC_COMPRESSION_DEFAULT, - resultProperties.get(TableProperties.ORC_COMPRESSION)); - Assert.assertEquals( - TableProperties.ORC_COMPRESSION_STRATEGY_DEFAULT, - resultProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); - } else { - Assert.assertEquals( - initProperties.get(TableProperties.ORC_COMPRESSION), - resultProperties.get(TableProperties.ORC_COMPRESSION)); - Assert.assertEquals( - initProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY), - resultProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); - } - - // Override compression to snappy and a different strategy - resultProperties = - appenderProperties( - table, - SimpleDataUtil.FLINK_SCHEMA, - ImmutableMap.of( - FlinkWriteOptions.WRITE_FORMAT.key(), - "ORC", - FlinkWriteOptions.COMPRESSION_CODEC.key(), - "snappy", - FlinkWriteOptions.COMPRESSION_STRATEGY.key(), - "speed")); - - Assert.assertEquals("snappy", resultProperties.get(TableProperties.ORC_COMPRESSION)); - Assert.assertEquals("speed", resultProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); - } - - private static OneInputStreamOperatorTestHarness createIcebergStreamWriter( - Table icebergTable, TableSchema flinkSchema, Map override) throws Exception { - RowType flinkRowType = FlinkSink.toFlinkRowType(icebergTable.schema(), flinkSchema); - FlinkWriteConf flinkWriteConfig = - new FlinkWriteConf( - icebergTable, override, new org.apache.flink.configuration.Configuration()); - - IcebergStreamWriter streamWriter = - FlinkSink.createStreamWriter(() -> icebergTable, flinkWriteConfig, flinkRowType, null); - OneInputStreamOperatorTestHarness harness = - new OneInputStreamOperatorTestHarness<>(streamWriter, 1, 1, 0); - - harness.setup(); - harness.open(); - - return harness; - } - - private static Map appenderProperties( - Table table, TableSchema schema, Map override) throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = - createIcebergStreamWriter(table, schema, override)) { - testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); - - testHarness.prepareSnapshotPreBarrier(1L); - DynFields.BoundField operatorField = - DynFields.builder() - .hiddenImpl(testHarness.getOperatorFactory().getClass(), "operator") - .build(testHarness.getOperatorFactory()); - DynFields.BoundField writerField = - DynFields.builder() - .hiddenImpl(IcebergStreamWriter.class, "writer") - .build(operatorField.get()); - DynFields.BoundField appenderField = - DynFields.builder() - .hiddenImpl(BaseTaskWriter.class, "appenderFactory") - .build(writerField.get()); - DynFields.BoundField> propsField = - DynFields.builder() - .hiddenImpl(FlinkAppenderFactory.class, "props") - .build(appenderField.get()); - return propsField.get(); - } - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java deleted file mode 100644 index 4ecbd1c12921..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java +++ /dev/null @@ -1,440 +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.iceberg.flink.sink; - -import static org.apache.iceberg.flink.SimpleDataUtil.createDelete; -import static org.apache.iceberg.flink.SimpleDataUtil.createInsert; -import static org.apache.iceberg.flink.SimpleDataUtil.createRecord; -import static org.apache.iceberg.flink.SimpleDataUtil.createUpdateAfter; -import static org.apache.iceberg.flink.SimpleDataUtil.createUpdateBefore; -import static org.apache.iceberg.types.Types.NestedField.required; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.time.OffsetDateTime; -import java.util.Arrays; -import java.util.List; -import java.util.stream.Collectors; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.types.logical.IntType; -import org.apache.flink.table.types.logical.LocalZonedTimestampType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.types.RowKind; -import org.apache.iceberg.ContentFile; -import org.apache.iceberg.FileContent; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.RowDelta; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SerializableTable; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.TableTestBase; -import org.apache.iceberg.TestTables; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.WriteResult; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.StructLikeSet; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestDeltaTaskWriter extends TableTestBase { - private static final int FORMAT_V2 = 2; - - private final FileFormat format; - - @Parameterized.Parameters(name = "FileFormat = {0}") - public static Object[][] parameters() { - return new Object[][] {{"avro"}, {"orc"}, {"parquet"}}; - } - - public TestDeltaTaskWriter(String fileFormat) { - super(FORMAT_V2); - this.format = FileFormat.fromString(fileFormat); - } - - @Override - @Before - public void setupTable() throws IOException { - this.tableDir = temp.newFolder(); - Assert.assertTrue(tableDir.delete()); // created by table create - - this.metadataDir = new File(tableDir, "metadata"); - } - - private int idFieldId() { - return table.schema().findField("id").fieldId(); - } - - private int dataFieldId() { - return table.schema().findField("data").fieldId(); - } - - private void testCdcEvents(boolean partitioned) throws IOException { - List equalityFieldIds = Lists.newArrayList(idFieldId()); - TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); - taskWriterFactory.initialize(1, 1); - - // Start the 1th transaction. - TaskWriter writer = taskWriterFactory.create(); - - writer.write(createInsert(1, "aaa")); - writer.write(createInsert(2, "bbb")); - writer.write(createInsert(3, "ccc")); - - // Update <2, 'bbb'> to <2, 'ddd'> - writer.write(createUpdateBefore(2, "bbb")); // 1 pos-delete and 1 eq-delete. - writer.write(createUpdateAfter(2, "ddd")); - - // Update <1, 'aaa'> to <1, 'eee'> - writer.write(createUpdateBefore(1, "aaa")); // 1 pos-delete and 1 eq-delete. - writer.write(createUpdateAfter(1, "eee")); - - // Insert <4, 'fff'> - writer.write(createInsert(4, "fff")); - // Insert <5, 'ggg'> - writer.write(createInsert(5, "ggg")); - - // Delete <3, 'ccc'> - writer.write(createDelete(3, "ccc")); // 1 pos-delete and 1 eq-delete. - - WriteResult result = writer.complete(); - Assert.assertEquals(partitioned ? 7 : 1, result.dataFiles().length); - Assert.assertEquals(partitioned ? 3 : 1, result.deleteFiles().length); - commitTransaction(result); - - Assert.assertEquals( - "Should have expected records.", - expectedRowSet( - createRecord(1, "eee"), - createRecord(2, "ddd"), - createRecord(4, "fff"), - createRecord(5, "ggg")), - actualRowSet("*")); - - // Start the 2nd transaction. - writer = taskWriterFactory.create(); - - // Update <2, 'ddd'> to <6, 'hhh'> - (Update both key and value) - writer.write(createUpdateBefore(2, "ddd")); // 1 eq-delete - writer.write(createUpdateAfter(6, "hhh")); - - // Update <5, 'ggg'> to <5, 'iii'> - writer.write(createUpdateBefore(5, "ggg")); // 1 eq-delete - writer.write(createUpdateAfter(5, "iii")); - - // Delete <4, 'fff'> - writer.write(createDelete(4, "fff")); // 1 eq-delete. - - result = writer.complete(); - Assert.assertEquals(partitioned ? 2 : 1, result.dataFiles().length); - Assert.assertEquals(partitioned ? 3 : 1, result.deleteFiles().length); - commitTransaction(result); - - Assert.assertEquals( - "Should have expected records", - expectedRowSet(createRecord(1, "eee"), createRecord(5, "iii"), createRecord(6, "hhh")), - actualRowSet("*")); - } - - @Test - public void testUnpartitioned() throws IOException { - createAndInitTable(false); - testCdcEvents(false); - } - - @Test - public void testPartitioned() throws IOException { - createAndInitTable(true); - testCdcEvents(true); - } - - private void testWritePureEqDeletes(boolean partitioned) throws IOException { - createAndInitTable(partitioned); - List equalityFieldIds = Lists.newArrayList(idFieldId()); - TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); - taskWriterFactory.initialize(1, 1); - - TaskWriter writer = taskWriterFactory.create(); - writer.write(createDelete(1, "aaa")); - writer.write(createDelete(2, "bbb")); - writer.write(createDelete(3, "ccc")); - - WriteResult result = writer.complete(); - Assert.assertEquals(0, result.dataFiles().length); - Assert.assertEquals(partitioned ? 3 : 1, result.deleteFiles().length); - commitTransaction(result); - - Assert.assertEquals("Should have no record", expectedRowSet(), actualRowSet("*")); - } - - @Test - public void testUnpartitionedPureEqDeletes() throws IOException { - testWritePureEqDeletes(false); - } - - @Test - public void testPartitionedPureEqDeletes() throws IOException { - testWritePureEqDeletes(true); - } - - private void testAbort(boolean partitioned) throws IOException { - createAndInitTable(partitioned); - List equalityFieldIds = Lists.newArrayList(idFieldId()); - TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); - taskWriterFactory.initialize(1, 1); - - TaskWriter writer = taskWriterFactory.create(); - for (int i = 0; i < 8_000; i += 2) { - writer.write(createUpdateBefore(i + 1, "aaa")); - writer.write(createUpdateAfter(i + 1, "aaa")); - - writer.write(createUpdateBefore(i + 2, "bbb")); - writer.write(createUpdateAfter(i + 2, "bbb")); - } - - // Assert the current data/delete file count. - List files = - Files.walk(Paths.get(tableDir.getPath(), "data")) - .filter(p -> p.toFile().isFile()) - .filter(p -> !p.toString().endsWith(".crc")) - .collect(Collectors.toList()); - Assert.assertEquals( - "Should have expected file count, but files are: " + files, - partitioned ? 4 : 2, - files.size()); - - writer.abort(); - for (Path file : files) { - Assert.assertFalse(Files.exists(file)); - } - } - - @Test - public void testUnpartitionedAbort() throws IOException { - testAbort(false); - } - - @Test - public void testPartitionedAbort() throws IOException { - testAbort(true); - } - - @Test - public void testPartitionedTableWithDataAsKey() throws IOException { - createAndInitTable(true); - List equalityFieldIds = Lists.newArrayList(dataFieldId()); - TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); - taskWriterFactory.initialize(1, 1); - - // Start the 1th transaction. - TaskWriter writer = taskWriterFactory.create(); - writer.write(createInsert(1, "aaa")); - writer.write(createInsert(2, "aaa")); - writer.write(createInsert(3, "bbb")); - writer.write(createInsert(4, "ccc")); - - WriteResult result = writer.complete(); - Assert.assertEquals(3, result.dataFiles().length); - Assert.assertEquals(1, result.deleteFiles().length); - commitTransaction(result); - - Assert.assertEquals( - "Should have expected records", - expectedRowSet(createRecord(2, "aaa"), createRecord(3, "bbb"), createRecord(4, "ccc")), - actualRowSet("*")); - - // Start the 2nd transaction. - writer = taskWriterFactory.create(); - writer.write(createInsert(5, "aaa")); - writer.write(createInsert(6, "bbb")); - writer.write(createDelete(7, "ccc")); // 1 eq-delete. - - result = writer.complete(); - Assert.assertEquals(2, result.dataFiles().length); - Assert.assertEquals(1, result.deleteFiles().length); - commitTransaction(result); - - Assert.assertEquals( - "Should have expected records", - expectedRowSet( - createRecord(2, "aaa"), - createRecord(5, "aaa"), - createRecord(3, "bbb"), - createRecord(6, "bbb")), - actualRowSet("*")); - } - - @Test - public void testPartitionedTableWithDataAndIdAsKey() throws IOException { - createAndInitTable(true); - List equalityFieldIds = Lists.newArrayList(dataFieldId(), idFieldId()); - TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); - taskWriterFactory.initialize(1, 1); - - TaskWriter writer = taskWriterFactory.create(); - writer.write(createInsert(1, "aaa")); - writer.write(createInsert(2, "aaa")); - - writer.write(createDelete(2, "aaa")); // 1 pos-delete. - - WriteResult result = writer.complete(); - Assert.assertEquals(1, result.dataFiles().length); - Assert.assertEquals(1, result.deleteFiles().length); - Assert.assertEquals( - Sets.newHashSet(FileContent.POSITION_DELETES), - Sets.newHashSet(result.deleteFiles()[0].content())); - commitTransaction(result); - - Assert.assertEquals( - "Should have expected records", expectedRowSet(createRecord(1, "aaa")), actualRowSet("*")); - } - - @Test - public void testEqualityColumnOnCustomPrecisionTSColumn() throws IOException { - Schema tableSchema = - new Schema( - required(3, "id", Types.IntegerType.get()), - required(4, "ts", Types.TimestampType.withZone())); - RowType flinkType = - new RowType( - false, - ImmutableList.of( - new RowType.RowField("id", new IntType()), - new RowType.RowField("ts", new LocalZonedTimestampType(3)))); - - this.table = create(tableSchema, PartitionSpec.unpartitioned()); - initTable(table); - - List equalityIds = ImmutableList.of(table.schema().findField("ts").fieldId()); - TaskWriterFactory taskWriterFactory = createTaskWriterFactory(flinkType, equalityIds); - taskWriterFactory.initialize(1, 1); - - TaskWriter writer = taskWriterFactory.create(); - RowDataSerializer serializer = new RowDataSerializer(flinkType); - OffsetDateTime start = OffsetDateTime.now(); - writer.write( - serializer.toBinaryRow( - GenericRowData.ofKind( - RowKind.INSERT, 1, TimestampData.fromInstant(start.toInstant())))); - writer.write( - serializer.toBinaryRow( - GenericRowData.ofKind( - RowKind.INSERT, 2, TimestampData.fromInstant(start.plusSeconds(1).toInstant())))); - writer.write( - serializer.toBinaryRow( - GenericRowData.ofKind( - RowKind.DELETE, 2, TimestampData.fromInstant(start.plusSeconds(1).toInstant())))); - - WriteResult result = writer.complete(); - // One data file - Assertions.assertThat(result.dataFiles().length).isEqualTo(1); - // One eq delete file + one pos delete file - Assertions.assertThat(result.deleteFiles().length).isEqualTo(2); - Assertions.assertThat( - Arrays.stream(result.deleteFiles()) - .map(ContentFile::content) - .collect(Collectors.toSet())) - .isEqualTo(Sets.newHashSet(FileContent.POSITION_DELETES, FileContent.EQUALITY_DELETES)); - commitTransaction(result); - - Record expectedRecord = GenericRecord.create(tableSchema); - expectedRecord.setField("id", 1); - int cutPrecisionNano = start.getNano() / 1000000 * 1000000; - expectedRecord.setField("ts", start.withNano(cutPrecisionNano)); - - Assertions.assertThat(actualRowSet("*")).isEqualTo(expectedRowSet(expectedRecord)); - } - - private void commitTransaction(WriteResult result) { - RowDelta rowDelta = table.newRowDelta(); - Arrays.stream(result.dataFiles()).forEach(rowDelta::addRows); - Arrays.stream(result.deleteFiles()).forEach(rowDelta::addDeletes); - rowDelta - .validateDeletedFiles() - .validateDataFilesExist(Lists.newArrayList(result.referencedDataFiles())) - .commit(); - } - - private StructLikeSet expectedRowSet(Record... records) { - return SimpleDataUtil.expectedRowSet(table, records); - } - - private StructLikeSet actualRowSet(String... columns) throws IOException { - return SimpleDataUtil.actualRowSet(table, columns); - } - - private TaskWriterFactory createTaskWriterFactory(List equalityFieldIds) { - return new RowDataTaskWriterFactory( - SerializableTable.copyOf(table), - FlinkSchemaUtil.convert(table.schema()), - 128 * 1024 * 1024, - format, - table.properties(), - equalityFieldIds, - false); - } - - private TaskWriterFactory createTaskWriterFactory( - RowType flinkType, List equalityFieldIds) { - return new RowDataTaskWriterFactory( - SerializableTable.copyOf(table), - flinkType, - 128 * 1024 * 1024, - format, - table.properties(), - equalityFieldIds, - true); - } - - private void createAndInitTable(boolean partitioned) { - if (partitioned) { - this.table = create(SCHEMA, PartitionSpec.builderFor(SCHEMA).identity("data").build()); - } else { - this.table = create(SCHEMA, PartitionSpec.unpartitioned()); - } - - initTable(table); - } - - private void initTable(TestTables.TestTable testTable) { - testTable - .updateProperties() - .set(TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, String.valueOf(8 * 1024)) - .defaultFormat(format) - .commit(); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java deleted file mode 100644 index d25b2792ac65..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java +++ /dev/null @@ -1,70 +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.iceberg.flink.sink; - -import java.util.List; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.io.TestAppenderFactory; -import org.apache.iceberg.util.ArrayUtil; -import org.apache.iceberg.util.StructLikeSet; - -public class TestFlinkAppenderFactory extends TestAppenderFactory { - - private final RowType rowType; - - public TestFlinkAppenderFactory(String fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - this.rowType = FlinkSchemaUtil.convert(SCHEMA); - } - - @Override - protected FileAppenderFactory createAppenderFactory( - List equalityFieldIds, Schema eqDeleteSchema, Schema posDeleteRowSchema) { - return new FlinkAppenderFactory( - table, - table.schema(), - rowType, - table.properties(), - table.spec(), - ArrayUtil.toIntArray(equalityFieldIds), - eqDeleteSchema, - posDeleteRowSchema); - } - - @Override - protected RowData createRow(Integer id, String data) { - return SimpleDataUtil.createRowData(id, data); - } - - @Override - protected StructLikeSet expectedRowSet(Iterable rows) { - StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); - for (RowData row : rows) { - RowDataWrapper wrapper = new RowDataWrapper(rowType, table.schema().asStruct()); - set.add(wrapper.wrap(row)); - } - return set; - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java deleted file mode 100644 index da45241256f5..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java +++ /dev/null @@ -1,71 +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.iceberg.flink.sink; - -import java.util.List; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.io.FileWriterFactory; -import org.apache.iceberg.io.TestFileWriterFactory; -import org.apache.iceberg.util.ArrayUtil; -import org.apache.iceberg.util.StructLikeSet; - -public class TestFlinkFileWriterFactory extends TestFileWriterFactory { - - public TestFlinkFileWriterFactory(FileFormat fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - } - - @Override - protected FileWriterFactory newWriterFactory( - Schema dataSchema, - List equalityFieldIds, - Schema equalityDeleteRowSchema, - Schema positionDeleteRowSchema) { - return FlinkFileWriterFactory.builderFor(table) - .dataSchema(table.schema()) - .dataFileFormat(format()) - .deleteFileFormat(format()) - .equalityFieldIds(ArrayUtil.toIntArray(equalityFieldIds)) - .equalityDeleteRowSchema(equalityDeleteRowSchema) - .positionDeleteRowSchema(positionDeleteRowSchema) - .build(); - } - - @Override - protected RowData toRow(Integer id, String data) { - return SimpleDataUtil.createRowData(id, data); - } - - @Override - protected StructLikeSet toSet(Iterable rows) { - StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); - RowType flinkType = FlinkSchemaUtil.convert(table.schema()); - for (RowData row : rows) { - RowDataWrapper wrapper = new RowDataWrapper(flinkType, table.schema().asStruct()); - set.add(wrapper.wrap(row)); - } - return set; - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java deleted file mode 100644 index 4bef4d4244e5..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java +++ /dev/null @@ -1,394 +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.iceberg.flink.sink; - -import java.io.IOException; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.data.RowData; -import org.apache.flink.test.util.MiniClusterWithClientResource; -import org.apache.flink.types.Row; -import org.apache.iceberg.DistributionMode; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.flink.FlinkWriteOptions; -import org.apache.iceberg.flink.HadoopCatalogResource; -import org.apache.iceberg.flink.MiniClusterResource; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestFlinkIcebergSink extends TestFlinkIcebergSinkBase { - - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); - - private TableLoader tableLoader; - - private final FileFormat format; - private final int parallelism; - private final boolean partitioned; - - @Parameterized.Parameters(name = "format={0}, parallelism = {1}, partitioned = {2}") - public static Object[][] parameters() { - return new Object[][] { - {"avro", 1, true}, - {"avro", 1, false}, - {"avro", 2, true}, - {"avro", 2, false}, - {"orc", 1, true}, - {"orc", 1, false}, - {"orc", 2, true}, - {"orc", 2, false}, - {"parquet", 1, true}, - {"parquet", 1, false}, - {"parquet", 2, true}, - {"parquet", 2, false} - }; - } - - public TestFlinkIcebergSink(String format, int parallelism, boolean partitioned) { - this.format = FileFormat.fromString(format); - this.parallelism = parallelism; - this.partitioned = partitioned; - } - - @Before - public void before() throws IOException { - table = - catalogResource - .catalog() - .createTable( - TestFixtures.TABLE_IDENTIFIER, - SimpleDataUtil.SCHEMA, - partitioned - ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() - : PartitionSpec.unpartitioned(), - ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); - - env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) - .enableCheckpointing(100) - .setParallelism(parallelism) - .setMaxParallelism(parallelism); - - tableLoader = catalogResource.tableLoader(); - } - - @Test - public void testWriteRowData() throws Exception { - List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); - DataStream dataStream = - env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) - .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); - - FlinkSink.forRowData(dataStream) - .table(table) - .tableLoader(tableLoader) - .writeParallelism(parallelism) - .append(); - - // Execute the program. - env.execute("Test Iceberg DataStream"); - - // Assert the iceberg table's records. - SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); - } - - private void testWriteRow(TableSchema tableSchema, DistributionMode distributionMode) - throws Exception { - List rows = createRows(""); - DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); - - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) - .table(table) - .tableLoader(tableLoader) - .tableSchema(tableSchema) - .writeParallelism(parallelism) - .distributionMode(distributionMode) - .append(); - - // Execute the program. - env.execute("Test Iceberg DataStream."); - - SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); - } - - private int partitionFiles(String partition) throws IOException { - return SimpleDataUtil.partitionDataFiles(table, ImmutableMap.of("data", partition)).size(); - } - - @Test - public void testWriteRow() throws Exception { - testWriteRow(null, DistributionMode.NONE); - } - - @Test - public void testWriteRowWithTableSchema() throws Exception { - testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); - } - - @Test - public void testJobNoneDistributeMode() throws Exception { - table - .updateProperties() - .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) - .commit(); - - testWriteRow(null, DistributionMode.NONE); - - if (parallelism > 1) { - if (partitioned) { - int files = partitionFiles("aaa") + partitionFiles("bbb") + partitionFiles("ccc"); - Assert.assertTrue("Should have more than 3 files in iceberg table.", files > 3); - } - } - } - - @Test - public void testJobHashDistributionMode() { - table - .updateProperties() - .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) - .commit(); - Assertions.assertThatThrownBy(() -> testWriteRow(null, DistributionMode.RANGE)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Flink does not support 'range' write distribution mode now."); - } - - @Test - public void testJobNullDistributionMode() throws Exception { - table - .updateProperties() - .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) - .commit(); - - testWriteRow(null, null); - - if (partitioned) { - Assert.assertEquals( - "There should be only 1 data file in partition 'aaa'", 1, partitionFiles("aaa")); - Assert.assertEquals( - "There should be only 1 data file in partition 'bbb'", 1, partitionFiles("bbb")); - Assert.assertEquals( - "There should be only 1 data file in partition 'ccc'", 1, partitionFiles("ccc")); - } - } - - @Test - public void testPartitionWriteMode() throws Exception { - testWriteRow(null, DistributionMode.HASH); - if (partitioned) { - Assert.assertEquals( - "There should be only 1 data file in partition 'aaa'", 1, partitionFiles("aaa")); - Assert.assertEquals( - "There should be only 1 data file in partition 'bbb'", 1, partitionFiles("bbb")); - Assert.assertEquals( - "There should be only 1 data file in partition 'ccc'", 1, partitionFiles("ccc")); - } - } - - @Test - public void testShuffleByPartitionWithSchema() throws Exception { - testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.HASH); - if (partitioned) { - Assert.assertEquals( - "There should be only 1 data file in partition 'aaa'", 1, partitionFiles("aaa")); - Assert.assertEquals( - "There should be only 1 data file in partition 'bbb'", 1, partitionFiles("bbb")); - Assert.assertEquals( - "There should be only 1 data file in partition 'ccc'", 1, partitionFiles("ccc")); - } - } - - @Test - public void testTwoSinksInDisjointedDAG() throws Exception { - Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); - - Table leftTable = - catalogResource - .catalog() - .createTable( - TableIdentifier.of("left"), - SimpleDataUtil.SCHEMA, - partitioned - ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() - : PartitionSpec.unpartitioned(), - props); - TableLoader leftTableLoader = - TableLoader.fromCatalog(catalogResource.catalogLoader(), TableIdentifier.of("left")); - - Table rightTable = - catalogResource - .catalog() - .createTable( - TableIdentifier.of("right"), - SimpleDataUtil.SCHEMA, - partitioned - ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() - : PartitionSpec.unpartitioned(), - props); - TableLoader rightTableLoader = - TableLoader.fromCatalog(catalogResource.catalogLoader(), TableIdentifier.of("right")); - - env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) - .enableCheckpointing(100) - .setParallelism(parallelism) - .setMaxParallelism(parallelism); - env.getConfig().disableAutoGeneratedUIDs(); - - List leftRows = createRows("left-"); - DataStream leftStream = - env.fromCollection(leftRows, ROW_TYPE_INFO) - .name("leftCustomSource") - .uid("leftCustomSource"); - FlinkSink.forRow(leftStream, SimpleDataUtil.FLINK_SCHEMA) - .table(leftTable) - .tableLoader(leftTableLoader) - .tableSchema(SimpleDataUtil.FLINK_SCHEMA) - .distributionMode(DistributionMode.NONE) - .uidPrefix("leftIcebergSink") - .append(); - - List rightRows = createRows("right-"); - DataStream rightStream = - env.fromCollection(rightRows, ROW_TYPE_INFO) - .name("rightCustomSource") - .uid("rightCustomSource"); - FlinkSink.forRow(rightStream, SimpleDataUtil.FLINK_SCHEMA) - .table(rightTable) - .tableLoader(rightTableLoader) - .tableSchema(SimpleDataUtil.FLINK_SCHEMA) - .writeParallelism(parallelism) - .distributionMode(DistributionMode.HASH) - .uidPrefix("rightIcebergSink") - .setSnapshotProperty("flink.test", TestFlinkIcebergSink.class.getName()) - .setSnapshotProperties(Collections.singletonMap("direction", "rightTable")) - .append(); - - // Execute the program. - env.execute("Test Iceberg DataStream."); - - SimpleDataUtil.assertTableRows(leftTable, convertToRowData(leftRows)); - SimpleDataUtil.assertTableRows(rightTable, convertToRowData(rightRows)); - - leftTable.refresh(); - Assert.assertNull(leftTable.currentSnapshot().summary().get("flink.test")); - Assert.assertNull(leftTable.currentSnapshot().summary().get("direction")); - rightTable.refresh(); - Assert.assertEquals( - TestFlinkIcebergSink.class.getName(), - rightTable.currentSnapshot().summary().get("flink.test")); - Assert.assertEquals("rightTable", rightTable.currentSnapshot().summary().get("direction")); - } - - @Test - public void testOverrideWriteConfigWithUnknownDistributionMode() { - Map newProps = Maps.newHashMap(); - newProps.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), "UNRECOGNIZED"); - - List rows = createRows(""); - DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); - - FlinkSink.Builder builder = - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) - .table(table) - .tableLoader(tableLoader) - .writeParallelism(parallelism) - .setAll(newProps); - Assertions.assertThatThrownBy(builder::append) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid distribution mode: UNRECOGNIZED"); - } - - @Test - public void testOverrideWriteConfigWithUnknownFileFormat() { - Map newProps = Maps.newHashMap(); - newProps.put(FlinkWriteOptions.WRITE_FORMAT.key(), "UNRECOGNIZED"); - - List rows = createRows(""); - DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); - - FlinkSink.Builder builder = - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) - .table(table) - .tableLoader(tableLoader) - .writeParallelism(parallelism) - .setAll(newProps); - Assertions.assertThatThrownBy(builder::append) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid file format: UNRECOGNIZED"); - } - - @Test - public void testWriteRowWithTableRefreshInterval() throws Exception { - List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); - DataStream dataStream = - env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) - .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); - - Configuration flinkConf = new Configuration(); - flinkConf.setString(FlinkWriteOptions.TABLE_REFRESH_INTERVAL.key(), "100ms"); - - FlinkSink.forRowData(dataStream) - .table(table) - .tableLoader(tableLoader) - .flinkConf(flinkConf) - .writeParallelism(parallelism) - .append(); - - // Execute the program. - env.execute("Test Iceberg DataStream"); - - // Assert the iceberg table's records. - SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java deleted file mode 100644 index b38aa6b50ce6..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java +++ /dev/null @@ -1,64 +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.iceberg.flink.sink; - -import java.util.List; -import java.util.stream.Collectors; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.util.DataFormatConverters; -import org.apache.flink.types.Row; -import org.apache.iceberg.Table; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.source.BoundedTestSource; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -public class TestFlinkIcebergSinkBase { - - protected Table table; - protected StreamExecutionEnvironment env; - protected static final TypeInformation ROW_TYPE_INFO = - new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); - - protected static final DataFormatConverters.RowConverter CONVERTER = - new DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes()); - - protected BoundedTestSource createBoundedSource(List rows) { - return new BoundedTestSource<>(rows.toArray(new Row[0])); - } - - protected List createRows(String prefix) { - return Lists.newArrayList( - Row.of(1, prefix + "aaa"), - Row.of(1, prefix + "bbb"), - Row.of(1, prefix + "ccc"), - Row.of(2, prefix + "aaa"), - Row.of(2, prefix + "bbb"), - Row.of(2, prefix + "ccc"), - Row.of(3, prefix + "aaa"), - Row.of(3, prefix + "bbb"), - Row.of(3, prefix + "ccc")); - } - - protected List convertToRowData(List rows) { - return rows.stream().map(CONVERTER::toInternal).collect(Collectors.toList()); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java deleted file mode 100644 index 16b4542b00d3..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java +++ /dev/null @@ -1,137 +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.iceberg.flink.sink; - -import java.io.IOException; -import java.util.List; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.test.util.MiniClusterWithClientResource; -import org.apache.flink.types.Row; -import org.apache.iceberg.DistributionMode; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.SnapshotRef; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.flink.HadoopCatalogResource; -import org.apache.iceberg.flink.MiniClusterResource; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestFlinkIcebergSinkBranch extends TestFlinkIcebergSinkBase { - - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); - - private final String branch; - private TableLoader tableLoader; - - @Parameterized.Parameters(name = "formatVersion = {0}, branch = {1}") - public static Object[] parameters() { - return new Object[] {"main", "testBranch"}; - } - - public TestFlinkIcebergSinkBranch(String branch) { - this.branch = branch; - } - - @Before - public void before() throws IOException { - table = - catalogResource - .catalog() - .createTable( - TestFixtures.TABLE_IDENTIFIER, - SimpleDataUtil.SCHEMA, - PartitionSpec.unpartitioned(), - ImmutableMap.of( - TableProperties.DEFAULT_FILE_FORMAT, - FileFormat.AVRO.name(), - TableProperties.FORMAT_VERSION, - "1")); - - env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) - .enableCheckpointing(100); - - tableLoader = catalogResource.tableLoader(); - } - - @Test - public void testWriteRowWithTableSchema() throws Exception { - testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); - verifyOtherBranchUnmodified(); - } - - private void testWriteRow(TableSchema tableSchema, DistributionMode distributionMode) - throws Exception { - List rows = createRows(""); - DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); - - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) - .table(table) - .tableLoader(tableLoader) - .tableSchema(tableSchema) - .toBranch(branch) - .distributionMode(distributionMode) - .append(); - - // Execute the program. - env.execute("Test Iceberg DataStream."); - - SimpleDataUtil.assertTableRows(table, convertToRowData(rows), branch); - SimpleDataUtil.assertTableRows( - table, - ImmutableList.of(), - branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH); - - verifyOtherBranchUnmodified(); - } - - private void verifyOtherBranchUnmodified() { - String otherBranch = - branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH; - if (otherBranch.equals(SnapshotRef.MAIN_BRANCH)) { - Assert.assertNull(table.currentSnapshot()); - } - - Assert.assertTrue(table.snapshot(otherBranch) == null); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java deleted file mode 100644 index b5c3bcf41734..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java +++ /dev/null @@ -1,236 +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.iceberg.flink.sink; - -import java.util.List; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.test.util.MiniClusterWithClientResource; -import org.apache.flink.types.Row; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.SnapshotRef; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.HadoopCatalogResource; -import org.apache.iceberg.flink.MiniClusterResource; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.source.BoundedTestSource; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestFlinkIcebergSinkV2 extends TestFlinkIcebergSinkV2Base { - - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); - - @Parameterized.Parameters( - name = "FileFormat = {0}, Parallelism = {1}, Partitioned={2}, WriteDistributionMode ={3}") - public static Object[][] parameters() { - return new Object[][] { - new Object[] {"avro", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, - new Object[] {"avro", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, - new Object[] {"avro", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, - new Object[] {"avro", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, - new Object[] {"orc", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, - new Object[] {"orc", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, - new Object[] {"orc", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, - new Object[] {"orc", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, - new Object[] {"parquet", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, - new Object[] {"parquet", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, - new Object[] {"parquet", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, - new Object[] {"parquet", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE} - }; - } - - public TestFlinkIcebergSinkV2( - String format, int parallelism, boolean partitioned, String writeDistributionMode) { - this.format = FileFormat.fromString(format); - this.parallelism = parallelism; - this.partitioned = partitioned; - this.writeDistributionMode = writeDistributionMode; - } - - @Before - public void setupTable() { - table = - catalogResource - .catalog() - .createTable( - TestFixtures.TABLE_IDENTIFIER, - SimpleDataUtil.SCHEMA, - partitioned - ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() - : PartitionSpec.unpartitioned(), - ImmutableMap.of( - TableProperties.DEFAULT_FILE_FORMAT, - format.name(), - TableProperties.FORMAT_VERSION, - String.valueOf(FORMAT_V2))); - - table - .updateProperties() - .set(TableProperties.DEFAULT_FILE_FORMAT, format.name()) - .set(TableProperties.WRITE_DISTRIBUTION_MODE, writeDistributionMode) - .commit(); - - env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) - .enableCheckpointing(100L) - .setParallelism(parallelism) - .setMaxParallelism(parallelism); - - tableLoader = catalogResource.tableLoader(); - } - - @Test - public void testCheckAndGetEqualityFieldIds() { - table - .updateSchema() - .allowIncompatibleChanges() - .addRequiredColumn("type", Types.StringType.get()) - .setIdentifierFields("type") - .commit(); - - DataStream dataStream = - env.addSource(new BoundedTestSource<>(ImmutableList.of()), ROW_TYPE_INFO); - FlinkSink.Builder builder = - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA).table(table); - - // Use schema identifier field IDs as equality field id list by default - Assert.assertEquals( - table.schema().identifierFieldIds(), - Sets.newHashSet(builder.checkAndGetEqualityFieldIds())); - - // Use user-provided equality field column as equality field id list - builder.equalityFieldColumns(Lists.newArrayList("id")); - Assert.assertEquals( - Sets.newHashSet(table.schema().findField("id").fieldId()), - Sets.newHashSet(builder.checkAndGetEqualityFieldIds())); - - builder.equalityFieldColumns(Lists.newArrayList("type")); - Assert.assertEquals( - Sets.newHashSet(table.schema().findField("type").fieldId()), - Sets.newHashSet(builder.checkAndGetEqualityFieldIds())); - } - - @Test - public void testChangeLogOnIdKey() throws Exception { - testChangeLogOnIdKey(SnapshotRef.MAIN_BRANCH); - } - - @Test - public void testUpsertOnlyDeletesOnDataKey() throws Exception { - List> elementsPerCheckpoint = - ImmutableList.of( - ImmutableList.of(row("+I", 1, "aaa")), - ImmutableList.of(row("-D", 1, "aaa"), row("-D", 2, "bbb"))); - - List> expectedRecords = - ImmutableList.of(ImmutableList.of(record(1, "aaa")), ImmutableList.of()); - - testChangeLogs( - ImmutableList.of("data"), - row -> row.getField(ROW_DATA_POS), - true, - elementsPerCheckpoint, - expectedRecords, - SnapshotRef.MAIN_BRANCH); - } - - @Test - public void testChangeLogOnDataKey() throws Exception { - testChangeLogOnDataKey(SnapshotRef.MAIN_BRANCH); - } - - @Test - public void testChangeLogOnIdDataKey() throws Exception { - testChangeLogOnIdDataKey(SnapshotRef.MAIN_BRANCH); - } - - @Test - public void testChangeLogOnSameKey() throws Exception { - testChangeLogOnSameKey(SnapshotRef.MAIN_BRANCH); - } - - @Test - public void testUpsertModeCheck() throws Exception { - DataStream dataStream = - env.addSource(new BoundedTestSource<>(ImmutableList.of()), ROW_TYPE_INFO); - FlinkSink.Builder builder = - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) - .tableLoader(tableLoader) - .tableSchema(SimpleDataUtil.FLINK_SCHEMA) - .writeParallelism(parallelism) - .upsert(true); - - Assertions.assertThatThrownBy( - () -> - builder - .equalityFieldColumns(ImmutableList.of("id", "data")) - .overwrite(true) - .append()) - .isInstanceOf(IllegalStateException.class) - .hasMessage( - "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); - - Assertions.assertThatThrownBy( - () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) - .isInstanceOf(IllegalStateException.class) - .hasMessage( - "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); - } - - @Test - public void testUpsertOnIdKey() throws Exception { - testUpsertOnIdKey(SnapshotRef.MAIN_BRANCH); - } - - @Test - public void testUpsertOnDataKey() throws Exception { - testUpsertOnDataKey(SnapshotRef.MAIN_BRANCH); - } - - @Test - public void testUpsertOnIdDataKey() throws Exception { - testUpsertOnIdDataKey(SnapshotRef.MAIN_BRANCH); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java deleted file mode 100644 index 507df9e35215..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java +++ /dev/null @@ -1,361 +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.iceberg.flink.sink; - -import java.io.IOException; -import java.util.List; -import java.util.Map; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.types.Row; -import org.apache.flink.types.RowKind; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.data.IcebergGenerics; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.source.BoundedTestSource; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.util.StructLikeSet; -import org.assertj.core.api.Assertions; -import org.junit.Assert; - -public class TestFlinkIcebergSinkV2Base { - - protected static final int FORMAT_V2 = 2; - protected static final TypeInformation ROW_TYPE_INFO = - new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); - - protected static final int ROW_ID_POS = 0; - protected static final int ROW_DATA_POS = 1; - - protected int parallelism = 1; - protected TableLoader tableLoader; - protected Table table; - protected StreamExecutionEnvironment env; - protected FileFormat format; - protected boolean partitioned; - protected String writeDistributionMode; - - protected static final Map ROW_KIND_MAP = - ImmutableMap.of( - "+I", RowKind.INSERT, - "-D", RowKind.DELETE, - "-U", RowKind.UPDATE_BEFORE, - "+U", RowKind.UPDATE_AFTER); - - protected Row row(String rowKind, int id, String data) { - RowKind kind = ROW_KIND_MAP.get(rowKind); - if (kind == null) { - throw new IllegalArgumentException("Unknown row kind: " + rowKind); - } - - return Row.ofKind(kind, id, data); - } - - protected void testUpsertOnIdDataKey(String branch) throws Exception { - List> elementsPerCheckpoint = - ImmutableList.of( - ImmutableList.of(row("+I", 1, "aaa"), row("+U", 1, "aaa"), row("+I", 2, "bbb")), - ImmutableList.of(row("+I", 1, "aaa"), row("-D", 2, "bbb"), row("+I", 2, "ccc")), - ImmutableList.of(row("+U", 1, "bbb"), row("-U", 1, "ccc"), row("-D", 1, "aaa"))); - - List> expectedRecords = - ImmutableList.of( - ImmutableList.of(record(1, "aaa"), record(2, "bbb")), - ImmutableList.of(record(1, "aaa"), record(2, "ccc")), - ImmutableList.of(record(1, "bbb"), record(2, "ccc"))); - testChangeLogs( - ImmutableList.of("id", "data"), - row -> Row.of(row.getField(ROW_ID_POS), row.getField(ROW_DATA_POS)), - true, - elementsPerCheckpoint, - expectedRecords, - branch); - } - - protected void testChangeLogOnIdDataKey(String branch) throws Exception { - List> elementsPerCheckpoint = - ImmutableList.of( - ImmutableList.of( - row("+I", 1, "aaa"), - row("-D", 1, "aaa"), - row("+I", 2, "bbb"), - row("+I", 1, "bbb"), - row("+I", 2, "aaa")), - ImmutableList.of(row("-U", 2, "aaa"), row("+U", 1, "ccc"), row("+I", 1, "aaa")), - ImmutableList.of(row("-D", 1, "bbb"), row("+I", 2, "aaa"))); - - List> expectedRecords = - ImmutableList.of( - ImmutableList.of(record(1, "bbb"), record(2, "aaa"), record(2, "bbb")), - ImmutableList.of( - record(1, "aaa"), record(1, "bbb"), record(1, "ccc"), record(2, "bbb")), - ImmutableList.of( - record(1, "aaa"), record(1, "ccc"), record(2, "aaa"), record(2, "bbb"))); - - testChangeLogs( - ImmutableList.of("data", "id"), - row -> Row.of(row.getField(ROW_ID_POS), row.getField(ROW_DATA_POS)), - false, - elementsPerCheckpoint, - expectedRecords, - branch); - } - - protected void testChangeLogOnSameKey(String branch) throws Exception { - List> elementsPerCheckpoint = - ImmutableList.of( - // Checkpoint #1 - ImmutableList.of(row("+I", 1, "aaa"), row("-D", 1, "aaa"), row("+I", 1, "aaa")), - // Checkpoint #2 - ImmutableList.of(row("-U", 1, "aaa"), row("+U", 1, "aaa")), - // Checkpoint #3 - ImmutableList.of(row("-D", 1, "aaa"), row("+I", 1, "aaa")), - // Checkpoint #4 - ImmutableList.of(row("-U", 1, "aaa"), row("+U", 1, "aaa"), row("+I", 1, "aaa"))); - - List> expectedRecords = - ImmutableList.of( - ImmutableList.of(record(1, "aaa")), - ImmutableList.of(record(1, "aaa")), - ImmutableList.of(record(1, "aaa")), - ImmutableList.of(record(1, "aaa"), record(1, "aaa"))); - - testChangeLogs( - ImmutableList.of("id", "data"), - row -> Row.of(row.getField(ROW_ID_POS), row.getField(ROW_DATA_POS)), - false, - elementsPerCheckpoint, - expectedRecords, - branch); - } - - protected void testChangeLogOnDataKey(String branch) throws Exception { - List> elementsPerCheckpoint = - ImmutableList.of( - ImmutableList.of( - row("+I", 1, "aaa"), - row("-D", 1, "aaa"), - row("+I", 2, "bbb"), - row("+I", 1, "bbb"), - row("+I", 2, "aaa")), - ImmutableList.of(row("-U", 2, "aaa"), row("+U", 1, "ccc"), row("+I", 1, "aaa")), - ImmutableList.of(row("-D", 1, "bbb"), row("+I", 2, "aaa"), row("+I", 2, "ccc"))); - - List> expectedRecords = - ImmutableList.of( - ImmutableList.of(record(1, "bbb"), record(2, "aaa")), - ImmutableList.of(record(1, "aaa"), record(1, "bbb"), record(1, "ccc")), - ImmutableList.of( - record(1, "aaa"), record(1, "ccc"), record(2, "aaa"), record(2, "ccc"))); - - testChangeLogs( - ImmutableList.of("data"), - row -> row.getField(ROW_DATA_POS), - false, - elementsPerCheckpoint, - expectedRecords, - branch); - } - - protected void testUpsertOnDataKey(String branch) throws Exception { - List> elementsPerCheckpoint = - ImmutableList.of( - ImmutableList.of(row("+I", 1, "aaa"), row("+I", 2, "aaa"), row("+I", 3, "bbb")), - ImmutableList.of(row("+U", 4, "aaa"), row("-U", 3, "bbb"), row("+U", 5, "bbb")), - ImmutableList.of(row("+I", 6, "aaa"), row("+U", 7, "bbb"))); - - List> expectedRecords = - ImmutableList.of( - ImmutableList.of(record(2, "aaa"), record(3, "bbb")), - ImmutableList.of(record(4, "aaa"), record(5, "bbb")), - ImmutableList.of(record(6, "aaa"), record(7, "bbb"))); - - testChangeLogs( - ImmutableList.of("data"), - row -> row.getField(ROW_DATA_POS), - true, - elementsPerCheckpoint, - expectedRecords, - branch); - } - - protected void testChangeLogOnIdKey(String branch) throws Exception { - List> elementsPerCheckpoint = - ImmutableList.of( - ImmutableList.of( - row("+I", 1, "aaa"), - row("-D", 1, "aaa"), - row("+I", 1, "bbb"), - row("+I", 2, "aaa"), - row("-D", 2, "aaa"), - row("+I", 2, "bbb")), - ImmutableList.of( - row("-U", 2, "bbb"), row("+U", 2, "ccc"), row("-D", 2, "ccc"), row("+I", 2, "ddd")), - ImmutableList.of( - row("-D", 1, "bbb"), - row("+I", 1, "ccc"), - row("-D", 1, "ccc"), - row("+I", 1, "ddd"))); - - List> expectedRecords = - ImmutableList.of( - ImmutableList.of(record(1, "bbb"), record(2, "bbb")), - ImmutableList.of(record(1, "bbb"), record(2, "ddd")), - ImmutableList.of(record(1, "ddd"), record(2, "ddd"))); - - if (partitioned && writeDistributionMode.equals(TableProperties.WRITE_DISTRIBUTION_MODE_HASH)) { - Assertions.assertThatThrownBy( - () -> - testChangeLogs( - ImmutableList.of("id"), - row -> row.getField(ROW_ID_POS), - false, - elementsPerCheckpoint, - expectedRecords, - branch)) - .isInstanceOf(IllegalStateException.class) - .hasMessageStartingWith( - "In 'hash' distribution mode with equality fields set, partition field") - .hasMessageContaining("should be included in equality fields:"); - } else { - testChangeLogs( - ImmutableList.of("id"), - row -> row.getField(ROW_ID_POS), - false, - elementsPerCheckpoint, - expectedRecords, - branch); - } - } - - protected void testUpsertOnIdKey(String branch) throws Exception { - List> elementsPerCheckpoint = - ImmutableList.of( - ImmutableList.of(row("+I", 1, "aaa"), row("+U", 1, "bbb")), - ImmutableList.of(row("+I", 1, "ccc")), - ImmutableList.of(row("+U", 1, "ddd"), row("+I", 1, "eee"))); - - List> expectedRecords = - ImmutableList.of( - ImmutableList.of(record(1, "bbb")), - ImmutableList.of(record(1, "ccc")), - ImmutableList.of(record(1, "eee"))); - - if (!partitioned) { - testChangeLogs( - ImmutableList.of("id"), - row -> row.getField(ROW_ID_POS), - true, - elementsPerCheckpoint, - expectedRecords, - branch); - } else { - Assertions.assertThatThrownBy( - () -> - testChangeLogs( - ImmutableList.of("id"), - row -> row.getField(ROW_ID_POS), - true, - elementsPerCheckpoint, - expectedRecords, - branch)) - .isInstanceOf(IllegalStateException.class) - .hasMessageContaining("should be included in equality fields"); - } - } - - protected void testChangeLogs( - List equalityFieldColumns, - KeySelector keySelector, - boolean insertAsUpsert, - List> elementsPerCheckpoint, - List> expectedRecordsPerCheckpoint, - String branch) - throws Exception { - DataStream dataStream = - env.addSource(new BoundedTestSource<>(elementsPerCheckpoint), ROW_TYPE_INFO); - - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) - .tableLoader(tableLoader) - .tableSchema(SimpleDataUtil.FLINK_SCHEMA) - .writeParallelism(parallelism) - .equalityFieldColumns(equalityFieldColumns) - .upsert(insertAsUpsert) - .toBranch(branch) - .append(); - - // Execute the program. - env.execute("Test Iceberg Change-Log DataStream."); - - table.refresh(); - List snapshots = findValidSnapshots(); - int expectedSnapshotNum = expectedRecordsPerCheckpoint.size(); - Assert.assertEquals( - "Should have the expected snapshot number", expectedSnapshotNum, snapshots.size()); - - for (int i = 0; i < expectedSnapshotNum; i++) { - long snapshotId = snapshots.get(i).snapshotId(); - List expectedRecords = expectedRecordsPerCheckpoint.get(i); - Assert.assertEquals( - "Should have the expected records for the checkpoint#" + i, - expectedRowSet(expectedRecords.toArray(new Record[0])), - actualRowSet(snapshotId, "*")); - } - } - - protected Record record(int id, String data) { - return SimpleDataUtil.createRecord(id, data); - } - - private List findValidSnapshots() { - List validSnapshots = Lists.newArrayList(); - for (Snapshot snapshot : table.snapshots()) { - if (snapshot.allManifests(table.io()).stream() - .anyMatch(m -> snapshot.snapshotId() == m.snapshotId())) { - validSnapshots.add(snapshot); - } - } - return validSnapshots; - } - - private StructLikeSet expectedRowSet(Record... records) { - return SimpleDataUtil.expectedRowSet(table, records); - } - - private StructLikeSet actualRowSet(long snapshotId, String... columns) throws IOException { - table.refresh(); - StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); - try (CloseableIterable reader = - IcebergGenerics.read(table).useSnapshot(snapshotId).select(columns).build()) { - reader.forEach(set::add); - } - return set; - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java deleted file mode 100644 index fed333848279..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java +++ /dev/null @@ -1,134 +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.iceberg.flink.sink; - -import java.io.IOException; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.test.util.MiniClusterWithClientResource; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.SnapshotRef; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.flink.HadoopCatalogResource; -import org.apache.iceberg.flink.MiniClusterResource; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestFlinkIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Base { - - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); - - private final String branch; - - @Parameterized.Parameters(name = "branch = {0}") - public static Object[] parameters() { - return new Object[] {"main", "testBranch"}; - } - - public TestFlinkIcebergSinkV2Branch(String branch) { - this.branch = branch; - } - - @Before - public void before() throws IOException { - table = - catalogResource - .catalog() - .createTable( - TestFixtures.TABLE_IDENTIFIER, - SimpleDataUtil.SCHEMA, - PartitionSpec.unpartitioned(), - ImmutableMap.of( - TableProperties.DEFAULT_FILE_FORMAT, - FileFormat.AVRO.name(), - TableProperties.FORMAT_VERSION, - "2")); - - env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) - .enableCheckpointing(100); - - tableLoader = catalogResource.tableLoader(); - } - - @Test - public void testChangeLogOnIdKey() throws Exception { - testChangeLogOnIdKey(branch); - verifyOtherBranchUnmodified(); - } - - @Test - public void testChangeLogOnDataKey() throws Exception { - testChangeLogOnDataKey(branch); - verifyOtherBranchUnmodified(); - } - - @Test - public void testChangeLogOnIdDataKey() throws Exception { - testChangeLogOnIdDataKey(branch); - verifyOtherBranchUnmodified(); - } - - @Test - public void testUpsertOnIdKey() throws Exception { - testUpsertOnIdKey(branch); - verifyOtherBranchUnmodified(); - } - - @Test - public void testUpsertOnDataKey() throws Exception { - testUpsertOnDataKey(branch); - verifyOtherBranchUnmodified(); - } - - @Test - public void testUpsertOnIdDataKey() throws Exception { - testUpsertOnIdDataKey(branch); - verifyOtherBranchUnmodified(); - } - - private void verifyOtherBranchUnmodified() { - String otherBranch = - branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH; - if (otherBranch.equals(SnapshotRef.MAIN_BRANCH)) { - Assert.assertNull(table.currentSnapshot()); - } - - Assert.assertTrue(table.snapshot(otherBranch) == null); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java deleted file mode 100644 index ce1f208a4b07..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java +++ /dev/null @@ -1,315 +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.iceberg.flink.sink; - -import static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Paths; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import java.util.concurrent.atomic.AtomicInteger; -import org.apache.flink.core.io.SimpleVersionedSerialization; -import org.apache.flink.core.io.SimpleVersionedSerializer; -import org.apache.flink.table.data.RowData; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.ManifestFiles; -import org.apache.iceberg.Table; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.io.WriteResult; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.util.Pair; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class TestFlinkManifest { - private static final Configuration CONF = new Configuration(); - - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); - - private Table table; - private FileAppenderFactory appenderFactory; - private final AtomicInteger fileCount = new AtomicInteger(0); - - @Before - public void before() throws IOException { - File folder = tempFolder.newFolder(); - String warehouse = folder.getAbsolutePath(); - - String tablePath = warehouse.concat("/test"); - Assert.assertTrue("Should create the table directory correctly.", new File(tablePath).mkdir()); - - // Construct the iceberg table. - table = SimpleDataUtil.createTable(tablePath, ImmutableMap.of(), false); - - int[] equalityFieldIds = - new int[] { - table.schema().findField("id").fieldId(), table.schema().findField("data").fieldId() - }; - this.appenderFactory = - new FlinkAppenderFactory( - table, - table.schema(), - FlinkSchemaUtil.convert(table.schema()), - table.properties(), - table.spec(), - equalityFieldIds, - table.schema(), - null); - } - - @Test - public void testIO() throws IOException { - String flinkJobId = newFlinkJobId(); - String operatorId = newOperatorUniqueId(); - for (long checkpointId = 1; checkpointId <= 3; checkpointId++) { - ManifestOutputFileFactory factory = - FlinkManifestUtil.createOutputFileFactory( - () -> table, table.properties(), flinkJobId, operatorId, 1, 1); - final long curCkpId = checkpointId; - - List dataFiles = generateDataFiles(10); - List eqDeleteFiles = generateEqDeleteFiles(5); - List posDeleteFiles = generatePosDeleteFiles(5); - DeltaManifests deltaManifests = - FlinkManifestUtil.writeCompletedFiles( - WriteResult.builder() - .addDataFiles(dataFiles) - .addDeleteFiles(eqDeleteFiles) - .addDeleteFiles(posDeleteFiles) - .build(), - () -> factory.create(curCkpId), - table.spec()); - - WriteResult result = - FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs()); - Assert.assertEquals("Size of data file list are not equal.", 10, result.deleteFiles().length); - for (int i = 0; i < dataFiles.size(); i++) { - TestHelpers.assertEquals(dataFiles.get(i), result.dataFiles()[i]); - } - Assert.assertEquals("Size of delete file list are not equal.", 10, result.dataFiles().length); - for (int i = 0; i < 5; i++) { - TestHelpers.assertEquals(eqDeleteFiles.get(i), result.deleteFiles()[i]); - } - for (int i = 0; i < 5; i++) { - TestHelpers.assertEquals(posDeleteFiles.get(i), result.deleteFiles()[5 + i]); - } - } - } - - @Test - public void testUserProvidedManifestLocation() throws IOException { - long checkpointId = 1; - String flinkJobId = newFlinkJobId(); - String operatorId = newOperatorUniqueId(); - File userProvidedFolder = tempFolder.newFolder(); - Map props = - ImmutableMap.of(FLINK_MANIFEST_LOCATION, userProvidedFolder.getAbsolutePath() + "///"); - ManifestOutputFileFactory factory = - new ManifestOutputFileFactory(() -> table, props, flinkJobId, operatorId, 1, 1); - - List dataFiles = generateDataFiles(5); - DeltaManifests deltaManifests = - FlinkManifestUtil.writeCompletedFiles( - WriteResult.builder().addDataFiles(dataFiles).build(), - () -> factory.create(checkpointId), - table.spec()); - - Assert.assertNotNull("Data manifest shouldn't be null", deltaManifests.dataManifest()); - Assert.assertNull("Delete manifest should be null", deltaManifests.deleteManifest()); - Assert.assertEquals( - "The newly created manifest file should be located under the user provided directory", - userProvidedFolder.toPath(), - Paths.get(deltaManifests.dataManifest().path()).getParent()); - - WriteResult result = - FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs()); - - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(5, result.dataFiles().length); - - Assert.assertEquals( - "Size of data file list are not equal.", dataFiles.size(), result.dataFiles().length); - for (int i = 0; i < dataFiles.size(); i++) { - TestHelpers.assertEquals(dataFiles.get(i), result.dataFiles()[i]); - } - } - - @Test - public void testVersionedSerializer() throws IOException { - long checkpointId = 1; - String flinkJobId = newFlinkJobId(); - String operatorId = newOperatorUniqueId(); - ManifestOutputFileFactory factory = - FlinkManifestUtil.createOutputFileFactory( - () -> table, table.properties(), flinkJobId, operatorId, 1, 1); - - List dataFiles = generateDataFiles(10); - List eqDeleteFiles = generateEqDeleteFiles(10); - List posDeleteFiles = generatePosDeleteFiles(10); - DeltaManifests expected = - FlinkManifestUtil.writeCompletedFiles( - WriteResult.builder() - .addDataFiles(dataFiles) - .addDeleteFiles(eqDeleteFiles) - .addDeleteFiles(posDeleteFiles) - .build(), - () -> factory.create(checkpointId), - table.spec()); - - byte[] versionedSerializeData = - SimpleVersionedSerialization.writeVersionAndSerialize( - DeltaManifestsSerializer.INSTANCE, expected); - DeltaManifests actual = - SimpleVersionedSerialization.readVersionAndDeSerialize( - DeltaManifestsSerializer.INSTANCE, versionedSerializeData); - TestHelpers.assertEquals(expected.dataManifest(), actual.dataManifest()); - TestHelpers.assertEquals(expected.deleteManifest(), actual.deleteManifest()); - - byte[] versionedSerializeData2 = - SimpleVersionedSerialization.writeVersionAndSerialize( - DeltaManifestsSerializer.INSTANCE, actual); - Assert.assertArrayEquals(versionedSerializeData, versionedSerializeData2); - } - - @Test - public void testCompatibility() throws IOException { - // The v2 deserializer should be able to deserialize the v1 binary. - long checkpointId = 1; - String flinkJobId = newFlinkJobId(); - String operatorId = newOperatorUniqueId(); - ManifestOutputFileFactory factory = - FlinkManifestUtil.createOutputFileFactory( - () -> table, table.properties(), flinkJobId, operatorId, 1, 1); - - List dataFiles = generateDataFiles(10); - ManifestFile manifest = - FlinkManifestUtil.writeDataFiles(factory.create(checkpointId), table.spec(), dataFiles); - byte[] dataV1 = - SimpleVersionedSerialization.writeVersionAndSerialize(new V1Serializer(), manifest); - - DeltaManifests delta = - SimpleVersionedSerialization.readVersionAndDeSerialize( - DeltaManifestsSerializer.INSTANCE, dataV1); - Assert.assertNull("Serialization v1 don't include delete files.", delta.deleteManifest()); - Assert.assertNotNull( - "Serialization v1 should not have null data manifest.", delta.dataManifest()); - TestHelpers.assertEquals(manifest, delta.dataManifest()); - - List actualFiles = - FlinkManifestUtil.readDataFiles(delta.dataManifest(), table.io(), table.specs()); - Assert.assertEquals(10, actualFiles.size()); - for (int i = 0; i < 10; i++) { - TestHelpers.assertEquals(dataFiles.get(i), actualFiles.get(i)); - } - } - - private static class V1Serializer implements SimpleVersionedSerializer { - - @Override - public int getVersion() { - return 1; - } - - @Override - public byte[] serialize(ManifestFile m) throws IOException { - return ManifestFiles.encode(m); - } - - @Override - public ManifestFile deserialize(int version, byte[] serialized) throws IOException { - return ManifestFiles.decode(serialized); - } - } - - private DataFile writeDataFile(String filename, List rows) throws IOException { - return SimpleDataUtil.writeFile( - table, - table.schema(), - table.spec(), - CONF, - table.location(), - FileFormat.PARQUET.addExtension(filename), - rows); - } - - private DeleteFile writeEqDeleteFile(String filename, List deletes) throws IOException { - return SimpleDataUtil.writeEqDeleteFile( - table, FileFormat.PARQUET, filename, appenderFactory, deletes); - } - - private DeleteFile writePosDeleteFile(String filename, List> positions) - throws IOException { - return SimpleDataUtil.writePosDeleteFile( - table, FileFormat.PARQUET, filename, appenderFactory, positions); - } - - private List generateDataFiles(int fileNum) throws IOException { - List rowDataList = Lists.newArrayList(); - List dataFiles = Lists.newArrayList(); - for (int i = 0; i < fileNum; i++) { - rowDataList.add(SimpleDataUtil.createRowData(i, "a" + i)); - dataFiles.add(writeDataFile("data-file-" + fileCount.incrementAndGet(), rowDataList)); - } - return dataFiles; - } - - private List generateEqDeleteFiles(int fileNum) throws IOException { - List rowDataList = Lists.newArrayList(); - List deleteFiles = Lists.newArrayList(); - for (int i = 0; i < fileNum; i++) { - rowDataList.add(SimpleDataUtil.createDelete(i, "a" + i)); - deleteFiles.add( - writeEqDeleteFile("eq-delete-file-" + fileCount.incrementAndGet(), rowDataList)); - } - return deleteFiles; - } - - private List generatePosDeleteFiles(int fileNum) throws IOException { - List> positions = Lists.newArrayList(); - List deleteFiles = Lists.newArrayList(); - for (int i = 0; i < fileNum; i++) { - positions.add(Pair.of("data-file-1", (long) i)); - deleteFiles.add( - writePosDeleteFile("pos-delete-file-" + fileCount.incrementAndGet(), positions)); - } - return deleteFiles; - } - - private static String newFlinkJobId() { - return UUID.randomUUID().toString(); - } - - private static String newOperatorUniqueId() { - return UUID.randomUUID().toString(); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java deleted file mode 100644 index 3951c2e70f65..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java +++ /dev/null @@ -1,71 +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.iceberg.flink.sink; - -import java.util.List; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.io.FileWriterFactory; -import org.apache.iceberg.io.TestPartitioningWriters; -import org.apache.iceberg.util.ArrayUtil; -import org.apache.iceberg.util.StructLikeSet; - -public class TestFlinkPartitioningWriters extends TestPartitioningWriters { - - public TestFlinkPartitioningWriters(FileFormat fileFormat) { - super(fileFormat); - } - - @Override - protected FileWriterFactory newWriterFactory( - Schema dataSchema, - List equalityFieldIds, - Schema equalityDeleteRowSchema, - Schema positionDeleteRowSchema) { - return FlinkFileWriterFactory.builderFor(table) - .dataSchema(table.schema()) - .dataFileFormat(format()) - .deleteFileFormat(format()) - .equalityFieldIds(ArrayUtil.toIntArray(equalityFieldIds)) - .equalityDeleteRowSchema(equalityDeleteRowSchema) - .positionDeleteRowSchema(positionDeleteRowSchema) - .build(); - } - - @Override - protected RowData toRow(Integer id, String data) { - return SimpleDataUtil.createRowData(id, data); - } - - @Override - protected StructLikeSet toSet(Iterable rows) { - StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); - RowType flinkType = FlinkSchemaUtil.convert(table.schema()); - for (RowData row : rows) { - RowDataWrapper wrapper = new RowDataWrapper(flinkType, table.schema().asStruct()); - set.add(wrapper.wrap(row)); - } - return set; - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java deleted file mode 100644 index 9e846efe6fc9..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java +++ /dev/null @@ -1,71 +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.iceberg.flink.sink; - -import java.util.List; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.io.FileWriterFactory; -import org.apache.iceberg.io.TestPositionDeltaWriters; -import org.apache.iceberg.util.ArrayUtil; -import org.apache.iceberg.util.StructLikeSet; - -public class TestFlinkPositionDeltaWriters extends TestPositionDeltaWriters { - - public TestFlinkPositionDeltaWriters(FileFormat fileFormat) { - super(fileFormat); - } - - @Override - protected FileWriterFactory newWriterFactory( - Schema dataSchema, - List equalityFieldIds, - Schema equalityDeleteRowSchema, - Schema positionDeleteRowSchema) { - return FlinkFileWriterFactory.builderFor(table) - .dataSchema(table.schema()) - .dataFileFormat(format()) - .deleteFileFormat(format()) - .equalityFieldIds(ArrayUtil.toIntArray(equalityFieldIds)) - .equalityDeleteRowSchema(equalityDeleteRowSchema) - .positionDeleteRowSchema(positionDeleteRowSchema) - .build(); - } - - @Override - protected RowData toRow(Integer id, String data) { - return SimpleDataUtil.createRowData(id, data); - } - - @Override - protected StructLikeSet toSet(Iterable rows) { - StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); - RowType flinkType = FlinkSchemaUtil.convert(table.schema()); - for (RowData row : rows) { - RowDataWrapper wrapper = new RowDataWrapper(flinkType, table.schema().asStruct()); - set.add(wrapper.wrap(row)); - } - return set; - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java deleted file mode 100644 index 07716b9c3e60..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java +++ /dev/null @@ -1,56 +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.iceberg.flink.sink; - -import java.util.List; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.io.FileWriterFactory; -import org.apache.iceberg.io.TestRollingFileWriters; -import org.apache.iceberg.util.ArrayUtil; - -public class TestFlinkRollingFileWriters extends TestRollingFileWriters { - - public TestFlinkRollingFileWriters(FileFormat fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - } - - @Override - protected FileWriterFactory newWriterFactory( - Schema dataSchema, - List equalityFieldIds, - Schema equalityDeleteRowSchema, - Schema positionDeleteRowSchema) { - return FlinkFileWriterFactory.builderFor(table) - .dataSchema(table.schema()) - .dataFileFormat(format()) - .deleteFileFormat(format()) - .equalityFieldIds(ArrayUtil.toIntArray(equalityFieldIds)) - .equalityDeleteRowSchema(equalityDeleteRowSchema) - .positionDeleteRowSchema(positionDeleteRowSchema) - .build(); - } - - @Override - protected RowData toRow(Integer id, String data) { - return SimpleDataUtil.createRowData(id, data); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java deleted file mode 100644 index e6d64ef2c720..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java +++ /dev/null @@ -1,60 +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.iceberg.flink.sink; - -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Table; -import org.apache.iceberg.io.FileWriterFactory; -import org.apache.iceberg.io.TestWriterMetrics; - -public class TestFlinkWriterMetrics extends TestWriterMetrics { - - public TestFlinkWriterMetrics(FileFormat fileFormat) { - super(fileFormat); - } - - @Override - protected FileWriterFactory newWriterFactory(Table sourceTable) { - return FlinkFileWriterFactory.builderFor(sourceTable) - .dataSchema(sourceTable.schema()) - .dataFileFormat(fileFormat) - .deleteFileFormat(fileFormat) - .positionDeleteRowSchema(sourceTable.schema()) - .build(); - } - - @Override - protected RowData toRow(Integer id, String data, boolean boolValue, Long longValue) { - GenericRowData nested = GenericRowData.of(boolValue, longValue); - GenericRowData row = GenericRowData.of(id, StringData.fromString(data), nested); - return row; - } - - @Override - public RowData toGenericRow(int value, int repeated) { - GenericRowData row = new GenericRowData(repeated); - for (int i = 0; i < repeated; i++) { - row.setField(i, value); - } - return row; - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java deleted file mode 100644 index a2fe092b7cf6..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java +++ /dev/null @@ -1,1152 +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.iceberg.flink.sink; - -import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; -import static org.apache.iceberg.flink.sink.IcebergFilesCommitter.MAX_CONTINUOUS_EMPTY_COMMITS; -import static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; -import static org.assertj.core.api.Assertions.assertThat; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.Collections; -import java.util.List; -import java.util.NavigableMap; -import java.util.SortedMap; -import java.util.stream.Collectors; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.state.ListState; -import org.apache.flink.api.common.state.OperatorStateStore; -import org.apache.flink.core.io.SimpleVersionedSerialization; -import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; -import org.apache.flink.runtime.jobgraph.OperatorID; -import org.apache.flink.runtime.operators.testutils.MockEnvironment; -import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; -import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; -import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; -import org.apache.flink.streaming.api.operators.BoundedOneInput; -import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; -import org.apache.flink.streaming.api.operators.StreamOperator; -import org.apache.flink.streaming.api.operators.StreamOperatorParameters; -import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; -import org.apache.flink.table.data.RowData; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.GenericManifestFile; -import org.apache.iceberg.ManifestContent; -import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.PartitionData; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.TableTestBase; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.flink.TestTableLoader; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.io.WriteResult; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.util.Pair; -import org.apache.iceberg.util.ThreadPools; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestIcebergFilesCommitter extends TableTestBase { - private static final Configuration CONF = new Configuration(); - - private File flinkManifestFolder; - - private final FileFormat format; - private final String branch; - - @Parameterized.Parameters(name = "FileFormat = {0}, FormatVersion = {1}, branch = {2}") - public static Object[][] parameters() { - return new Object[][] { - new Object[] {"avro", 1, "main"}, - new Object[] {"avro", 2, "test-branch"}, - new Object[] {"parquet", 1, "main"}, - new Object[] {"parquet", 2, "test-branch"}, - new Object[] {"orc", 1, "main"}, - new Object[] {"orc", 2, "test-branch"} - }; - } - - public TestIcebergFilesCommitter(String format, int formatVersion, String branch) { - super(formatVersion); - this.format = FileFormat.fromString(format); - this.branch = branch; - } - - @Override - @Before - public void setupTable() throws IOException { - flinkManifestFolder = temp.newFolder(); - - this.tableDir = temp.newFolder(); - this.metadataDir = new File(tableDir, "metadata"); - Assert.assertTrue(tableDir.delete()); - - // Construct the iceberg table. - table = create(SimpleDataUtil.SCHEMA, PartitionSpec.unpartitioned()); - - table - .updateProperties() - .set(DEFAULT_FILE_FORMAT, format.name()) - .set(FLINK_MANIFEST_LOCATION, flinkManifestFolder.getAbsolutePath()) - .set(MAX_CONTINUOUS_EMPTY_COMMITS, "1") - .commit(); - } - - @Test - public void testCommitTxnWithoutDataFiles() throws Exception { - long checkpointId = 0; - long timestamp = 0; - JobID jobId = new JobID(); - OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.setup(); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - SimpleDataUtil.assertTableRows(table, Lists.newArrayList(), branch); - assertSnapshotSize(0); - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - // It's better to advance the max-committed-checkpoint-id in iceberg snapshot, so that the - // future flink job - // failover won't fail. - for (int i = 1; i <= 3; i++) { - harness.snapshot(++checkpointId, ++timestamp); - assertFlinkManifests(0); - - harness.notifyOfCompletedCheckpoint(checkpointId); - assertFlinkManifests(0); - - assertSnapshotSize(i); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); - } - } - } - - @Test - public void testMaxContinuousEmptyCommits() throws Exception { - table.updateProperties().set(MAX_CONTINUOUS_EMPTY_COMMITS, "3").commit(); - - JobID jobId = new JobID(); - long checkpointId = 0; - long timestamp = 0; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.setup(); - harness.open(); - - assertSnapshotSize(0); - - for (int i = 1; i <= 9; i++) { - harness.snapshot(++checkpointId, ++timestamp); - harness.notifyOfCompletedCheckpoint(checkpointId); - - assertSnapshotSize(i / 3); - } - } - } - - private WriteResult of(DataFile dataFile) { - return WriteResult.builder().addDataFiles(dataFile).build(); - } - - @Test - public void testCommitTxn() throws Exception { - // Test with 3 continues checkpoints: - // 1. snapshotState for checkpoint#1 - // 2. notifyCheckpointComplete for checkpoint#1 - // 3. snapshotState for checkpoint#2 - // 4. notifyCheckpointComplete for checkpoint#2 - // 5. snapshotState for checkpoint#3 - // 6. notifyCheckpointComplete for checkpoint#3 - long timestamp = 0; - - JobID jobID = new JobID(); - OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobID)) { - harness.setup(); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - assertSnapshotSize(0); - - List rows = Lists.newArrayListWithExpectedSize(3); - for (int i = 1; i <= 3; i++) { - RowData rowData = SimpleDataUtil.createRowData(i, "hello" + i); - DataFile dataFile = writeDataFile("data-" + i, ImmutableList.of(rowData)); - harness.processElement(of(dataFile), ++timestamp); - rows.add(rowData); - - harness.snapshot(i, ++timestamp); - assertFlinkManifests(1); - - harness.notifyOfCompletedCheckpoint(i); - assertFlinkManifests(0); - - SimpleDataUtil.assertTableRows(table, ImmutableList.copyOf(rows), branch); - assertSnapshotSize(i); - assertMaxCommittedCheckpointId(jobID, operatorId, i); - Assert.assertEquals( - TestIcebergFilesCommitter.class.getName(), - SimpleDataUtil.latestSnapshot(table, branch).summary().get("flink.test")); - } - } - } - - @Test - public void testOrderedEventsBetweenCheckpoints() throws Exception { - // It's possible that two checkpoints happen in the following orders: - // 1. snapshotState for checkpoint#1; - // 2. snapshotState for checkpoint#2; - // 3. notifyCheckpointComplete for checkpoint#1; - // 4. notifyCheckpointComplete for checkpoint#2; - long timestamp = 0; - - JobID jobId = new JobID(); - OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.setup(); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - RowData row1 = SimpleDataUtil.createRowData(1, "hello"); - DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - - harness.processElement(of(dataFile1), ++timestamp); - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - // 1. snapshotState for checkpoint#1 - long firstCheckpointId = 1; - harness.snapshot(firstCheckpointId, ++timestamp); - assertFlinkManifests(1); - - RowData row2 = SimpleDataUtil.createRowData(2, "world"); - DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); - harness.processElement(of(dataFile2), ++timestamp); - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - // 2. snapshotState for checkpoint#2 - long secondCheckpointId = 2; - harness.snapshot(secondCheckpointId, ++timestamp); - assertFlinkManifests(2); - - // 3. notifyCheckpointComplete for checkpoint#1 - harness.notifyOfCompletedCheckpoint(firstCheckpointId); - SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); - assertMaxCommittedCheckpointId(jobId, operatorId, firstCheckpointId); - assertFlinkManifests(1); - - // 4. notifyCheckpointComplete for checkpoint#2 - harness.notifyOfCompletedCheckpoint(secondCheckpointId); - SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); - assertMaxCommittedCheckpointId(jobId, operatorId, secondCheckpointId); - assertFlinkManifests(0); - } - } - - @Test - public void testDisorderedEventsBetweenCheckpoints() throws Exception { - // It's possible that the two checkpoints happen in the following orders: - // 1. snapshotState for checkpoint#1; - // 2. snapshotState for checkpoint#2; - // 3. notifyCheckpointComplete for checkpoint#2; - // 4. notifyCheckpointComplete for checkpoint#1; - long timestamp = 0; - - JobID jobId = new JobID(); - OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.setup(); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - RowData row1 = SimpleDataUtil.createRowData(1, "hello"); - DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - - harness.processElement(of(dataFile1), ++timestamp); - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - // 1. snapshotState for checkpoint#1 - long firstCheckpointId = 1; - harness.snapshot(firstCheckpointId, ++timestamp); - assertFlinkManifests(1); - - RowData row2 = SimpleDataUtil.createRowData(2, "world"); - DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); - harness.processElement(of(dataFile2), ++timestamp); - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - // 2. snapshotState for checkpoint#2 - long secondCheckpointId = 2; - harness.snapshot(secondCheckpointId, ++timestamp); - assertFlinkManifests(2); - - // 3. notifyCheckpointComplete for checkpoint#2 - harness.notifyOfCompletedCheckpoint(secondCheckpointId); - SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); - assertMaxCommittedCheckpointId(jobId, operatorId, secondCheckpointId); - assertFlinkManifests(0); - - // 4. notifyCheckpointComplete for checkpoint#1 - harness.notifyOfCompletedCheckpoint(firstCheckpointId); - SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); - assertMaxCommittedCheckpointId(jobId, operatorId, secondCheckpointId); - assertFlinkManifests(0); - } - } - - @Test - public void testRecoveryFromValidSnapshot() throws Exception { - long checkpointId = 0; - long timestamp = 0; - List expectedRows = Lists.newArrayList(); - OperatorSubtaskState snapshot; - - JobID jobId = new JobID(); - OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.setup(); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - assertSnapshotSize(0); - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - RowData row = SimpleDataUtil.createRowData(1, "hello"); - expectedRows.add(row); - DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row)); - - harness.processElement(of(dataFile1), ++timestamp); - snapshot = harness.snapshot(++checkpointId, ++timestamp); - assertFlinkManifests(1); - - harness.notifyOfCompletedCheckpoint(checkpointId); - assertFlinkManifests(0); - - SimpleDataUtil.assertTableRows(table, ImmutableList.of(row), branch); - assertSnapshotSize(1); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); - } - - // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.getStreamConfig().setOperatorID(operatorId); - harness.setup(); - harness.initializeState(snapshot); - harness.open(); - - SimpleDataUtil.assertTableRows(table, expectedRows, branch); - assertSnapshotSize(1); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); - - RowData row = SimpleDataUtil.createRowData(2, "world"); - expectedRows.add(row); - DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); - - harness.snapshot(++checkpointId, ++timestamp); - assertFlinkManifests(1); - - harness.notifyOfCompletedCheckpoint(checkpointId); - assertFlinkManifests(0); - - SimpleDataUtil.assertTableRows(table, expectedRows, branch); - assertSnapshotSize(2); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); - } - } - - @Test - public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Exception { - // We've two steps in checkpoint: 1. snapshotState(ckp); 2. notifyCheckpointComplete(ckp). It's - // possible that we - // flink job will restore from a checkpoint with only step#1 finished. - long checkpointId = 0; - long timestamp = 0; - OperatorSubtaskState snapshot; - List expectedRows = Lists.newArrayList(); - JobID jobId = new JobID(); - OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.setup(); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - assertSnapshotSize(0); - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - RowData row = SimpleDataUtil.createRowData(1, "hello"); - expectedRows.add(row); - DataFile dataFile = writeDataFile("data-1", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); - - snapshot = harness.snapshot(++checkpointId, ++timestamp); - SimpleDataUtil.assertTableRows(table, ImmutableList.of(), branch); - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - assertFlinkManifests(1); - } - - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.getStreamConfig().setOperatorID(operatorId); - harness.setup(); - harness.initializeState(snapshot); - harness.open(); - - // All flink manifests should be cleaned because it has committed the unfinished iceberg - // transaction. - assertFlinkManifests(0); - - SimpleDataUtil.assertTableRows(table, expectedRows, branch); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); - - harness.snapshot(++checkpointId, ++timestamp); - // Did not write any new record, so it won't generate new manifest. - assertFlinkManifests(0); - - harness.notifyOfCompletedCheckpoint(checkpointId); - assertFlinkManifests(0); - - SimpleDataUtil.assertTableRows(table, expectedRows, branch); - assertSnapshotSize(2); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); - - RowData row = SimpleDataUtil.createRowData(2, "world"); - expectedRows.add(row); - DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); - - snapshot = harness.snapshot(++checkpointId, ++timestamp); - assertFlinkManifests(1); - } - - // Redeploying flink job from external checkpoint. - JobID newJobId = new JobID(); - try (OneInputStreamOperatorTestHarness harness = - createStreamSink(newJobId)) { - harness.setup(); - harness.initializeState(snapshot); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - // All flink manifests should be cleaned because it has committed the unfinished iceberg - // transaction. - assertFlinkManifests(0); - - assertMaxCommittedCheckpointId(newJobId, operatorId, -1); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); - SimpleDataUtil.assertTableRows(table, expectedRows, branch); - assertSnapshotSize(3); - - RowData row = SimpleDataUtil.createRowData(3, "foo"); - expectedRows.add(row); - DataFile dataFile = writeDataFile("data-3", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); - - harness.snapshot(++checkpointId, ++timestamp); - assertFlinkManifests(1); - - harness.notifyOfCompletedCheckpoint(checkpointId); - assertFlinkManifests(0); - - SimpleDataUtil.assertTableRows(table, expectedRows, branch); - assertSnapshotSize(4); - assertMaxCommittedCheckpointId(newJobId, operatorId, checkpointId); - } - } - - @Test - public void testStartAnotherJobToWriteSameTable() throws Exception { - long checkpointId = 0; - long timestamp = 0; - List rows = Lists.newArrayList(); - List tableRows = Lists.newArrayList(); - - JobID oldJobId = new JobID(); - OperatorID oldOperatorId; - try (OneInputStreamOperatorTestHarness harness = - createStreamSink(oldJobId)) { - harness.setup(); - harness.open(); - oldOperatorId = harness.getOperator().getOperatorID(); - - assertSnapshotSize(0); - assertMaxCommittedCheckpointId(oldJobId, oldOperatorId, -1L); - - for (int i = 1; i <= 3; i++) { - rows.add(SimpleDataUtil.createRowData(i, "hello" + i)); - tableRows.addAll(rows); - - DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); - harness.processElement(of(dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); - assertFlinkManifests(1); - - harness.notifyOfCompletedCheckpoint(checkpointId); - assertFlinkManifests(0); - - SimpleDataUtil.assertTableRows(table, tableRows, branch); - assertSnapshotSize(i); - assertMaxCommittedCheckpointId(oldJobId, oldOperatorId, checkpointId); - } - } - - // The new started job will start with checkpoint = 1 again. - checkpointId = 0; - timestamp = 0; - JobID newJobId = new JobID(); - OperatorID newOperatorId; - try (OneInputStreamOperatorTestHarness harness = - createStreamSink(newJobId)) { - harness.setup(); - harness.open(); - newOperatorId = harness.getOperator().getOperatorID(); - - assertSnapshotSize(3); - assertMaxCommittedCheckpointId(oldJobId, oldOperatorId, 3); - assertMaxCommittedCheckpointId(newJobId, newOperatorId, -1); - - rows.add(SimpleDataUtil.createRowData(2, "world")); - tableRows.addAll(rows); - - DataFile dataFile = writeDataFile("data-new-1", rows); - harness.processElement(of(dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); - assertFlinkManifests(1); - - harness.notifyOfCompletedCheckpoint(checkpointId); - assertFlinkManifests(0); - SimpleDataUtil.assertTableRows(table, tableRows, branch); - assertSnapshotSize(4); - assertMaxCommittedCheckpointId(newJobId, newOperatorId, checkpointId); - } - } - - @Test - public void testMultipleJobsWriteSameTable() throws Exception { - long timestamp = 0; - List tableRows = Lists.newArrayList(); - - JobID[] jobs = new JobID[] {new JobID(), new JobID(), new JobID()}; - OperatorID[] operatorIds = - new OperatorID[] {new OperatorID(), new OperatorID(), new OperatorID()}; - for (int i = 0; i < 20; i++) { - int jobIndex = i % 3; - int checkpointId = i / 3; - JobID jobId = jobs[jobIndex]; - OperatorID operatorId = operatorIds[jobIndex]; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.getStreamConfig().setOperatorID(operatorId); - harness.setup(); - harness.open(); - - assertSnapshotSize(i); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId == 0 ? -1 : checkpointId); - - List rows = Lists.newArrayList(SimpleDataUtil.createRowData(i, "word-" + i)); - tableRows.addAll(rows); - - DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); - harness.processElement(of(dataFile), ++timestamp); - harness.snapshot(checkpointId + 1, ++timestamp); - assertFlinkManifests(1); - - harness.notifyOfCompletedCheckpoint(checkpointId + 1); - assertFlinkManifests(0); - SimpleDataUtil.assertTableRows(table, tableRows, branch); - assertSnapshotSize(i + 1); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId + 1); - } - } - } - - @Test - public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { - long checkpointId = 0; - long timestamp = 0; - List expectedRows = Lists.newArrayList(); - OperatorSubtaskState snapshot1; - OperatorSubtaskState snapshot2; - - JobID jobId = new JobID(); - OperatorID operatorId1 = new OperatorID(); - OperatorID operatorId2 = new OperatorID(); - try (OneInputStreamOperatorTestHarness harness1 = createStreamSink(jobId); - OneInputStreamOperatorTestHarness harness2 = createStreamSink(jobId)) { - harness1.getStreamConfig().setOperatorID(operatorId1); - harness1.setup(); - harness1.open(); - harness2.getStreamConfig().setOperatorID(operatorId2); - harness2.setup(); - harness2.open(); - - assertSnapshotSize(0); - assertMaxCommittedCheckpointId(jobId, operatorId1, -1L); - assertMaxCommittedCheckpointId(jobId, operatorId2, -1L); - - RowData row1 = SimpleDataUtil.createRowData(1, "hello1"); - expectedRows.add(row1); - DataFile dataFile1 = writeDataFile("data-1-1", ImmutableList.of(row1)); - - harness1.processElement(of(dataFile1), ++timestamp); - snapshot1 = harness1.snapshot(++checkpointId, ++timestamp); - - RowData row2 = SimpleDataUtil.createRowData(1, "hello2"); - expectedRows.add(row2); - DataFile dataFile2 = writeDataFile("data-1-2", ImmutableList.of(row2)); - - harness2.processElement(of(dataFile2), ++timestamp); - snapshot2 = harness2.snapshot(checkpointId, ++timestamp); - assertFlinkManifests(2); - - // Only notify one of the committers - harness1.notifyOfCompletedCheckpoint(checkpointId); - assertFlinkManifests(1); - - // Only the first row is committed at this point - SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); - assertSnapshotSize(1); - assertMaxCommittedCheckpointId(jobId, operatorId1, checkpointId); - assertMaxCommittedCheckpointId(jobId, operatorId2, -1); - } - - // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness harness1 = createStreamSink(jobId); - OneInputStreamOperatorTestHarness harness2 = createStreamSink(jobId)) { - harness1.getStreamConfig().setOperatorID(operatorId1); - harness1.setup(); - harness1.initializeState(snapshot1); - harness1.open(); - - harness2.getStreamConfig().setOperatorID(operatorId2); - harness2.setup(); - harness2.initializeState(snapshot2); - harness2.open(); - - // All flink manifests should be cleaned because it has committed the unfinished iceberg - // transaction. - assertFlinkManifests(0); - - SimpleDataUtil.assertTableRows(table, expectedRows, branch); - assertSnapshotSize(2); - assertMaxCommittedCheckpointId(jobId, operatorId1, checkpointId); - assertMaxCommittedCheckpointId(jobId, operatorId2, checkpointId); - - RowData row1 = SimpleDataUtil.createRowData(2, "world1"); - expectedRows.add(row1); - DataFile dataFile1 = writeDataFile("data-2-1", ImmutableList.of(row1)); - - harness1.processElement(of(dataFile1), ++timestamp); - harness1.snapshot(++checkpointId, ++timestamp); - - RowData row2 = SimpleDataUtil.createRowData(2, "world2"); - expectedRows.add(row2); - DataFile dataFile2 = writeDataFile("data-2-2", ImmutableList.of(row2)); - harness2.processElement(of(dataFile2), ++timestamp); - harness2.snapshot(checkpointId, ++timestamp); - - assertFlinkManifests(2); - - harness1.notifyOfCompletedCheckpoint(checkpointId); - harness2.notifyOfCompletedCheckpoint(checkpointId); - assertFlinkManifests(0); - - SimpleDataUtil.assertTableRows(table, expectedRows, branch); - assertSnapshotSize(4); - assertMaxCommittedCheckpointId(jobId, operatorId1, checkpointId); - assertMaxCommittedCheckpointId(jobId, operatorId2, checkpointId); - } - } - - @Test - public void testBoundedStream() throws Exception { - JobID jobId = new JobID(); - OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.setup(); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - assertFlinkManifests(0); - assertSnapshotSize(0); - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - List tableRows = Lists.newArrayList(SimpleDataUtil.createRowData(1, "word-1")); - - DataFile dataFile = writeDataFile("data-1", tableRows); - harness.processElement(of(dataFile), 1); - ((BoundedOneInput) harness.getOneInputOperator()).endInput(); - - assertFlinkManifests(0); - SimpleDataUtil.assertTableRows(table, tableRows, branch); - assertSnapshotSize(1); - assertMaxCommittedCheckpointId(jobId, operatorId, Long.MAX_VALUE); - Assert.assertEquals( - TestIcebergFilesCommitter.class.getName(), - SimpleDataUtil.latestSnapshot(table, branch).summary().get("flink.test")); - } - } - - @Test - public void testFlinkManifests() throws Exception { - long timestamp = 0; - final long checkpoint = 10; - - JobID jobId = new JobID(); - OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.setup(); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - RowData row1 = SimpleDataUtil.createRowData(1, "hello"); - DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - - harness.processElement(of(dataFile1), ++timestamp); - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - // 1. snapshotState for checkpoint#1 - harness.snapshot(checkpoint, ++timestamp); - List manifestPaths = assertFlinkManifests(1); - Path manifestPath = manifestPaths.get(0); - Assert.assertEquals( - "File name should have the expected pattern.", - String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1), - manifestPath.getFileName().toString()); - - // 2. Read the data files from manifests and assert. - List dataFiles = - FlinkManifestUtil.readDataFiles( - createTestingManifestFile(manifestPath), table.io(), table.specs()); - Assert.assertEquals(1, dataFiles.size()); - TestHelpers.assertEquals(dataFile1, dataFiles.get(0)); - - // 3. notifyCheckpointComplete for checkpoint#1 - harness.notifyOfCompletedCheckpoint(checkpoint); - SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); - assertFlinkManifests(0); - } - } - - @Test - public void testDeleteFiles() throws Exception { - Assume.assumeFalse("Only support equality-delete in format v2.", formatVersion < 2); - - long timestamp = 0; - long checkpoint = 10; - - JobID jobId = new JobID(); - OperatorID operatorId; - FileAppenderFactory appenderFactory = createDeletableAppenderFactory(); - - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.setup(); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - RowData row1 = SimpleDataUtil.createInsert(1, "aaa"); - DataFile dataFile1 = writeDataFile("data-file-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - // 1. snapshotState for checkpoint#1 - harness.snapshot(checkpoint, ++timestamp); - List manifestPaths = assertFlinkManifests(1); - Path manifestPath = manifestPaths.get(0); - Assert.assertEquals( - "File name should have the expected pattern.", - String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1), - manifestPath.getFileName().toString()); - - // 2. Read the data files from manifests and assert. - List dataFiles = - FlinkManifestUtil.readDataFiles( - createTestingManifestFile(manifestPath), table.io(), table.specs()); - Assert.assertEquals(1, dataFiles.size()); - TestHelpers.assertEquals(dataFile1, dataFiles.get(0)); - - // 3. notifyCheckpointComplete for checkpoint#1 - harness.notifyOfCompletedCheckpoint(checkpoint); - SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); - assertFlinkManifests(0); - - // 4. process both data files and delete files. - RowData row2 = SimpleDataUtil.createInsert(2, "bbb"); - DataFile dataFile2 = writeDataFile("data-file-2", ImmutableList.of(row2)); - - RowData delete1 = SimpleDataUtil.createDelete(1, "aaa"); - DeleteFile deleteFile1 = - writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete1)); - harness.processElement( - WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile1).build(), - ++timestamp); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); - - // 5. snapshotState for checkpoint#2 - harness.snapshot(++checkpoint, ++timestamp); - assertFlinkManifests(2); - - // 6. notifyCheckpointComplete for checkpoint#2 - harness.notifyOfCompletedCheckpoint(checkpoint); - SimpleDataUtil.assertTableRows(table, ImmutableList.of(row2), branch); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); - assertFlinkManifests(0); - } - } - - @Test - public void testCommitTwoCheckpointsInSingleTxn() throws Exception { - Assume.assumeFalse("Only support equality-delete in format v2.", formatVersion < 2); - - long timestamp = 0; - long checkpoint = 10; - - JobID jobId = new JobID(); - OperatorID operatorId; - FileAppenderFactory appenderFactory = createDeletableAppenderFactory(); - - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.setup(); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - RowData insert1 = SimpleDataUtil.createInsert(1, "aaa"); - RowData insert2 = SimpleDataUtil.createInsert(2, "bbb"); - RowData delete3 = SimpleDataUtil.createDelete(3, "ccc"); - DataFile dataFile1 = writeDataFile("data-file-1", ImmutableList.of(insert1, insert2)); - DeleteFile deleteFile1 = - writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete3)); - harness.processElement( - WriteResult.builder().addDataFiles(dataFile1).addDeleteFiles(deleteFile1).build(), - ++timestamp); - - // The 1th snapshotState. - harness.snapshot(checkpoint, ++timestamp); - - RowData insert4 = SimpleDataUtil.createInsert(4, "ddd"); - RowData delete2 = SimpleDataUtil.createDelete(2, "bbb"); - DataFile dataFile2 = writeDataFile("data-file-2", ImmutableList.of(insert4)); - DeleteFile deleteFile2 = - writeEqDeleteFile(appenderFactory, "delete-file-2", ImmutableList.of(delete2)); - harness.processElement( - WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile2).build(), - ++timestamp); - - // The 2nd snapshotState. - harness.snapshot(++checkpoint, ++timestamp); - - // Notify the 2nd snapshot to complete. - harness.notifyOfCompletedCheckpoint(checkpoint); - SimpleDataUtil.assertTableRows(table, ImmutableList.of(insert1, insert4), branch); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); - assertFlinkManifests(0); - Assert.assertEquals( - "Should have committed 2 txn.", 2, ImmutableList.copyOf(table.snapshots()).size()); - } - } - - @Test - public void testSpecEvolution() throws Exception { - long timestamp = 0; - int checkpointId = 0; - List rows = Lists.newArrayList(); - JobID jobId = new JobID(); - - OperatorID operatorId; - OperatorSubtaskState snapshot; - DataFile dataFile; - int specId; - - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.setup(); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - assertSnapshotSize(0); - - checkpointId++; - RowData rowData = SimpleDataUtil.createRowData(checkpointId, "hello" + checkpointId); - // table unpartitioned - dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(rowData)); - harness.processElement(of(dataFile), ++timestamp); - rows.add(rowData); - harness.snapshot(checkpointId, ++timestamp); - - specId = - getStagingManifestSpecId(harness.getOperator().getOperatorStateBackend(), checkpointId); - assertThat(specId).isEqualTo(table.spec().specId()); - - harness.notifyOfCompletedCheckpoint(checkpointId); - - // Change partition spec - table.refresh(); - PartitionSpec oldSpec = table.spec(); - table.updateSpec().addField("id").commit(); - - checkpointId++; - rowData = SimpleDataUtil.createRowData(checkpointId, "hello" + checkpointId); - // write data with old partition spec - dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(rowData), oldSpec, null); - harness.processElement(of(dataFile), ++timestamp); - rows.add(rowData); - snapshot = harness.snapshot(checkpointId, ++timestamp); - - specId = - getStagingManifestSpecId(harness.getOperator().getOperatorStateBackend(), checkpointId); - assertThat(specId).isEqualTo(oldSpec.specId()); - - harness.notifyOfCompletedCheckpoint(checkpointId); - - assertFlinkManifests(0); - - SimpleDataUtil.assertTableRows(table, ImmutableList.copyOf(rows), branch); - assertSnapshotSize(checkpointId); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); - } - - // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.getStreamConfig().setOperatorID(operatorId); - harness.setup(); - harness.initializeState(snapshot); - harness.open(); - - SimpleDataUtil.assertTableRows(table, rows, branch); - assertSnapshotSize(checkpointId); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); - - checkpointId++; - RowData row = SimpleDataUtil.createRowData(checkpointId, "world" + checkpointId); - StructLike partition = new PartitionData(table.spec().partitionType()); - partition.set(0, checkpointId); - dataFile = - writeDataFile("data-" + checkpointId, ImmutableList.of(row), table.spec(), partition); - harness.processElement(of(dataFile), ++timestamp); - rows.add(row); - harness.snapshot(checkpointId, ++timestamp); - assertFlinkManifests(1); - - specId = - getStagingManifestSpecId(harness.getOperator().getOperatorStateBackend(), checkpointId); - assertThat(specId).isEqualTo(table.spec().specId()); - - harness.notifyOfCompletedCheckpoint(checkpointId); - assertFlinkManifests(0); - - SimpleDataUtil.assertTableRows(table, rows, branch); - assertSnapshotSize(checkpointId); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); - } - } - - private int getStagingManifestSpecId(OperatorStateStore operatorStateStore, long checkPointId) - throws Exception { - ListState> checkpointsState = - operatorStateStore.getListState(IcebergFilesCommitter.buildStateDescriptor()); - NavigableMap statedDataFiles = - Maps.newTreeMap(checkpointsState.get().iterator().next()); - DeltaManifests deltaManifests = - SimpleVersionedSerialization.readVersionAndDeSerialize( - DeltaManifestsSerializer.INSTANCE, statedDataFiles.get(checkPointId)); - return deltaManifests.dataManifest().partitionSpecId(); - } - - private DeleteFile writeEqDeleteFile( - FileAppenderFactory appenderFactory, String filename, List deletes) - throws IOException { - return SimpleDataUtil.writeEqDeleteFile(table, format, filename, appenderFactory, deletes); - } - - private DeleteFile writePosDeleteFile( - FileAppenderFactory appenderFactory, - String filename, - List> positions) - throws IOException { - return SimpleDataUtil.writePosDeleteFile(table, format, filename, appenderFactory, positions); - } - - private FileAppenderFactory createDeletableAppenderFactory() { - int[] equalityFieldIds = - new int[] { - table.schema().findField("id").fieldId(), table.schema().findField("data").fieldId() - }; - return new FlinkAppenderFactory( - table, - table.schema(), - FlinkSchemaUtil.convert(table.schema()), - table.properties(), - table.spec(), - equalityFieldIds, - table.schema(), - null); - } - - private ManifestFile createTestingManifestFile(Path manifestPath) { - return new GenericManifestFile( - manifestPath.toAbsolutePath().toString(), - manifestPath.toFile().length(), - 0, - ManifestContent.DATA, - 0, - 0, - 0L, - 0, - 0, - 0, - 0, - 0, - 0, - null, - null); - } - - private List assertFlinkManifests(int expectedCount) throws IOException { - List manifests = - Files.list(flinkManifestFolder.toPath()) - .filter(p -> !p.toString().endsWith(".crc")) - .collect(Collectors.toList()); - Assert.assertEquals( - String.format("Expected %s flink manifests, but the list is: %s", expectedCount, manifests), - expectedCount, - manifests.size()); - return manifests; - } - - private DataFile writeDataFile(String filename, List rows) throws IOException { - return SimpleDataUtil.writeFile( - table, - table.schema(), - table.spec(), - CONF, - table.location(), - format.addExtension(filename), - rows); - } - - private DataFile writeDataFile( - String filename, List rows, PartitionSpec spec, StructLike partition) - throws IOException { - return SimpleDataUtil.writeFile( - table, - table.schema(), - spec, - CONF, - table.location(), - format.addExtension(filename), - rows, - partition); - } - - private void assertMaxCommittedCheckpointId(JobID jobID, OperatorID operatorID, long expectedId) { - table.refresh(); - long actualId = - IcebergFilesCommitter.getMaxCommittedCheckpointId( - table, jobID.toString(), operatorID.toHexString(), branch); - Assert.assertEquals(expectedId, actualId); - } - - private void assertSnapshotSize(int expectedSnapshotSize) { - table.refresh(); - Assert.assertEquals(expectedSnapshotSize, Lists.newArrayList(table.snapshots()).size()); - } - - private OneInputStreamOperatorTestHarness createStreamSink(JobID jobID) - throws Exception { - TestOperatorFactory factory = TestOperatorFactory.of(table.location(), branch, table.spec()); - return new OneInputStreamOperatorTestHarness<>(factory, createEnvironment(jobID)); - } - - private static MockEnvironment createEnvironment(JobID jobID) { - return new MockEnvironmentBuilder() - .setTaskName("test task") - .setManagedMemorySize(32 * 1024) - .setInputSplitProvider(new MockInputSplitProvider()) - .setBufferSize(256) - .setTaskConfiguration(new org.apache.flink.configuration.Configuration()) - .setExecutionConfig(new ExecutionConfig()) - .setMaxParallelism(16) - .setJobID(jobID) - .build(); - } - - private static class TestOperatorFactory extends AbstractStreamOperatorFactory - implements OneInputStreamOperatorFactory { - private final String tablePath; - private final String branch; - private final PartitionSpec spec; - - private TestOperatorFactory(String tablePath, String branch, PartitionSpec spec) { - this.tablePath = tablePath; - this.branch = branch; - this.spec = spec; - } - - private static TestOperatorFactory of(String tablePath, String branch, PartitionSpec spec) { - return new TestOperatorFactory(tablePath, branch, spec); - } - - @Override - @SuppressWarnings("unchecked") - public > T createStreamOperator( - StreamOperatorParameters param) { - IcebergFilesCommitter committer = - new IcebergFilesCommitter( - new TestTableLoader(tablePath), - false, - Collections.singletonMap("flink.test", TestIcebergFilesCommitter.class.getName()), - ThreadPools.WORKER_THREAD_POOL_SIZE, - branch, - spec); - committer.setup(param.getContainingTask(), param.getStreamConfig(), param.getOutput()); - return (T) committer; - } - - @Override - public Class getStreamOperatorClass(ClassLoader classLoader) { - return IcebergFilesCommitter.class; - } - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java deleted file mode 100644 index 0968f89f55e0..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java +++ /dev/null @@ -1,388 +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.iceberg.flink.sink; - -import java.io.File; -import java.io.IOException; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.Set; -import org.apache.flink.streaming.api.operators.BoundedOneInput; -import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.LocatedFileStatus; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.RemoteIterator; -import org.apache.iceberg.AppendFiles; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -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.flink.FlinkWriteConf; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.hadoop.HadoopTables; -import org.apache.iceberg.io.WriteResult; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestIcebergStreamWriter { - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); - - private Table table; - - private final FileFormat format; - private final boolean partitioned; - - @Parameterized.Parameters(name = "format = {0}, partitioned = {1}") - public static Object[][] parameters() { - return new Object[][] { - {"avro", true}, - {"avro", false}, - {"orc", true}, - {"orc", false}, - {"parquet", true}, - {"parquet", false} - }; - } - - public TestIcebergStreamWriter(String format, boolean partitioned) { - this.format = FileFormat.fromString(format); - this.partitioned = partitioned; - } - - @Before - public void before() throws IOException { - File folder = tempFolder.newFolder(); - // Construct the iceberg table. - Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); - table = SimpleDataUtil.createTable(folder.getAbsolutePath(), props, partitioned); - } - - @Test - public void testWritingTable() throws Exception { - long checkpointId = 1L; - try (OneInputStreamOperatorTestHarness testHarness = - createIcebergStreamWriter()) { - // The first checkpoint - testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); - testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 1); - testHarness.processElement(SimpleDataUtil.createRowData(3, "hello"), 1); - - testHarness.prepareSnapshotPreBarrier(checkpointId); - long expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); - - checkpointId = checkpointId + 1; - - // The second checkpoint - testHarness.processElement(SimpleDataUtil.createRowData(4, "foo"), 1); - testHarness.processElement(SimpleDataUtil.createRowData(5, "bar"), 2); - - testHarness.prepareSnapshotPreBarrier(checkpointId); - expectedDataFiles = partitioned ? 4 : 2; - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); - - // Commit the iceberg transaction. - AppendFiles appendFiles = table.newAppend(); - Arrays.stream(result.dataFiles()).forEach(appendFiles::appendFile); - appendFiles.commit(); - - // Assert the table records. - SimpleDataUtil.assertTableRecords( - table, - Lists.newArrayList( - SimpleDataUtil.createRecord(1, "hello"), - SimpleDataUtil.createRecord(2, "world"), - SimpleDataUtil.createRecord(3, "hello"), - SimpleDataUtil.createRecord(4, "foo"), - SimpleDataUtil.createRecord(5, "bar"))); - } - } - - @Test - public void testSnapshotTwice() throws Exception { - long checkpointId = 1; - long timestamp = 1; - try (OneInputStreamOperatorTestHarness testHarness = - createIcebergStreamWriter()) { - testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), timestamp++); - testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), timestamp); - - testHarness.prepareSnapshotPreBarrier(checkpointId++); - long expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); - - // snapshot again immediately. - for (int i = 0; i < 5; i++) { - testHarness.prepareSnapshotPreBarrier(checkpointId++); - - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); - } - } - } - - @Test - public void testTableWithoutSnapshot() throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = - createIcebergStreamWriter()) { - Assert.assertEquals(0, testHarness.extractOutputValues().size()); - } - // Even if we closed the iceberg stream writer, there's no orphan data file. - Assert.assertEquals(0, scanDataFiles().size()); - - try (OneInputStreamOperatorTestHarness testHarness = - createIcebergStreamWriter()) { - testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); - // Still not emit the data file yet, because there is no checkpoint. - Assert.assertEquals(0, testHarness.extractOutputValues().size()); - } - // Once we closed the iceberg stream writer, there will left an orphan data file. - Assert.assertEquals(1, scanDataFiles().size()); - } - - private Set scanDataFiles() throws IOException { - Path dataDir = new Path(table.location(), "data"); - FileSystem fs = FileSystem.get(new Configuration()); - if (!fs.exists(dataDir)) { - return ImmutableSet.of(); - } else { - Set paths = Sets.newHashSet(); - RemoteIterator iterators = fs.listFiles(dataDir, true); - while (iterators.hasNext()) { - LocatedFileStatus status = iterators.next(); - if (status.isFile()) { - Path path = status.getPath(); - if (path.getName().endsWith("." + format.toString().toLowerCase())) { - paths.add(path.toString()); - } - } - } - return paths; - } - } - - @Test - public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = - createIcebergStreamWriter()) { - testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); - testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 2); - - Assertions.assertThat(testHarness.getOneInputOperator()).isInstanceOf(BoundedOneInput.class); - ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); - - long expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); - - ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); - - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - // Datafiles should not be sent again - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); - } - } - - @Test - public void testBoundedStreamTriggeredEndInputBeforeTriggeringCheckpoint() throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = - createIcebergStreamWriter()) { - testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); - testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 2); - - testHarness.endInput(); - - long expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); - - testHarness.prepareSnapshotPreBarrier(1L); - - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - // It should be ensured that after endInput is triggered, when prepareSnapshotPreBarrier - // is triggered, write should only send WriteResult once - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); - } - } - - @Test - public void testTableWithTargetFileSize() throws Exception { - // Adjust the target-file-size in table properties. - table - .updateProperties() - .set(TableProperties.WRITE_TARGET_FILE_SIZE_BYTES, "4") // ~4 bytes; low enough to trigger - .commit(); - - List rows = Lists.newArrayListWithCapacity(8000); - List records = Lists.newArrayListWithCapacity(8000); - for (int i = 0; i < 2000; i++) { - for (String data : new String[] {"a", "b", "c", "d"}) { - rows.add(SimpleDataUtil.createRowData(i, data)); - records.add(SimpleDataUtil.createRecord(i, data)); - } - } - - try (OneInputStreamOperatorTestHarness testHarness = - createIcebergStreamWriter()) { - for (RowData row : rows) { - testHarness.processElement(row, 1); - } - - // snapshot the operator. - testHarness.prepareSnapshotPreBarrier(1); - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(8, result.dataFiles().length); - - // Assert that the data file have the expected records. - for (DataFile dataFile : result.dataFiles()) { - Assert.assertEquals(1000, dataFile.recordCount()); - } - - // Commit the iceberg transaction. - AppendFiles appendFiles = table.newAppend(); - Arrays.stream(result.dataFiles()).forEach(appendFiles::appendFile); - appendFiles.commit(); - } - - // Assert the table records. - SimpleDataUtil.assertTableRecords(table, records); - } - - @Test - public void testPromotedFlinkDataType() throws Exception { - Schema iSchema = - new Schema( - Types.NestedField.required(1, "tinyint", Types.IntegerType.get()), - Types.NestedField.required(2, "smallint", Types.IntegerType.get()), - Types.NestedField.optional(3, "int", Types.IntegerType.get())); - TableSchema flinkSchema = - TableSchema.builder() - .field("tinyint", DataTypes.TINYINT().notNull()) - .field("smallint", DataTypes.SMALLINT().notNull()) - .field("int", DataTypes.INT().nullable()) - .build(); - - PartitionSpec spec; - if (partitioned) { - spec = - PartitionSpec.builderFor(iSchema) - .identity("smallint") - .identity("tinyint") - .identity("int") - .build(); - } else { - spec = PartitionSpec.unpartitioned(); - } - - String location = tempFolder.newFolder().getAbsolutePath(); - Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); - Table icebergTable = new HadoopTables().create(iSchema, spec, props, location); - - List rows = - Lists.newArrayList( - GenericRowData.of((byte) 0x01, (short) -32768, 101), - GenericRowData.of((byte) 0x02, (short) 0, 102), - GenericRowData.of((byte) 0x03, (short) 32767, 103)); - - Record record = GenericRecord.create(iSchema); - List expected = - Lists.newArrayList( - record.copy(ImmutableMap.of("tinyint", 1, "smallint", -32768, "int", 101)), - record.copy(ImmutableMap.of("tinyint", 2, "smallint", 0, "int", 102)), - record.copy(ImmutableMap.of("tinyint", 3, "smallint", 32767, "int", 103))); - - try (OneInputStreamOperatorTestHarness testHarness = - createIcebergStreamWriter(icebergTable, flinkSchema)) { - for (RowData row : rows) { - testHarness.processElement(row, 1); - } - testHarness.prepareSnapshotPreBarrier(1); - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(partitioned ? 3 : 1, result.dataFiles().length); - - // Commit the iceberg transaction. - AppendFiles appendFiles = icebergTable.newAppend(); - Arrays.stream(result.dataFiles()).forEach(appendFiles::appendFile); - appendFiles.commit(); - } - - SimpleDataUtil.assertTableRecords(location, expected); - } - - private OneInputStreamOperatorTestHarness createIcebergStreamWriter() - throws Exception { - return createIcebergStreamWriter(table, SimpleDataUtil.FLINK_SCHEMA); - } - - private OneInputStreamOperatorTestHarness createIcebergStreamWriter( - Table icebergTable, TableSchema flinkSchema) throws Exception { - RowType flinkRowType = FlinkSink.toFlinkRowType(icebergTable.schema(), flinkSchema); - FlinkWriteConf flinkWriteConfig = - new FlinkWriteConf( - icebergTable, Maps.newHashMap(), new org.apache.flink.configuration.Configuration()); - - IcebergStreamWriter streamWriter = - FlinkSink.createStreamWriter(() -> icebergTable, flinkWriteConfig, flinkRowType, null); - OneInputStreamOperatorTestHarness harness = - new OneInputStreamOperatorTestHarness<>(streamWriter, 1, 1, 0); - - harness.setup(); - harness.open(); - - return harness; - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java deleted file mode 100644 index 79be96d20902..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java +++ /dev/null @@ -1,252 +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.iceberg.flink.sink; - -import java.util.List; -import java.util.stream.Collectors; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.types.logical.RowType; -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.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.data.RandomRowData; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Test; - -public class TestRowDataPartitionKey { - private static final Schema SCHEMA = - new Schema( - Types.NestedField.required(0, "boolType", Types.BooleanType.get()), - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.required(2, "longType", Types.LongType.get()), - Types.NestedField.required(3, "dateType", Types.DateType.get()), - Types.NestedField.required(4, "timeType", Types.TimeType.get()), - Types.NestedField.required(5, "stringType", Types.StringType.get()), - Types.NestedField.required(6, "timestampWithoutZone", Types.TimestampType.withoutZone()), - Types.NestedField.required(7, "timestampWithZone", Types.TimestampType.withZone()), - Types.NestedField.required(8, "fixedType", Types.FixedType.ofLength(5)), - Types.NestedField.required(9, "uuidType", Types.UUIDType.get()), - Types.NestedField.required(10, "binaryType", Types.BinaryType.get()), - Types.NestedField.required(11, "decimalType1", Types.DecimalType.of(18, 3)), - Types.NestedField.required(12, "decimalType2", Types.DecimalType.of(10, 5)), - Types.NestedField.required(13, "decimalType3", Types.DecimalType.of(38, 19)), - Types.NestedField.required(14, "floatType", Types.FloatType.get()), - Types.NestedField.required(15, "doubleType", Types.DoubleType.get())); - - private static final List SUPPORTED_PRIMITIVES = - SCHEMA.asStruct().fields().stream().map(Types.NestedField::name).collect(Collectors.toList()); - - private static final Schema NESTED_SCHEMA = - new Schema( - Types.NestedField.required( - 1, - "structType", - Types.StructType.of( - Types.NestedField.optional(2, "innerStringType", Types.StringType.get()), - Types.NestedField.optional(3, "innerIntegerType", Types.IntegerType.get())))); - - @Test - public void testNullPartitionValue() { - Schema schema = - new Schema( - Types.NestedField.optional(1, "id", Types.IntegerType.get()), - Types.NestedField.optional(2, "data", Types.StringType.get())); - - PartitionSpec spec = PartitionSpec.builderFor(schema).identity("data").build(); - - List rows = - Lists.newArrayList( - GenericRowData.of(1, StringData.fromString("a")), - GenericRowData.of(2, StringData.fromString("b")), - GenericRowData.of(3, null)); - - RowDataWrapper rowWrapper = - new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); - - for (RowData row : rows) { - PartitionKey partitionKey = new PartitionKey(spec, schema); - partitionKey.partition(rowWrapper.wrap(row)); - Assert.assertEquals(partitionKey.size(), 1); - - String expectedStr = row.isNullAt(1) ? null : row.getString(1).toString(); - Assert.assertEquals(expectedStr, partitionKey.get(0, String.class)); - } - } - - @Test - public void testPartitionWithOneNestedField() { - RowDataWrapper rowWrapper = - new RowDataWrapper(FlinkSchemaUtil.convert(NESTED_SCHEMA), NESTED_SCHEMA.asStruct()); - List records = RandomGenericData.generate(NESTED_SCHEMA, 10, 1991); - List rows = Lists.newArrayList(RandomRowData.convert(NESTED_SCHEMA, records)); - - PartitionSpec spec1 = - PartitionSpec.builderFor(NESTED_SCHEMA).identity("structType.innerStringType").build(); - PartitionSpec spec2 = - PartitionSpec.builderFor(NESTED_SCHEMA).identity("structType.innerIntegerType").build(); - - for (int i = 0; i < rows.size(); i++) { - RowData row = rows.get(i); - Record record = (Record) records.get(i).get(0); - - PartitionKey partitionKey1 = new PartitionKey(spec1, NESTED_SCHEMA); - partitionKey1.partition(rowWrapper.wrap(row)); - Assert.assertEquals(partitionKey1.size(), 1); - - Assert.assertEquals(record.get(0), partitionKey1.get(0, String.class)); - - PartitionKey partitionKey2 = new PartitionKey(spec2, NESTED_SCHEMA); - partitionKey2.partition(rowWrapper.wrap(row)); - Assert.assertEquals(partitionKey2.size(), 1); - - Assert.assertEquals(record.get(1), partitionKey2.get(0, Integer.class)); - } - } - - @Test - public void testPartitionMultipleNestedField() { - RowDataWrapper rowWrapper = - new RowDataWrapper(FlinkSchemaUtil.convert(NESTED_SCHEMA), NESTED_SCHEMA.asStruct()); - List records = RandomGenericData.generate(NESTED_SCHEMA, 10, 1992); - List rows = Lists.newArrayList(RandomRowData.convert(NESTED_SCHEMA, records)); - - PartitionSpec spec1 = - PartitionSpec.builderFor(NESTED_SCHEMA) - .identity("structType.innerIntegerType") - .identity("structType.innerStringType") - .build(); - PartitionSpec spec2 = - PartitionSpec.builderFor(NESTED_SCHEMA) - .identity("structType.innerStringType") - .identity("structType.innerIntegerType") - .build(); - - PartitionKey pk1 = new PartitionKey(spec1, NESTED_SCHEMA); - PartitionKey pk2 = new PartitionKey(spec2, NESTED_SCHEMA); - - for (int i = 0; i < rows.size(); i++) { - RowData row = rows.get(i); - Record record = (Record) records.get(i).get(0); - - pk1.partition(rowWrapper.wrap(row)); - Assert.assertEquals(2, pk1.size()); - - Assert.assertEquals(record.get(1), pk1.get(0, Integer.class)); - Assert.assertEquals(record.get(0), pk1.get(1, String.class)); - - pk2.partition(rowWrapper.wrap(row)); - Assert.assertEquals(2, pk2.size()); - - Assert.assertEquals(record.get(0), pk2.get(0, String.class)); - Assert.assertEquals(record.get(1), pk2.get(1, Integer.class)); - } - } - - @Test - public void testPartitionValueTypes() { - RowType rowType = FlinkSchemaUtil.convert(SCHEMA); - RowDataWrapper rowWrapper = new RowDataWrapper(rowType, SCHEMA.asStruct()); - InternalRecordWrapper recordWrapper = new InternalRecordWrapper(SCHEMA.asStruct()); - - List records = RandomGenericData.generate(SCHEMA, 10, 1993); - List rows = Lists.newArrayList(RandomRowData.convert(SCHEMA, records)); - - for (String column : SUPPORTED_PRIMITIVES) { - PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity(column).build(); - Class[] javaClasses = spec.javaClasses(); - - PartitionKey pk = new PartitionKey(spec, SCHEMA); - PartitionKey expectedPK = new PartitionKey(spec, SCHEMA); - - for (int j = 0; j < rows.size(); j++) { - RowData row = rows.get(j); - Record record = records.get(j); - - pk.partition(rowWrapper.wrap(row)); - expectedPK.partition(recordWrapper.wrap(record)); - - Assert.assertEquals( - "Partition with column " + column + " should have one field.", 1, pk.size()); - - if (column.equals("timeType")) { - Assert.assertEquals( - "Partition with column " + column + " should have the expected values", - expectedPK.get(0, Long.class) / 1000, - pk.get(0, Long.class) / 1000); - } else { - Assert.assertEquals( - "Partition with column " + column + " should have the expected values", - expectedPK.get(0, javaClasses[0]), - pk.get(0, javaClasses[0])); - } - } - } - } - - @Test - public void testNestedPartitionValues() { - Schema nestedSchema = new Schema(Types.NestedField.optional(1001, "nested", SCHEMA.asStruct())); - RowType rowType = FlinkSchemaUtil.convert(nestedSchema); - - RowDataWrapper rowWrapper = new RowDataWrapper(rowType, nestedSchema.asStruct()); - InternalRecordWrapper recordWrapper = new InternalRecordWrapper(nestedSchema.asStruct()); - - List records = RandomGenericData.generate(nestedSchema, 10, 1994); - List rows = Lists.newArrayList(RandomRowData.convert(nestedSchema, records)); - - for (String supportedPrimitive : SUPPORTED_PRIMITIVES) { - String column = String.format("nested.%s", supportedPrimitive); - - PartitionSpec spec = PartitionSpec.builderFor(nestedSchema).identity(column).build(); - Class[] javaClasses = spec.javaClasses(); - - PartitionKey pk = new PartitionKey(spec, nestedSchema); - PartitionKey expectedPK = new PartitionKey(spec, nestedSchema); - - for (int j = 0; j < rows.size(); j++) { - pk.partition(rowWrapper.wrap(rows.get(j))); - expectedPK.partition(recordWrapper.wrap(records.get(j))); - - Assert.assertEquals( - "Partition with nested column " + column + " should have one field.", 1, pk.size()); - - if (column.equals("nested.timeType")) { - Assert.assertEquals( - "Partition with nested column " + column + " should have the expected values.", - expectedPK.get(0, Long.class) / 1000, - pk.get(0, Long.class) / 1000); - } else { - Assert.assertEquals( - "Partition with nested column " + column + " should have the expected values.", - expectedPK.get(0, javaClasses[0]), - pk.get(0, javaClasses[0])); - } - } - } - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java deleted file mode 100644 index 112dbb511310..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java +++ /dev/null @@ -1,243 +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.iceberg.flink.sink; - -import java.io.File; -import java.io.IOException; -import java.util.List; -import java.util.Map; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.iceberg.AppendFiles; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.SerializableTable; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.data.RandomRowData; -import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestTaskWriters { - private static final Configuration CONF = new Configuration(); - private static final long TARGET_FILE_SIZE = 128 * 1024 * 1024; - - @Rule public final TemporaryFolder tempFolder = new TemporaryFolder(); - - @Parameterized.Parameters(name = "format = {0}, partitioned = {1}") - public static Object[][] parameters() { - return new Object[][] { - {"avro", true}, - {"avro", false}, - {"orc", true}, - {"orc", false}, - {"parquet", true}, - {"parquet", false} - }; - } - - private final FileFormat format; - private final boolean partitioned; - - private Table table; - - public TestTaskWriters(String format, boolean partitioned) { - this.format = FileFormat.fromString(format); - this.partitioned = partitioned; - } - - @Before - public void before() throws IOException { - File folder = tempFolder.newFolder(); - // Construct the iceberg table with the specified file format. - Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); - table = SimpleDataUtil.createTable(folder.getAbsolutePath(), props, partitioned); - } - - @Test - public void testWriteZeroRecord() throws IOException { - try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { - taskWriter.close(); - - DataFile[] dataFiles = taskWriter.dataFiles(); - Assert.assertNotNull(dataFiles); - Assert.assertEquals(0, dataFiles.length); - - // Close again. - taskWriter.close(); - dataFiles = taskWriter.dataFiles(); - Assert.assertNotNull(dataFiles); - Assert.assertEquals(0, dataFiles.length); - } - } - - @Test - public void testCloseTwice() throws IOException { - try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { - taskWriter.write(SimpleDataUtil.createRowData(1, "hello")); - taskWriter.write(SimpleDataUtil.createRowData(2, "world")); - taskWriter.close(); // The first close - taskWriter.close(); // The second close - - int expectedFiles = partitioned ? 2 : 1; - DataFile[] dataFiles = taskWriter.dataFiles(); - Assert.assertEquals(expectedFiles, dataFiles.length); - - FileSystem fs = FileSystem.get(CONF); - for (DataFile dataFile : dataFiles) { - Assert.assertTrue(fs.exists(new Path(dataFile.path().toString()))); - } - } - } - - @Test - public void testAbort() throws IOException { - try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { - taskWriter.write(SimpleDataUtil.createRowData(1, "hello")); - taskWriter.write(SimpleDataUtil.createRowData(2, "world")); - - taskWriter.abort(); - DataFile[] dataFiles = taskWriter.dataFiles(); - - int expectedFiles = partitioned ? 2 : 1; - Assert.assertEquals(expectedFiles, dataFiles.length); - - FileSystem fs = FileSystem.get(CONF); - for (DataFile dataFile : dataFiles) { - Assert.assertFalse(fs.exists(new Path(dataFile.path().toString()))); - } - } - } - - @Test - public void testCompleteFiles() throws IOException { - try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { - taskWriter.write(SimpleDataUtil.createRowData(1, "a")); - taskWriter.write(SimpleDataUtil.createRowData(2, "b")); - taskWriter.write(SimpleDataUtil.createRowData(3, "c")); - taskWriter.write(SimpleDataUtil.createRowData(4, "d")); - - DataFile[] dataFiles = taskWriter.dataFiles(); - int expectedFiles = partitioned ? 4 : 1; - Assert.assertEquals(expectedFiles, dataFiles.length); - - dataFiles = taskWriter.dataFiles(); - Assert.assertEquals(expectedFiles, dataFiles.length); - - FileSystem fs = FileSystem.get(CONF); - for (DataFile dataFile : dataFiles) { - Assert.assertTrue(fs.exists(new Path(dataFile.path().toString()))); - } - - AppendFiles appendFiles = table.newAppend(); - for (DataFile dataFile : dataFiles) { - appendFiles.appendFile(dataFile); - } - appendFiles.commit(); - - // Assert the data rows. - SimpleDataUtil.assertTableRecords( - table, - Lists.newArrayList( - SimpleDataUtil.createRecord(1, "a"), - SimpleDataUtil.createRecord(2, "b"), - SimpleDataUtil.createRecord(3, "c"), - SimpleDataUtil.createRecord(4, "d"))); - } - } - - @Test - public void testRollingWithTargetFileSize() throws IOException { - try (TaskWriter taskWriter = createTaskWriter(4)) { - List rows = Lists.newArrayListWithCapacity(8000); - List records = Lists.newArrayListWithCapacity(8000); - for (int i = 0; i < 2000; i++) { - for (String data : new String[] {"a", "b", "c", "d"}) { - rows.add(SimpleDataUtil.createRowData(i, data)); - records.add(SimpleDataUtil.createRecord(i, data)); - } - } - - for (RowData row : rows) { - taskWriter.write(row); - } - - DataFile[] dataFiles = taskWriter.dataFiles(); - Assert.assertEquals(8, dataFiles.length); - - AppendFiles appendFiles = table.newAppend(); - for (DataFile dataFile : dataFiles) { - appendFiles.appendFile(dataFile); - } - appendFiles.commit(); - - // Assert the data rows. - SimpleDataUtil.assertTableRecords(table, records); - } - } - - @Test - public void testRandomData() throws IOException { - try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { - Iterable rows = RandomRowData.generate(SimpleDataUtil.SCHEMA, 100, 1996); - for (RowData row : rows) { - taskWriter.write(row); - } - - taskWriter.close(); - DataFile[] dataFiles = taskWriter.dataFiles(); - AppendFiles appendFiles = table.newAppend(); - for (DataFile dataFile : dataFiles) { - appendFiles.appendFile(dataFile); - } - appendFiles.commit(); - - // Assert the data rows. - SimpleDataUtil.assertTableRows(table, Lists.newArrayList(rows)); - } - } - - private TaskWriter createTaskWriter(long targetFileSize) { - TaskWriterFactory taskWriterFactory = - new RowDataTaskWriterFactory( - SerializableTable.copyOf(table), - (RowType) SimpleDataUtil.FLINK_SCHEMA.toRowDataType().getLogicalType(), - targetFileSize, - format, - table.properties(), - null, - false); - taskWriterFactory.initialize(1, 1); - return taskWriterFactory.create(); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java deleted file mode 100644 index dd7fcafe5336..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java +++ /dev/null @@ -1,61 +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.iceberg.flink.sink.shuffle; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.util.Map; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.VarCharType; -import org.junit.Test; - -public class TestAggregatedStatistics { - - @Test - public void mergeDataStatisticTest() { - GenericRowData rowDataA = GenericRowData.of(StringData.fromString("a")); - GenericRowData rowDataB = GenericRowData.of(StringData.fromString("b")); - - AggregatedStatistics> aggregatedStatistics = - new AggregatedStatistics<>( - 1, - MapDataStatisticsSerializer.fromKeySerializer( - new RowDataSerializer(RowType.of(new VarCharType())))); - MapDataStatistics mapDataStatistics1 = new MapDataStatistics(); - mapDataStatistics1.add(rowDataA); - mapDataStatistics1.add(rowDataA); - mapDataStatistics1.add(rowDataB); - aggregatedStatistics.mergeDataStatistic("testOperator", 1, mapDataStatistics1); - MapDataStatistics mapDataStatistics2 = new MapDataStatistics(); - mapDataStatistics2.add(rowDataA); - aggregatedStatistics.mergeDataStatistic("testOperator", 1, mapDataStatistics2); - assertThat(aggregatedStatistics.dataStatistics().statistics().get(rowDataA)) - .isEqualTo( - mapDataStatistics1.statistics().get(rowDataA) - + mapDataStatistics2.statistics().get(rowDataA)); - assertThat(aggregatedStatistics.dataStatistics().statistics().get(rowDataB)) - .isEqualTo( - mapDataStatistics1.statistics().get(rowDataB) - + mapDataStatistics2.statistics().getOrDefault(rowDataB, 0L)); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java deleted file mode 100644 index 48e4e4d8f966..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java +++ /dev/null @@ -1,177 +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.iceberg.flink.sink.shuffle; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.util.Map; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.binary.BinaryRowData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.VarCharType; -import org.junit.Before; -import org.junit.Test; - -public class TestAggregatedStatisticsTracker { - private static final int NUM_SUBTASKS = 2; - private final RowType rowType = RowType.of(new VarCharType()); - // When coordinator handles events from operator, DataStatisticsUtil#deserializeDataStatistics - // deserializes bytes into BinaryRowData - private final BinaryRowData binaryRowDataA = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("a"))); - private final BinaryRowData binaryRowDataB = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("b"))); - private final TypeSerializer rowSerializer = new RowDataSerializer(rowType); - private final TypeSerializer>> - statisticsSerializer = MapDataStatisticsSerializer.fromKeySerializer(rowSerializer); - private AggregatedStatisticsTracker> - aggregatedStatisticsTracker; - - @Before - public void before() throws Exception { - aggregatedStatisticsTracker = - new AggregatedStatisticsTracker<>("testOperator", statisticsSerializer, NUM_SUBTASKS); - } - - @Test - public void receiveNewerDataStatisticEvent() { - MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(binaryRowDataA); - DataStatisticsEvent> - checkpoint1Subtask0DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint1Subtask0DataStatisticEvent)) - .isNull(); - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()).isEqualTo(1); - - MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(binaryRowDataA); - DataStatisticsEvent> - checkpoint2Subtask0DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint2Subtask0DataStatisticEvent)) - .isNull(); - // Checkpoint 2 is newer than checkpoint1, thus dropping in progress statistics for checkpoint1 - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()).isEqualTo(2); - } - - @Test - public void receiveOlderDataStatisticEventTest() { - MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(binaryRowDataA); - checkpoint2Subtask0DataStatistic.add(binaryRowDataB); - checkpoint2Subtask0DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> - checkpoint3Subtask0DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint3Subtask0DataStatisticEvent)) - .isNull(); - - MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - checkpoint1Subtask1DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> - checkpoint1Subtask1DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); - // Receive event from old checkpoint, aggregatedStatisticsAggregatorTracker won't return - // completed statistics and in progress statistics won't be updated - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 1, checkpoint1Subtask1DataStatisticEvent)) - .isNull(); - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()).isEqualTo(2); - } - - @Test - public void receiveCompletedDataStatisticEvent() { - MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(binaryRowDataA); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> - checkpoint1Subtask0DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint1Subtask0DataStatisticEvent)) - .isNull(); - - MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - checkpoint1Subtask1DataStatistic.add(binaryRowDataA); - checkpoint1Subtask1DataStatistic.add(binaryRowDataA); - checkpoint1Subtask1DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> - checkpoint1Subtask1DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); - // Receive data statistics from all subtasks at checkpoint 1 - AggregatedStatistics> completedStatistics = - aggregatedStatisticsTracker.updateAndCheckCompletion( - 1, checkpoint1Subtask1DataStatisticEvent); - - assertThat(completedStatistics).isNotNull(); - assertThat(completedStatistics.checkpointId()).isEqualTo(1); - MapDataStatistics globalDataStatistics = - (MapDataStatistics) completedStatistics.dataStatistics(); - assertThat((long) globalDataStatistics.statistics().get(binaryRowDataA)) - .isEqualTo( - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataA) - + checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataA)); - assertThat((long) globalDataStatistics.statistics().get(binaryRowDataB)) - .isEqualTo( - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataB) - + checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataB)); - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()) - .isEqualTo(completedStatistics.checkpointId() + 1); - - MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(binaryRowDataA); - DataStatisticsEvent> - checkpoint2Subtask0DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint2Subtask0DataStatisticEvent)) - .isNull(); - assertThat(completedStatistics.checkpointId()).isEqualTo(1); - - MapDataStatistics checkpoint2Subtask1DataStatistic = new MapDataStatistics(); - checkpoint2Subtask1DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> - checkpoint2Subtask1DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask1DataStatistic, statisticsSerializer); - // Receive data statistics from all subtasks at checkpoint 2 - completedStatistics = - aggregatedStatisticsTracker.updateAndCheckCompletion( - 1, checkpoint2Subtask1DataStatisticEvent); - - assertThat(completedStatistics).isNotNull(); - assertThat(completedStatistics.checkpointId()).isEqualTo(2); - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()) - .isEqualTo(completedStatistics.checkpointId() + 1); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java deleted file mode 100644 index 3be12ea1965b..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java +++ /dev/null @@ -1,172 +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.iceberg.flink.sink.shuffle; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -import java.util.Map; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.runtime.jobgraph.OperatorID; -import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; -import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.binary.BinaryRowData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.VarCharType; -import org.apache.flink.util.ExceptionUtils; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Before; -import org.junit.Test; - -public class TestDataStatisticsCoordinator { - private static final String OPERATOR_NAME = "TestCoordinator"; - private static final OperatorID TEST_OPERATOR_ID = new OperatorID(1234L, 5678L); - private static final int NUM_SUBTASKS = 2; - private TypeSerializer>> - statisticsSerializer; - - private EventReceivingTasks receivingTasks; - private DataStatisticsCoordinator> - dataStatisticsCoordinator; - - @Before - public void before() throws Exception { - receivingTasks = EventReceivingTasks.createForRunningTasks(); - statisticsSerializer = - MapDataStatisticsSerializer.fromKeySerializer( - new RowDataSerializer(RowType.of(new VarCharType()))); - - dataStatisticsCoordinator = - new DataStatisticsCoordinator<>( - OPERATOR_NAME, - new MockOperatorCoordinatorContext(TEST_OPERATOR_ID, NUM_SUBTASKS), - statisticsSerializer); - } - - private void tasksReady() throws Exception { - dataStatisticsCoordinator.start(); - setAllTasksReady(NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); - } - - @Test - public void testThrowExceptionWhenNotStarted() { - String failureMessage = "The coordinator of TestCoordinator has not started yet."; - - assertThatThrownBy( - () -> - dataStatisticsCoordinator.handleEventFromOperator( - 0, - DataStatisticsEvent.create(0, new MapDataStatistics(), statisticsSerializer))) - .isInstanceOf(IllegalStateException.class) - .hasMessage(failureMessage); - assertThatThrownBy(() -> dataStatisticsCoordinator.subtaskFailed(0, null)) - .isInstanceOf(IllegalStateException.class) - .hasMessage(failureMessage); - assertThatThrownBy(() -> dataStatisticsCoordinator.checkpointCoordinator(0, null)) - .isInstanceOf(IllegalStateException.class) - .hasMessage(failureMessage); - } - - @Test - public void testDataStatisticsEventHandling() throws Exception { - tasksReady(); - // When coordinator handles events from operator, DataStatisticsUtil#deserializeDataStatistics - // deserializes bytes into BinaryRowData - RowType rowType = RowType.of(new VarCharType()); - BinaryRowData binaryRowDataA = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("a"))); - BinaryRowData binaryRowDataB = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("b"))); - BinaryRowData binaryRowDataC = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("c"))); - - MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(binaryRowDataA); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - checkpoint1Subtask0DataStatistic.add(binaryRowDataC); - checkpoint1Subtask0DataStatistic.add(binaryRowDataC); - checkpoint1Subtask0DataStatistic.add(binaryRowDataC); - DataStatisticsEvent> - checkpoint1Subtask0DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); - MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - checkpoint1Subtask1DataStatistic.add(binaryRowDataA); - checkpoint1Subtask1DataStatistic.add(binaryRowDataB); - checkpoint1Subtask1DataStatistic.add(binaryRowDataC); - checkpoint1Subtask1DataStatistic.add(binaryRowDataC); - DataStatisticsEvent> - checkpoint1Subtask1DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); - // Handle events from operators for checkpoint 1 - dataStatisticsCoordinator.handleEventFromOperator(0, checkpoint1Subtask0DataStatisticEvent); - dataStatisticsCoordinator.handleEventFromOperator(1, checkpoint1Subtask1DataStatisticEvent); - - waitForCoordinatorToProcessActions(dataStatisticsCoordinator); - // Verify global data statistics is the aggregation of all subtasks data statistics - MapDataStatistics globalDataStatistics = - (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(globalDataStatistics.statistics()) - .containsExactlyInAnyOrderEntriesOf( - ImmutableMap.of( - binaryRowDataA, - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataA) - + (long) checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataA), - binaryRowDataB, - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataB) - + (long) checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataB), - binaryRowDataC, - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataC) - + (long) checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataC))); - } - - static void setAllTasksReady( - int subtasks, - DataStatisticsCoordinator> dataStatisticsCoordinator, - EventReceivingTasks receivingTasks) { - for (int i = 0; i < subtasks; i++) { - dataStatisticsCoordinator.subtaskReady(i, receivingTasks.createGatewayForSubtask(i)); - } - } - - static void waitForCoordinatorToProcessActions( - DataStatisticsCoordinator> coordinator) { - CompletableFuture future = new CompletableFuture<>(); - coordinator.callInCoordinatorThread( - () -> { - future.complete(null); - return null; - }, - "Coordinator fails to process action"); - - try { - future.get(); - } catch (InterruptedException e) { - throw new AssertionError("test interrupted"); - } catch (ExecutionException e) { - ExceptionUtils.rethrow(ExceptionUtils.stripExecutionException(e)); - } - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java deleted file mode 100644 index 072332383ada..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java +++ /dev/null @@ -1,147 +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.iceberg.flink.sink.shuffle; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.util.Map; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.runtime.jobgraph.OperatorID; -import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; -import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; -import org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.binary.BinaryRowData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.VarCharType; -import org.junit.Before; -import org.junit.Test; - -public class TestDataStatisticsCoordinatorProvider { - private static final OperatorID OPERATOR_ID = new OperatorID(); - private static final int NUM_SUBTASKS = 1; - - private DataStatisticsCoordinatorProvider> provider; - private EventReceivingTasks receivingTasks; - private TypeSerializer>> - statisticsSerializer; - - @Before - public void before() { - statisticsSerializer = - MapDataStatisticsSerializer.fromKeySerializer( - new RowDataSerializer(RowType.of(new VarCharType()))); - provider = - new DataStatisticsCoordinatorProvider<>( - "DataStatisticsCoordinatorProvider", OPERATOR_ID, statisticsSerializer); - receivingTasks = EventReceivingTasks.createForRunningTasks(); - } - - @Test - @SuppressWarnings("unchecked") - public void testCheckpointAndReset() throws Exception { - RowType rowType = RowType.of(new VarCharType()); - // When coordinator handles events from operator, DataStatisticsUtil#deserializeDataStatistics - // deserializes bytes into BinaryRowData - BinaryRowData binaryRowDataA = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("a"))); - BinaryRowData binaryRowDataB = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("b"))); - BinaryRowData binaryRowDataC = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("c"))); - BinaryRowData binaryRowDataD = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("d"))); - BinaryRowData binaryRowDataE = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("e"))); - - RecreateOnResetOperatorCoordinator coordinator = - (RecreateOnResetOperatorCoordinator) - provider.create(new MockOperatorCoordinatorContext(OPERATOR_ID, NUM_SUBTASKS)); - DataStatisticsCoordinator> dataStatisticsCoordinator = - (DataStatisticsCoordinator>) - coordinator.getInternalCoordinator(); - - // Start the coordinator - coordinator.start(); - TestDataStatisticsCoordinator.setAllTasksReady( - NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); - MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(binaryRowDataA); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - checkpoint1Subtask0DataStatistic.add(binaryRowDataC); - DataStatisticsEvent> - checkpoint1Subtask0DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); - - // Handle events from operators for checkpoint 1 - coordinator.handleEventFromOperator(0, checkpoint1Subtask0DataStatisticEvent); - TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); - // Verify checkpoint 1 global data statistics - MapDataStatistics checkpoint1GlobalDataStatistics = - (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(checkpoint1GlobalDataStatistics.statistics()) - .isEqualTo(checkpoint1Subtask0DataStatistic.statistics()); - byte[] checkpoint1Bytes = waitForCheckpoint(1L, dataStatisticsCoordinator); - - MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(binaryRowDataD); - checkpoint2Subtask0DataStatistic.add(binaryRowDataE); - checkpoint2Subtask0DataStatistic.add(binaryRowDataE); - DataStatisticsEvent> - checkpoint2Subtask0DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); - // Handle events from operators for checkpoint 2 - coordinator.handleEventFromOperator(0, checkpoint2Subtask0DataStatisticEvent); - TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); - // Verify checkpoint 2 global data statistics - MapDataStatistics checkpoint2GlobalDataStatistics = - (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(checkpoint2GlobalDataStatistics.statistics()) - .isEqualTo(checkpoint2Subtask0DataStatistic.statistics()); - waitForCheckpoint(2L, dataStatisticsCoordinator); - - // Reset coordinator to checkpoint 1 - coordinator.resetToCheckpoint(1L, checkpoint1Bytes); - DataStatisticsCoordinator> - restoredDataStatisticsCoordinator = - (DataStatisticsCoordinator>) - coordinator.getInternalCoordinator(); - assertThat(dataStatisticsCoordinator).isNotEqualTo(restoredDataStatisticsCoordinator); - // Verify restored data statistics - MapDataStatistics restoredAggregateDataStatistics = - (MapDataStatistics) - restoredDataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(restoredAggregateDataStatistics.statistics()) - .isEqualTo(checkpoint1GlobalDataStatistics.statistics()); - } - - private byte[] waitForCheckpoint( - long checkpointId, - DataStatisticsCoordinator> coordinator) - throws InterruptedException, ExecutionException { - CompletableFuture future = new CompletableFuture<>(); - coordinator.checkpointCoordinator(checkpointId, future); - return future.get(); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java deleted file mode 100644 index 880cb3d5516f..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java +++ /dev/null @@ -1,227 +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.iceberg.flink.sink.shuffle; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.state.OperatorStateStore; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.fs.CloseableRegistry; -import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; -import org.apache.flink.runtime.execution.Environment; -import org.apache.flink.runtime.operators.coordination.MockOperatorEventGateway; -import org.apache.flink.runtime.operators.testutils.MockEnvironment; -import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; -import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; -import org.apache.flink.runtime.state.AbstractStateBackend; -import org.apache.flink.runtime.state.StateInitializationContext; -import org.apache.flink.runtime.state.StateInitializationContextImpl; -import org.apache.flink.runtime.state.TestTaskStateManager; -import org.apache.flink.runtime.state.hashmap.HashMapStateBackend; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask; -import org.apache.flink.streaming.runtime.tasks.StreamMockEnvironment; -import org.apache.flink.streaming.util.MockOutput; -import org.apache.flink.streaming.util.MockStreamConfig; -import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.binary.BinaryRowData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.VarCharType; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -public class TestDataStatisticsOperator { - private final RowType rowType = RowType.of(new VarCharType()); - private final TypeSerializer rowSerializer = new RowDataSerializer(rowType); - private final GenericRowData genericRowDataA = GenericRowData.of(StringData.fromString("a")); - private final GenericRowData genericRowDataB = GenericRowData.of(StringData.fromString("b")); - // When operator hands events from coordinator, DataStatisticsUtil#deserializeDataStatistics - // deserializes bytes into BinaryRowData - private final BinaryRowData binaryRowDataA = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("a"))); - private final BinaryRowData binaryRowDataB = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("b"))); - private final BinaryRowData binaryRowDataC = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("c"))); - private final TypeSerializer>> - statisticsSerializer = MapDataStatisticsSerializer.fromKeySerializer(rowSerializer); - private DataStatisticsOperator> operator; - - private Environment getTestingEnvironment() { - return new StreamMockEnvironment( - new Configuration(), - new Configuration(), - new ExecutionConfig(), - 1L, - new MockInputSplitProvider(), - 1, - new TestTaskStateManager()); - } - - @Before - public void before() throws Exception { - this.operator = createOperator(); - Environment env = getTestingEnvironment(); - this.operator.setup( - new OneInputStreamTask(env), - new MockStreamConfig(new Configuration(), 1), - new MockOutput<>(Lists.newArrayList())); - } - - private DataStatisticsOperator> createOperator() { - MockOperatorEventGateway mockGateway = new MockOperatorEventGateway(); - KeySelector keySelector = - new KeySelector() { - private static final long serialVersionUID = 7662520075515707428L; - - @Override - public RowData getKey(RowData value) { - return value; - } - }; - - return new DataStatisticsOperator<>( - "testOperator", keySelector, mockGateway, statisticsSerializer); - } - - @After - public void clean() throws Exception { - operator.close(); - } - - @Test - public void testProcessElement() throws Exception { - try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> - testHarness = createHarness(this.operator)) { - StateInitializationContext stateContext = getStateContext(); - operator.initializeState(stateContext); - operator.processElement(new StreamRecord<>(genericRowDataA)); - operator.processElement(new StreamRecord<>(genericRowDataA)); - operator.processElement(new StreamRecord<>(genericRowDataB)); - assertThat(operator.localDataStatistics()).isInstanceOf(MapDataStatistics.class); - MapDataStatistics mapDataStatistics = (MapDataStatistics) operator.localDataStatistics(); - Map statsMap = mapDataStatistics.statistics(); - assertThat(statsMap).hasSize(2); - assertThat(statsMap) - .containsExactlyInAnyOrderEntriesOf( - ImmutableMap.of(genericRowDataA, 2L, genericRowDataB, 1L)); - testHarness.endInput(); - } - } - - @Test - public void testOperatorOutput() throws Exception { - try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> - testHarness = createHarness(this.operator)) { - testHarness.processElement(new StreamRecord<>(genericRowDataA)); - testHarness.processElement(new StreamRecord<>(genericRowDataB)); - testHarness.processElement(new StreamRecord<>(genericRowDataB)); - - List recordsOutput = - testHarness.extractOutputValues().stream() - .filter(DataStatisticsOrRecord::hasRecord) - .map(DataStatisticsOrRecord::record) - .collect(Collectors.toList()); - assertThat(recordsOutput) - .containsExactlyInAnyOrderElementsOf( - ImmutableList.of(genericRowDataA, genericRowDataB, genericRowDataB)); - } - } - - @Test - public void testRestoreState() throws Exception { - OperatorSubtaskState snapshot; - try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> - testHarness1 = createHarness(this.operator)) { - DataStatistics> mapDataStatistics = - new MapDataStatistics(); - mapDataStatistics.add(binaryRowDataA); - mapDataStatistics.add(binaryRowDataA); - mapDataStatistics.add(binaryRowDataB); - mapDataStatistics.add(binaryRowDataC); - operator.handleOperatorEvent( - DataStatisticsEvent.create(0, mapDataStatistics, statisticsSerializer)); - assertThat(operator.globalDataStatistics()).isInstanceOf(MapDataStatistics.class); - assertThat(operator.globalDataStatistics().statistics()) - .containsExactlyInAnyOrderEntriesOf( - ImmutableMap.of(binaryRowDataA, 2L, binaryRowDataB, 1L, binaryRowDataC, 1L)); - snapshot = testHarness1.snapshot(1L, 0); - } - - // Use the snapshot to initialize state for another new operator and then verify that the global - // statistics for the new operator is same as before - DataStatisticsOperator> restoredOperator = - createOperator(); - try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> - testHarness2 = new OneInputStreamOperatorTestHarness<>(restoredOperator, 2, 2, 1)) { - testHarness2.setup(); - testHarness2.initializeState(snapshot); - assertThat(restoredOperator.globalDataStatistics()).isInstanceOf(MapDataStatistics.class); - assertThat(restoredOperator.globalDataStatistics().statistics()) - .containsExactlyInAnyOrderEntriesOf( - ImmutableMap.of(binaryRowDataA, 2L, binaryRowDataB, 1L, binaryRowDataC, 1L)); - } - } - - private StateInitializationContext getStateContext() throws Exception { - MockEnvironment env = new MockEnvironmentBuilder().build(); - AbstractStateBackend abstractStateBackend = new HashMapStateBackend(); - CloseableRegistry cancelStreamRegistry = new CloseableRegistry(); - OperatorStateStore operatorStateStore = - abstractStateBackend.createOperatorStateBackend( - env, "test-operator", Collections.emptyList(), cancelStreamRegistry); - return new StateInitializationContextImpl(null, operatorStateStore, null, null, null); - } - - private OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> - createHarness( - final DataStatisticsOperator> - dataStatisticsOperator) - throws Exception { - - OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> - harness = new OneInputStreamOperatorTestHarness<>(dataStatisticsOperator, 1, 1, 0); - harness.setup( - new DataStatisticsOrRecordSerializer<>( - MapDataStatisticsSerializer.fromKeySerializer(rowSerializer), rowSerializer)); - harness.open(); - return harness; - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java deleted file mode 100644 index a08578a4c106..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java +++ /dev/null @@ -1,170 +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.iceberg.flink.source; - -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Supplier; -import java.util.stream.Stream; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.ProviderContext; -import org.apache.flink.table.connector.source.DataStreamScanProvider; -import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.connector.source.ScanTableSource; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.util.DataFormatConverters; -import org.apache.flink.table.factories.DynamicTableSourceFactory; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.utils.TableSchemaUtils; -import org.apache.flink.types.Row; -import org.apache.flink.types.RowKind; -import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; - -public class BoundedTableFactory implements DynamicTableSourceFactory { - private static final AtomicInteger DATA_SET_ID = new AtomicInteger(0); - private static final Map>> DATA_SETS = Maps.newHashMap(); - - private static final ConfigOption DATA_ID = - ConfigOptions.key("data-id").stringType().noDefaultValue(); - - public static String registerDataSet(List> dataSet) { - String dataSetId = String.valueOf(DATA_SET_ID.incrementAndGet()); - DATA_SETS.put(dataSetId, dataSet); - return dataSetId; - } - - public static void clearDataSets() { - DATA_SETS.clear(); - } - - @Override - public DynamicTableSource createDynamicTableSource(Context context) { - TableSchema tableSchema = - TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema()); - - Configuration configuration = Configuration.fromMap(context.getCatalogTable().getOptions()); - String dataId = configuration.getString(DATA_ID); - Preconditions.checkArgument( - DATA_SETS.containsKey(dataId), "data-id %s does not found in registered data set.", dataId); - - return new BoundedTableSource(DATA_SETS.get(dataId), tableSchema); - } - - @Override - public String factoryIdentifier() { - return "BoundedSource"; - } - - @Override - public Set> requiredOptions() { - return ImmutableSet.of(); - } - - @Override - public Set> optionalOptions() { - return ImmutableSet.of(DATA_ID); - } - - private static class BoundedTableSource implements ScanTableSource { - - private final List> elementsPerCheckpoint; - private final TableSchema tableSchema; - - private BoundedTableSource(List> elementsPerCheckpoint, TableSchema tableSchema) { - this.elementsPerCheckpoint = elementsPerCheckpoint; - this.tableSchema = tableSchema; - } - - private BoundedTableSource(BoundedTableSource toCopy) { - this.elementsPerCheckpoint = toCopy.elementsPerCheckpoint; - this.tableSchema = toCopy.tableSchema; - } - - @Override - public ChangelogMode getChangelogMode() { - Supplier> supplier = () -> elementsPerCheckpoint.stream().flatMap(List::stream); - - // Add the INSERT row kind by default. - ChangelogMode.Builder builder = ChangelogMode.newBuilder().addContainedKind(RowKind.INSERT); - - if (supplier.get().anyMatch(r -> r.getKind() == RowKind.DELETE)) { - builder.addContainedKind(RowKind.DELETE); - } - - if (supplier.get().anyMatch(r -> r.getKind() == RowKind.UPDATE_BEFORE)) { - builder.addContainedKind(RowKind.UPDATE_BEFORE); - } - - if (supplier.get().anyMatch(r -> r.getKind() == RowKind.UPDATE_AFTER)) { - builder.addContainedKind(RowKind.UPDATE_AFTER); - } - - return builder.build(); - } - - @Override - public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) { - return new DataStreamScanProvider() { - @Override - public DataStream produceDataStream( - ProviderContext providerContext, StreamExecutionEnvironment env) { - boolean checkpointEnabled = env.getCheckpointConfig().isCheckpointingEnabled(); - SourceFunction source = - new BoundedTestSource<>(elementsPerCheckpoint, checkpointEnabled); - - RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType(); - // Converter to convert the Row to RowData. - DataFormatConverters.RowConverter rowConverter = - new DataFormatConverters.RowConverter(tableSchema.getFieldDataTypes()); - - return env.addSource(source, new RowTypeInfo(tableSchema.getFieldTypes())) - .map(rowConverter::toInternal, FlinkCompatibilityUtil.toTypeInfo(rowType)); - } - - @Override - public boolean isBounded() { - return true; - } - }; - } - - @Override - public DynamicTableSource copy() { - return new BoundedTableSource(this); - } - - @Override - public String asSummaryString() { - return "Bounded test table source"; - } - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java deleted file mode 100644 index 7b435d059845..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java +++ /dev/null @@ -1,108 +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.iceberg.flink.source; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.concurrent.atomic.AtomicInteger; -import org.apache.flink.api.common.state.CheckpointListener; -import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** - * A stream source that: 1) emits the elements from elementsPerCheckpoint.get(0) without allowing - * checkpoints. 2) then waits for the checkpoint to complete. 3) emits the elements from - * elementsPerCheckpoint.get(1) without allowing checkpoints. 4) then waits for the checkpoint to - * complete. 5) ... - * - *

    Util all the list from elementsPerCheckpoint are exhausted. - */ -public final class BoundedTestSource implements SourceFunction, CheckpointListener { - - private final List> elementsPerCheckpoint; - private final boolean checkpointEnabled; - private volatile boolean running = true; - - private final AtomicInteger numCheckpointsComplete = new AtomicInteger(0); - - /** Emits all those elements in several checkpoints. */ - public BoundedTestSource(List> elementsPerCheckpoint, boolean checkpointEnabled) { - this.elementsPerCheckpoint = elementsPerCheckpoint; - this.checkpointEnabled = checkpointEnabled; - } - - public BoundedTestSource(List> elementsPerCheckpoint) { - this(elementsPerCheckpoint, true); - } - - /** Emits all those elements in a single checkpoint. */ - public BoundedTestSource(T... elements) { - this(Collections.singletonList(Arrays.asList(elements))); - } - - @Override - public void run(SourceContext ctx) throws Exception { - if (!checkpointEnabled) { - Preconditions.checkArgument( - elementsPerCheckpoint.size() <= 1, - "There should be at most one list in the elementsPerCheckpoint when checkpoint is disabled."); - elementsPerCheckpoint.stream().flatMap(List::stream).forEach(ctx::collect); - return; - } - - for (List elements : elementsPerCheckpoint) { - - final int checkpointToAwait; - synchronized (ctx.getCheckpointLock()) { - // Let's say checkpointToAwait = numCheckpointsComplete.get() + delta, in fact the value of - // delta should not - // affect the final table records because we only need to make sure that there will be - // exactly - // elementsPerCheckpoint.size() checkpoints to emit each records buffer from the original - // elementsPerCheckpoint. - // Even if the checkpoints that emitted results are not continuous, the correctness of the - // data should not be - // affected in the end. Setting the delta to be 2 is introducing the variable that produce - // un-continuous - // checkpoints that emit the records buffer from elementsPerCheckpoints. - checkpointToAwait = numCheckpointsComplete.get() + 2; - for (T element : elements) { - ctx.collect(element); - } - } - - synchronized (ctx.getCheckpointLock()) { - while (running && numCheckpointsComplete.get() < checkpointToAwait) { - ctx.getCheckpointLock().wait(1); - } - } - } - } - - @Override - public void notifyCheckpointComplete(long checkpointId) throws Exception { - numCheckpointsComplete.incrementAndGet(); - } - - @Override - public void cancel() { - running = false; - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java deleted file mode 100644 index 7aa2b8034bc5..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java +++ /dev/null @@ -1,87 +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.iceberg.flink.source; - -import java.util.List; -import java.util.stream.Collectors; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; -import org.apache.flink.types.Row; -import org.apache.flink.types.RowKind; -import org.apache.iceberg.flink.FlinkTestBase; -import org.apache.iceberg.flink.MiniClusterResource; -import org.junit.After; -import org.junit.Rule; -import org.junit.rules.TestName; - -public class ChangeLogTableTestBase extends FlinkTestBase { - private volatile TableEnvironment tEnv = null; - - @Rule public TestName name = new TestName(); - - @After - public void clean() { - sql("DROP TABLE IF EXISTS %s", name.getMethodName()); - BoundedTableFactory.clearDataSets(); - } - - @Override - protected TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - if (tEnv == null) { - EnvironmentSettings settings = - EnvironmentSettings.newInstance().inStreamingMode().build(); - - StreamExecutionEnvironment env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) - .enableCheckpointing(400) - .setMaxParallelism(1) - .setParallelism(1); - - tEnv = StreamTableEnvironment.create(env, settings); - } - } - } - return tEnv; - } - - protected static Row insertRow(Object... values) { - return Row.ofKind(RowKind.INSERT, values); - } - - protected static Row deleteRow(Object... values) { - return Row.ofKind(RowKind.DELETE, values); - } - - protected static Row updateBeforeRow(Object... values) { - return Row.ofKind(RowKind.UPDATE_BEFORE, values); - } - - protected static Row updateAfterRow(Object... values) { - return Row.ofKind(RowKind.UPDATE_AFTER, values); - } - - protected static List listJoin(List> lists) { - return lists.stream().flatMap(List::stream).collect(Collectors.toList()); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java deleted file mode 100644 index 3a8071523b7c..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java +++ /dev/null @@ -1,132 +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.iceberg.flink.source; - -import java.io.File; -import java.util.List; -import java.util.stream.Collectors; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.BaseCombinedScanTask; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.hadoop.HadoopCatalog; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.util.ThreadPools; -import org.junit.Assert; -import org.junit.rules.TemporaryFolder; - -public class SplitHelpers { - - private SplitHelpers() {} - - /** - * This create a list of IcebergSourceSplit from real files - *

  • Create a new Hadoop table under the {@code temporaryFolder} - *
  • write {@code fileCount} number of files to the new Iceberg table - *
  • Discover the splits from the table and partition the splits by the {@code filePerSplit} - * limit - *
  • Delete the Hadoop table - * - *

    Since the table and data files are deleted before this method return, caller shouldn't - * attempt to read the data files. - * - *

    By default, v1 Iceberg table is created. For v2 table use {@link - * SplitHelpers#createSplitsFromTransientHadoopTable(TemporaryFolder, int, int, String)} - * - * @param temporaryFolder Folder to place the data to - * @param fileCount The number of files to create and add to the table - * @param filesPerSplit The number of files used for a split - */ - public static List createSplitsFromTransientHadoopTable( - TemporaryFolder temporaryFolder, int fileCount, int filesPerSplit) throws Exception { - return createSplitsFromTransientHadoopTable(temporaryFolder, fileCount, filesPerSplit, "1"); - } - - /** - * This create a list of IcebergSourceSplit from real files - *

  • Create a new Hadoop table under the {@code temporaryFolder} - *
  • write {@code fileCount} number of files to the new Iceberg table - *
  • Discover the splits from the table and partition the splits by the {@code filePerSplit} - * limit - *
  • Delete the Hadoop table - * - *

    Since the table and data files are deleted before this method return, caller shouldn't - * attempt to read the data files. - * - * @param temporaryFolder Folder to place the data to - * @param fileCount The number of files to create and add to the table - * @param filesPerSplit The number of files used for a split - * @param version The table version to create - */ - public static List createSplitsFromTransientHadoopTable( - TemporaryFolder temporaryFolder, int fileCount, int filesPerSplit, String version) - throws Exception { - final File warehouseFile = temporaryFolder.newFolder(); - Assert.assertTrue(warehouseFile.delete()); - final String warehouse = "file:" + warehouseFile; - Configuration hadoopConf = new Configuration(); - final HadoopCatalog catalog = new HadoopCatalog(hadoopConf, warehouse); - ImmutableMap properties = - ImmutableMap.of(TableProperties.FORMAT_VERSION, version); - try { - final Table table = - catalog.createTable( - TestFixtures.TABLE_IDENTIFIER, - TestFixtures.SCHEMA, - PartitionSpec.unpartitioned(), - null, - properties); - final GenericAppenderHelper dataAppender = - new GenericAppenderHelper(table, FileFormat.PARQUET, temporaryFolder); - for (int i = 0; i < fileCount; ++i) { - List records = RandomGenericData.generate(TestFixtures.SCHEMA, 2, i); - dataAppender.appendToTable(records); - } - - final ScanContext scanContext = ScanContext.builder().build(); - final List splits = - FlinkSplitPlanner.planIcebergSourceSplits( - table, scanContext, ThreadPools.getWorkerPool()); - return splits.stream() - .flatMap( - split -> { - List> filesList = - Lists.partition(Lists.newArrayList(split.task().files()), filesPerSplit); - return filesList.stream() - .map(files -> new BaseCombinedScanTask(files)) - .map( - combinedScanTask -> - IcebergSourceSplit.fromCombinedScanTask(combinedScanTask)); - }) - .collect(Collectors.toList()); - } finally { - catalog.dropTable(TestFixtures.TABLE_IDENTIFIER); - catalog.close(); - } - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java deleted file mode 100644 index e4e48ca67f66..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java +++ /dev/null @@ -1,60 +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.iceberg.flink.source; - -import java.util.List; -import java.util.Map; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.TableResult; -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -public class SqlHelpers { - private SqlHelpers() {} - - public static List sql(TableEnvironment tableEnv, String query, Object... args) { - TableResult tableResult = tableEnv.executeSql(String.format(query, args)); - try (CloseableIterator iter = tableResult.collect()) { - List results = Lists.newArrayList(iter); - return results; - } catch (Exception e) { - throw new RuntimeException("Failed to collect table result", e); - } - } - - public static String sqlOptionsToString(Map sqlOptions) { - StringBuilder builder = new StringBuilder(); - sqlOptions.forEach((key, value) -> builder.append(optionToKv(key, value)).append(",")); - String optionStr = builder.toString(); - if (optionStr.endsWith(",")) { - optionStr = optionStr.substring(0, optionStr.length() - 1); - } - - if (!optionStr.isEmpty()) { - optionStr = String.format("/*+ OPTIONS(%s)*/", optionStr); - } - - return optionStr; - } - - private static String optionToKv(String key, Object value) { - return "'" + key + "'='" + value + "'"; - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java deleted file mode 100644 index 7b5f9328694c..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java +++ /dev/null @@ -1,87 +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.iceberg.flink.source; - -import java.util.List; -import java.util.Objects; -import java.util.stream.Collectors; -import org.apache.flink.types.Row; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Streams; -import org.junit.Assert; -import org.junit.Test; - -public class TestBoundedTableFactory extends ChangeLogTableTestBase { - - @Test - public void testEmptyDataSet() { - String table = name.getMethodName(); - List> emptyDataSet = ImmutableList.of(); - - String dataId = BoundedTableFactory.registerDataSet(emptyDataSet); - sql( - "CREATE TABLE %s(id INT, data STRING) WITH ('connector'='BoundedSource', 'data-id'='%s')", - table, dataId); - - Assert.assertEquals( - "Should have caught empty change log set.", - ImmutableList.of(), - sql("SELECT * FROM %s", table)); - } - - @Test - public void testBoundedTableFactory() { - String table = name.getMethodName(); - List> dataSet = - ImmutableList.of( - ImmutableList.of( - insertRow(1, "aaa"), - deleteRow(1, "aaa"), - insertRow(1, "bbb"), - insertRow(2, "aaa"), - deleteRow(2, "aaa"), - insertRow(2, "bbb")), - ImmutableList.of( - updateBeforeRow(2, "bbb"), - updateAfterRow(2, "ccc"), - deleteRow(2, "ccc"), - insertRow(2, "ddd")), - ImmutableList.of( - deleteRow(1, "bbb"), - insertRow(1, "ccc"), - deleteRow(1, "ccc"), - insertRow(1, "ddd"))); - - String dataId = BoundedTableFactory.registerDataSet(dataSet); - sql( - "CREATE TABLE %s(id INT, data STRING) WITH ('connector'='BoundedSource', 'data-id'='%s')", - table, dataId); - - List rowSet = dataSet.stream().flatMap(Streams::stream).collect(Collectors.toList()); - Assert.assertEquals( - "Should have the expected change log events.", rowSet, sql("SELECT * FROM %s", table)); - - Assert.assertEquals( - "Should have the expected change log events", - rowSet.stream() - .filter(r -> Objects.equals(r.getField(1), "aaa")) - .collect(Collectors.toList()), - sql("SELECT * FROM %s WHERE data='aaa'", table)); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java deleted file mode 100644 index 73d03710d32c..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java +++ /dev/null @@ -1,193 +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.iceberg.flink.source; - -import static org.apache.iceberg.types.Types.NestedField.required; - -import java.io.IOException; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.types.Row; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Types; -import org.junit.Assume; -import org.junit.Test; - -/** Test {@link FlinkInputFormat}. */ -public class TestFlinkInputFormat extends TestFlinkSource { - - public TestFlinkInputFormat(String fileFormat) { - super(fileFormat); - } - - @Override - protected List run( - FlinkSource.Builder formatBuilder, - Map sqlOptions, - String sqlFilter, - String... sqlSelectedFields) - throws Exception { - return runFormat(formatBuilder.tableLoader(tableLoader()).buildFormat()); - } - - @Test - public void testNestedProjection() throws Exception { - Schema schema = - new Schema( - required(1, "data", Types.StringType.get()), - required( - 2, - "nested", - Types.StructType.of( - Types.NestedField.required(3, "f1", Types.StringType.get()), - Types.NestedField.required(4, "f2", Types.StringType.get()), - Types.NestedField.required(5, "f3", Types.LongType.get()))), - required(6, "id", Types.LongType.get())); - - Table table = catalogResource.catalog().createTable(TableIdentifier.of("default", "t"), schema); - - List writeRecords = RandomGenericData.generate(schema, 2, 0L); - new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(writeRecords); - - // Schema: [data, nested[f1, f2, f3], id] - // Projection: [nested.f2, data] - // The Flink SQL output: [f2, data] - // The FlinkInputFormat output: [nested[f2], data] - - TableSchema projectedSchema = - TableSchema.builder() - .field("nested", DataTypes.ROW(DataTypes.FIELD("f2", DataTypes.STRING()))) - .field("data", DataTypes.STRING()) - .build(); - List result = - runFormat( - FlinkSource.forRowData() - .tableLoader(tableLoader()) - .project(projectedSchema) - .buildFormat()); - - List expected = Lists.newArrayList(); - for (Record record : writeRecords) { - Row nested = Row.of(((Record) record.get(1)).get(1)); - expected.add(Row.of(nested, record.get(0))); - } - - TestHelpers.assertRows(result, expected); - } - - @Test - public void testBasicProjection() throws IOException { - Schema writeSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get()), - Types.NestedField.optional(2, "time", Types.TimestampType.withZone())); - - Table table = - catalogResource.catalog().createTable(TableIdentifier.of("default", "t"), writeSchema); - - List writeRecords = RandomGenericData.generate(writeSchema, 2, 0L); - new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(writeRecords); - - TableSchema projectedSchema = - TableSchema.builder() - .field("id", DataTypes.BIGINT()) - .field("data", DataTypes.STRING()) - .build(); - List result = - runFormat( - FlinkSource.forRowData() - .tableLoader(tableLoader()) - .project(projectedSchema) - .buildFormat()); - - List expected = Lists.newArrayList(); - for (Record record : writeRecords) { - expected.add(Row.of(record.get(0), record.get(1))); - } - - TestHelpers.assertRows(result, expected); - } - - @Test - public void testReadPartitionColumn() throws Exception { - Assume.assumeTrue("Temporary skip ORC", FileFormat.ORC != fileFormat); - - Schema nestedSchema = - new Schema( - Types.NestedField.optional(1, "id", Types.LongType.get()), - Types.NestedField.optional( - 2, - "struct", - Types.StructType.of( - Types.NestedField.optional(3, "innerId", Types.LongType.get()), - Types.NestedField.optional(4, "innerName", Types.StringType.get())))); - PartitionSpec spec = - PartitionSpec.builderFor(nestedSchema).identity("struct.innerName").build(); - - Table table = - catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, nestedSchema, spec); - List records = RandomGenericData.generate(nestedSchema, 10, 0L); - GenericAppenderHelper appender = new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER); - for (Record record : records) { - org.apache.iceberg.TestHelpers.Row partition = - org.apache.iceberg.TestHelpers.Row.of(record.get(1, Record.class).get(1)); - appender.appendToTable(partition, Collections.singletonList(record)); - } - - TableSchema projectedSchema = - TableSchema.builder() - .field("struct", DataTypes.ROW(DataTypes.FIELD("innerName", DataTypes.STRING()))) - .build(); - List result = - runFormat( - FlinkSource.forRowData() - .tableLoader(tableLoader()) - .project(projectedSchema) - .buildFormat()); - - List expected = Lists.newArrayList(); - for (Record record : records) { - Row nested = Row.of(((Record) record.get(1)).get(1)); - expected.add(Row.of(nested)); - } - - TestHelpers.assertRows(result, expected); - } - - private List runFormat(FlinkInputFormat inputFormat) throws IOException { - RowType rowType = FlinkSchemaUtil.convert(inputFormat.projectedSchema()); - return TestHelpers.readRows(inputFormat, rowType); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java deleted file mode 100644 index b2f914e51299..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java +++ /dev/null @@ -1,76 +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.iceberg.flink.source; - -import java.io.IOException; -import java.util.Map; -import org.apache.flink.table.types.logical.RowType; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.flink.CatalogLoader; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.util.StructLikeSet; - -public class TestFlinkInputFormatReaderDeletes extends TestFlinkReaderDeletesBase { - - public TestFlinkInputFormatReaderDeletes(FileFormat inputFormat) { - super(inputFormat); - } - - @Override - protected StructLikeSet rowSet(String tableName, Table testTable, String... columns) - throws IOException { - Schema projected = testTable.schema().select(columns); - RowType rowType = FlinkSchemaUtil.convert(projected); - Map properties = Maps.newHashMap(); - properties.put( - CatalogProperties.WAREHOUSE_LOCATION, - hiveConf.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname)); - properties.put(CatalogProperties.URI, hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname)); - properties.put( - CatalogProperties.CLIENT_POOL_SIZE, - Integer.toString(hiveConf.getInt("iceberg.hive.client-pool-size", 5))); - CatalogLoader hiveCatalogLoader = CatalogLoader.hive(catalog.name(), hiveConf, properties); - FlinkInputFormat inputFormat = - FlinkSource.forRowData() - .tableLoader( - TableLoader.fromCatalog( - hiveCatalogLoader, TableIdentifier.of("default", tableName))) - .project(FlinkSchemaUtil.toSchema(rowType)) - .buildFormat(); - - StructLikeSet set = StructLikeSet.create(projected.asStruct()); - TestHelpers.readRowData(inputFormat, rowType) - .forEach( - rowData -> { - RowDataWrapper wrapper = new RowDataWrapper(rowType, projected.asStruct()); - set.add(wrapper.wrap(rowData)); - }); - - return set; - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java deleted file mode 100644 index 1d52acb2fe7b..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java +++ /dev/null @@ -1,71 +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.iceberg.flink.source; - -import java.io.IOException; -import java.util.List; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.TestMergingMetrics; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.HadoopTableResource; -import org.apache.iceberg.flink.RowDataConverter; -import org.apache.iceberg.flink.sink.FlinkAppenderFactory; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.rules.TemporaryFolder; - -public class TestFlinkMergingMetrics extends TestMergingMetrics { - - @ClassRule public static final TemporaryFolder TEMP_FOLDER = new TemporaryFolder(); - - @Rule - public final HadoopTableResource tableResource = - new HadoopTableResource(TEMP_FOLDER, "test_db", "test_table", SCHEMA); - - public TestFlinkMergingMetrics(FileFormat fileFormat) { - super(fileFormat); - } - - @Override - protected FileAppender writeAndGetAppender(List records) throws IOException { - RowType flinkSchema = FlinkSchemaUtil.convert(SCHEMA); - - FileAppender appender = - new FlinkAppenderFactory( - tableResource.table(), - SCHEMA, - flinkSchema, - ImmutableMap.of(), - PartitionSpec.unpartitioned(), - null, - null, - null) - .newAppender(org.apache.iceberg.Files.localOutput(temp.newFile()), fileFormat); - try (FileAppender fileAppender = appender) { - records.stream().map(r -> RowDataConverter.convert(SCHEMA, r)).forEach(fileAppender::add); - } - return appender; - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java deleted file mode 100644 index 5ecf4f4536bb..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java +++ /dev/null @@ -1,829 +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.iceberg.flink.source; - -import java.io.IOException; -import java.time.Instant; -import java.util.Comparator; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import org.apache.avro.generic.GenericData; -import org.apache.commons.collections.ListUtils; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.types.Row; -import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.FileContent; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Files; -import org.apache.iceberg.HasTableOperations; -import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.MetadataTableType; -import org.apache.iceberg.MetadataTableUtils; -import org.apache.iceberg.MetricsUtil; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableMetadata; -import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.data.FileHelpers; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.flink.FlinkCatalogTestBase; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.InputFile; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.SnapshotUtil; -import org.junit.After; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Before; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestFlinkMetaDataTable extends FlinkCatalogTestBase { - private static final String TABLE_NAME = "test_table"; - private final FileFormat format = FileFormat.AVRO; - private static final TemporaryFolder TEMP = new TemporaryFolder(); - private final boolean isPartition; - - public TestFlinkMetaDataTable(String catalogName, Namespace baseNamespace, Boolean isPartition) { - super(catalogName, baseNamespace); - this.isPartition = isPartition; - } - - @Parameterized.Parameters(name = "catalogName={0}, baseNamespace={1}, isPartition={2}") - public static Iterable parameters() { - List parameters = Lists.newArrayList(); - - for (Boolean isPartition : new Boolean[] {true, false}) { - String catalogName = "testhadoop"; - Namespace baseNamespace = Namespace.of("default"); - parameters.add(new Object[] {catalogName, baseNamespace, isPartition}); - } - return parameters; - } - - @Override - protected TableEnvironment getTableEnv() { - Configuration configuration = super.getTableEnv().getConfig().getConfiguration(); - configuration.set(CoreOptions.DEFAULT_PARALLELISM, 1); - return super.getTableEnv(); - } - - @Before - public void before() { - super.before(); - sql("USE CATALOG %s", catalogName); - sql("CREATE DATABASE %s", flinkDatabase); - sql("USE %s", DATABASE); - if (isPartition) { - sql( - "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) PARTITIONED BY (data) WITH ('format-version'='2', 'write.format.default'='%s')", - TABLE_NAME, format.name()); - sql("INSERT INTO %s VALUES (1,'a',10),(2,'a',20)", TABLE_NAME); - sql("INSERT INTO %s VALUES (1,'b',10),(2,'b',20)", TABLE_NAME); - } else { - sql( - "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('format-version'='2', 'write.format.default'='%s')", - TABLE_NAME, format.name()); - sql( - "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)", - TABLE_NAME); - sql("INSERT INTO %s VALUES (4,'iceberg',10)", TABLE_NAME); - } - } - - @Override - @After - public void clean() { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); - super.clean(); - } - - @Test - public void testSnapshots() { - String sql = String.format("SELECT * FROM %s$snapshots ", TABLE_NAME); - List result = sql(sql); - - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); - Iterator snapshots = table.snapshots().iterator(); - for (Row row : result) { - Snapshot next = snapshots.next(); - Assert.assertEquals( - "Should have expected timestamp", - ((Instant) row.getField(0)).toEpochMilli(), - next.timestampMillis()); - Assert.assertEquals("Should have expected snapshot id", next.snapshotId(), row.getField(1)); - Assert.assertEquals("Should have expected parent id", next.parentId(), row.getField(2)); - Assert.assertEquals("Should have expected operation", next.operation(), row.getField(3)); - Assert.assertEquals( - "Should have expected manifest list location", - row.getField(4), - next.manifestListLocation()); - Assert.assertEquals("Should have expected summary", next.summary(), row.getField(5)); - } - } - - @Test - public void testHistory() { - String sql = String.format("SELECT * FROM %s$history ", TABLE_NAME); - List result = sql(sql); - - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); - Iterator snapshots = table.snapshots().iterator(); - for (Row row : result) { - Snapshot next = snapshots.next(); - Assert.assertEquals( - "Should have expected made_current_at", - ((Instant) row.getField(0)).toEpochMilli(), - next.timestampMillis()); - Assert.assertEquals("Should have expected snapshot id", next.snapshotId(), row.getField(1)); - Assert.assertEquals("Should have expected parent id", next.parentId(), row.getField(2)); - - Assert.assertEquals( - "Should have expected is current ancestor", - SnapshotUtil.isAncestorOf(table, table.currentSnapshot().snapshotId(), next.snapshotId()), - row.getField(3)); - } - } - - @Test - public void testManifests() { - String sql = String.format("SELECT * FROM %s$manifests ", TABLE_NAME); - List result = sql(sql); - - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); - List expectedDataManifests = dataManifests(table); - - for (int i = 0; i < result.size(); i++) { - Row row = result.get(i); - ManifestFile manifestFile = expectedDataManifests.get(i); - Assert.assertEquals( - "Should have expected content", manifestFile.content().id(), row.getField(0)); - Assert.assertEquals("Should have expected path", manifestFile.path(), row.getField(1)); - Assert.assertEquals("Should have expected length", manifestFile.length(), row.getField(2)); - Assert.assertEquals( - "Should have expected partition_spec_id", - manifestFile.partitionSpecId(), - row.getField(3)); - Assert.assertEquals( - "Should have expected added_snapshot_id", manifestFile.snapshotId(), row.getField(4)); - Assert.assertEquals( - "Should have expected added_data_files_count", - manifestFile.addedFilesCount(), - row.getField(5)); - Assert.assertEquals( - "Should have expected existing_data_files_count", - manifestFile.existingFilesCount(), - row.getField(6)); - Assert.assertEquals( - "Should have expected deleted_data_files_count", - manifestFile.deletedFilesCount(), - row.getField(7)); - } - } - - @Test - public void testAllManifests() { - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); - - String sql = String.format("SELECT * FROM %s$all_manifests ", TABLE_NAME); - List result = sql(sql); - - List expectedDataManifests = allDataManifests(table); - - Assert.assertEquals(expectedDataManifests.size(), result.size()); - for (int i = 0; i < result.size(); i++) { - Row row = result.get(i); - ManifestFile manifestFile = expectedDataManifests.get(i); - Assert.assertEquals( - "Should have expected content", manifestFile.content().id(), row.getField(0)); - Assert.assertEquals("Should have expected path", manifestFile.path(), row.getField(1)); - Assert.assertEquals("Should have expected length", manifestFile.length(), row.getField(2)); - Assert.assertEquals( - "Should have expected partition_spec_id", - manifestFile.partitionSpecId(), - row.getField(3)); - Assert.assertEquals( - "Should have expected added_snapshot_id", manifestFile.snapshotId(), row.getField(4)); - Assert.assertEquals( - "Should have expected added_data_files_count", - manifestFile.addedFilesCount(), - row.getField(5)); - Assert.assertEquals( - "Should have expected existing_data_files_count", - manifestFile.existingFilesCount(), - row.getField(6)); - Assert.assertEquals( - "Should have expected deleted_data_files_count", - manifestFile.deletedFilesCount(), - row.getField(7)); - } - } - - @Test - public void testUnPartitionedTable() throws IOException { - Assume.assumeFalse(isPartition); - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); - - Schema deleteRowSchema = table.schema().select("id"); - Record dataDelete = GenericRecord.create(deleteRowSchema); - List dataDeletes = Lists.newArrayList(dataDelete.copy("id", 1)); - - TEMP.create(); - DeleteFile eqDeletes = - FileHelpers.writeDeleteFile( - table, Files.localOutput(TEMP.newFile()), dataDeletes, deleteRowSchema); - table.newRowDelta().addDeletes(eqDeletes).commit(); - - List expectedDataManifests = dataManifests(table); - List expectedDeleteManifests = deleteManifests(table); - - Assert.assertEquals("Should have 2 data manifest", 2, expectedDataManifests.size()); - Assert.assertEquals("Should have 1 delete manifest", 1, expectedDeleteManifests.size()); - - Schema entriesTableSchema = - MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.from("entries")) - .schema(); - - // check delete files table - Schema deleteFilesTableSchema = - MetadataTableUtils.createMetadataTableInstance( - table, MetadataTableType.from("delete_files")) - .schema(); - - List deleteColumns = - deleteFilesTableSchema.columns().stream() - .map(Types.NestedField::name) - .filter(c -> !c.equals(MetricsUtil.READABLE_METRICS)) - .collect(Collectors.toList()); - String deleteNames = - deleteColumns.stream().map(n -> "`" + n + "`").collect(Collectors.joining(",")); - - deleteFilesTableSchema = deleteFilesTableSchema.select(deleteColumns); - - List actualDeleteFiles = sql("SELECT %s FROM %s$delete_files", deleteNames, TABLE_NAME); - Assert.assertEquals("Metadata table should return 1 delete file", 1, actualDeleteFiles.size()); - - List expectedDeleteFiles = - expectedEntries( - table, FileContent.EQUALITY_DELETES, entriesTableSchema, expectedDeleteManifests, null); - Assert.assertEquals("Should be 1 delete file manifest entry", 1, expectedDeleteFiles.size()); - TestHelpers.assertEquals( - deleteFilesTableSchema, expectedDeleteFiles.get(0), actualDeleteFiles.get(0)); - - // Check data files table - Schema filesTableSchema = - MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.from("files")) - .schema(); - - List columns = - filesTableSchema.columns().stream() - .map(Types.NestedField::name) - .filter(c -> !c.equals(MetricsUtil.READABLE_METRICS)) - .collect(Collectors.toList()); - String names = columns.stream().map(n -> "`" + n + "`").collect(Collectors.joining(",")); - - filesTableSchema = filesTableSchema.select(columns); - - List actualDataFiles = sql("SELECT %s FROM %s$data_files", names, TABLE_NAME); - Assert.assertEquals("Metadata table should return 2 data file", 2, actualDataFiles.size()); - - List expectedDataFiles = - expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, null); - Assert.assertEquals("Should be 2 data file manifest entry", 2, expectedDataFiles.size()); - TestHelpers.assertEquals(filesTableSchema, expectedDataFiles.get(0), actualDataFiles.get(0)); - - // check all files table - List actualFiles = sql("SELECT %s FROM %s$files ORDER BY content", names, TABLE_NAME); - Assert.assertEquals("Metadata table should return 3 files", 3, actualFiles.size()); - - List expectedFiles = - Stream.concat(expectedDataFiles.stream(), expectedDeleteFiles.stream()) - .collect(Collectors.toList()); - Assert.assertEquals("Should have 3 files manifest entries", 3, expectedFiles.size()); - TestHelpers.assertEquals(filesTableSchema, expectedFiles.get(0), actualFiles.get(0)); - TestHelpers.assertEquals(filesTableSchema, expectedFiles.get(1), actualFiles.get(1)); - } - - @Test - public void testPartitionedTable() throws Exception { - Assume.assumeFalse(!isPartition); - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); - - Schema deleteRowSchema = table.schema().select("id", "data"); - Record dataDelete = GenericRecord.create(deleteRowSchema); - TEMP.create(); - - Map deleteRow = Maps.newHashMap(); - deleteRow.put("id", 1); - deleteRow.put("data", "a"); - DeleteFile eqDeletes = - FileHelpers.writeDeleteFile( - table, - Files.localOutput(TEMP.newFile()), - org.apache.iceberg.TestHelpers.Row.of("a"), - Lists.newArrayList(dataDelete.copy(deleteRow)), - deleteRowSchema); - table.newRowDelta().addDeletes(eqDeletes).commit(); - - deleteRow.put("data", "b"); - DeleteFile eqDeletes2 = - FileHelpers.writeDeleteFile( - table, - Files.localOutput(TEMP.newFile()), - org.apache.iceberg.TestHelpers.Row.of("b"), - Lists.newArrayList(dataDelete.copy(deleteRow)), - deleteRowSchema); - table.newRowDelta().addDeletes(eqDeletes2).commit(); - - Schema entriesTableSchema = - MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.from("entries")) - .schema(); - - List expectedDataManifests = dataManifests(table); - List expectedDeleteManifests = deleteManifests(table); - - Assert.assertEquals("Should have 2 data manifests", 2, expectedDataManifests.size()); - Assert.assertEquals("Should have 2 delete manifests", 2, expectedDeleteManifests.size()); - - Table deleteFilesTable = - MetadataTableUtils.createMetadataTableInstance( - table, MetadataTableType.from("delete_files")); - Schema filesTableSchema = deleteFilesTable.schema(); - - List columns = - filesTableSchema.columns().stream() - .map(Types.NestedField::name) - .filter(c -> !c.equals(MetricsUtil.READABLE_METRICS)) - .collect(Collectors.toList()); - String names = columns.stream().map(n -> "`" + n + "`").collect(Collectors.joining(",")); - - filesTableSchema = filesTableSchema.select(columns); - - // Check delete files table - List expectedDeleteFiles = - expectedEntries( - table, FileContent.EQUALITY_DELETES, entriesTableSchema, expectedDeleteManifests, "a"); - Assert.assertEquals( - "Should have one delete file manifest entry", 1, expectedDeleteFiles.size()); - - List actualDeleteFiles = - sql("SELECT %s FROM %s$delete_files WHERE `partition`.`data`='a'", names, TABLE_NAME); - - Assert.assertEquals( - "Metadata table should return one delete file", 1, actualDeleteFiles.size()); - TestHelpers.assertEquals( - filesTableSchema, expectedDeleteFiles.get(0), actualDeleteFiles.get(0)); - - // Check data files table - List expectedDataFiles = - expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, "a"); - Assert.assertEquals("Should have one data file manifest entry", 1, expectedDataFiles.size()); - - List actualDataFiles = - sql("SELECT %s FROM %s$data_files WHERE `partition`.`data`='a'", names, TABLE_NAME); - Assert.assertEquals("Metadata table should return one data file", 1, actualDataFiles.size()); - TestHelpers.assertEquals(filesTableSchema, expectedDataFiles.get(0), actualDataFiles.get(0)); - - List actualPartitionsWithProjection = - sql("SELECT file_count FROM %s$partitions ", TABLE_NAME); - Assert.assertEquals( - "Metadata table should return two partitions record", - 2, - actualPartitionsWithProjection.size()); - for (int i = 0; i < 2; ++i) { - Assert.assertEquals(1, actualPartitionsWithProjection.get(i).getField(0)); - } - - // Check files table - List expectedFiles = - Stream.concat(expectedDataFiles.stream(), expectedDeleteFiles.stream()) - .collect(Collectors.toList()); - Assert.assertEquals("Should have two file manifest entries", 2, expectedFiles.size()); - - List actualFiles = - sql( - "SELECT %s FROM %s$files WHERE `partition`.`data`='a' ORDER BY content", - names, TABLE_NAME); - Assert.assertEquals("Metadata table should return two files", 2, actualFiles.size()); - TestHelpers.assertEquals(filesTableSchema, expectedFiles.get(0), actualFiles.get(0)); - TestHelpers.assertEquals(filesTableSchema, expectedFiles.get(1), actualFiles.get(1)); - } - - @Test - public void testAllFilesUnpartitioned() throws Exception { - Assume.assumeFalse(isPartition); - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); - - Schema deleteRowSchema = table.schema().select("id", "data"); - Record dataDelete = GenericRecord.create(deleteRowSchema); - TEMP.create(); - - Map deleteRow = Maps.newHashMap(); - deleteRow.put("id", 1); - DeleteFile eqDeletes = - FileHelpers.writeDeleteFile( - table, - Files.localOutput(TEMP.newFile()), - Lists.newArrayList(dataDelete.copy(deleteRow)), - deleteRowSchema); - table.newRowDelta().addDeletes(eqDeletes).commit(); - - List expectedDataManifests = dataManifests(table); - Assert.assertEquals("Should have 2 data manifest", 2, expectedDataManifests.size()); - List expectedDeleteManifests = deleteManifests(table); - Assert.assertEquals("Should have 1 delete manifest", 1, expectedDeleteManifests.size()); - - // Clear table to test whether 'all_files' can read past files - table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit(); - - Schema entriesTableSchema = - MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.from("entries")) - .schema(); - Schema filesTableSchema = - MetadataTableUtils.createMetadataTableInstance( - table, MetadataTableType.from("all_data_files")) - .schema(); - - List columns = - filesTableSchema.columns().stream() - .map(Types.NestedField::name) - .filter(c -> !c.equals(MetricsUtil.READABLE_METRICS)) - .collect(Collectors.toList()); - String names = columns.stream().map(n -> "`" + n + "`").collect(Collectors.joining(",")); - - filesTableSchema = filesTableSchema.select(columns); - - // Check all data files table - List actualDataFiles = - sql("SELECT %s FROM %s$all_data_files order by record_count ", names, TABLE_NAME); - - List expectedDataFiles = - expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, null); - Assert.assertEquals("Should be 2 data file manifest entry", 2, expectedDataFiles.size()); - Assert.assertEquals("Metadata table should return 2 data file", 2, actualDataFiles.size()); - TestHelpers.assertEquals(filesTableSchema, expectedDataFiles, actualDataFiles); - - // Check all delete files table - List actualDeleteFiles = sql("SELECT %s FROM %s$all_delete_files", names, TABLE_NAME); - List expectedDeleteFiles = - expectedEntries( - table, FileContent.EQUALITY_DELETES, entriesTableSchema, expectedDeleteManifests, null); - Assert.assertEquals("Should be one delete file manifest entry", 1, expectedDeleteFiles.size()); - Assert.assertEquals( - "Metadata table should return one delete file", 1, actualDeleteFiles.size()); - TestHelpers.assertEquals( - filesTableSchema, expectedDeleteFiles.get(0), actualDeleteFiles.get(0)); - - // Check all files table - List actualFiles = - sql("SELECT %s FROM %s$all_files ORDER BY content, record_count asc", names, TABLE_NAME); - List expectedFiles = - ListUtils.union(expectedDataFiles, expectedDeleteFiles); - expectedFiles.sort(Comparator.comparing(r -> ((Integer) r.get("content")))); - Assert.assertEquals("Metadata table should return 3 files", 3, actualFiles.size()); - TestHelpers.assertEquals(filesTableSchema, expectedFiles, actualFiles); - } - - @Test - public void testAllFilesPartitioned() throws Exception { - Assume.assumeFalse(!isPartition); - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); - - // Create delete file - Schema deleteRowSchema = table.schema().select("id"); - Record dataDelete = GenericRecord.create(deleteRowSchema); - TEMP.create(); - - Map deleteRow = Maps.newHashMap(); - deleteRow.put("id", 1); - DeleteFile eqDeletes = - FileHelpers.writeDeleteFile( - table, - Files.localOutput(TEMP.newFile()), - org.apache.iceberg.TestHelpers.Row.of("a"), - Lists.newArrayList(dataDelete.copy(deleteRow)), - deleteRowSchema); - DeleteFile eqDeletes2 = - FileHelpers.writeDeleteFile( - table, - Files.localOutput(TEMP.newFile()), - org.apache.iceberg.TestHelpers.Row.of("b"), - Lists.newArrayList(dataDelete.copy(deleteRow)), - deleteRowSchema); - table.newRowDelta().addDeletes(eqDeletes).addDeletes(eqDeletes2).commit(); - - List expectedDataManifests = dataManifests(table); - Assert.assertEquals("Should have 2 data manifests", 2, expectedDataManifests.size()); - List expectedDeleteManifests = deleteManifests(table); - Assert.assertEquals("Should have 1 delete manifest", 1, expectedDeleteManifests.size()); - - // Clear table to test whether 'all_files' can read past files - table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit(); - - Schema entriesTableSchema = - MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.from("entries")) - .schema(); - Schema filesTableSchema = - MetadataTableUtils.createMetadataTableInstance( - table, MetadataTableType.from("all_data_files")) - .schema(); - - List columns = - filesTableSchema.columns().stream() - .map(Types.NestedField::name) - .filter(c -> !c.equals(MetricsUtil.READABLE_METRICS)) - .collect(Collectors.toList()); - String names = columns.stream().map(n -> "`" + n + "`").collect(Collectors.joining(",")); - - filesTableSchema = filesTableSchema.select(columns); - - // Check all data files table - List actualDataFiles = - sql("SELECT %s FROM %s$all_data_files WHERE `partition`.`data`='a'", names, TABLE_NAME); - List expectedDataFiles = - expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, "a"); - Assert.assertEquals("Should be one data file manifest entry", 1, expectedDataFiles.size()); - Assert.assertEquals("Metadata table should return one data file", 1, actualDataFiles.size()); - TestHelpers.assertEquals(filesTableSchema, expectedDataFiles.get(0), actualDataFiles.get(0)); - - // Check all delete files table - List actualDeleteFiles = - sql("SELECT %s FROM %s$all_delete_files WHERE `partition`.`data`='a'", names, TABLE_NAME); - List expectedDeleteFiles = - expectedEntries( - table, FileContent.EQUALITY_DELETES, entriesTableSchema, expectedDeleteManifests, "a"); - Assert.assertEquals("Should be one data file manifest entry", 1, expectedDeleteFiles.size()); - Assert.assertEquals("Metadata table should return one data file", 1, actualDeleteFiles.size()); - - TestHelpers.assertEquals( - filesTableSchema, expectedDeleteFiles.get(0), actualDeleteFiles.get(0)); - - // Check all files table - List actualFiles = - sql( - "SELECT %s FROM %s$all_files WHERE `partition`.`data`='a' ORDER BY content", - names, TABLE_NAME); - List expectedFiles = - ListUtils.union(expectedDataFiles, expectedDeleteFiles); - expectedFiles.sort(Comparator.comparing(r -> ((Integer) r.get("content")))); - Assert.assertEquals("Metadata table should return two files", 2, actualFiles.size()); - TestHelpers.assertEquals(filesTableSchema, expectedFiles, actualFiles); - } - - @Test - public void testMetadataLogEntries() { - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); - - Long currentSnapshotId = table.currentSnapshot().snapshotId(); - TableMetadata tableMetadata = ((HasTableOperations) table).operations().current(); - Snapshot currentSnapshot = tableMetadata.currentSnapshot(); - Snapshot parentSnapshot = table.snapshot(currentSnapshot.parentId()); - List metadataLogEntries = - Lists.newArrayList(tableMetadata.previousFiles()); - - // Check metadataLog table - List metadataLogs = sql("SELECT * FROM %s$metadata_log_entries", TABLE_NAME); - - Assert.assertEquals("metadataLogEntries table should return 3 row", 3, metadataLogs.size()); - Row metadataLog = metadataLogs.get(0); - Assert.assertEquals( - Instant.ofEpochMilli(metadataLogEntries.get(0).timestampMillis()), - metadataLog.getField("timestamp")); - Assert.assertEquals(metadataLogEntries.get(0).file(), metadataLog.getField("file")); - Assert.assertNull(metadataLog.getField("latest_snapshot_id")); - Assert.assertNull(metadataLog.getField("latest_schema_id")); - Assert.assertNull(metadataLog.getField("latest_sequence_number")); - - metadataLog = metadataLogs.get(1); - Assert.assertEquals( - Instant.ofEpochMilli(metadataLogEntries.get(1).timestampMillis()), - metadataLog.getField("timestamp")); - Assert.assertEquals(metadataLogEntries.get(1).file(), metadataLog.getField("file")); - Assert.assertEquals(parentSnapshot.snapshotId(), metadataLog.getField("latest_snapshot_id")); - Assert.assertEquals(parentSnapshot.schemaId(), metadataLog.getField("latest_schema_id")); - Assert.assertEquals( - parentSnapshot.sequenceNumber(), metadataLog.getField("latest_sequence_number")); - - metadataLog = metadataLogs.get(2); - Assert.assertEquals( - Instant.ofEpochMilli(currentSnapshot.timestampMillis()), metadataLog.getField("timestamp")); - Assert.assertEquals(tableMetadata.metadataFileLocation(), metadataLog.getField("file")); - Assert.assertEquals(currentSnapshot.snapshotId(), metadataLog.getField("latest_snapshot_id")); - Assert.assertEquals(currentSnapshot.schemaId(), metadataLog.getField("latest_schema_id")); - Assert.assertEquals( - currentSnapshot.sequenceNumber(), metadataLog.getField("latest_sequence_number")); - - // test filtering - List metadataLogWithFilters = - sql( - "SELECT * FROM %s$metadata_log_entries WHERE latest_snapshot_id = %s", - TABLE_NAME, currentSnapshotId); - Assert.assertEquals( - "metadataLogEntries table should return 1 row", 1, metadataLogWithFilters.size()); - - metadataLog = metadataLogWithFilters.get(0); - Assert.assertEquals( - Instant.ofEpochMilli(tableMetadata.currentSnapshot().timestampMillis()), - metadataLog.getField("timestamp")); - Assert.assertEquals(tableMetadata.metadataFileLocation(), metadataLog.getField("file")); - Assert.assertEquals( - tableMetadata.currentSnapshot().snapshotId(), metadataLog.getField("latest_snapshot_id")); - Assert.assertEquals( - tableMetadata.currentSnapshot().schemaId(), metadataLog.getField("latest_schema_id")); - Assert.assertEquals( - tableMetadata.currentSnapshot().sequenceNumber(), - metadataLog.getField("latest_sequence_number")); - - // test projection - List metadataFiles = - metadataLogEntries.stream() - .map(TableMetadata.MetadataLogEntry::file) - .collect(Collectors.toList()); - metadataFiles.add(tableMetadata.metadataFileLocation()); - List metadataLogWithProjection = - sql("SELECT file FROM %s$metadata_log_entries", TABLE_NAME); - Assert.assertEquals( - "metadataLogEntries table should return 3 rows", 3, metadataLogWithProjection.size()); - for (int i = 0; i < metadataFiles.size(); i++) { - Assert.assertEquals(metadataFiles.get(i), metadataLogWithProjection.get(i).getField("file")); - } - } - - @Test - public void testSnapshotReferencesMetatable() { - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); - - Long currentSnapshotId = table.currentSnapshot().snapshotId(); - - // Create branch - table - .manageSnapshots() - .createBranch("testBranch", currentSnapshotId) - .setMaxRefAgeMs("testBranch", 10) - .setMinSnapshotsToKeep("testBranch", 20) - .setMaxSnapshotAgeMs("testBranch", 30) - .commit(); - // Create Tag - table - .manageSnapshots() - .createTag("testTag", currentSnapshotId) - .setMaxRefAgeMs("testTag", 50) - .commit(); - // Check refs table - List references = sql("SELECT * FROM %s$refs", TABLE_NAME); - Assert.assertEquals("Refs table should return 3 rows", 3, references.size()); - List branches = sql("SELECT * FROM %s$refs WHERE type='BRANCH'", TABLE_NAME); - Assert.assertEquals("Refs table should return 2 branches", 2, branches.size()); - List tags = sql("SELECT * FROM %s$refs WHERE type='TAG'", TABLE_NAME); - Assert.assertEquals("Refs table should return 1 tag", 1, tags.size()); - - // Check branch entries in refs table - List mainBranch = - sql("SELECT * FROM %s$refs WHERE name='main' AND type='BRANCH'", TABLE_NAME); - Assert.assertEquals("main", mainBranch.get(0).getFieldAs("name")); - Assert.assertEquals("BRANCH", mainBranch.get(0).getFieldAs("type")); - Assert.assertEquals(currentSnapshotId, mainBranch.get(0).getFieldAs("snapshot_id")); - - List testBranch = - sql("SELECT * FROM %s$refs WHERE name='testBranch' AND type='BRANCH'", TABLE_NAME); - Assert.assertEquals("testBranch", testBranch.get(0).getFieldAs("name")); - Assert.assertEquals("BRANCH", testBranch.get(0).getFieldAs("type")); - Assert.assertEquals(currentSnapshotId, testBranch.get(0).getFieldAs("snapshot_id")); - Assert.assertEquals(Long.valueOf(10), testBranch.get(0).getFieldAs("max_reference_age_in_ms")); - Assert.assertEquals(Integer.valueOf(20), testBranch.get(0).getFieldAs("min_snapshots_to_keep")); - Assert.assertEquals(Long.valueOf(30), testBranch.get(0).getFieldAs("max_snapshot_age_in_ms")); - - // Check tag entries in refs table - List testTag = - sql("SELECT * FROM %s$refs WHERE name='testTag' AND type='TAG'", TABLE_NAME); - Assert.assertEquals("testTag", testTag.get(0).getFieldAs("name")); - Assert.assertEquals("TAG", testTag.get(0).getFieldAs("type")); - Assert.assertEquals(currentSnapshotId, testTag.get(0).getFieldAs("snapshot_id")); - Assert.assertEquals(Long.valueOf(50), testTag.get(0).getFieldAs("max_reference_age_in_ms")); - - // Check projection in refs table - List testTagProjection = - sql( - "SELECT name,type,snapshot_id,max_reference_age_in_ms,min_snapshots_to_keep FROM %s$refs where type='TAG'", - TABLE_NAME); - Assert.assertEquals("testTag", testTagProjection.get(0).getFieldAs("name")); - Assert.assertEquals("TAG", testTagProjection.get(0).getFieldAs("type")); - Assert.assertEquals(currentSnapshotId, testTagProjection.get(0).getFieldAs("snapshot_id")); - Assert.assertEquals( - Long.valueOf(50), testTagProjection.get(0).getFieldAs("max_reference_age_in_ms")); - Assert.assertNull(testTagProjection.get(0).getFieldAs("min_snapshots_to_keep")); - - List mainBranchProjection = - sql("SELECT name, type FROM %s$refs WHERE name='main' AND type = 'BRANCH'", TABLE_NAME); - Assert.assertEquals("main", mainBranchProjection.get(0).getFieldAs("name")); - Assert.assertEquals("BRANCH", mainBranchProjection.get(0).getFieldAs("type")); - - List testBranchProjection = - sql( - "SELECT type, name, max_reference_age_in_ms, snapshot_id FROM %s$refs WHERE name='testBranch' AND type = 'BRANCH'", - TABLE_NAME); - Assert.assertEquals("testBranch", testBranchProjection.get(0).getFieldAs("name")); - Assert.assertEquals("BRANCH", testBranchProjection.get(0).getFieldAs("type")); - Assert.assertEquals(currentSnapshotId, testBranchProjection.get(0).getFieldAs("snapshot_id")); - Assert.assertEquals( - Long.valueOf(10), testBranchProjection.get(0).getFieldAs("max_reference_age_in_ms")); - } - - /** - * Find matching manifest entries of an Iceberg table - * - * @param table iceberg table - * @param expectedContent file content to populate on entries - * @param entriesTableSchema schema of Manifest entries - * @param manifestsToExplore manifests to explore of the table - * @param partValue partition value that manifest entries must match, or null to skip filtering - */ - private List expectedEntries( - Table table, - FileContent expectedContent, - Schema entriesTableSchema, - List manifestsToExplore, - String partValue) - throws IOException { - List expected = Lists.newArrayList(); - for (ManifestFile manifest : manifestsToExplore) { - InputFile in = table.io().newInputFile(manifest.path()); - try (CloseableIterable rows = - Avro.read(in).project(entriesTableSchema).build()) { - for (GenericData.Record record : rows) { - if ((Integer) record.get("status") < 2 /* added or existing */) { - GenericData.Record file = (GenericData.Record) record.get("data_file"); - if (partitionMatch(file, partValue)) { - asMetadataRecord(file, expectedContent); - expected.add(file); - } - } - } - } - } - return expected; - } - - // Populate certain fields derived in the metadata tables - private void asMetadataRecord(GenericData.Record file, FileContent content) { - file.put(0, content.id()); - file.put(3, 0); // specId - } - - private boolean partitionMatch(GenericData.Record file, String partValue) { - if (partValue == null) { - return true; - } - GenericData.Record partition = (GenericData.Record) file.get(4); - return partValue.equals(partition.get(0).toString()); - } - - private List dataManifests(Table table) { - return table.currentSnapshot().dataManifests(table.io()); - } - - private List allDataManifests(Table table) { - List manifests = Lists.newArrayList(); - for (Snapshot snapshot : table.snapshots()) { - manifests.addAll(snapshot.dataManifests(table.io())); - } - return manifests; - } - - private List deleteManifests(Table table) { - return table.currentSnapshot().deleteManifests(table.io()); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java deleted file mode 100644 index 987d79fed3c3..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java +++ /dev/null @@ -1,110 +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.iceberg.flink.source; - -import java.util.Map; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.iceberg.BaseTable; -import org.apache.iceberg.CatalogUtil; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableMetadata; -import org.apache.iceberg.TableOperations; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.data.DeleteReadTests; -import org.apache.iceberg.hive.HiveCatalog; -import org.apache.iceberg.hive.TestHiveMetastore; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public abstract class TestFlinkReaderDeletesBase extends DeleteReadTests { - - @ClassRule public static final TemporaryFolder TEMP_FOLDER = new TemporaryFolder(); - - protected static String databaseName = "default"; - - protected static HiveConf hiveConf = null; - protected static HiveCatalog catalog = null; - private static TestHiveMetastore metastore = null; - - protected final FileFormat format; - - @Parameterized.Parameters(name = "fileFormat={0}") - public static Object[][] parameters() { - return new Object[][] { - new Object[] {FileFormat.PARQUET}, - new Object[] {FileFormat.AVRO}, - new Object[] {FileFormat.ORC} - }; - } - - TestFlinkReaderDeletesBase(FileFormat fileFormat) { - this.format = fileFormat; - } - - @BeforeClass - public static void startMetastore() { - metastore = new TestHiveMetastore(); - metastore.start(); - hiveConf = metastore.hiveConf(); - catalog = - (HiveCatalog) - CatalogUtil.loadCatalog( - HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf); - } - - @AfterClass - public static void stopMetastore() throws Exception { - metastore.stop(); - catalog = null; - } - - @Override - protected Table createTable(String name, Schema schema, PartitionSpec spec) { - Map props = Maps.newHashMap(); - props.put(TableProperties.DEFAULT_FILE_FORMAT, format.name()); - - Table table = catalog.createTable(TableIdentifier.of(databaseName, name), schema, spec, props); - TableOperations ops = ((BaseTable) table).operations(); - TableMetadata meta = ops.current(); - ops.commit(meta, meta.upgradeToFormatVersion(2)); - - return table; - } - - @Override - protected void dropTable(String name) { - catalog.dropTable(TableIdentifier.of(databaseName, name)); - } - - @Override - protected boolean expectPruned() { - return false; - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java deleted file mode 100644 index b4301b2fc4d7..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java +++ /dev/null @@ -1,540 +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.iceberg.flink.source; - -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import org.apache.flink.test.util.MiniClusterWithClientResource; -import org.apache.flink.types.Row; -import org.apache.iceberg.AppendFiles; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.flink.HadoopCatalogResource; -import org.apache.iceberg.flink.MiniClusterResource; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.DateTimeUtil; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public abstract class TestFlinkScan { - - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); - - // parametrized variables - protected final FileFormat fileFormat; - - @Parameterized.Parameters(name = "format={0}") - public static Object[] parameters() { - return new Object[] {"avro", "parquet", "orc"}; - } - - TestFlinkScan(String fileFormat) { - this.fileFormat = FileFormat.fromString(fileFormat); - } - - protected TableLoader tableLoader() { - return catalogResource.tableLoader(); - } - - protected abstract List runWithProjection(String... projected) throws Exception; - - protected abstract List runWithFilter( - Expression filter, String sqlFilter, boolean caseSensitive) throws Exception; - - protected List runWithFilter(Expression filter, String sqlFilter) throws Exception { - return runWithFilter(filter, sqlFilter, true); - } - - protected abstract List runWithOptions(Map options) throws Exception; - - protected abstract List run() throws Exception; - - @Test - public void testUnpartitionedTable() throws Exception { - Table table = - catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); - List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); - new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(expectedRecords); - TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); - } - - @Test - public void testPartitionedTable() throws Exception { - Table table = - catalogResource - .catalog() - .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); - List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); - expectedRecords.get(0).set(2, "2020-03-20"); - new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER) - .appendToTable(org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), expectedRecords); - TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); - } - - @Test - public void testProjection() throws Exception { - Table table = - catalogResource - .catalog() - .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); - List inputRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); - new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER) - .appendToTable(org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), inputRecords); - assertRows(runWithProjection("data"), Row.of(inputRecords.get(0).get(0))); - } - - @Test - public void testIdentityPartitionProjections() throws Exception { - Schema logSchema = - new Schema( - Types.NestedField.optional(1, "id", Types.IntegerType.get()), - Types.NestedField.optional(2, "dt", Types.StringType.get()), - Types.NestedField.optional(3, "level", Types.StringType.get()), - Types.NestedField.optional(4, "message", Types.StringType.get())); - PartitionSpec spec = - PartitionSpec.builderFor(logSchema).identity("dt").identity("level").build(); - - Table table = - catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, logSchema, spec); - List inputRecords = RandomGenericData.generate(logSchema, 10, 0L); - - int idx = 0; - AppendFiles append = table.newAppend(); - for (Record record : inputRecords) { - record.set(1, "2020-03-2" + idx); - record.set(2, Integer.toString(idx)); - append.appendFile( - new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER) - .writeFile( - org.apache.iceberg.TestHelpers.Row.of("2020-03-2" + idx, Integer.toString(idx)), - ImmutableList.of(record))); - idx += 1; - } - append.commit(); - - // individual fields - validateIdentityPartitionProjections(table, Collections.singletonList("dt"), inputRecords); - validateIdentityPartitionProjections(table, Collections.singletonList("level"), inputRecords); - validateIdentityPartitionProjections(table, Collections.singletonList("message"), inputRecords); - validateIdentityPartitionProjections(table, Collections.singletonList("id"), inputRecords); - // field pairs - validateIdentityPartitionProjections(table, Arrays.asList("dt", "message"), inputRecords); - validateIdentityPartitionProjections(table, Arrays.asList("level", "message"), inputRecords); - validateIdentityPartitionProjections(table, Arrays.asList("dt", "level"), inputRecords); - // out-of-order pairs - validateIdentityPartitionProjections(table, Arrays.asList("message", "dt"), inputRecords); - validateIdentityPartitionProjections(table, Arrays.asList("message", "level"), inputRecords); - validateIdentityPartitionProjections(table, Arrays.asList("level", "dt"), inputRecords); - // out-of-order triplets - validateIdentityPartitionProjections( - table, Arrays.asList("dt", "level", "message"), inputRecords); - validateIdentityPartitionProjections( - table, Arrays.asList("level", "dt", "message"), inputRecords); - validateIdentityPartitionProjections( - table, Arrays.asList("dt", "message", "level"), inputRecords); - validateIdentityPartitionProjections( - table, Arrays.asList("level", "message", "dt"), inputRecords); - validateIdentityPartitionProjections( - table, Arrays.asList("message", "dt", "level"), inputRecords); - validateIdentityPartitionProjections( - table, Arrays.asList("message", "level", "dt"), inputRecords); - } - - private void validateIdentityPartitionProjections( - Table table, List projectedFields, List inputRecords) throws Exception { - List rows = runWithProjection(projectedFields.toArray(new String[0])); - - for (int pos = 0; pos < inputRecords.size(); pos++) { - Record inputRecord = inputRecords.get(pos); - Row actualRecord = rows.get(pos); - - for (int i = 0; i < projectedFields.size(); i++) { - String name = projectedFields.get(i); - Assert.assertEquals( - "Projected field " + name + " should match", - inputRecord.getField(name), - actualRecord.getField(i)); - } - } - } - - @Test - public void testSnapshotReads() throws Exception { - Table table = - catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); - - GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER); - - List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); - helper.appendToTable(expectedRecords); - long snapshotId = table.currentSnapshot().snapshotId(); - - long timestampMillis = table.currentSnapshot().timestampMillis(); - - // produce another timestamp - waitUntilAfter(timestampMillis); - helper.appendToTable(RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L)); - - TestHelpers.assertRecords( - runWithOptions(ImmutableMap.of("snapshot-id", Long.toString(snapshotId))), - expectedRecords, - TestFixtures.SCHEMA); - TestHelpers.assertRecords( - runWithOptions(ImmutableMap.of("as-of-timestamp", Long.toString(timestampMillis))), - expectedRecords, - TestFixtures.SCHEMA); - } - - @Test - public void testTagReads() throws Exception { - Table table = - catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); - - GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER); - - List expectedRecords1 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); - helper.appendToTable(expectedRecords1); - long snapshotId = table.currentSnapshot().snapshotId(); - - table.manageSnapshots().createTag("t1", snapshotId).commit(); - - TestHelpers.assertRecords( - runWithOptions(ImmutableMap.of("tag", "t1")), expectedRecords1, TestFixtures.SCHEMA); - - List expectedRecords2 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); - helper.appendToTable(expectedRecords2); - snapshotId = table.currentSnapshot().snapshotId(); - - table.manageSnapshots().replaceTag("t1", snapshotId).commit(); - - List expectedRecords = Lists.newArrayList(); - expectedRecords.addAll(expectedRecords1); - expectedRecords.addAll(expectedRecords2); - TestHelpers.assertRecords( - runWithOptions(ImmutableMap.of("tag", "t1")), expectedRecords, TestFixtures.SCHEMA); - } - - @Test - public void testBranchReads() throws Exception { - Table table = - catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); - - GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER); - - List expectedRecordsBase = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); - helper.appendToTable(expectedRecordsBase); - long snapshotId = table.currentSnapshot().snapshotId(); - - String branchName = "b1"; - table.manageSnapshots().createBranch(branchName, snapshotId).commit(); - - List expectedRecordsForBranch = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); - helper.appendToTable(branchName, expectedRecordsForBranch); - - List expectedRecordsForMain = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); - helper.appendToTable(expectedRecordsForMain); - - List branchExpectedRecords = Lists.newArrayList(); - branchExpectedRecords.addAll(expectedRecordsBase); - branchExpectedRecords.addAll(expectedRecordsForBranch); - - TestHelpers.assertRecords( - runWithOptions(ImmutableMap.of("branch", branchName)), - branchExpectedRecords, - TestFixtures.SCHEMA); - - List mainExpectedRecords = Lists.newArrayList(); - mainExpectedRecords.addAll(expectedRecordsBase); - mainExpectedRecords.addAll(expectedRecordsForMain); - - TestHelpers.assertRecords(run(), mainExpectedRecords, TestFixtures.SCHEMA); - } - - @Test - public void testIncrementalReadViaTag() throws Exception { - Table table = - catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); - - GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER); - - List records1 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); - helper.appendToTable(records1); - long snapshotId1 = table.currentSnapshot().snapshotId(); - String startTag = "t1"; - table.manageSnapshots().createTag(startTag, snapshotId1).commit(); - - List records2 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 1L); - helper.appendToTable(records2); - - List records3 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 2L); - helper.appendToTable(records3); - long snapshotId3 = table.currentSnapshot().snapshotId(); - String endTag = "t2"; - table.manageSnapshots().createTag(endTag, snapshotId3).commit(); - - helper.appendToTable(RandomGenericData.generate(TestFixtures.SCHEMA, 1, 3L)); - - List expected = Lists.newArrayList(); - expected.addAll(records2); - expected.addAll(records3); - - TestHelpers.assertRecords( - runWithOptions( - ImmutableMap.builder() - .put("start-tag", startTag) - .put("end-tag", endTag) - .buildOrThrow()), - expected, - TestFixtures.SCHEMA); - - TestHelpers.assertRecords( - runWithOptions( - ImmutableMap.builder() - .put("start-snapshot-id", Long.toString(snapshotId1)) - .put("end-tag", endTag) - .buildOrThrow()), - expected, - TestFixtures.SCHEMA); - - TestHelpers.assertRecords( - runWithOptions( - ImmutableMap.builder() - .put("start-tag", startTag) - .put("end-snapshot-id", Long.toString(snapshotId3)) - .buildOrThrow()), - expected, - TestFixtures.SCHEMA); - Assertions.assertThatThrownBy( - () -> - runWithOptions( - ImmutableMap.builder() - .put("start-tag", startTag) - .put("end-tag", endTag) - .put("start-snapshot-id", Long.toString(snapshotId1)) - .buildOrThrow())) - .isInstanceOf(Exception.class) - .hasMessage("START_SNAPSHOT_ID and START_TAG cannot both be set."); - Assertions.assertThatThrownBy( - () -> - runWithOptions( - ImmutableMap.builder() - .put("start-tag", startTag) - .put("end-tag", endTag) - .put("end-snapshot-id", Long.toString(snapshotId3)) - .buildOrThrow())) - .isInstanceOf(Exception.class) - .hasMessage("END_SNAPSHOT_ID and END_TAG cannot both be set."); - } - - @Test - public void testIncrementalRead() throws Exception { - Table table = - catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); - - GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER); - - List records1 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); - helper.appendToTable(records1); - long snapshotId1 = table.currentSnapshot().snapshotId(); - - // snapshot 2 - List records2 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 1L); - helper.appendToTable(records2); - - List records3 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 2L); - helper.appendToTable(records3); - long snapshotId3 = table.currentSnapshot().snapshotId(); - - // snapshot 4 - helper.appendToTable(RandomGenericData.generate(TestFixtures.SCHEMA, 1, 3L)); - - List expected2 = Lists.newArrayList(); - expected2.addAll(records2); - expected2.addAll(records3); - TestHelpers.assertRecords( - runWithOptions( - ImmutableMap.builder() - .put("start-snapshot-id", Long.toString(snapshotId1)) - .put("end-snapshot-id", Long.toString(snapshotId3)) - .buildOrThrow()), - expected2, - TestFixtures.SCHEMA); - } - - @Test - public void testFilterExpPartition() throws Exception { - Table table = - catalogResource - .catalog() - .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); - - List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); - expectedRecords.get(0).set(2, "2020-03-20"); - expectedRecords.get(1).set(2, "2020-03-20"); - - GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER); - DataFile dataFile1 = - helper.writeFile(org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), expectedRecords); - DataFile dataFile2 = - helper.writeFile( - org.apache.iceberg.TestHelpers.Row.of("2020-03-21", 0), - RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L)); - helper.appendToTable(dataFile1, dataFile2); - TestHelpers.assertRecords( - runWithFilter(Expressions.equal("dt", "2020-03-20"), "where dt='2020-03-20'", true), - expectedRecords, - TestFixtures.SCHEMA); - } - - private void testFilterExp(Expression filter, String sqlFilter, boolean caseSensitive) - throws Exception { - Table table = - catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); - - List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 3, 0L); - expectedRecords.get(0).set(0, "a"); - expectedRecords.get(1).set(0, "b"); - expectedRecords.get(2).set(0, "c"); - - GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER); - DataFile dataFile = helper.writeFile(expectedRecords); - helper.appendToTable(dataFile); - - List actual = - runWithFilter(Expressions.greaterThanOrEqual("data", "b"), "where data>='b'", true); - - TestHelpers.assertRecords(actual, expectedRecords.subList(1, 3), TestFixtures.SCHEMA); - } - - @Test - public void testFilterExp() throws Exception { - testFilterExp(Expressions.greaterThanOrEqual("data", "b"), "where data>='b'", true); - } - - @Test - public void testFilterExpCaseInsensitive() throws Exception { - // sqlFilter does not support case-insensitive filtering: - // https://issues.apache.org/jira/browse/FLINK-16175 - testFilterExp(Expressions.greaterThanOrEqual("DATA", "b"), "where data>='b'", false); - } - - @Test - public void testPartitionTypes() throws Exception { - Schema typesSchema = - new Schema( - Types.NestedField.optional(1, "id", Types.IntegerType.get()), - Types.NestedField.optional(2, "decimal", Types.DecimalType.of(38, 18)), - Types.NestedField.optional(3, "str", Types.StringType.get()), - Types.NestedField.optional(4, "binary", Types.BinaryType.get()), - Types.NestedField.optional(5, "date", Types.DateType.get()), - Types.NestedField.optional(6, "time", Types.TimeType.get()), - Types.NestedField.optional(7, "timestamp", Types.TimestampType.withoutZone())); - PartitionSpec spec = - PartitionSpec.builderFor(typesSchema) - .identity("decimal") - .identity("str") - .identity("binary") - .identity("date") - .identity("time") - .identity("timestamp") - .build(); - - Table table = - catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, typesSchema, spec); - List records = RandomGenericData.generate(typesSchema, 10, 0L); - GenericAppenderHelper appender = new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER); - for (Record record : records) { - org.apache.iceberg.TestHelpers.Row partition = - org.apache.iceberg.TestHelpers.Row.of( - record.get(1), - record.get(2), - record.get(3), - record.get(4) == null ? null : DateTimeUtil.daysFromDate((LocalDate) record.get(4)), - record.get(5) == null ? null : DateTimeUtil.microsFromTime((LocalTime) record.get(5)), - record.get(6) == null - ? null - : DateTimeUtil.microsFromTimestamp((LocalDateTime) record.get(6))); - appender.appendToTable(partition, Collections.singletonList(record)); - } - - TestHelpers.assertRecords(run(), records, typesSchema); - } - - @Test - public void testCustomizedFlinkDataTypes() throws Exception { - Schema schema = - new Schema( - Types.NestedField.required( - 1, - "map", - Types.MapType.ofRequired(2, 3, Types.StringType.get(), Types.StringType.get())), - Types.NestedField.required( - 4, "arr", Types.ListType.ofRequired(5, Types.StringType.get()))); - Table table = catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, schema); - List records = RandomGenericData.generate(schema, 10, 0L); - GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER); - helper.appendToTable(records); - TestHelpers.assertRecords(run(), records, schema); - } - - private static void assertRows(List results, Row... expected) { - TestHelpers.assertRows(results, Arrays.asList(expected)); - } - - private static void waitUntilAfter(long timestampMillis) { - long current = System.currentTimeMillis(); - while (current <= timestampMillis) { - current = System.currentTimeMillis(); - } - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java deleted file mode 100644 index 023166801b19..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java +++ /dev/null @@ -1,74 +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.iceberg.flink.source; - -import java.io.IOException; -import java.util.List; -import java.util.Map; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.config.TableConfigOptions; -import org.apache.flink.types.Row; -import org.junit.Before; - -/** Test Flink SELECT SQLs. */ -public class TestFlinkScanSql extends TestFlinkSource { - - private volatile TableEnvironment tEnv; - - public TestFlinkScanSql(String fileFormat) { - super(fileFormat); - } - - @Before - public void before() throws IOException { - SqlHelpers.sql( - getTableEnv(), - "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - catalogResource.warehouse()); - SqlHelpers.sql(getTableEnv(), "use catalog iceberg_catalog"); - getTableEnv() - .getConfig() - .getConfiguration() - .set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); - } - - private TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - if (tEnv == null) { - this.tEnv = - TableEnvironment.create(EnvironmentSettings.newInstance().inBatchMode().build()); - } - } - } - return tEnv; - } - - @Override - protected List run( - FlinkSource.Builder formatBuilder, - Map sqlOptions, - String sqlFilter, - String... sqlSelectedFields) { - String select = String.join(",", sqlSelectedFields); - String optionStr = SqlHelpers.sqlOptionsToString(sqlOptions); - return SqlHelpers.sql(getTableEnv(), "select %s from t %s %s", select, optionStr, sqlFilter); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java deleted file mode 100644 index 2b55bee6e54c..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java +++ /dev/null @@ -1,94 +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.iceberg.flink.source; - -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import org.apache.flink.table.api.TableColumn; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.types.Row; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; - -public abstract class TestFlinkSource extends TestFlinkScan { - - TestFlinkSource(String fileFormat) { - super(fileFormat); - } - - @Override - protected List runWithProjection(String... projected) throws Exception { - TableSchema.Builder builder = TableSchema.builder(); - TableSchema schema = - FlinkSchemaUtil.toSchema( - FlinkSchemaUtil.convert( - catalogResource.catalog().loadTable(TestFixtures.TABLE_IDENTIFIER).schema())); - for (String field : projected) { - TableColumn column = schema.getTableColumn(field).get(); - builder.field(column.getName(), column.getType()); - } - return run(FlinkSource.forRowData().project(builder.build()), Maps.newHashMap(), "", projected); - } - - @Override - protected List runWithFilter(Expression filter, String sqlFilter, boolean caseSensitive) - throws Exception { - FlinkSource.Builder builder = - FlinkSource.forRowData().filters(Collections.singletonList(filter)); - Map options = Maps.newHashMap(); - options.put("case-sensitive", Boolean.toString(caseSensitive)); - return run(builder, options, sqlFilter, "*"); - } - - @Override - protected List runWithOptions(Map options) throws Exception { - FlinkSource.Builder builder = FlinkSource.forRowData(); - Optional.ofNullable(options.get("case-sensitive")) - .ifPresent(value -> builder.caseSensitive(Boolean.getBoolean(value))); - Optional.ofNullable(options.get("snapshot-id")) - .ifPresent(value -> builder.snapshotId(Long.parseLong(value))); - Optional.ofNullable(options.get("tag")).ifPresent(value -> builder.tag(value)); - Optional.ofNullable(options.get("branch")).ifPresent(value -> builder.branch(value)); - Optional.ofNullable(options.get("start-tag")).ifPresent(value -> builder.startTag(value)); - Optional.ofNullable(options.get("end-tag")).ifPresent(value -> builder.endTag(value)); - Optional.ofNullable(options.get("start-snapshot-id")) - .ifPresent(value -> builder.startSnapshotId(Long.parseLong(value))); - Optional.ofNullable(options.get("end-snapshot-id")) - .ifPresent(value -> builder.endSnapshotId(Long.parseLong(value))); - Optional.ofNullable(options.get("as-of-timestamp")) - .ifPresent(value -> builder.asOfTimestamp(Long.parseLong(value))); - return run(builder, options, "", "*"); - } - - @Override - protected List run() throws Exception { - return run(FlinkSource.forRowData(), Maps.newHashMap(), "", "*"); - } - - protected abstract List run( - FlinkSource.Builder formatBuilder, - Map sqlOptions, - String sqlFilter, - String... sqlSelectedFields) - throws Exception; -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java deleted file mode 100644 index 1814ff8f8542..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java +++ /dev/null @@ -1,58 +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.iceberg.flink.source; - -import java.util.List; -import org.apache.flink.types.Row; -import org.apache.iceberg.flink.FlinkReadOptions; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; - -public class TestFlinkSourceConfig extends TestFlinkTableSource { - private static final String TABLE = "test_table"; - - @Test - public void testFlinkSessionConfig() { - getTableEnv().getConfig().set(FlinkReadOptions.STREAMING_OPTION, true); - Assertions.assertThatThrownBy( - () -> sql("SELECT * FROM %s /*+ OPTIONS('as-of-timestamp'='1')*/", TABLE)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot set as-of-timestamp option for streaming reader"); - } - - @Test - public void testFlinkHintConfig() { - List result = - sql( - "SELECT * FROM %s /*+ OPTIONS('as-of-timestamp'='%d','streaming'='false')*/", - TABLE, System.currentTimeMillis()); - Assert.assertEquals(3, result.size()); - } - - @Test - public void testReadOptionHierarchy() { - getTableEnv().getConfig().set(FlinkReadOptions.LIMIT_OPTION, 1L); - List result = sql("SELECT * FROM %s", TABLE); - Assert.assertEquals(1, result.size()); - - result = sql("SELECT * FROM %s /*+ OPTIONS('limit'='3')*/", TABLE); - Assert.assertEquals(3, result.size()); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java deleted file mode 100644 index affd90c347dd..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java +++ /dev/null @@ -1,85 +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.iceberg.flink.source; - -import java.io.IOException; -import java.util.List; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.PipelineOptions; -import org.apache.flink.table.api.config.TableConfigOptions; -import org.apache.flink.types.Row; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Table; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.junit.Test; - -/** Use the FlinkSource */ -public class TestFlinkSourceSql extends TestSqlBase { - @Override - public void before() throws IOException { - SqlHelpers.sql( - getTableEnv(), - "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - catalogResource.warehouse()); - SqlHelpers.sql(getTableEnv(), "use catalog iceberg_catalog"); - getTableEnv() - .getConfig() - .getConfiguration() - .set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); - } - - @Test - public void testInferParallelismWithGlobalSetting() throws IOException { - Configuration cfg = getTableEnv().getConfig().getConfiguration(); - cfg.set(PipelineOptions.MAX_PARALLELISM, 1); - - Table table = - catalogResource - .catalog() - .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, null); - - GenericAppenderHelper helper = - new GenericAppenderHelper(table, FileFormat.PARQUET, TEMPORARY_FOLDER); - List expectedRecords = Lists.newArrayList(); - long maxFileLen = 0; - for (int i = 0; i < 5; i++) { - List records = RandomGenericData.generate(TestFixtures.SCHEMA, 2, i); - DataFile dataFile = helper.writeFile(null, records); - helper.appendToTable(dataFile); - expectedRecords.addAll(records); - maxFileLen = Math.max(dataFile.fileSizeInBytes(), maxFileLen); - } - - // Make sure to generate multiple CombinedScanTasks - SqlHelpers.sql( - getTableEnv(), - "ALTER TABLE t SET ('read.split.open-file-cost'='1', 'read.split.target-size'='%s')", - maxFileLen); - - List results = run(Maps.newHashMap(), "", "*"); - org.apache.iceberg.flink.TestHelpers.assertRecords( - results, expectedRecords, TestFixtures.SCHEMA); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java deleted file mode 100644 index c3e23d34faf5..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java +++ /dev/null @@ -1,646 +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.iceberg.flink.source; - -import java.io.File; -import java.io.IOException; -import java.util.List; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.table.api.SqlParserException; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.types.Row; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.events.Listeners; -import org.apache.iceberg.events.ScanEvent; -import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.flink.FlinkTestBase; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; - -public class TestFlinkTableSource extends FlinkTestBase { - - private static final String CATALOG_NAME = "test_catalog"; - private static final String DATABASE_NAME = "test_db"; - private static final String TABLE_NAME = "test_table"; - private final FileFormat format = FileFormat.AVRO; - private static String warehouse; - - private int scanEventCount = 0; - private ScanEvent lastScanEvent = null; - - public TestFlinkTableSource() { - // register a scan event listener to validate pushdown - Listeners.register( - event -> { - scanEventCount += 1; - lastScanEvent = event; - }, - ScanEvent.class); - } - - @Override - protected TableEnvironment getTableEnv() { - super.getTableEnv().getConfig().getConfiguration().set(CoreOptions.DEFAULT_PARALLELISM, 1); - return super.getTableEnv(); - } - - @BeforeClass - public static void createWarehouse() throws IOException { - File warehouseFile = TEMPORARY_FOLDER.newFolder(); - Assert.assertTrue("The warehouse should be deleted", warehouseFile.delete()); - // before variables - warehouse = "file:" + warehouseFile; - } - - @Before - public void before() { - sql( - "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - CATALOG_NAME, warehouse); - sql("USE CATALOG %s", CATALOG_NAME); - sql("CREATE DATABASE %s", DATABASE_NAME); - sql("USE %s", DATABASE_NAME); - sql( - "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('write.format.default'='%s')", - TABLE_NAME, format.name()); - sql( - "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)", - TABLE_NAME); - - this.scanEventCount = 0; - this.lastScanEvent = null; - } - - @After - public void clean() { - sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, TABLE_NAME); - sql("DROP DATABASE IF EXISTS %s", DATABASE_NAME); - sql("DROP CATALOG IF EXISTS %s", CATALOG_NAME); - } - - @Test - public void testLimitPushDown() { - Assertions.assertThatThrownBy(() -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME)) - .as("Invalid limit number: -1 ") - .isInstanceOf(SqlParserException.class) - .hasMessageContaining("SQL parse failed. Encountered \"-\""); - - Assert.assertEquals( - "Should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size()); - - String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME); - List resultExceed = sql(sqlLimitExceed); - Assert.assertEquals("Should have 3 records", 3, resultExceed.size()); - List expectedList = - Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); - assertSameElements(expectedList, resultExceed); - - String querySql = String.format("SELECT * FROM %s LIMIT 1", TABLE_NAME); - String explain = getTableEnv().explainSql(querySql); - String expectedExplain = "limit=[1]"; - Assert.assertTrue("Explain should contain LimitPushDown", explain.contains(expectedExplain)); - List result = sql(querySql); - Assert.assertEquals("Should have 1 record", 1, result.size()); - Assertions.assertThat(result).containsAnyElementsOf(expectedList); - - String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME); - List mixedResult = sql(sqlMixed); - Assert.assertEquals("Should have 1 record", 1, mixedResult.size()); - Assert.assertEquals( - "Should produce the expected records", Row.of(1, "iceberg", 10.0), mixedResult.get(0)); - } - - @Test - public void testNoFilterPushDown() { - String sql = String.format("SELECT * FROM %s ", TABLE_NAME); - List result = sql(sql); - List expectedRecords = - Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); - assertSameElements(expectedRecords, result); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); - } - - @Test - public void testFilterPushDownEqual() { - String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME); - String expectedFilter = "ref(name=\"id\") == 1"; - - List result = sql(sqlLiteralRight); - Assert.assertEquals("Should have 1 record", 1, result.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), result.get(0)); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownEqualNull() { - String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME); - - List result = sql(sqlEqualNull); - Assert.assertEquals("Should have 0 record", 0, result.size()); - Assert.assertNull("Should not push down a filter", lastScanEvent); - } - - @Test - public void testFilterPushDownEqualLiteralOnLeft() { - String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME); - String expectedFilter = "ref(name=\"id\") == 1"; - - List resultLeft = sql(sqlLiteralLeft); - Assert.assertEquals("Should have 1 record", 1, resultLeft.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultLeft.get(0)); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownNoEqual() { - String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME); - String expectedFilter = "ref(name=\"id\") != 1"; - - List resultNE = sql(sqlNE); - Assert.assertEquals("Should have 2 records", 2, resultNE.size()); - - List expectedNE = Lists.newArrayList(Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); - assertSameElements(expectedNE, resultNE); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownNoEqualNull() { - String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME); - - List resultNE = sql(sqlNotEqualNull); - Assert.assertEquals("Should have 0 records", 0, resultNE.size()); - Assert.assertNull("Should not push down a filter", lastScanEvent); - } - - @Test - public void testFilterPushDownAnd() { - String sqlAnd = - String.format("SELECT * FROM %s WHERE id = 1 AND data = 'iceberg' ", TABLE_NAME); - - List resultAnd = sql(sqlAnd); - Assert.assertEquals("Should have 1 record", 1, resultAnd.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultAnd.get(0)); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - String expected = "(ref(name=\"id\") == 1 and ref(name=\"data\") == \"iceberg\")"; - Assert.assertEquals( - "Should contain the push down filter", expected, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownOr() { - String sqlOr = String.format("SELECT * FROM %s WHERE id = 1 OR data = 'b' ", TABLE_NAME); - String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"data\") == \"b\")"; - - List resultOr = sql(sqlOr); - Assert.assertEquals("Should have 2 record", 2, resultOr.size()); - - List expectedOR = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); - assertSameElements(expectedOR, resultOr); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownGreaterThan() { - String sqlGT = String.format("SELECT * FROM %s WHERE id > 1 ", TABLE_NAME); - String expectedFilter = "ref(name=\"id\") > 1"; - - List resultGT = sql(sqlGT); - Assert.assertEquals("Should have 2 record", 2, resultGT.size()); - - List expectedGT = Lists.newArrayList(Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); - assertSameElements(expectedGT, resultGT); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownGreaterThanNull() { - String sqlGT = String.format("SELECT * FROM %s WHERE data > null ", TABLE_NAME); - - List resultGT = sql(sqlGT); - Assert.assertEquals("Should have 0 record", 0, resultGT.size()); - Assert.assertNull("Should not push down a filter", lastScanEvent); - } - - @Test - public void testFilterPushDownGreaterThanLiteralOnLeft() { - String sqlGT = String.format("SELECT * FROM %s WHERE 3 > id ", TABLE_NAME); - String expectedFilter = "ref(name=\"id\") < 3"; - - List resultGT = sql(sqlGT); - Assert.assertEquals("Should have 2 records", 2, resultGT.size()); - - List expectedGT = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); - assertSameElements(expectedGT, resultGT); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownGreaterThanEqual() { - String sqlGTE = String.format("SELECT * FROM %s WHERE id >= 2 ", TABLE_NAME); - String expectedFilter = "ref(name=\"id\") >= 2"; - - List resultGTE = sql(sqlGTE); - Assert.assertEquals("Should have 2 records", 2, resultGTE.size()); - - List expectedGTE = Lists.newArrayList(Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); - assertSameElements(expectedGTE, resultGTE); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownGreaterThanEqualNull() { - String sqlGTE = String.format("SELECT * FROM %s WHERE data >= null ", TABLE_NAME); - - List resultGT = sql(sqlGTE); - Assert.assertEquals("Should have 0 record", 0, resultGT.size()); - Assert.assertNull("Should not push down a filter", lastScanEvent); - } - - @Test - public void testFilterPushDownGreaterThanEqualLiteralOnLeft() { - String sqlGTE = String.format("SELECT * FROM %s WHERE 2 >= id ", TABLE_NAME); - String expectedFilter = "ref(name=\"id\") <= 2"; - - List resultGTE = sql(sqlGTE); - Assert.assertEquals("Should have 2 records", 2, resultGTE.size()); - - List expectedGTE = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); - assertSameElements(expectedGTE, resultGTE); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownLessThan() { - String sqlLT = String.format("SELECT * FROM %s WHERE id < 2 ", TABLE_NAME); - String expectedFilter = "ref(name=\"id\") < 2"; - - List resultLT = sql(sqlLT); - Assert.assertEquals("Should have 1 record", 1, resultLT.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultLT.get(0)); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownLessThanNull() { - String sqlLT = String.format("SELECT * FROM %s WHERE data < null ", TABLE_NAME); - - List resultGT = sql(sqlLT); - Assert.assertEquals("Should have 0 record", 0, resultGT.size()); - Assert.assertNull("Should not push down a filter", lastScanEvent); - } - - @Test - public void testFilterPushDownLessThanLiteralOnLeft() { - String sqlLT = String.format("SELECT * FROM %s WHERE 2 < id ", TABLE_NAME); - String expectedFilter = "ref(name=\"id\") > 2"; - - List resultLT = sql(sqlLT); - Assert.assertEquals("Should have 1 record", 1, resultLT.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(3, null, 30.0), resultLT.get(0)); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownLessThanEqual() { - String sqlLTE = String.format("SELECT * FROM %s WHERE id <= 1 ", TABLE_NAME); - String expectedFilter = "ref(name=\"id\") <= 1"; - - List resultLTE = sql(sqlLTE); - Assert.assertEquals("Should have 1 record", 1, resultLTE.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultLTE.get(0)); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownLessThanEqualNull() { - String sqlLTE = String.format("SELECT * FROM %s WHERE data <= null ", TABLE_NAME); - - List resultGT = sql(sqlLTE); - Assert.assertEquals("Should have 0 record", 0, resultGT.size()); - Assert.assertNull("Should not push down a filter", lastScanEvent); - } - - @Test - public void testFilterPushDownLessThanEqualLiteralOnLeft() { - String sqlLTE = String.format("SELECT * FROM %s WHERE 3 <= id ", TABLE_NAME); - String expectedFilter = "ref(name=\"id\") >= 3"; - - List resultLTE = sql(sqlLTE); - Assert.assertEquals("Should have 1 record", 1, resultLTE.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(3, null, 30.0), resultLTE.get(0)); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownIn() { - String sqlIN = String.format("SELECT * FROM %s WHERE id IN (1,2) ", TABLE_NAME); - String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"id\") == 2)"; - List resultIN = sql(sqlIN); - Assert.assertEquals("Should have 2 records", 2, resultIN.size()); - - List expectedIN = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); - assertSameElements(expectedIN, resultIN); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownInNull() { - String sqlInNull = - String.format("SELECT * FROM %s WHERE data IN ('iceberg',NULL) ", TABLE_NAME); - - List result = sql(sqlInNull); - Assert.assertEquals("Should have 1 record", 1, result.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), result.get(0)); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); - } - - @Test - public void testFilterPushDownNotIn() { - String sqlNotIn = String.format("SELECT * FROM %s WHERE id NOT IN (3,2) ", TABLE_NAME); - - List resultNotIn = sql(sqlNotIn); - Assert.assertEquals("Should have 1 record", 1, resultNotIn.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultNotIn.get(0)); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - String expectedScan = "(ref(name=\"id\") != 2 and ref(name=\"id\") != 3)"; - Assert.assertEquals( - "Should contain the push down filter", expectedScan, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownNotInNull() { - String sqlNotInNull = String.format("SELECT * FROM %s WHERE id NOT IN (1,2,NULL) ", TABLE_NAME); - List resultGT = sql(sqlNotInNull); - Assert.assertEquals("Should have 0 record", 0, resultGT.size()); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); - } - - @Test - public void testFilterPushDownIsNotNull() { - String sqlNotNull = String.format("SELECT * FROM %s WHERE data IS NOT NULL", TABLE_NAME); - String expectedFilter = "not_null(ref(name=\"data\"))"; - - List resultNotNull = sql(sqlNotNull); - Assert.assertEquals("Should have 2 record", 2, resultNotNull.size()); - - List expected = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); - assertSameElements(expected, resultNotNull); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownIsNull() { - String sqlNull = String.format("SELECT * FROM %s WHERE data IS NULL", TABLE_NAME); - String expectedFilter = "is_null(ref(name=\"data\"))"; - - List resultNull = sql(sqlNull); - Assert.assertEquals("Should have 1 record", 1, resultNull.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(3, null, 30.0), resultNull.get(0)); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownNot() { - String sqlNot = String.format("SELECT * FROM %s WHERE NOT (id = 1 OR id = 2 ) ", TABLE_NAME); - - List resultNot = sql(sqlNot); - Assert.assertEquals("Should have 1 record", 1, resultNot.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(3, null, 30.0), resultNot.get(0)); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - String expectedFilter = "(ref(name=\"id\") != 1 and ref(name=\"id\") != 2)"; - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownBetween() { - String sqlBetween = String.format("SELECT * FROM %s WHERE id BETWEEN 1 AND 2 ", TABLE_NAME); - - List resultBetween = sql(sqlBetween); - Assert.assertEquals("Should have 2 record", 2, resultBetween.size()); - - List expectedBetween = - Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); - assertSameElements(expectedBetween, resultBetween); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - String expected = "(ref(name=\"id\") >= 1 and ref(name=\"id\") <= 2)"; - Assert.assertEquals( - "Should contain the push down filter", expected, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownNotBetween() { - String sqlNotBetween = - String.format("SELECT * FROM %s WHERE id NOT BETWEEN 2 AND 3 ", TABLE_NAME); - String expectedFilter = "(ref(name=\"id\") < 2 or ref(name=\"id\") > 3)"; - - List resultNotBetween = sql(sqlNotBetween); - Assert.assertEquals("Should have 1 record", 1, resultNotBetween.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultNotBetween.get(0)); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownLike() { - String expectedFilter = "ref(name=\"data\") startsWith \"\"ice\"\""; - - String sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'ice%%' "; - List resultLike = sql(sqlLike); - Assert.assertEquals("Should have 1 record", 1, resultLike.size()); - Assert.assertEquals( - "The like result should produce the expected record", - Row.of(1, "iceberg", 10.0), - resultLike.get(0)); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterNotPushDownLike() { - Row expectRecord = Row.of(1, "iceberg", 10.0); - String sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i' "; - List resultLike = sql(sqlNoPushDown); - Assert.assertEquals("Should have 1 record", 0, resultLike.size()); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); - - sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i%%' "; - resultLike = sql(sqlNoPushDown); - Assert.assertEquals("Should have 1 record", 1, resultLike.size()); - Assert.assertEquals("Should produce the expected record", expectRecord, resultLike.get(0)); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); - - sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%ice%%g' "; - resultLike = sql(sqlNoPushDown); - Assert.assertEquals("Should have 1 record", 1, resultLike.size()); - Assert.assertEquals("Should produce the expected record", expectRecord, resultLike.get(0)); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); - - sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%' "; - resultLike = sql(sqlNoPushDown); - Assert.assertEquals("Should have 3 records", 3, resultLike.size()); - List expectedRecords = - Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); - assertSameElements(expectedRecords, resultLike); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); - - sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'iceber_' "; - resultLike = sql(sqlNoPushDown); - Assert.assertEquals("Should have 1 record", 1, resultLike.size()); - Assert.assertEquals("Should produce the expected record", expectRecord, resultLike.get(0)); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); - - sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'i%%g' "; - resultLike = sql(sqlNoPushDown); - Assert.assertEquals("Should have 1 record", 1, resultLike.size()); - Assert.assertEquals("Should produce the expected record", expectRecord, resultLike.get(0)); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); - } - - @Test - public void testFilterPushDown2Literal() { - String sql2Literal = String.format("SELECT * FROM %s WHERE 1 > 0 ", TABLE_NAME); - List result = sql(sql2Literal); - List expectedRecords = - Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); - assertSameElements(expectedRecords, result); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); - } - - /** - * NaN is not supported by flink now, so we add the test case to assert the parse error, when we - * upgrade the flink that supports NaN, we will delele the method, and add some test case to test - * NaN. - */ - @Test - public void testSqlParseError() { - String sqlParseErrorEqual = - String.format("SELECT * FROM %s WHERE d = CAST('NaN' AS DOUBLE) ", TABLE_NAME); - Assertions.assertThatThrownBy(() -> sql(sqlParseErrorEqual)) - .isInstanceOf(NumberFormatException.class) - .hasMessageContaining("Infinite or NaN"); - - String sqlParseErrorNotEqual = - String.format("SELECT * FROM %s WHERE d <> CAST('NaN' AS DOUBLE) ", TABLE_NAME); - Assertions.assertThatThrownBy(() -> sql(sqlParseErrorNotEqual)) - .isInstanceOf(NumberFormatException.class) - .hasMessageContaining("Infinite or NaN"); - - String sqlParseErrorGT = - String.format("SELECT * FROM %s WHERE d > CAST('NaN' AS DOUBLE) ", TABLE_NAME); - Assertions.assertThatThrownBy(() -> sql(sqlParseErrorGT)) - .isInstanceOf(NumberFormatException.class) - .hasMessageContaining("Infinite or NaN"); - - String sqlParseErrorLT = - String.format("SELECT * FROM %s WHERE d < CAST('NaN' AS DOUBLE) ", TABLE_NAME); - Assertions.assertThatThrownBy(() -> sql(sqlParseErrorLT)) - .isInstanceOf(NumberFormatException.class) - .hasMessageContaining("Infinite or NaN"); - - String sqlParseErrorGTE = - String.format("SELECT * FROM %s WHERE d >= CAST('NaN' AS DOUBLE) ", TABLE_NAME); - Assertions.assertThatThrownBy(() -> sql(sqlParseErrorGTE)) - .isInstanceOf(NumberFormatException.class) - .hasMessageContaining("Infinite or NaN"); - - String sqlParseErrorLTE = - String.format("SELECT * FROM %s WHERE d <= CAST('NaN' AS DOUBLE) ", TABLE_NAME); - Assertions.assertThatThrownBy(() -> sql(sqlParseErrorLTE)) - .isInstanceOf(NumberFormatException.class) - .hasMessageContaining("Infinite or NaN"); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java deleted file mode 100644 index a80f87d648d5..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java +++ /dev/null @@ -1,134 +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.iceberg.flink.source; - -import java.util.Collections; -import java.util.List; -import java.util.Map; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.TableColumn; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.data.RowData; -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.data.RowDataToRowMapper; -import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestIcebergSourceBounded extends TestFlinkScan { - - public TestIcebergSourceBounded(String fileFormat) { - super(fileFormat); - } - - @Override - protected List runWithProjection(String... projected) throws Exception { - Schema icebergTableSchema = - catalogResource.catalog().loadTable(TestFixtures.TABLE_IDENTIFIER).schema(); - TableSchema.Builder builder = TableSchema.builder(); - TableSchema schema = FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(icebergTableSchema)); - for (String field : projected) { - TableColumn column = schema.getTableColumn(field).get(); - builder.field(column.getName(), column.getType()); - } - TableSchema flinkSchema = builder.build(); - Schema projectedSchema = FlinkSchemaUtil.convert(icebergTableSchema, flinkSchema); - return run(projectedSchema, Lists.newArrayList(), Maps.newHashMap(), "", projected); - } - - @Override - protected List runWithFilter(Expression filter, String sqlFilter, boolean caseSensitive) - throws Exception { - Map options = Maps.newHashMap(); - options.put("case-sensitive", Boolean.toString(caseSensitive)); - return run(null, Collections.singletonList(filter), options, sqlFilter, "*"); - } - - @Override - protected List runWithOptions(Map options) throws Exception { - return run(null, Lists.newArrayList(), options, "", "*"); - } - - @Override - protected List run() throws Exception { - return run(null, Lists.newArrayList(), Maps.newHashMap(), "", "*"); - } - - protected List run( - Schema projectedSchema, - List filters, - Map options, - String sqlFilter, - String... sqlSelectedFields) - throws Exception { - - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(1); - Configuration config = new Configuration(); - config.setInteger(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 128); - Table table; - try (TableLoader tableLoader = tableLoader()) { - tableLoader.open(); - table = tableLoader.loadTable(); - } - - IcebergSource.Builder sourceBuilder = - IcebergSource.forRowData() - .tableLoader(tableLoader()) - .table(table) - .assignerFactory(new SimpleSplitAssignerFactory()) - .flinkConfig(config); - if (projectedSchema != null) { - sourceBuilder.project(projectedSchema); - } - - sourceBuilder.filters(filters); - sourceBuilder.properties(options); - - DataStream stream = - env.fromSource( - sourceBuilder.build(), - WatermarkStrategy.noWatermarks(), - "testBasicRead", - TypeInformation.of(RowData.class)) - .map( - new RowDataToRowMapper( - FlinkSchemaUtil.convert( - projectedSchema == null ? table.schema() : projectedSchema))); - - try (CloseableIterator iter = stream.executeAndCollect()) { - return Lists.newArrayList(iter); - } - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java deleted file mode 100644 index 0337f3597053..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java +++ /dev/null @@ -1,203 +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.iceberg.flink.source; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import org.apache.avro.generic.GenericRecord; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.formats.avro.typeutils.GenericRecordAvroTypeInfo; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.test.util.MiniClusterWithClientResource; -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.avro.AvroSchemaUtil; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.HadoopCatalogResource; -import org.apache.iceberg.flink.MiniClusterResource; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.flink.data.RowDataToRowMapper; -import org.apache.iceberg.flink.sink.AvroGenericRecordToRowDataMapper; -import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; -import org.apache.iceberg.flink.source.reader.AvroGenericRecordReaderFunction; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.types.TypeUtil; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestIcebergSourceBoundedGenericRecord { - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); - - @Parameterized.Parameters(name = "format={0}, parallelism = {1}") - public static Object[][] parameters() { - return new Object[][] { - {"avro", 2}, - {"parquet", 2}, - {"orc", 2} - }; - } - - private final FileFormat fileFormat; - private final int parallelism; - - public TestIcebergSourceBoundedGenericRecord(String format, int parallelism) { - this.fileFormat = FileFormat.valueOf(format.toUpperCase(Locale.ENGLISH)); - this.parallelism = parallelism; - } - - @Test - public void testUnpartitionedTable() throws Exception { - Table table = - catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); - List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); - new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(expectedRecords); - TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); - } - - @Test - public void testPartitionedTable() throws Exception { - String dateStr = "2020-03-20"; - Table table = - catalogResource - .catalog() - .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); - List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); - for (int i = 0; i < expectedRecords.size(); ++i) { - expectedRecords.get(i).setField("dt", dateStr); - } - - new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER) - .appendToTable(org.apache.iceberg.TestHelpers.Row.of(dateStr, 0), expectedRecords); - TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); - } - - @Test - public void testProjection() throws Exception { - Table table = - catalogResource - .catalog() - .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); - List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); - new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER) - .appendToTable(org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), expectedRecords); - // select the "data" field (fieldId == 1) - Schema projectedSchema = TypeUtil.select(TestFixtures.SCHEMA, Sets.newHashSet(1)); - List expectedRows = - Arrays.asList(Row.of(expectedRecords.get(0).get(0)), Row.of(expectedRecords.get(1).get(0))); - TestHelpers.assertRows( - run(projectedSchema, Collections.emptyList(), Collections.emptyMap()), expectedRows); - } - - private List run() throws Exception { - return run(null, Collections.emptyList(), Collections.emptyMap()); - } - - private List run( - Schema projectedSchema, List filters, Map options) - throws Exception { - - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(parallelism); - env.getConfig().enableObjectReuse(); - - Configuration config = new Configuration(); - config.setInteger(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 128); - Table table; - try (TableLoader tableLoader = catalogResource.tableLoader()) { - tableLoader.open(); - table = tableLoader.loadTable(); - } - - AvroGenericRecordReaderFunction readerFunction = - new AvroGenericRecordReaderFunction( - TestFixtures.TABLE_IDENTIFIER.name(), - new Configuration(), - table.schema(), - null, - null, - false, - table.io(), - table.encryption(), - filters); - - IcebergSource.Builder sourceBuilder = - IcebergSource.builder() - .tableLoader(catalogResource.tableLoader()) - .readerFunction(readerFunction) - .assignerFactory(new SimpleSplitAssignerFactory()) - .flinkConfig(config); - if (projectedSchema != null) { - sourceBuilder.project(projectedSchema); - } - - sourceBuilder.filters(filters); - sourceBuilder.setAll(options); - - Schema readSchema = projectedSchema != null ? projectedSchema : table.schema(); - RowType rowType = FlinkSchemaUtil.convert(readSchema); - org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(readSchema, TestFixtures.TABLE_IDENTIFIER.name()); - - DataStream stream = - env.fromSource( - sourceBuilder.build(), - WatermarkStrategy.noWatermarks(), - "testBasicRead", - new GenericRecordAvroTypeInfo(avroSchema)) - // There are two reasons for converting GenericRecord back to Row. - // 1. Avro GenericRecord/Schema is not serializable. - // 2. leverage the TestHelpers.assertRecords for validation. - .map(AvroGenericRecordToRowDataMapper.forAvroSchema(avroSchema)) - .map(new RowDataToRowMapper(rowType)); - - try (CloseableIterator iter = stream.executeAndCollect()) { - return Lists.newArrayList(iter); - } - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java deleted file mode 100644 index 3652e0bb56df..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java +++ /dev/null @@ -1,80 +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.iceberg.flink.source; - -import java.io.IOException; -import java.util.List; -import java.util.Map; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.config.TableConfigOptions; -import org.apache.flink.types.Row; -import org.apache.iceberg.Schema; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.junit.Before; - -public class TestIcebergSourceBoundedSql extends TestIcebergSourceBounded { - private volatile TableEnvironment tEnv; - - public TestIcebergSourceBoundedSql(String fileFormat) { - super(fileFormat); - } - - @Before - public void before() throws IOException { - Configuration tableConf = getTableEnv().getConfig().getConfiguration(); - tableConf.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), true); - SqlHelpers.sql( - getTableEnv(), - "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - catalogResource.warehouse()); - SqlHelpers.sql(getTableEnv(), "use catalog iceberg_catalog"); - getTableEnv() - .getConfig() - .getConfiguration() - .set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); - } - - private TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - if (tEnv == null) { - this.tEnv = - TableEnvironment.create(EnvironmentSettings.newInstance().inBatchMode().build()); - } - } - } - return tEnv; - } - - @Override - protected List run( - Schema projectedSchema, - List filters, - Map options, - String sqlFilter, - String... sqlSelectedFields) - throws Exception { - String select = String.join(",", sqlSelectedFields); - String optionStr = SqlHelpers.sqlOptionsToString(options); - return SqlHelpers.sql(getTableEnv(), "select %s from t %s %s", select, optionStr, sqlFilter); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java deleted file mode 100644 index 31e9733fcd60..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java +++ /dev/null @@ -1,421 +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.iceberg.flink.source; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.time.Duration; -import java.util.Collection; -import java.util.List; -import java.util.concurrent.atomic.AtomicLong; -import java.util.stream.Collectors; -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.JobStatus; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.client.program.ClusterClient; -import org.apache.flink.runtime.client.JobStatusMessage; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.data.RowData; -import org.apache.flink.test.util.MiniClusterWithClientResource; -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.HadoopTableResource; -import org.apache.iceberg.flink.MiniClusterResource; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.flink.data.RowDataToRowMapper; -import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.awaitility.Awaitility; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class TestIcebergSourceContinuous { - - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @Rule - public final HadoopTableResource tableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); - - private final AtomicLong randomSeed = new AtomicLong(0L); - - @Test - public void testTableScanThenIncremental() throws Exception { - GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); - - // snapshot1 - List batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch1); - - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .monitorInterval(Duration.ofMillis(10L)) - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - - try (CloseableIterator iter = - createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { - List result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); - - // snapshot2 - List batch2 = - RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch2); - tableResource.table().currentSnapshot().snapshotId(); - - List result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); - - // snapshot3 - List batch3 = - RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch3); - tableResource.table().currentSnapshot().snapshotId(); - - List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); - } - } - - @Test - public void testTableScanThenIncrementalAfterExpiration() throws Exception { - GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); - - // snapshot1 - List batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch1); - long snapshotId = tableResource.table().currentSnapshot().snapshotId(); - - // snapshot2 - List batch2 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch2); - - tableResource.table().expireSnapshots().expireSnapshotId(snapshotId).commit(); - - Assert.assertEquals(1, tableResource.table().history().size()); - - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .monitorInterval(Duration.ofMillis(10L)) - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - - Assert.assertEquals( - FlinkSplitPlanner.ScanMode.BATCH, FlinkSplitPlanner.checkScanMode(scanContext)); - - try (CloseableIterator iter = - createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { - List result1 = waitForResult(iter, 4); - List initialRecords = Lists.newArrayList(); - initialRecords.addAll(batch1); - initialRecords.addAll(batch2); - TestHelpers.assertRecords(result1, initialRecords, tableResource.table().schema()); - - // snapshot3 - List batch3 = - RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch3); - tableResource.table().currentSnapshot().snapshotId(); - - List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); - } - } - - @Test - public void testEarliestSnapshot() throws Exception { - GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); - - // snapshot0 - List batch0 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch0); - - // snapshot1 - List batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch1); - - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .monitorInterval(Duration.ofMillis(10L)) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) - .build(); - - try (CloseableIterator iter = - createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { - List result1 = waitForResult(iter, 4); - List combinedBatch0AndBatch1 = Lists.newArrayList(batch0); - combinedBatch0AndBatch1.addAll(batch1); - TestHelpers.assertRecords(result1, combinedBatch0AndBatch1, tableResource.table().schema()); - - // snapshot2 - List batch2 = - RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch2); - - List result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); - - // snapshot3 - List batch3 = - RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch3); - - List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); - } - } - - @Test - public void testLatestSnapshot() throws Exception { - GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); - - // snapshot0 - List batch0 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch0); - - // snapshot1 - List batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch1); - - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .monitorInterval(Duration.ofMillis(10L)) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) - .build(); - - try (CloseableIterator iter = - createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { - // we want to make sure job is running first so that enumerator can - // start from the latest snapshot before inserting the next batch2 below. - waitUntilJobIsRunning(MINI_CLUSTER_RESOURCE.getClusterClient()); - - // inclusive behavior for starting snapshot - List result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); - - // snapshot2 - List batch2 = - RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch2); - - List result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); - - // snapshot3 - List batch3 = - RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch3); - - List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); - } - } - - @Test - public void testSpecificSnapshotId() throws Exception { - GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); - - // snapshot0 - List batch0 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch0); - long snapshot0 = tableResource.table().currentSnapshot().snapshotId(); - - // snapshot1 - List batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch1); - long snapshot1 = tableResource.table().currentSnapshot().snapshotId(); - - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .monitorInterval(Duration.ofMillis(10L)) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) - .startSnapshotId(snapshot1) - .build(); - - try (CloseableIterator iter = - createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { - List result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); - - // snapshot2 - List batch2 = - RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch2); - - List result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); - - // snapshot3 - List batch3 = - RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch3); - - List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); - } - } - - @Test - public void testSpecificSnapshotTimestamp() throws Exception { - GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); - - // snapshot0 - List batch0 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch0); - long snapshot0Timestamp = tableResource.table().currentSnapshot().timestampMillis(); - - // sleep for 2 ms to make sure snapshot1 has a higher timestamp value - Thread.sleep(2); - - // snapshot1 - List batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch1); - long snapshot1Timestamp = tableResource.table().currentSnapshot().timestampMillis(); - - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .monitorInterval(Duration.ofMillis(10L)) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) - .startSnapshotTimestamp(snapshot1Timestamp) - .build(); - - try (CloseableIterator iter = - createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { - // consume data from snapshot1 - List result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); - - // snapshot2 - List batch2 = - RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch2); - - List result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); - - // snapshot3 - List batch3 = - RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch3); - - List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); - } - } - - private DataStream createStream(ScanContext scanContext) throws Exception { - // start the source and collect output - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(1); - DataStream stream = - env.fromSource( - IcebergSource.forRowData() - .tableLoader(tableResource.tableLoader()) - .assignerFactory(new SimpleSplitAssignerFactory()) - .streaming(scanContext.isStreaming()) - .streamingStartingStrategy(scanContext.streamingStartingStrategy()) - .startSnapshotTimestamp(scanContext.startSnapshotTimestamp()) - .startSnapshotId(scanContext.startSnapshotId()) - .monitorInterval(Duration.ofMillis(10L)) - .build(), - WatermarkStrategy.noWatermarks(), - "icebergSource", - TypeInformation.of(RowData.class)) - .map(new RowDataToRowMapper(FlinkSchemaUtil.convert(tableResource.table().schema()))); - return stream; - } - - public static List waitForResult(CloseableIterator iter, int limit) { - List results = Lists.newArrayListWithCapacity(limit); - while (results.size() < limit) { - if (iter.hasNext()) { - results.add(iter.next()); - } else { - break; - } - } - return results; - } - - public static void waitUntilJobIsRunning(ClusterClient client) { - Awaitility.await("job should be running") - .atMost(Duration.ofSeconds(30)) - .pollInterval(Duration.ofMillis(10)) - .untilAsserted(() -> assertThat(getRunningJobs(client)).isNotEmpty()); - } - - public static List getRunningJobs(ClusterClient client) throws Exception { - Collection statusMessages = client.listJobs().get(); - return statusMessages.stream() - .filter(status -> status.getJobState() == JobStatus.RUNNING) - .map(JobStatusMessage::getJobId) - .collect(Collectors.toList()); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java deleted file mode 100644 index 7d991ee603c9..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java +++ /dev/null @@ -1,297 +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.iceberg.flink.source; - -import java.time.Duration; -import java.util.List; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.atomic.AtomicInteger; -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.execution.JobClient; -import org.apache.flink.runtime.highavailability.nonha.embedded.HaLeadershipControl; -import org.apache.flink.runtime.minicluster.MiniCluster; -import org.apache.flink.runtime.minicluster.RpcServiceSharing; -import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.data.RowData; -import org.apache.flink.test.util.MiniClusterWithClientResource; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.HadoopTableResource; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.sink.FlinkSink; -import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class TestIcebergSourceFailover { - - private static final int PARALLELISM = 4; - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @Rule - public final MiniClusterWithClientResource miniClusterResource = - new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(1) - .setNumberSlotsPerTaskManager(PARALLELISM) - .setRpcServiceSharing(RpcServiceSharing.DEDICATED) - .withHaLeadershipControl() - .build()); - - @Rule - public final HadoopTableResource sourceTableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, schema()); - - @Rule - public final HadoopTableResource sinkTableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.SINK_TABLE, schema()); - - protected IcebergSource.Builder sourceBuilder() { - Configuration config = new Configuration(); - config.setInteger(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 128); - return IcebergSource.forRowData() - .tableLoader(sourceTableResource.tableLoader()) - .assignerFactory(new SimpleSplitAssignerFactory()) - .flinkConfig(config); - } - - protected Schema schema() { - return TestFixtures.SCHEMA; - } - - protected List generateRecords(int numRecords, long seed) { - return RandomGenericData.generate(schema(), numRecords, seed); - } - - protected void assertRecords(Table table, List expectedRecords, Duration timeout) - throws Exception { - SimpleDataUtil.assertTableRecords(table, expectedRecords, timeout); - } - - @Test - public void testBoundedWithTaskManagerFailover() throws Exception { - testBoundedIcebergSource(FailoverType.TM); - } - - @Test - public void testBoundedWithJobManagerFailover() throws Exception { - testBoundedIcebergSource(FailoverType.JM); - } - - private void testBoundedIcebergSource(FailoverType failoverType) throws Exception { - List expectedRecords = Lists.newArrayList(); - GenericAppenderHelper dataAppender = - new GenericAppenderHelper( - sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); - for (int i = 0; i < 4; ++i) { - List records = generateRecords(2, i); - expectedRecords.addAll(records); - dataAppender.appendToTable(records); - } - - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(PARALLELISM); - env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); - - DataStream stream = - env.fromSource( - sourceBuilder().build(), - WatermarkStrategy.noWatermarks(), - "IcebergSource", - TypeInformation.of(RowData.class)); - - DataStream streamFailingInTheMiddleOfReading = - RecordCounterToFail.wrapWithFailureAfter(stream, expectedRecords.size() / 2); - - // CollectStreamSink from DataStream#executeAndCollect() doesn't guarantee - // exactly-once behavior. When Iceberg sink, we can verify end-to-end - // exactly-once. Here we mainly about source exactly-once behavior. - FlinkSink.forRowData(streamFailingInTheMiddleOfReading) - .table(sinkTableResource.table()) - .tableLoader(sinkTableResource.tableLoader()) - .append(); - - JobClient jobClient = env.executeAsync("Bounded Iceberg Source Failover Test"); - JobID jobId = jobClient.getJobID(); - - RecordCounterToFail.waitToFail(); - triggerFailover( - failoverType, - jobId, - RecordCounterToFail::continueProcessing, - miniClusterResource.getMiniCluster()); - - assertRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); - } - - @Test - public void testContinuousWithTaskManagerFailover() throws Exception { - testContinuousIcebergSource(FailoverType.TM); - } - - @Test - public void testContinuousWithJobManagerFailover() throws Exception { - testContinuousIcebergSource(FailoverType.JM); - } - - private void testContinuousIcebergSource(FailoverType failoverType) throws Exception { - GenericAppenderHelper dataAppender = - new GenericAppenderHelper( - sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); - List expectedRecords = Lists.newArrayList(); - - List batch = generateRecords(2, 0); - expectedRecords.addAll(batch); - dataAppender.appendToTable(batch); - - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(PARALLELISM); - env.enableCheckpointing(10L); - Configuration config = new Configuration(); - config.setInteger(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 128); - - DataStream stream = - env.fromSource( - sourceBuilder() - .streaming(true) - .monitorInterval(Duration.ofMillis(10)) - .streamingStartingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(), - WatermarkStrategy.noWatermarks(), - "IcebergSource", - TypeInformation.of(RowData.class)); - - // CollectStreamSink from DataStream#executeAndCollect() doesn't guarantee - // exactly-once behavior. When Iceberg sink, we can verify end-to-end - // exactly-once. Here we mainly about source exactly-once behavior. - FlinkSink.forRowData(stream) - .table(sinkTableResource.table()) - .tableLoader(sinkTableResource.tableLoader()) - .append(); - - JobClient jobClient = env.executeAsync("Continuous Iceberg Source Failover Test"); - JobID jobId = jobClient.getJobID(); - - for (int i = 1; i < 5; i++) { - Thread.sleep(10); - List records = generateRecords(2, i); - expectedRecords.addAll(records); - dataAppender.appendToTable(records); - if (i == 2) { - triggerFailover(failoverType, jobId, () -> {}, miniClusterResource.getMiniCluster()); - } - } - - // wait longer for continuous source to reduce flakiness - // because CI servers tend to be overloaded. - assertRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); - } - - // ------------------------------------------------------------------------ - // test utilities copied from Flink's FileSourceTextLinesITCase - // ------------------------------------------------------------------------ - - private enum FailoverType { - NONE, - TM, - JM - } - - private static void triggerFailover( - FailoverType type, JobID jobId, Runnable afterFailAction, MiniCluster miniCluster) - throws Exception { - switch (type) { - case NONE: - afterFailAction.run(); - break; - case TM: - restartTaskManager(afterFailAction, miniCluster); - break; - case JM: - triggerJobManagerFailover(jobId, afterFailAction, miniCluster); - break; - } - } - - private static void triggerJobManagerFailover( - JobID jobId, Runnable afterFailAction, MiniCluster miniCluster) throws Exception { - HaLeadershipControl haLeadershipControl = miniCluster.getHaLeadershipControl().get(); - haLeadershipControl.revokeJobMasterLeadership(jobId).get(); - afterFailAction.run(); - haLeadershipControl.grantJobMasterLeadership(jobId).get(); - } - - private static void restartTaskManager(Runnable afterFailAction, MiniCluster miniCluster) - throws Exception { - miniCluster.terminateTaskManager(0).get(); - afterFailAction.run(); - miniCluster.startTaskManager(); - } - - private static class RecordCounterToFail { - - private static AtomicInteger records; - private static CompletableFuture fail; - private static CompletableFuture continueProcessing; - - private static DataStream wrapWithFailureAfter(DataStream stream, int failAfter) { - - records = new AtomicInteger(); - fail = new CompletableFuture<>(); - continueProcessing = new CompletableFuture<>(); - return stream.map( - record -> { - boolean reachedFailPoint = records.incrementAndGet() > failAfter; - boolean notFailedYet = !fail.isDone(); - if (notFailedYet && reachedFailPoint) { - fail.complete(null); - continueProcessing.get(); - } - return record; - }); - } - - private static void waitToFail() throws ExecutionException, InterruptedException { - fail.get(); - } - - private static void continueProcessing() { - continueProcessing.complete(null); - } - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java deleted file mode 100644 index f7dc931c506c..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java +++ /dev/null @@ -1,112 +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.iceberg.flink.source; - -import java.time.Duration; -import java.time.Instant; -import java.time.LocalDateTime; -import java.time.ZoneId; -import java.time.ZoneOffset; -import java.util.List; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; -import java.util.stream.Collectors; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.types.Comparators; -import org.apache.iceberg.util.StructLikeWrapper; -import org.awaitility.Awaitility; - -public class TestIcebergSourceFailoverWithWatermarkExtractor extends TestIcebergSourceFailover { - // Increment ts by 15 minutes for each generateRecords batch - private static final long RECORD_BATCH_TS_INCREMENT_MILLI = TimeUnit.MINUTES.toMillis(15); - // Within a batch, increment ts by 1 second - private static final long RECORD_TS_INCREMENT_MILLI = TimeUnit.SECONDS.toMillis(1); - - private final AtomicLong tsMilli = new AtomicLong(System.currentTimeMillis()); - - @Override - protected IcebergSource.Builder sourceBuilder() { - return IcebergSource.builder() - .tableLoader(sourceTableResource.tableLoader()) - .watermarkColumn("ts") - .project(TestFixtures.TS_SCHEMA); - } - - @Override - protected Schema schema() { - return TestFixtures.TS_SCHEMA; - } - - @Override - protected List generateRecords(int numRecords, long seed) { - // Override the ts field to create a more realistic situation for event time alignment - tsMilli.addAndGet(RECORD_BATCH_TS_INCREMENT_MILLI); - return RandomGenericData.generate(schema(), numRecords, seed).stream() - .peek( - record -> { - LocalDateTime ts = - LocalDateTime.ofInstant( - Instant.ofEpochMilli(tsMilli.addAndGet(RECORD_TS_INCREMENT_MILLI)), - ZoneId.of("Z")); - record.setField("ts", ts); - }) - .collect(Collectors.toList()); - } - - /** - * This override is needed because {@link Comparators} used by {@link StructLikeWrapper} retrieves - * Timestamp type using Long type as inner class, while the {@link RandomGenericData} generates - * {@link LocalDateTime} for {@code TimestampType.withoutZone()}. This method normalizes the - * {@link LocalDateTime} to a Long type so that Comparators can continue to work. - */ - @Override - protected void assertRecords(Table table, List expectedRecords, Duration timeout) - throws Exception { - List expectedNormalized = convertLocalDateTimeToMilli(expectedRecords); - Awaitility.await("expected list of records should be produced") - .atMost(timeout) - .untilAsserted( - () -> { - SimpleDataUtil.equalsRecords( - expectedNormalized, - convertLocalDateTimeToMilli(SimpleDataUtil.tableRecords(table)), - table.schema()); - SimpleDataUtil.assertRecordsEqual( - expectedNormalized, - convertLocalDateTimeToMilli(SimpleDataUtil.tableRecords(table)), - table.schema()); - }); - } - - private List convertLocalDateTimeToMilli(List records) { - return records.stream() - .peek( - r -> { - LocalDateTime localDateTime = ((LocalDateTime) r.getField("ts")); - r.setField("ts", localDateTime.atZone(ZoneOffset.UTC).toInstant().toEpochMilli()); - }) - .collect(Collectors.toList()); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java deleted file mode 100644 index 2974f4bc94a2..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java +++ /dev/null @@ -1,114 +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.iceberg.flink.source; - -import java.io.IOException; -import java.util.List; -import java.util.Map; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.test.util.MiniClusterWithClientResource; -import org.apache.flink.util.CloseableIterator; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.flink.CatalogLoader; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.util.StructLikeSet; -import org.junit.ClassRule; -import org.junit.rules.TemporaryFolder; - -public class TestIcebergSourceReaderDeletes extends TestFlinkReaderDeletesBase { - - private static final int PARALLELISM = 4; - - @ClassRule public static final TemporaryFolder TMP_FOLDER = new TemporaryFolder(); - - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER = - new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(1) - .setNumberSlotsPerTaskManager(PARALLELISM) - .build()); - - public TestIcebergSourceReaderDeletes(FileFormat inputFormat) { - super(inputFormat); - } - - @Override - protected StructLikeSet rowSet(String tableName, Table testTable, String... columns) - throws IOException { - Schema projected = testTable.schema().select(columns); - RowType rowType = FlinkSchemaUtil.convert(projected); - - Map properties = Maps.newHashMap(); - properties.put( - CatalogProperties.WAREHOUSE_LOCATION, - hiveConf.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname)); - properties.put(CatalogProperties.URI, hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname)); - properties.put( - CatalogProperties.CLIENT_POOL_SIZE, - Integer.toString(hiveConf.getInt("iceberg.hive.client-pool-size", 5))); - CatalogLoader hiveCatalogLoader = CatalogLoader.hive(catalog.name(), hiveConf, properties); - TableLoader hiveTableLoader = - TableLoader.fromCatalog(hiveCatalogLoader, TableIdentifier.of("default", tableName)); - hiveTableLoader.open(); - try (TableLoader tableLoader = hiveTableLoader) { - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(1); - DataStream stream = - env.fromSource( - IcebergSource.builder() - .tableLoader(tableLoader) - .assignerFactory(new SimpleSplitAssignerFactory()) - .project(projected) - .build(), - WatermarkStrategy.noWatermarks(), - "testBasicRead", - TypeInformation.of(RowData.class)); - - try (CloseableIterator iter = stream.executeAndCollect()) { - List rowDataList = Lists.newArrayList(iter); - StructLikeSet set = StructLikeSet.create(projected.asStruct()); - rowDataList.forEach( - rowData -> { - RowDataWrapper wrapper = new RowDataWrapper(rowType, projected.asStruct()); - set.add(wrapper.wrap(rowData)); - }); - return set; - } catch (Exception e) { - throw new IOException("Failed to collect result", e); - } - } - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java deleted file mode 100644 index e66ae79c28f8..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java +++ /dev/null @@ -1,42 +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.iceberg.flink.source; - -import java.io.IOException; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.table.api.config.TableConfigOptions; -import org.apache.iceberg.flink.FlinkConfigOptions; - -/** Use the IcebergSource (FLIP-27) */ -public class TestIcebergSourceSql extends TestSqlBase { - @Override - public void before() throws IOException { - Configuration tableConf = getTableEnv().getConfig().getConfiguration(); - tableConf.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), true); - SqlHelpers.sql( - getTableEnv(), - "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - catalogResource.warehouse()); - SqlHelpers.sql(getTableEnv(), "use catalog iceberg_catalog"); - getTableEnv() - .getConfig() - .getConfiguration() - .set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java deleted file mode 100644 index ac4d07fa52d3..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java +++ /dev/null @@ -1,485 +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.iceberg.flink.source; - -import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT; -import static org.assertj.core.api.AssertionsForClassTypes.assertThat; - -import java.io.Serializable; -import java.time.Duration; -import java.time.Instant; -import java.time.LocalDateTime; -import java.time.ZoneId; -import java.util.Collection; -import java.util.List; -import java.util.Optional; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.stream.Collectors; -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.eventtime.SerializableTimestampAssigner; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.core.execution.JobClient; -import org.apache.flink.metrics.Gauge; -import org.apache.flink.runtime.metrics.MetricNames; -import org.apache.flink.runtime.minicluster.RpcServiceSharing; -import org.apache.flink.runtime.testutils.InMemoryReporter; -import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction; -import org.apache.flink.streaming.api.operators.collect.CollectResultIterator; -import org.apache.flink.streaming.api.operators.collect.CollectSinkOperator; -import org.apache.flink.streaming.api.operators.collect.CollectSinkOperatorFactory; -import org.apache.flink.streaming.api.operators.collect.CollectStreamSink; -import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; -import org.apache.flink.streaming.api.windowing.time.Time; -import org.apache.flink.streaming.api.windowing.windows.TimeWindow; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.test.util.MiniClusterWithClientResource; -import org.apache.flink.util.CloseableIterator; -import org.apache.flink.util.Collector; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.HadoopTableResource; -import org.apache.iceberg.flink.RowDataConverter; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.awaitility.Awaitility; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class TestIcebergSourceWithWatermarkExtractor implements Serializable { - private static final InMemoryReporter reporter = InMemoryReporter.createWithRetainedMetrics(); - private static final int PARALLELISM = 4; - private static final String SOURCE_NAME = "IcebergSource"; - private static final int RECORD_NUM_FOR_2_SPLITS = 200; - private static final ConcurrentMap windows = Maps.newConcurrentMap(); - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @Rule - public final MiniClusterWithClientResource miniClusterResource = - new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(1) - .setNumberSlotsPerTaskManager(PARALLELISM) - .setRpcServiceSharing(RpcServiceSharing.DEDICATED) - .setConfiguration( - reporter.addToConfiguration( - // disable classloader check as Avro may cache class in the serializers. - new Configuration().set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false))) - .withHaLeadershipControl() - .build()); - - @Rule - public final HadoopTableResource sourceTableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.TS_SCHEMA); - - /** - * This is an integration test for watermark handling and windowing. Integration testing the - * following features: - * - *

      - *
    • - Ordering of the splits - *
    • - Emitting of watermarks - *
    • - Firing windows based on watermarks - *
    - * - *

    The test generates 4 splits - * - *

      - *
    • - Split 1 - Watermark 100 min - *
    • - Split 2, 3 - Watermark 0 min - *
    • - Split 4 - Watermark 6 min - *
    - * - *

    Creates a source with 5 minutes tumbling window with parallelism 1 (to prevent concurrency - * issues). - * - *

    Checks that windows are handled correctly based on the emitted watermarks, and splits are - * read in the following order: - * - *

      - *
    • - Split 2, 3 - *
    • - Split 4 - *
    • - Split 1 - *
    - * - *

    As a result the window aggregator emits the records based on in Split 2-3, and Split 4 data. - * - *

    Add 2 more splits, so the task manager close the windows for the original 4 splits and emit - * the appropriate aggregated records. - */ - @Test - public void testWindowing() throws Exception { - GenericAppenderHelper dataAppender = appender(); - List expectedRecords = Lists.newArrayList(); - - // Generate records with the following pattern: - // - File 1 - Later records (Watermark 6000000) - // - Split 1 - 2 records (100, "file_1-recordTs_100"), (103, "file_1-recordTs_103") - // - File 2 - First records (Watermark 0) - // - Split 1 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... - // - Split 2 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... - // - File 3 - Parallel write for the first records (Watermark 360000) - // - Split 1 - 2 records (6, "file_3-recordTs_6"), (7, "file_3-recordTs_7") - List batch = - ImmutableList.of( - generateRecord(100, "file_1-recordTs_100"), - generateRecord(101, "file_1-recordTs_101"), - generateRecord(103, "file_1-recordTs_103")); - expectedRecords.addAll(batch); - dataAppender.appendToTable(batch); - - batch = Lists.newArrayListWithCapacity(100); - for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { - // Generate records where the timestamps are out of order, but still between 0-5 minutes - batch.add(generateRecord(4 - i % 5, "file_2-recordTs_" + i)); - } - expectedRecords.addAll(batch); - dataAppender.appendToTable(batch); - - batch = - ImmutableList.of( - generateRecord(6, "file_3-recordTs_6"), generateRecord(7, "file_3-recordTs_7")); - expectedRecords.addAll(batch); - dataAppender.appendToTable(batch); - - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(1); - - DataStream stream = - env.fromSource( - source(), - WatermarkStrategy.noWatermarks() - .withTimestampAssigner(new RowDataTimestampAssigner()), - SOURCE_NAME, - TypeInformation.of(RowData.class)); - - stream - .windowAll(TumblingEventTimeWindows.of(Time.minutes(5))) - .apply( - new AllWindowFunction() { - @Override - public void apply( - TimeWindow window, Iterable values, Collector out) { - // Emit RowData which contains the window start time, and the record count in - // that window - AtomicInteger count = new AtomicInteger(0); - values.forEach(a -> count.incrementAndGet()); - out.collect(row(window.getStart(), count.get())); - windows.put(window.getStart(), count.get()); - } - }); - - // Use static variable to collect the windows, since other solutions were flaky - windows.clear(); - env.executeAsync("Iceberg Source Windowing Test"); - - // Wait for the 2 first windows from File 2 and File 3 - Awaitility.await() - .pollInterval(Duration.ofMillis(10)) - .atMost(30, TimeUnit.SECONDS) - .until( - () -> - windows.equals( - ImmutableMap.of(0L, RECORD_NUM_FOR_2_SPLITS, TimeUnit.MINUTES.toMillis(5), 2))); - - // Write data so the windows containing test data are closed - dataAppender.appendToTable( - dataAppender.writeFile(ImmutableList.of(generateRecord(1500, "last-record")))); - - // Wait for last test record window from File 1 - Awaitility.await() - .pollInterval(Duration.ofMillis(10)) - .atMost(30, TimeUnit.SECONDS) - .until( - () -> - windows.equals( - ImmutableMap.of( - 0L, - RECORD_NUM_FOR_2_SPLITS, - TimeUnit.MINUTES.toMillis(5), - 2, - TimeUnit.MINUTES.toMillis(100), - 3))); - } - - /** - * This is an integration test for watermark handling and throttling. Integration testing the - * following: - * - *

      - *
    • - Emitting of watermarks - *
    • - Watermark alignment - *
    - * - *

    The test generates 3 splits - * - *

      - *
    • - Split 1 - Watermark 100 min - *
    • - Split 2, 3 - Watermark 0 min - *
    - * - * The splits are read in the following order: - * - *
      - *
    • - Split 2, 3 (Task Manager 1, Task Manager 2) - *
    • - Split 1 (Task Manager 1 or ask Manager 2 depending on scheduling) - *
    - * - * Reading split 1 will cause the watermark alignment to pause reading for the given task manager. - * - *

    The status of the watermark alignment is checked by the alignment related metrics. - * - *

    Adding new records with old timestamps to the table will enable the running reader to - * continue reading the files, but the watermark alignment will still prevent the paused reader to - * continue. - * - *

    After adding some records with new timestamps the blocked reader is un-paused, and both ot - * the readers continue reading. - */ - @Test - public void testThrottling() throws Exception { - GenericAppenderHelper dataAppender = appender(); - - // Generate records with the following pattern: - // - File 1 - Later records (Watermark 6000000) - // - Split 1 - 2 records (100, "file_1-recordTs_100"), (103, "file_1-recordTs_103") - // - File 2 - First records (Watermark 0) - // - Split 1 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... - // - Split 2 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... - List batch = - ImmutableList.of( - generateRecord(100, "file_1-recordTs_100"), generateRecord(103, "file_1-recordTs_103")); - dataAppender.appendToTable(batch); - - batch = Lists.newArrayListWithCapacity(100); - for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { - batch.add(generateRecord(4 - i % 5, "file_2-recordTs_" + i)); - } - - dataAppender.appendToTable(batch); - - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(2); - - DataStream stream = - env.fromSource( - source(), - WatermarkStrategy.noWatermarks() - .withWatermarkAlignment("iceberg", Duration.ofMinutes(20), Duration.ofMillis(10)), - SOURCE_NAME, - TypeInformation.of(RowData.class)); - - // Flink 1.15 only change - start - CollectResultIterator resultStream = addCollectSink(stream); - - // Start the job - JobClient jobClient = env.executeAsync("Iceberg Source Throttling Test"); - resultStream.setJobClient(jobClient); - try (CollectResultIterator resultIterator = resultStream) { - // Flink 1.15 only change - end - - // Check that the read the non-blocked data - // The first RECORD_NUM_FOR_2_SPLITS should be read - // 1 or more from the runaway reader should be arrived depending on thread scheduling - waitForRecords(resultIterator, RECORD_NUM_FOR_2_SPLITS + 1); - - // Get the drift metric, wait for it to be created and reach the expected state - // (100 min - 20 min - 0 min) - // Also this validates that the WatermarkAlignment is working - Awaitility.await() - .pollInterval(Duration.ofMillis(10)) - .atMost(30, TimeUnit.SECONDS) - .until( - () -> - findAlignmentDriftMetric(jobClient.getJobID(), TimeUnit.MINUTES.toMillis(80)) - .isPresent()); - Gauge drift = - findAlignmentDriftMetric(jobClient.getJobID(), TimeUnit.MINUTES.toMillis(80)).get(); - - // Add some old records with 2 splits, so even if the blocked gets one split, the other reader - // one gets one as well - List newBatch1 = - ImmutableList.of( - generateRecord(15, "file_3-recordTs_15"), - generateRecord(16, "file_3-recordTs_16"), - generateRecord(17, "file_3-recordTs_17")); - List newBatch2 = - ImmutableList.of( - generateRecord(15, "file_4-recordTs_15"), - generateRecord(16, "file_4-recordTs_16"), - generateRecord(17, "file_4-recordTs_17")); - dataAppender.appendToTable( - dataAppender.writeFile(newBatch1), dataAppender.writeFile(newBatch2)); - // The records received will highly depend on scheduling - // We minimally get 3 records from the non-blocked reader - // We might get 1 record from the blocked reader (as part of the previous batch - - // file_1-recordTs_100) - // We might get 3 records form the non-blocked reader if it gets both new splits - waitForRecords(resultIterator, 3); - - // Get the drift metric, wait for it to be created and reach the expected state (100 min - 20 - // min - 15 min) - Awaitility.await() - .pollInterval(Duration.ofMillis(10)) - .atMost(30, TimeUnit.SECONDS) - .until(() -> drift.getValue() == TimeUnit.MINUTES.toMillis(65)); - - // Add some new records which should unblock the throttled reader - batch = - ImmutableList.of( - generateRecord(90, "file_5-recordTs_90"), generateRecord(91, "file_5-recordTs_91")); - dataAppender.appendToTable(batch); - // We should get all the records at this point - waitForRecords(resultIterator, 6); - - // Wait for the new drift to decrease below the allowed drift to signal the normal state - Awaitility.await() - .pollInterval(Duration.ofMillis(10)) - .atMost(30, TimeUnit.SECONDS) - .until(() -> drift.getValue() < TimeUnit.MINUTES.toMillis(20)); - } - } - - protected IcebergSource source() { - return IcebergSource.builder() - .tableLoader(sourceTableResource.tableLoader()) - .watermarkColumn("ts") - .project(TestFixtures.TS_SCHEMA) - .splitSize(100L) - .streaming(true) - .monitorInterval(Duration.ofMillis(2)) - .streamingStartingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - } - - protected Record generateRecord(int minutes, String str) { - // Override the ts field to create a more realistic situation for event time alignment - Record record = GenericRecord.create(TestFixtures.TS_SCHEMA); - LocalDateTime ts = - LocalDateTime.ofInstant( - Instant.ofEpochMilli(Time.of(minutes, TimeUnit.MINUTES).toMilliseconds()), - ZoneId.of("Z")); - record.setField("ts", ts); - record.setField("str", str); - return record; - } - - protected void assertRecords( - Collection expectedRecords, CloseableIterator iterator) throws Exception { - Set expected = - expectedRecords.stream() - .map(e -> RowDataConverter.convert(TestFixtures.TS_SCHEMA, e)) - .collect(Collectors.toSet()); - Assert.assertEquals(expected, waitForRecords(iterator, expectedRecords.size())); - } - - protected Set waitForRecords(CloseableIterator iterator, int num) { - Set received = Sets.newHashSetWithExpectedSize(num); - assertThat( - CompletableFuture.supplyAsync( - () -> { - int count = 0; - while (count < num && iterator.hasNext()) { - received.add(iterator.next()); - count++; - } - - if (count < num) { - throw new IllegalStateException(String.format("Fail to get %d records.", num)); - } - - return true; - })) - .succeedsWithin(DEFAULT_COLLECT_DATA_TIMEOUT); - - return received; - } - - private Optional> findAlignmentDriftMetric(JobID jobID, long withValue) { - String metricsName = SOURCE_NAME + ".*" + MetricNames.WATERMARK_ALIGNMENT_DRIFT; - return reporter.findMetrics(jobID, metricsName).values().stream() - .map(m -> (Gauge) m) - .filter(m -> m.getValue() == withValue) - .findFirst(); - } - - private GenericAppenderHelper appender() { - // We need to create multiple splits, so we need to generate parquet files with multiple offsets - org.apache.hadoop.conf.Configuration hadoopConf = new org.apache.hadoop.conf.Configuration(); - hadoopConf.set("write.parquet.page-size-bytes", "64"); - hadoopConf.set("write.parquet.row-group-size-bytes", "64"); - return new GenericAppenderHelper( - sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER, hadoopConf); - } - - private static RowData row(long time, long count) { - GenericRowData result = new GenericRowData(2); - result.setField(0, time); - result.setField(1, String.valueOf(count)); - return result; - } - - private static class RowDataTimestampAssigner implements SerializableTimestampAssigner { - @Override - public long extractTimestamp(RowData element, long recordTimestamp) { - return element.getTimestamp(0, 0).getMillisecond(); - } - } - - // Flink 1.15 only method - private CollectResultIterator addCollectSink(DataStream stream) { - TypeSerializer serializer = - stream.getType().createSerializer(stream.getExecutionConfig()); - String accumulatorName = "dataStreamCollect_" + UUID.randomUUID(); - CollectSinkOperatorFactory factory = - new CollectSinkOperatorFactory<>(serializer, accumulatorName); - CollectSinkOperator operator = (CollectSinkOperator) factory.getOperator(); - CollectStreamSink sink = new CollectStreamSink<>(stream, factory); - sink.name("Data stream collect sink"); - stream.getExecutionEnvironment().addOperator(sink.getTransformation()); - return new CollectResultIterator<>( - operator.getOperatorIdFuture(), - serializer, - accumulatorName, - stream.getExecutionEnvironment().getCheckpointConfig()); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java deleted file mode 100644 index cb6fda18a1ee..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java +++ /dev/null @@ -1,301 +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.iceberg.flink.source; - -import static org.apache.iceberg.types.Types.NestedField.optional; -import static org.apache.iceberg.types.Types.NestedField.required; - -import java.io.IOException; -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import java.util.Base64; -import java.util.List; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.types.Row; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.Files; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.data.FileHelpers; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.FlinkCatalogTestBase; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Types; -import org.junit.After; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runners.Parameterized; - -public class TestMetadataTableReadableMetrics extends FlinkCatalogTestBase { - private static final String TABLE_NAME = "test_table"; - - public TestMetadataTableReadableMetrics(String catalogName, Namespace baseNamespace) { - super(catalogName, baseNamespace); - } - - @Parameterized.Parameters(name = "catalogName={0}, baseNamespace={1}") - public static Iterable parameters() { - List parameters = Lists.newArrayList(); - String catalogName = "testhive"; - Namespace baseNamespace = Namespace.empty(); - parameters.add(new Object[] {catalogName, baseNamespace}); - return parameters; - } - - @Override - protected TableEnvironment getTableEnv() { - Configuration configuration = super.getTableEnv().getConfig().getConfiguration(); - configuration.set(CoreOptions.DEFAULT_PARALLELISM, 1); - return super.getTableEnv(); - } - - @Rule public TemporaryFolder temp = new TemporaryFolder(); - - private static final Types.StructType LEAF_STRUCT_TYPE = - Types.StructType.of( - optional(1, "leafLongCol", Types.LongType.get()), - optional(2, "leafDoubleCol", Types.DoubleType.get())); - - private static final Types.StructType NESTED_STRUCT_TYPE = - Types.StructType.of(required(3, "leafStructCol", LEAF_STRUCT_TYPE)); - - private static final Schema NESTED_SCHEMA = - new Schema(required(4, "nestedStructCol", NESTED_STRUCT_TYPE)); - - private static final Schema PRIMITIVE_SCHEMA = - new Schema( - required(1, "booleanCol", Types.BooleanType.get()), - required(2, "intCol", Types.IntegerType.get()), - required(3, "longCol", Types.LongType.get()), - required(4, "floatCol", Types.FloatType.get()), - required(5, "doubleCol", Types.DoubleType.get()), - optional(6, "decimalCol", Types.DecimalType.of(10, 2)), - optional(7, "stringCol", Types.StringType.get()), - optional(8, "fixedCol", Types.FixedType.ofLength(3)), - optional(9, "binaryCol", Types.BinaryType.get())); - - private Table createPrimitiveTable() throws IOException { - Table table = - catalog.createTable( - TableIdentifier.of(DATABASE, TABLE_NAME), - PRIMITIVE_SCHEMA, - PartitionSpec.unpartitioned(), - ImmutableMap.of()); - List records = - Lists.newArrayList( - createPrimitiveRecord( - false, - 1, - 1L, - 0, - 1.0D, - new BigDecimal("1.00"), - "1", - Base64.getDecoder().decode("1111"), - ByteBuffer.wrap(Base64.getDecoder().decode("1111"))), - createPrimitiveRecord( - true, - 2, - 2L, - 0, - 2.0D, - new BigDecimal("2.00"), - "2", - Base64.getDecoder().decode("2222"), - ByteBuffer.wrap(Base64.getDecoder().decode("2222"))), - createPrimitiveRecord(false, 1, 1, Float.NaN, Double.NaN, null, "1", null, null), - createPrimitiveRecord( - false, 2, 2L, Float.NaN, 2.0D, new BigDecimal("2.00"), "2", null, null)); - - DataFile dataFile = - FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), records); - table.newAppend().appendFile(dataFile).commit(); - return table; - } - - private void createNestedTable() throws IOException { - Table table = - validationCatalog.createTable( - TableIdentifier.of(DATABASE, TABLE_NAME), - NESTED_SCHEMA, - PartitionSpec.unpartitioned(), - ImmutableMap.of()); - - List records = - Lists.newArrayList( - createNestedRecord(0L, 0.0), - createNestedRecord(1L, Double.NaN), - createNestedRecord(null, null)); - DataFile dataFile = - FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), records); - table.newAppend().appendFile(dataFile).commit(); - } - - @Before - public void before() { - super.before(); - sql("USE CATALOG %s", catalogName); - sql("CREATE DATABASE %s", DATABASE); - sql("USE %s", DATABASE); - } - - @Override - @After - public void clean() { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); - super.clean(); - } - - protected GenericRecord createPrimitiveRecord( - boolean booleanCol, - int intCol, - long longCol, - float floatCol, - double doubleCol, - BigDecimal decimalCol, - String stringCol, - byte[] fixedCol, - ByteBuffer binaryCol) { - GenericRecord record = GenericRecord.create(PRIMITIVE_SCHEMA); - record.set(0, booleanCol); - record.set(1, intCol); - record.set(2, longCol); - record.set(3, floatCol); - record.set(4, doubleCol); - record.set(5, decimalCol); - record.set(6, stringCol); - record.set(7, fixedCol); - record.set(8, binaryCol); - return record; - } - - private GenericRecord createNestedRecord(Long longCol, Double doubleCol) { - GenericRecord record = GenericRecord.create(NESTED_SCHEMA); - GenericRecord nested = GenericRecord.create(NESTED_STRUCT_TYPE); - GenericRecord leaf = GenericRecord.create(LEAF_STRUCT_TYPE); - leaf.set(0, longCol); - leaf.set(1, doubleCol); - nested.set(0, leaf); - record.set(0, nested); - return record; - } - - protected Object[] row(Object... values) { - return values; - } - - @Test - public void testPrimitiveColumns() throws Exception { - createPrimitiveTable(); - List result = sql("SELECT readable_metrics FROM %s$files", TABLE_NAME); - - Row binaryCol = - Row.of( - 59L, - 4L, - 2L, - null, - Base64.getDecoder().decode("1111"), - Base64.getDecoder().decode("2222")); - Row booleanCol = Row.of(44L, 4L, 0L, null, false, true); - Row decimalCol = Row.of(97L, 4L, 1L, null, new BigDecimal("1.00"), new BigDecimal("2.00")); - Row doubleCol = Row.of(99L, 4L, 0L, 1L, 1.0D, 2.0D); - Row fixedCol = - Row.of( - 55L, - 4L, - 2L, - null, - Base64.getDecoder().decode("1111"), - Base64.getDecoder().decode("2222")); - Row floatCol = Row.of(90L, 4L, 0L, 2L, 0f, 0f); - Row intCol = Row.of(91L, 4L, 0L, null, 1, 2); - Row longCol = Row.of(91L, 4L, 0L, null, 1L, 2L); - Row stringCol = Row.of(99L, 4L, 0L, null, "1", "2"); - - List expected = - Lists.newArrayList( - Row.of( - Row.of( - binaryCol, - booleanCol, - decimalCol, - doubleCol, - fixedCol, - floatCol, - intCol, - longCol, - stringCol))); - TestHelpers.assertRows(result, expected); - } - - @Test - public void testSelectPrimitiveValues() throws Exception { - createPrimitiveTable(); - - TestHelpers.assertRows( - sql( - "SELECT readable_metrics.intCol.lower_bound, readable_metrics.booleanCol.upper_bound FROM %s$files", - TABLE_NAME), - ImmutableList.of(Row.of(1, true))); - - TestHelpers.assertRows( - sql("SELECT content, readable_metrics.longCol.value_count FROM %s$files", TABLE_NAME), - ImmutableList.of(Row.of(0, 4L))); - - TestHelpers.assertRows( - sql("SELECT readable_metrics.longCol.value_count, content FROM %s$files", TABLE_NAME), - ImmutableList.of(Row.of(4L, 0))); - } - - @Test - public void testSelectNestedValues() throws Exception { - createNestedTable(); - TestHelpers.assertRows( - sql( - "SELECT readable_metrics.`nestedStructCol.leafStructCol.leafLongCol`.lower_bound, " - + "readable_metrics.`nestedStructCol.leafStructCol.leafDoubleCol`.value_count FROM %s$files", - TABLE_NAME), - ImmutableList.of(Row.of(0L, 3L))); - } - - @Test - public void testNestedValues() throws Exception { - createNestedTable(); - - Row leafDoubleCol = Row.of(53L, 3L, 1L, 1L, 0.0D, 0.0D); - Row leafLongCol = Row.of(54L, 3L, 1L, null, 0L, 1L); - Row metrics = Row.of(Row.of(leafDoubleCol, leafLongCol)); - - TestHelpers.assertRows( - sql("SELECT readable_metrics FROM %s$files", TABLE_NAME), ImmutableList.of(metrics)); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java deleted file mode 100644 index 25ecec23d216..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java +++ /dev/null @@ -1,186 +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.iceberg.flink.source; - -import java.io.IOException; -import java.util.List; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.RowDelta; -import org.apache.iceberg.SerializableTable; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.flink.data.RowDataProjection; -import org.apache.iceberg.flink.sink.RowDataTaskWriterFactory; -import org.apache.iceberg.flink.sink.TaskWriterFactory; -import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.WriteResult; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestProjectMetaColumn { - - @Rule public final TemporaryFolder folder = new TemporaryFolder(); - private final FileFormat format; - - @Parameterized.Parameters(name = "fileFormat={0}") - public static Iterable parameters() { - return Lists.newArrayList( - new Object[] {FileFormat.PARQUET}, - new Object[] {FileFormat.ORC}, - new Object[] {FileFormat.AVRO}); - } - - public TestProjectMetaColumn(FileFormat format) { - this.format = format; - } - - private void testSkipToRemoveMetaColumn(int formatVersion) throws IOException { - // Create the table with given format version. - String location = folder.getRoot().getAbsolutePath(); - Table table = - SimpleDataUtil.createTable( - location, - ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)), - false); - - List rows = - Lists.newArrayList( - SimpleDataUtil.createInsert(1, "AAA"), - SimpleDataUtil.createInsert(2, "BBB"), - SimpleDataUtil.createInsert(3, "CCC")); - writeAndCommit(table, ImmutableList.of(), false, rows); - - FlinkInputFormat input = - FlinkSource.forRowData().tableLoader(TableLoader.fromHadoopTable(location)).buildFormat(); - - List results = Lists.newArrayList(); - TestHelpers.readRowData( - input, - rowData -> { - // If project to remove the meta columns, it will get a RowDataProjection. - Assert.assertTrue(rowData instanceof GenericRowData); - results.add(TestHelpers.copyRowData(rowData, SimpleDataUtil.ROW_TYPE)); - }); - - // Assert the results. - TestHelpers.assertRows(rows, results, SimpleDataUtil.ROW_TYPE); - } - - @Test - public void testV1SkipToRemoveMetaColumn() throws IOException { - testSkipToRemoveMetaColumn(1); - } - - @Test - public void testV2SkipToRemoveMetaColumn() throws IOException { - testSkipToRemoveMetaColumn(2); - } - - @Test - public void testV2RemoveMetaColumn() throws Exception { - // Create the v2 table. - String location = folder.getRoot().getAbsolutePath(); - Table table = - SimpleDataUtil.createTable( - location, ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"), false); - - List rows = - Lists.newArrayList( - SimpleDataUtil.createInsert(1, "AAA"), - SimpleDataUtil.createDelete(1, "AAA"), - SimpleDataUtil.createInsert(2, "AAA"), - SimpleDataUtil.createInsert(2, "BBB")); - int eqFieldId = table.schema().findField("data").fieldId(); - writeAndCommit(table, ImmutableList.of(eqFieldId), true, rows); - - FlinkInputFormat input = - FlinkSource.forRowData().tableLoader(TableLoader.fromHadoopTable(location)).buildFormat(); - - List results = Lists.newArrayList(); - TestHelpers.readRowData( - input, - rowData -> { - // If project to remove the meta columns, it will get a RowDataProjection. - Assert.assertTrue(rowData instanceof RowDataProjection); - results.add(TestHelpers.copyRowData(rowData, SimpleDataUtil.ROW_TYPE)); - }); - - // Assert the results. - TestHelpers.assertRows( - ImmutableList.of( - SimpleDataUtil.createInsert(2, "AAA"), SimpleDataUtil.createInsert(2, "BBB")), - results, - SimpleDataUtil.ROW_TYPE); - } - - private void writeAndCommit( - Table table, List eqFieldIds, boolean upsert, List rows) - throws IOException { - TaskWriter writer = createTaskWriter(table, eqFieldIds, upsert); - try (TaskWriter io = writer) { - for (RowData row : rows) { - io.write(row); - } - } - - RowDelta delta = table.newRowDelta(); - WriteResult result = writer.complete(); - - for (DataFile dataFile : result.dataFiles()) { - delta.addRows(dataFile); - } - - for (DeleteFile deleteFile : result.deleteFiles()) { - delta.addDeletes(deleteFile); - } - - delta.commit(); - } - - private TaskWriter createTaskWriter( - Table table, List equalityFieldIds, boolean upsert) { - TaskWriterFactory taskWriterFactory = - new RowDataTaskWriterFactory( - SerializableTable.copyOf(table), - SimpleDataUtil.ROW_TYPE, - TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT, - format, - table.properties(), - equalityFieldIds, - upsert); - - taskWriterFactory.initialize(1, 1); - return taskWriterFactory.create(); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java deleted file mode 100644 index 485035787d6d..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java +++ /dev/null @@ -1,35 +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.iceberg.flink.source; - -import org.apache.avro.generic.GenericRecord; -import org.apache.iceberg.flink.AvroGenericRecordConverterBase; -import org.apache.iceberg.flink.DataGenerator; -import org.junit.Assert; - -public class TestRowDataToAvroGenericRecordConverter extends AvroGenericRecordConverterBase { - @Override - protected void testConverter(DataGenerator dataGenerator) { - RowDataToAvroGenericRecordConverter converter = - RowDataToAvroGenericRecordConverter.fromAvroSchema(dataGenerator.avroSchema()); - GenericRecord expected = dataGenerator.generateAvroGenericRecord(); - GenericRecord actual = converter.apply(dataGenerator.generateFlinkRowData()); - Assert.assertEquals(expected, actual); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java deleted file mode 100644 index 317301260f66..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java +++ /dev/null @@ -1,60 +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.iceberg.flink.source; - -import java.io.IOException; -import org.apache.flink.configuration.Configuration; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.junit.Assert; -import org.junit.Test; - -public class TestSourceUtil { - @Test - public void testInferedParallelism() throws IOException { - Configuration configuration = new Configuration(); - // Empty table, infer parallelism should be at least 1 - int parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 0); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); - - // 2 splits (max infer is the default value 100 , max > splits num), the parallelism is splits - // num : 2 - parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 2, parallelism); - - // 2 splits and limit is 1 , max infer parallelism is default 100, - // which is greater than splits num and limit, the parallelism is the limit value : 1 - parallelism = SourceUtil.inferParallelism(configuration, 1, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); - - // 2 splits and max infer parallelism is 1 (max < splits num), the parallelism is 1 - configuration.setInteger(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX, 1); - parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); - - // 2 splits, max infer parallelism is 1, limit is 3, the parallelism is max infer parallelism : - // 1 - parallelism = SourceUtil.inferParallelism(configuration, 3, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); - - // 2 splits, infer parallelism is disabled, the parallelism is flink default parallelism 1 - configuration.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); - parallelism = SourceUtil.inferParallelism(configuration, 3, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java deleted file mode 100644 index dda46033143e..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java +++ /dev/null @@ -1,158 +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.iceberg.flink.source; - -import java.io.IOException; -import java.util.List; -import java.util.Map; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.test.util.MiniClusterWithClientResource; -import org.apache.flink.types.Row; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Table; -import org.apache.iceberg.TestHelpers; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.HadoopCatalogResource; -import org.apache.iceberg.flink.MiniClusterResource; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.junit.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -/** Test other more advanced usage of SQL. They don't need to run for every file format. */ -public abstract class TestSqlBase { - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); - - private volatile TableEnvironment tEnv; - - protected TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - if (tEnv == null) { - this.tEnv = - TableEnvironment.create(EnvironmentSettings.newInstance().inBatchMode().build()); - } - } - } - return tEnv; - } - - @Before - public abstract void before() throws IOException; - - @Test - public void testResiduals() throws Exception { - Table table = - catalogResource - .catalog() - .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); - - List writeRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); - writeRecords.get(0).set(1, 123L); - writeRecords.get(0).set(2, "2020-03-20"); - writeRecords.get(1).set(1, 456L); - writeRecords.get(1).set(2, "2020-03-20"); - - GenericAppenderHelper helper = - new GenericAppenderHelper(table, FileFormat.PARQUET, TEMPORARY_FOLDER); - - List expectedRecords = Lists.newArrayList(); - expectedRecords.add(writeRecords.get(0)); - - DataFile dataFile1 = helper.writeFile(TestHelpers.Row.of("2020-03-20", 0), writeRecords); - DataFile dataFile2 = - helper.writeFile( - TestHelpers.Row.of("2020-03-21", 0), - RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L)); - helper.appendToTable(dataFile1, dataFile2); - - org.apache.iceberg.flink.TestHelpers.assertRecords( - run(Maps.newHashMap(), "where dt='2020-03-20' and id=123", "*"), - expectedRecords, - TestFixtures.SCHEMA); - } - - @Test - public void testExposeLocality() throws Exception { - Table table = - catalogResource - .catalog() - .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); - - TableLoader tableLoader = TableLoader.fromHadoopTable(table.location()); - List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 10, 0L); - expectedRecords.forEach(expectedRecord -> expectedRecord.set(2, "2020-03-20")); - - GenericAppenderHelper helper = - new GenericAppenderHelper(table, FileFormat.PARQUET, TEMPORARY_FOLDER); - DataFile dataFile = helper.writeFile(TestHelpers.Row.of("2020-03-20", 0), expectedRecords); - helper.appendToTable(dataFile); - - // test sql api - Configuration tableConf = getTableEnv().getConfig().getConfiguration(); - tableConf.setBoolean( - FlinkConfigOptions.TABLE_EXEC_ICEBERG_EXPOSE_SPLIT_LOCALITY_INFO.key(), false); - - List results = SqlHelpers.sql(getTableEnv(), "select * from t"); - org.apache.iceberg.flink.TestHelpers.assertRecords( - results, expectedRecords, TestFixtures.SCHEMA); - - // test table api - tableConf.setBoolean( - FlinkConfigOptions.TABLE_EXEC_ICEBERG_EXPOSE_SPLIT_LOCALITY_INFO.key(), true); - FlinkSource.Builder builder = FlinkSource.forRowData().tableLoader(tableLoader).table(table); - - // When running with CI or local, `localityEnabled` will be false even if this configuration is - // enabled - Assert.assertFalse( - "Expose split locality info should be false.", - SourceUtil.isLocalityEnabled(table, tableConf, true)); - - results = run(Maps.newHashMap(), "where dt='2020-03-20'", "*"); - org.apache.iceberg.flink.TestHelpers.assertRecords( - results, expectedRecords, TestFixtures.SCHEMA); - } - - protected List run( - Map options, String sqlFilter, String... sqlSelectedFields) { - String select = String.join(",", sqlSelectedFields); - String optionStr = SqlHelpers.sqlOptionsToString(options); - return SqlHelpers.sql(getTableEnv(), "select %s from t %s %s", select, optionStr, sqlFilter); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java deleted file mode 100644 index eaba615ce910..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java +++ /dev/null @@ -1,316 +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.iceberg.flink.source; - -import java.io.IOException; -import java.util.Iterator; -import java.util.List; -import org.apache.flink.core.execution.JobClient; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.TableResult; -import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; -import org.apache.flink.table.api.config.TableConfigOptions; -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Table; -import org.apache.iceberg.TestHelpers; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.FlinkCatalogTestBase; -import org.apache.iceberg.flink.MiniClusterResource; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -public class TestStreamScanSql extends FlinkCatalogTestBase { - private static final String TABLE = "test_table"; - private static final FileFormat FORMAT = FileFormat.PARQUET; - - private TableEnvironment tEnv; - - public TestStreamScanSql(String catalogName, Namespace baseNamespace) { - super(catalogName, baseNamespace); - } - - @Override - protected TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - if (tEnv == null) { - EnvironmentSettings.Builder settingsBuilder = - EnvironmentSettings.newInstance().inStreamingMode(); - - StreamExecutionEnvironment env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); - env.enableCheckpointing(400); - - StreamTableEnvironment streamTableEnv = - StreamTableEnvironment.create(env, settingsBuilder.build()); - streamTableEnv - .getConfig() - .getConfiguration() - .set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); - tEnv = streamTableEnv; - } - } - } - return tEnv; - } - - @Override - @Before - public void before() { - super.before(); - sql("CREATE DATABASE %s", flinkDatabase); - sql("USE CATALOG %s", catalogName); - sql("USE %s", DATABASE); - } - - @Override - @After - public void clean() { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); - super.clean(); - } - - private void insertRows(String partition, Table table, Row... rows) throws IOException { - GenericAppenderHelper appender = new GenericAppenderHelper(table, FORMAT, TEMPORARY_FOLDER); - - GenericRecord gRecord = GenericRecord.create(table.schema()); - List records = Lists.newArrayList(); - for (Row row : rows) { - records.add( - gRecord.copy( - "id", row.getField(0), - "data", row.getField(1), - "dt", row.getField(2))); - } - - if (partition != null) { - appender.appendToTable(TestHelpers.Row.of(partition, 0), records); - } else { - appender.appendToTable(records); - } - } - - private void insertRows(Table table, Row... rows) throws IOException { - insertRows(null, table, rows); - } - - private void assertRows(List expectedRows, Iterator iterator) { - for (Row expectedRow : expectedRows) { - Assert.assertTrue("Should have more records", iterator.hasNext()); - - Row actualRow = iterator.next(); - Assert.assertEquals("Should have expected fields", 3, actualRow.getArity()); - Assert.assertEquals( - "Should have expected id", expectedRow.getField(0), actualRow.getField(0)); - Assert.assertEquals( - "Should have expected data", expectedRow.getField(1), actualRow.getField(1)); - Assert.assertEquals( - "Should have expected dt", expectedRow.getField(2), actualRow.getField(2)); - } - } - - @Test - public void testUnPartitionedTable() throws Exception { - sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); - - TableResult result = - exec("SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s')*/", TABLE); - try (CloseableIterator iterator = result.collect()) { - - Row row1 = Row.of(1, "aaa", "2021-01-01"); - insertRows(table, row1); - assertRows(ImmutableList.of(row1), iterator); - - Row row2 = Row.of(2, "bbb", "2021-01-01"); - insertRows(table, row2); - assertRows(ImmutableList.of(row2), iterator); - } - result.getJobClient().ifPresent(JobClient::cancel); - } - - @Test - public void testPartitionedTable() throws Exception { - sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR) PARTITIONED BY (dt)", TABLE); - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); - - TableResult result = - exec("SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s')*/", TABLE); - try (CloseableIterator iterator = result.collect()) { - Row row1 = Row.of(1, "aaa", "2021-01-01"); - insertRows("2021-01-01", table, row1); - assertRows(ImmutableList.of(row1), iterator); - - Row row2 = Row.of(2, "bbb", "2021-01-02"); - insertRows("2021-01-02", table, row2); - assertRows(ImmutableList.of(row2), iterator); - - Row row3 = Row.of(1, "aaa", "2021-01-02"); - insertRows("2021-01-02", table, row3); - assertRows(ImmutableList.of(row3), iterator); - - Row row4 = Row.of(2, "bbb", "2021-01-01"); - insertRows("2021-01-01", table, row4); - assertRows(ImmutableList.of(row4), iterator); - } - result.getJobClient().ifPresent(JobClient::cancel); - } - - @Test - public void testConsumeFromBeginning() throws Exception { - sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); - - Row row1 = Row.of(1, "aaa", "2021-01-01"); - Row row2 = Row.of(2, "bbb", "2021-01-01"); - insertRows(table, row1, row2); - - TableResult result = - exec("SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s')*/", TABLE); - try (CloseableIterator iterator = result.collect()) { - assertRows(ImmutableList.of(row1, row2), iterator); - - Row row3 = Row.of(3, "ccc", "2021-01-01"); - insertRows(table, row3); - assertRows(ImmutableList.of(row3), iterator); - - Row row4 = Row.of(4, "ddd", "2021-01-01"); - insertRows(table, row4); - assertRows(ImmutableList.of(row4), iterator); - } - result.getJobClient().ifPresent(JobClient::cancel); - } - - @Test - public void testConsumeFilesWithBranch() throws Exception { - sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); - Row row1 = Row.of(1, "aaa", "2021-01-01"); - Row row2 = Row.of(2, "bbb", "2021-01-01"); - insertRows(table, row1, row2); - Assertions.assertThatThrownBy( - () -> - exec( - "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', 'branch'='b1')*/", - TABLE)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot scan table using ref b1 configured for streaming reader yet"); - } - - @Test - public void testConsumeFromStartSnapshotId() throws Exception { - sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); - - // Produce two snapshots. - Row row1 = Row.of(1, "aaa", "2021-01-01"); - Row row2 = Row.of(2, "bbb", "2021-01-01"); - insertRows(table, row1); - insertRows(table, row2); - - long startSnapshotId = table.currentSnapshot().snapshotId(); - - Row row3 = Row.of(3, "ccc", "2021-01-01"); - Row row4 = Row.of(4, "ddd", "2021-01-01"); - insertRows(table, row3, row4); - - TableResult result = - exec( - "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', " - + "'start-snapshot-id'='%d')*/", - TABLE, startSnapshotId); - try (CloseableIterator iterator = result.collect()) { - // the start snapshot(row2) is exclusive. - assertRows(ImmutableList.of(row3, row4), iterator); - - Row row5 = Row.of(5, "eee", "2021-01-01"); - Row row6 = Row.of(6, "fff", "2021-01-01"); - insertRows(table, row5, row6); - assertRows(ImmutableList.of(row5, row6), iterator); - - Row row7 = Row.of(7, "ggg", "2021-01-01"); - insertRows(table, row7); - assertRows(ImmutableList.of(row7), iterator); - } - result.getJobClient().ifPresent(JobClient::cancel); - } - - @Test - public void testConsumeFromStartTag() throws Exception { - sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); - - // Produce two snapshots. - Row row1 = Row.of(1, "aaa", "2021-01-01"); - Row row2 = Row.of(2, "bbb", "2021-01-01"); - insertRows(table, row1); - insertRows(table, row2); - - String tagName = "t1"; - long startSnapshotId = table.currentSnapshot().snapshotId(); - table.manageSnapshots().createTag(tagName, startSnapshotId).commit(); - - Row row3 = Row.of(3, "ccc", "2021-01-01"); - Row row4 = Row.of(4, "ddd", "2021-01-01"); - insertRows(table, row3, row4); - - TableResult result = - exec( - "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', " - + "'start-tag'='%s')*/", - TABLE, tagName); - try (CloseableIterator iterator = result.collect()) { - // the start snapshot(row2) is exclusive. - assertRows(ImmutableList.of(row3, row4), iterator); - - Row row5 = Row.of(5, "eee", "2021-01-01"); - Row row6 = Row.of(6, "fff", "2021-01-01"); - insertRows(table, row5, row6); - assertRows(ImmutableList.of(row5, row6), iterator); - - Row row7 = Row.of(7, "ggg", "2021-01-01"); - insertRows(table, row7); - assertRows(ImmutableList.of(row7), iterator); - } - result.getJobClient().ifPresent(JobClient::cancel); - Assertions.assertThatThrownBy( - () -> - exec( - "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', 'start-tag'='%s', " - + "'start-snapshot-id'='%d' )*/", - TABLE, tagName, startSnapshotId)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("START_SNAPSHOT_ID and START_TAG cannot both be set."); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java deleted file mode 100644 index 8af1dd883f4c..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java +++ /dev/null @@ -1,405 +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.iceberg.flink.source; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.io.File; -import java.io.IOException; -import java.time.Duration; -import java.util.List; -import java.util.concurrent.CountDownLatch; -import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; -import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.flink.streaming.api.operators.StreamSource; -import org.apache.flink.streaming.api.watermark.Watermark; -import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness; -import org.apache.flink.table.data.RowData; -import org.apache.flink.types.Row; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.TableTestBase; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.flink.TestTableLoader; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.SnapshotUtil; -import org.apache.iceberg.util.ThreadPools; -import org.assertj.core.api.Assertions; -import org.awaitility.Awaitility; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestStreamingMonitorFunction extends TableTestBase { - - private static final Schema SCHEMA = - new Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.required(2, "data", Types.StringType.get())); - private static final FileFormat DEFAULT_FORMAT = FileFormat.PARQUET; - private static final long WAIT_TIME_MILLIS = 10 * 1000L; - - @Parameterized.Parameters(name = "FormatVersion={0}") - public static Iterable parameters() { - return ImmutableList.of(new Object[] {1}, new Object[] {2}); - } - - public TestStreamingMonitorFunction(int formatVersion) { - super(formatVersion); - } - - @Before - @Override - public void setupTable() throws IOException { - this.tableDir = temp.newFolder(); - this.metadataDir = new File(tableDir, "metadata"); - Assert.assertTrue(tableDir.delete()); - - // Construct the iceberg table. - table = create(SCHEMA, PartitionSpec.unpartitioned()); - } - - private void runSourceFunctionInTask( - TestSourceContext sourceContext, StreamingMonitorFunction function) { - Thread task = - new Thread( - () -> { - try { - function.run(sourceContext); - } catch (Exception e) { - throw new RuntimeException(e); - } - }); - task.start(); - } - - @Test - public void testConsumeWithoutStartSnapshotId() throws Exception { - List> recordsList = generateRecordsAndCommitTxn(10); - ScanContext scanContext = ScanContext.builder().monitorInterval(Duration.ofMillis(100)).build(); - - StreamingMonitorFunction function = createFunction(scanContext); - try (AbstractStreamOperatorTestHarness harness = createHarness(function)) { - harness.setup(); - harness.open(); - - TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); - runSourceFunctionInTask(sourceContext, function); - - awaitExpectedSplits(sourceContext); - - // Stop the stream task. - function.close(); - - TestHelpers.assertRecords( - sourceContext.toRows(), Lists.newArrayList(Iterables.concat(recordsList)), SCHEMA); - } - } - - @Test - public void testConsumeFromStartSnapshotId() throws Exception { - // Commit the first five transactions. - generateRecordsAndCommitTxn(5); - long startSnapshotId = table.currentSnapshot().snapshotId(); - - // Commit the next five transactions. - List> recordsList = generateRecordsAndCommitTxn(5); - - ScanContext scanContext = - ScanContext.builder() - .monitorInterval(Duration.ofMillis(100)) - .startSnapshotId(startSnapshotId) - .build(); - - StreamingMonitorFunction function = createFunction(scanContext); - try (AbstractStreamOperatorTestHarness harness = createHarness(function)) { - harness.setup(); - harness.open(); - - TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); - runSourceFunctionInTask(sourceContext, function); - - awaitExpectedSplits(sourceContext); - - // Stop the stream task. - function.close(); - - TestHelpers.assertRecords( - sourceContext.toRows(), Lists.newArrayList(Iterables.concat(recordsList)), SCHEMA); - } - } - - @Test - public void testConsumeFromStartTag() throws Exception { - // Commit the first five transactions. - generateRecordsAndCommitTxn(5); - long startSnapshotId = table.currentSnapshot().snapshotId(); - String tagName = "t1"; - table.manageSnapshots().createTag(tagName, startSnapshotId).commit(); - - // Commit the next five transactions. - List> recordsList = generateRecordsAndCommitTxn(5); - - ScanContext scanContext = - ScanContext.builder().monitorInterval(Duration.ofMillis(100)).startTag(tagName).build(); - - StreamingMonitorFunction function = createFunction(scanContext); - try (AbstractStreamOperatorTestHarness harness = createHarness(function)) { - harness.setup(); - harness.open(); - - TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); - runSourceFunctionInTask(sourceContext, function); - - awaitExpectedSplits(sourceContext); - - // Stop the stream task. - function.close(); - - TestHelpers.assertRecords( - sourceContext.toRows(), Lists.newArrayList(Iterables.concat(recordsList)), SCHEMA); - } - } - - @Test - public void testCheckpointRestore() throws Exception { - List> recordsList = generateRecordsAndCommitTxn(10); - ScanContext scanContext = ScanContext.builder().monitorInterval(Duration.ofMillis(100)).build(); - - StreamingMonitorFunction func = createFunction(scanContext); - OperatorSubtaskState state; - try (AbstractStreamOperatorTestHarness harness = createHarness(func)) { - harness.setup(); - harness.open(); - - TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); - runSourceFunctionInTask(sourceContext, func); - - awaitExpectedSplits(sourceContext); - - state = harness.snapshot(1, 1); - - // Stop the stream task. - func.close(); - - TestHelpers.assertRecords( - sourceContext.toRows(), Lists.newArrayList(Iterables.concat(recordsList)), SCHEMA); - } - - List> newRecordsList = generateRecordsAndCommitTxn(10); - StreamingMonitorFunction newFunc = createFunction(scanContext); - try (AbstractStreamOperatorTestHarness harness = createHarness(newFunc)) { - harness.setup(); - // Recover to process the remaining snapshots. - harness.initializeState(state); - harness.open(); - - TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); - runSourceFunctionInTask(sourceContext, newFunc); - - awaitExpectedSplits(sourceContext); - - // Stop the stream task. - newFunc.close(); - - TestHelpers.assertRecords( - sourceContext.toRows(), Lists.newArrayList(Iterables.concat(newRecordsList)), SCHEMA); - } - } - - private void awaitExpectedSplits(TestSourceContext sourceContext) { - Awaitility.await("expected splits should be produced") - .atMost(Duration.ofMillis(WAIT_TIME_MILLIS)) - .untilAsserted( - () -> { - assertThat(sourceContext.latch.getCount()).isEqualTo(0); - assertThat(sourceContext.splits).as("Should produce the expected splits").hasSize(1); - }); - } - - @Test - public void testInvalidMaxPlanningSnapshotCount() { - ScanContext scanContext1 = - ScanContext.builder() - .monitorInterval(Duration.ofMillis(100)) - .maxPlanningSnapshotCount(0) - .build(); - Assertions.assertThatThrownBy(() -> createFunction(scanContext1)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("The max-planning-snapshot-count must be greater than zero"); - - ScanContext scanContext2 = - ScanContext.builder() - .monitorInterval(Duration.ofMillis(100)) - .maxPlanningSnapshotCount(-10) - .build(); - Assertions.assertThatThrownBy(() -> createFunction(scanContext2)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("The max-planning-snapshot-count must be greater than zero"); - } - - @Test - public void testConsumeWithMaxPlanningSnapshotCount() throws Exception { - generateRecordsAndCommitTxn(10); - - // Use the oldest snapshot as starting to avoid the initial case. - long oldestSnapshotId = SnapshotUtil.oldestAncestor(table).snapshotId(); - - ScanContext scanContext = - ScanContext.builder() - .monitorInterval(Duration.ofMillis(100)) - .splitSize(1000L) - .startSnapshotId(oldestSnapshotId) - .maxPlanningSnapshotCount(Integer.MAX_VALUE) - .build(); - - FlinkInputSplit[] expectedSplits = - FlinkSplitPlanner.planInputSplits(table, scanContext, ThreadPools.getWorkerPool()); - - Assert.assertEquals("should produce 9 splits", 9, expectedSplits.length); - - // This covers three cases that maxPlanningSnapshotCount is less than, equal or greater than the - // total splits number - for (int maxPlanningSnapshotCount : ImmutableList.of(1, 9, 15)) { - scanContext = - ScanContext.builder() - .monitorInterval(Duration.ofMillis(500)) - .startSnapshotId(oldestSnapshotId) - .splitSize(1000L) - .maxPlanningSnapshotCount(maxPlanningSnapshotCount) - .build(); - - StreamingMonitorFunction function = createFunction(scanContext); - try (AbstractStreamOperatorTestHarness harness = createHarness(function)) { - harness.setup(); - harness.open(); - - CountDownLatch latch = new CountDownLatch(1); - TestSourceContext sourceContext = new TestSourceContext(latch); - function.sourceContext(sourceContext); - function.monitorAndForwardSplits(); - - if (maxPlanningSnapshotCount < 10) { - Assert.assertEquals( - "Should produce same splits as max-planning-snapshot-count", - maxPlanningSnapshotCount, - sourceContext.splits.size()); - } - } - } - } - - private List> generateRecordsAndCommitTxn(int commitTimes) throws IOException { - List> expectedRecords = Lists.newArrayList(); - for (int i = 0; i < commitTimes; i++) { - List records = RandomGenericData.generate(SCHEMA, 100, 0L); - expectedRecords.add(records); - - // Commit those records to iceberg table. - writeRecords(records); - } - return expectedRecords; - } - - private void writeRecords(List records) throws IOException { - GenericAppenderHelper appender = new GenericAppenderHelper(table, DEFAULT_FORMAT, temp); - appender.appendToTable(records); - } - - private StreamingMonitorFunction createFunction(ScanContext scanContext) { - return new StreamingMonitorFunction( - TestTableLoader.of(tableDir.getAbsolutePath()), scanContext); - } - - private AbstractStreamOperatorTestHarness createHarness( - StreamingMonitorFunction function) throws Exception { - StreamSource streamSource = - new StreamSource<>(function); - return new AbstractStreamOperatorTestHarness<>(streamSource, 1, 1, 0); - } - - private class TestSourceContext implements SourceFunction.SourceContext { - private final List splits = Lists.newArrayList(); - private final Object checkpointLock = new Object(); - private final CountDownLatch latch; - - TestSourceContext(CountDownLatch latch) { - this.latch = latch; - } - - @Override - public void collect(FlinkInputSplit element) { - splits.add(element); - latch.countDown(); - } - - @Override - public void collectWithTimestamp(FlinkInputSplit element, long timestamp) { - collect(element); - } - - @Override - public void emitWatermark(Watermark mark) {} - - @Override - public void markAsTemporarilyIdle() {} - - @Override - public Object getCheckpointLock() { - return checkpointLock; - } - - @Override - public void close() {} - - private List toRows() throws IOException { - FlinkInputFormat format = - FlinkSource.forRowData() - .tableLoader(TestTableLoader.of(tableDir.getAbsolutePath())) - .buildFormat(); - - List rows = Lists.newArrayList(); - for (FlinkInputSplit split : splits) { - format.open(split); - - RowData element = null; - try { - while (!format.reachedEnd()) { - element = format.nextRecord(element); - rows.add(Row.of(element.getInt(0), element.getString(1).toString())); - } - } finally { - format.close(); - } - } - - return rows; - } - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java deleted file mode 100644 index e258a197edf3..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java +++ /dev/null @@ -1,287 +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.iceberg.flink.source; - -import java.io.File; -import java.io.IOException; -import java.util.Collections; -import java.util.List; -import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; -import org.apache.flink.streaming.api.TimeCharacteristic; -import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; -import org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor; -import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxDefaultAction; -import org.apache.flink.streaming.runtime.tasks.mailbox.SteppingMailboxProcessor; -import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; -import org.apache.flink.table.data.RowData; -import org.apache.flink.types.Row; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.TableTestBase; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.flink.TestTableLoader; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.SnapshotUtil; -import org.apache.iceberg.util.ThreadPools; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestStreamingReaderOperator extends TableTestBase { - - private static final Schema SCHEMA = - new Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.required(2, "data", Types.StringType.get())); - private static final FileFormat DEFAULT_FORMAT = FileFormat.PARQUET; - - @Parameterized.Parameters(name = "FormatVersion={0}") - public static Iterable parameters() { - return ImmutableList.of(new Object[] {1}, new Object[] {2}); - } - - public TestStreamingReaderOperator(int formatVersion) { - super(formatVersion); - } - - @Before - @Override - public void setupTable() throws IOException { - this.tableDir = temp.newFolder(); - this.metadataDir = new File(tableDir, "metadata"); - Assert.assertTrue(tableDir.delete()); - - // Construct the iceberg table. - table = create(SCHEMA, PartitionSpec.unpartitioned()); - } - - @Test - public void testProcessAllRecords() throws Exception { - List> expectedRecords = generateRecordsAndCommitTxn(10); - - List splits = generateSplits(); - Assert.assertEquals("Should have 10 splits", 10, splits.size()); - - try (OneInputStreamOperatorTestHarness harness = createReader()) { - harness.setup(); - harness.open(); - - SteppingMailboxProcessor processor = createLocalMailbox(harness); - - List expected = Lists.newArrayList(); - for (int i = 0; i < splits.size(); i++) { - // Process this element to enqueue to mail-box. - harness.processElement(splits.get(i), -1); - - // Run the mail-box once to read all records from the given split. - Assert.assertTrue("Should processed 1 split", processor.runMailboxStep()); - - // Assert the output has expected elements. - expected.addAll(expectedRecords.get(i)); - TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); - } - } - } - - @Test - public void testTriggerCheckpoint() throws Exception { - // Received emitted splits: split1, split2, split3, checkpoint request is triggered when reading - // records from - // split1. - List> expectedRecords = generateRecordsAndCommitTxn(3); - - List splits = generateSplits(); - Assert.assertEquals("Should have 3 splits", 3, splits.size()); - - long timestamp = 0; - try (OneInputStreamOperatorTestHarness harness = createReader()) { - harness.setup(); - harness.open(); - - SteppingMailboxProcessor processor = createLocalMailbox(harness); - - harness.processElement(splits.get(0), ++timestamp); - harness.processElement(splits.get(1), ++timestamp); - harness.processElement(splits.get(2), ++timestamp); - - // Trigger snapshot state, it will start to work once all records from split0 are read. - processor.getMainMailboxExecutor().execute(() -> harness.snapshot(1, 3), "Trigger snapshot"); - - Assert.assertTrue("Should have processed the split0", processor.runMailboxStep()); - Assert.assertTrue( - "Should have processed the snapshot state action", processor.runMailboxStep()); - - TestHelpers.assertRecords(readOutputValues(harness), expectedRecords.get(0), SCHEMA); - - // Read records from split1. - Assert.assertTrue("Should have processed the split1", processor.runMailboxStep()); - - // Read records from split2. - Assert.assertTrue("Should have processed the split2", processor.runMailboxStep()); - - TestHelpers.assertRecords( - readOutputValues(harness), Lists.newArrayList(Iterables.concat(expectedRecords)), SCHEMA); - } - } - - @Test - public void testCheckpointRestore() throws Exception { - List> expectedRecords = generateRecordsAndCommitTxn(15); - - List splits = generateSplits(); - Assert.assertEquals("Should have 10 splits", 15, splits.size()); - - OperatorSubtaskState state; - List expected = Lists.newArrayList(); - try (OneInputStreamOperatorTestHarness harness = createReader()) { - harness.setup(); - harness.open(); - - // Enqueue all the splits. - for (FlinkInputSplit split : splits) { - harness.processElement(split, -1); - } - - // Read all records from the first five splits. - SteppingMailboxProcessor localMailbox = createLocalMailbox(harness); - for (int i = 0; i < 5; i++) { - expected.addAll(expectedRecords.get(i)); - Assert.assertTrue("Should have processed the split#" + i, localMailbox.runMailboxStep()); - - TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); - } - - // Snapshot state now, there're 10 splits left in the state. - state = harness.snapshot(1, 1); - } - - expected.clear(); - try (OneInputStreamOperatorTestHarness harness = createReader()) { - harness.setup(); - // Recover to process the remaining splits. - harness.initializeState(state); - harness.open(); - - SteppingMailboxProcessor localMailbox = createLocalMailbox(harness); - - for (int i = 5; i < 10; i++) { - expected.addAll(expectedRecords.get(i)); - Assert.assertTrue("Should have processed one split#" + i, localMailbox.runMailboxStep()); - - TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); - } - - // Let's process the final 5 splits now. - for (int i = 10; i < 15; i++) { - expected.addAll(expectedRecords.get(i)); - harness.processElement(splits.get(i), 1); - - Assert.assertTrue("Should have processed the split#" + i, localMailbox.runMailboxStep()); - TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); - } - } - } - - private List readOutputValues( - OneInputStreamOperatorTestHarness harness) { - List results = Lists.newArrayList(); - for (RowData rowData : harness.extractOutputValues()) { - results.add(Row.of(rowData.getInt(0), rowData.getString(1).toString())); - } - return results; - } - - private List> generateRecordsAndCommitTxn(int commitTimes) throws IOException { - List> expectedRecords = Lists.newArrayList(); - for (int i = 0; i < commitTimes; i++) { - List records = RandomGenericData.generate(SCHEMA, 100, 0L); - expectedRecords.add(records); - - // Commit those records to iceberg table. - writeRecords(records); - } - return expectedRecords; - } - - private void writeRecords(List records) throws IOException { - GenericAppenderHelper appender = new GenericAppenderHelper(table, DEFAULT_FORMAT, temp); - appender.appendToTable(records); - } - - private List generateSplits() { - List inputSplits = Lists.newArrayList(); - - List snapshotIds = SnapshotUtil.currentAncestorIds(table); - for (int i = snapshotIds.size() - 1; i >= 0; i--) { - ScanContext scanContext; - if (i == snapshotIds.size() - 1) { - // Generate the splits from the first snapshot. - scanContext = ScanContext.builder().useSnapshotId(snapshotIds.get(i)).build(); - } else { - // Generate the splits between the previous snapshot and current snapshot. - scanContext = - ScanContext.builder() - .startSnapshotId(snapshotIds.get(i + 1)) - .endSnapshotId(snapshotIds.get(i)) - .build(); - } - - Collections.addAll( - inputSplits, - FlinkSplitPlanner.planInputSplits(table, scanContext, ThreadPools.getWorkerPool())); - } - - return inputSplits; - } - - private OneInputStreamOperatorTestHarness createReader() - throws Exception { - // This input format is used to opening the emitted split. - FlinkInputFormat inputFormat = - FlinkSource.forRowData() - .tableLoader(TestTableLoader.of(tableDir.getAbsolutePath())) - .buildFormat(); - - OneInputStreamOperatorFactory factory = - StreamingReaderOperator.factory(inputFormat); - OneInputStreamOperatorTestHarness harness = - new OneInputStreamOperatorTestHarness<>(factory, 1, 1, 0); - harness.getStreamConfig().setTimeCharacteristic(TimeCharacteristic.ProcessingTime); - - return harness; - } - - private SteppingMailboxProcessor createLocalMailbox( - OneInputStreamOperatorTestHarness harness) { - return new SteppingMailboxProcessor( - MailboxDefaultAction.Controller::suspendDefaultAction, - harness.getTaskMailbox(), - StreamTaskActionExecutor.IMMEDIATE); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java deleted file mode 100644 index 090b304942c6..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java +++ /dev/null @@ -1,130 +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.iceberg.flink.source.assigner; - -import java.util.Collection; -import java.util.List; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.iceberg.flink.source.SplitHelpers; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public abstract class SplitAssignerTestBase { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @Test - public void testEmptyInitialization() { - SplitAssigner assigner = splitAssigner(); - assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); - } - - /** Test a sequence of interactions for StaticEnumerator */ - @Test - public void testStaticEnumeratorSequence() throws Exception { - SplitAssigner assigner = splitAssigner(); - assigner.onDiscoveredSplits(createSplits(4, 1, "1")); - - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - assertSnapshot(assigner, 1); - assigner.onUnassignedSplits(createSplits(1, 1, "1")); - assertSnapshot(assigner, 2); - - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); - assertSnapshot(assigner, 0); - } - - /** Test a sequence of interactions for ContinuousEnumerator */ - @Test - public void testContinuousEnumeratorSequence() throws Exception { - SplitAssigner assigner = splitAssigner(); - assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); - - List splits1 = createSplits(1, 1, "1"); - assertAvailableFuture(assigner, 1, () -> assigner.onDiscoveredSplits(splits1)); - List splits2 = createSplits(1, 1, "1"); - assertAvailableFuture(assigner, 1, () -> assigner.onUnassignedSplits(splits2)); - - assigner.onDiscoveredSplits(createSplits(2, 1, "1")); - assertSnapshot(assigner, 2); - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); - assertSnapshot(assigner, 0); - } - - private void assertAvailableFuture( - SplitAssigner assigner, int splitCount, Runnable addSplitsRunnable) { - // register callback - AtomicBoolean futureCompleted = new AtomicBoolean(); - CompletableFuture future = assigner.isAvailable(); - future.thenAccept(ignored -> futureCompleted.set(true)); - // calling isAvailable again should return the same object reference - // note that thenAccept will return a new future. - // we want to assert the same instance on the assigner returned future - Assert.assertSame(future, assigner.isAvailable()); - - // now add some splits - addSplitsRunnable.run(); - Assert.assertEquals(true, futureCompleted.get()); - - for (int i = 0; i < splitCount; ++i) { - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - } - assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); - assertSnapshot(assigner, 0); - } - - protected void assertGetNext(SplitAssigner assigner, GetSplitResult.Status expectedStatus) { - GetSplitResult result = assigner.getNext(null); - Assert.assertEquals(expectedStatus, result.status()); - switch (expectedStatus) { - case AVAILABLE: - Assert.assertNotNull(result.split()); - break; - case CONSTRAINED: - case UNAVAILABLE: - Assert.assertNull(result.split()); - break; - default: - Assert.fail("Unknown status: " + expectedStatus); - } - } - - protected void assertSnapshot(SplitAssigner assigner, int splitCount) { - Collection stateBeforeGet = assigner.state(); - Assert.assertEquals(splitCount, stateBeforeGet.size()); - } - - protected List createSplits(int fileCount, int filesPerSplit, String version) - throws Exception { - return SplitHelpers.createSplitsFromTransientHadoopTable( - TEMPORARY_FOLDER, fileCount, filesPerSplit, version); - } - - protected abstract SplitAssigner splitAssigner(); -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java deleted file mode 100644 index 8994f3054abe..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java +++ /dev/null @@ -1,43 +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.iceberg.flink.source.assigner; - -import org.apache.iceberg.flink.source.SplitHelpers; -import org.junit.Test; - -public class TestDefaultSplitAssigner extends SplitAssignerTestBase { - @Override - protected SplitAssigner splitAssigner() { - return new DefaultSplitAssigner(null); - } - - /** Test the assigner when multiple files are in a single split */ - @Test - public void testMultipleFilesInASplit() throws Exception { - SplitAssigner assigner = splitAssigner(); - assigner.onDiscoveredSplits( - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 4, 2)); - - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - assertSnapshot(assigner, 1); - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); - assertSnapshot(assigner, 0); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java deleted file mode 100644 index e78634e6b873..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java +++ /dev/null @@ -1,80 +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.iceberg.flink.source.assigner; - -import java.util.List; -import org.apache.iceberg.ContentFile; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.SerializableComparator; -import org.apache.iceberg.flink.source.split.SplitComparators; -import org.apache.iceberg.util.SerializationUtil; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; - -public class TestFileSequenceNumberBasedSplitAssigner extends SplitAssignerTestBase { - @Override - protected SplitAssigner splitAssigner() { - return new OrderedSplitAssignerFactory(SplitComparators.fileSequenceNumber()).createAssigner(); - } - - /** Test the assigner when multiple files are in a single split */ - @Test - public void testMultipleFilesInAnIcebergSplit() { - SplitAssigner assigner = splitAssigner(); - Assertions.assertThatThrownBy( - () -> assigner.onDiscoveredSplits(createSplits(4, 2, "2")), - "Multiple files in a split is not allowed") - .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining("Please use 'split-open-file-cost'"); - } - - /** Test sorted splits */ - @Test - public void testSplitSort() throws Exception { - SplitAssigner assigner = splitAssigner(); - List splits = createSplits(5, 1, "2"); - - assigner.onDiscoveredSplits(splits.subList(3, 5)); - assigner.onDiscoveredSplits(splits.subList(0, 1)); - assigner.onDiscoveredSplits(splits.subList(1, 3)); - - assertGetNext(assigner, 1L); - assertGetNext(assigner, 2L); - assertGetNext(assigner, 3L); - assertGetNext(assigner, 4L); - assertGetNext(assigner, 5L); - - assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); - } - - @Test - public void testSerializable() { - byte[] bytes = SerializationUtil.serializeToBytes(SplitComparators.fileSequenceNumber()); - SerializableComparator comparator = - SerializationUtil.deserializeFromBytes(bytes); - Assert.assertNotNull(comparator); - } - - private void assertGetNext(SplitAssigner assigner, Long expectedSequenceNumber) { - GetSplitResult result = assigner.getNext(null); - ContentFile file = result.split().task().files().iterator().next().file(); - Assert.assertEquals(expectedSequenceNumber, file.fileSequenceNumber()); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java deleted file mode 100644 index e1fc63fda918..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java +++ /dev/null @@ -1,146 +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.iceberg.flink.source.assigner; - -import static org.apache.iceberg.types.Types.NestedField.required; - -import java.io.IOException; -import java.time.Instant; -import java.time.LocalDateTime; -import java.time.ZoneOffset; -import java.time.temporal.ChronoUnit; -import java.util.List; -import java.util.concurrent.TimeUnit; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.GenericAppenderFactory; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.source.reader.ColumnStatsWatermarkExtractor; -import org.apache.iceberg.flink.source.reader.ReaderUtil; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.SerializableComparator; -import org.apache.iceberg.flink.source.split.SplitComparators; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.SerializationUtil; -import org.junit.Assert; -import org.junit.Test; - -public class TestWatermarkBasedSplitAssigner extends SplitAssignerTestBase { - public static final Schema SCHEMA = - new Schema(required(1, "timestamp_column", Types.TimestampType.withoutZone())); - private static final GenericAppenderFactory APPENDER_FACTORY = new GenericAppenderFactory(SCHEMA); - - @Override - protected SplitAssigner splitAssigner() { - return new OrderedSplitAssignerFactory( - SplitComparators.watermark( - new ColumnStatsWatermarkExtractor(SCHEMA, "timestamp_column", null))) - .createAssigner(); - } - - /** Test the assigner when multiple files are in a single split */ - @Test - public void testMultipleFilesInAnIcebergSplit() { - SplitAssigner assigner = splitAssigner(); - assigner.onDiscoveredSplits(createSplits(4, 2, "2")); - - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); - } - - /** Test sorted splits */ - @Test - public void testSplitSort() { - SplitAssigner assigner = splitAssigner(); - - Instant now = Instant.now(); - List splits = - IntStream.range(0, 5) - .mapToObj(i -> splitFromInstant(now.plus(i, ChronoUnit.MINUTES))) - .collect(Collectors.toList()); - - assigner.onDiscoveredSplits(splits.subList(3, 5)); - assigner.onDiscoveredSplits(splits.subList(0, 1)); - assigner.onDiscoveredSplits(splits.subList(1, 3)); - - assertGetNext(assigner, splits.get(0)); - assertGetNext(assigner, splits.get(1)); - assertGetNext(assigner, splits.get(2)); - assertGetNext(assigner, splits.get(3)); - assertGetNext(assigner, splits.get(4)); - - assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); - } - - @Test - public void testSerializable() { - byte[] bytes = - SerializationUtil.serializeToBytes( - SplitComparators.watermark( - new ColumnStatsWatermarkExtractor( - TestFixtures.SCHEMA, "id", TimeUnit.MILLISECONDS))); - SerializableComparator comparator = - SerializationUtil.deserializeFromBytes(bytes); - Assert.assertNotNull(comparator); - } - - private void assertGetNext(SplitAssigner assigner, IcebergSourceSplit split) { - GetSplitResult result = assigner.getNext(null); - Assert.assertEquals(result.split(), split); - } - - @Override - protected List createSplits( - int fileCount, int filesPerSplit, String version) { - return IntStream.range(0, fileCount / filesPerSplit) - .mapToObj( - splitNum -> - splitFromRecords( - IntStream.range(0, filesPerSplit) - .mapToObj( - fileNum -> - RandomGenericData.generate( - SCHEMA, 2, splitNum * filesPerSplit + fileNum)) - .collect(Collectors.toList()))) - .collect(Collectors.toList()); - } - - private IcebergSourceSplit splitFromInstant(Instant instant) { - Record record = GenericRecord.create(SCHEMA); - record.set(0, LocalDateTime.ofInstant(instant, ZoneOffset.UTC)); - return splitFromRecords(ImmutableList.of(ImmutableList.of(record))); - } - - private IcebergSourceSplit splitFromRecords(List> records) { - try { - return IcebergSourceSplit.fromCombinedScanTask( - ReaderUtil.createCombinedScanTask( - records, TEMPORARY_FOLDER, FileFormat.PARQUET, APPENDER_FACTORY)); - } catch (IOException e) { - throw new RuntimeException("Split creation exception", e); - } - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java deleted file mode 100644 index ebc92df02360..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java +++ /dev/null @@ -1,97 +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.iceberg.flink.source.enumerator; - -import java.io.IOException; -import java.util.List; -import java.util.NavigableMap; -import java.util.TreeMap; -import org.apache.iceberg.flink.source.ScanContext; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -class ManualContinuousSplitPlanner implements ContinuousSplitPlanner { - private final int maxPlanningSnapshotCount; - // track splits per snapshot - private final NavigableMap> splits; - private long latestSnapshotId; - private int remainingFailures; - - ManualContinuousSplitPlanner(ScanContext scanContext, int expectedFailures) { - this.maxPlanningSnapshotCount = scanContext.maxPlanningSnapshotCount(); - this.splits = new TreeMap<>(); - this.latestSnapshotId = 0L; - this.remainingFailures = expectedFailures; - } - - @Override - public synchronized ContinuousEnumerationResult planSplits( - IcebergEnumeratorPosition lastPosition) { - if (remainingFailures > 0) { - remainingFailures--; - throw new RuntimeException("Expected failure at planning"); - } - - long fromSnapshotIdExclusive = 0; - if (lastPosition != null && lastPosition.snapshotId() != null) { - fromSnapshotIdExclusive = lastPosition.snapshotId(); - } - - Preconditions.checkArgument( - fromSnapshotIdExclusive <= latestSnapshotId, - "last enumerated snapshotId is greater than the latestSnapshotId"); - if (fromSnapshotIdExclusive == latestSnapshotId) { - // already discovered everything. - return new ContinuousEnumerationResult(Lists.newArrayList(), lastPosition, lastPosition); - } - - // find the subset of snapshots to return discovered splits - long toSnapshotIdInclusive; - if (latestSnapshotId - fromSnapshotIdExclusive > maxPlanningSnapshotCount) { - toSnapshotIdInclusive = fromSnapshotIdExclusive + maxPlanningSnapshotCount; - } else { - toSnapshotIdInclusive = latestSnapshotId; - } - - List discoveredSplits = Lists.newArrayList(); - NavigableMap> discoveredView = - splits.subMap(fromSnapshotIdExclusive, false, toSnapshotIdInclusive, true); - discoveredView.forEach((snapshotId, snapshotSplits) -> discoveredSplits.addAll(snapshotSplits)); - ContinuousEnumerationResult result = - new ContinuousEnumerationResult( - discoveredSplits, - lastPosition, - // use the snapshot Id as snapshot timestamp. - IcebergEnumeratorPosition.of(toSnapshotIdInclusive, toSnapshotIdInclusive)); - return result; - } - - /** - * Add a collection of new splits. A monotonically increased snapshotId is assigned to each batch - * of splits added by this method. - */ - public synchronized void addSplits(List newSplits) { - latestSnapshotId += 1; - splits.put(latestSnapshotId, newSplits); - } - - @Override - public void close() throws IOException {} -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java deleted file mode 100644 index 5b0ed39745c5..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java +++ /dev/null @@ -1,353 +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.iceberg.flink.source.enumerator; - -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.stream.Collectors; -import org.apache.flink.api.connector.source.SplitEnumeratorContext; -import org.apache.flink.connector.testutils.source.reader.TestingSplitEnumeratorContext; -import org.apache.iceberg.flink.source.ScanContext; -import org.apache.iceberg.flink.source.SplitHelpers; -import org.apache.iceberg.flink.source.StreamingStartingStrategy; -import org.apache.iceberg.flink.source.assigner.DefaultSplitAssigner; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; -import org.apache.iceberg.flink.source.split.SplitRequestEvent; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class TestContinuousIcebergEnumerator { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @Test - public void testDiscoverSplitWhenNoReaderRegistered() throws Exception { - TestingSplitEnumeratorContext enumeratorContext = - new TestingSplitEnumeratorContext<>(4); - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 0); - ContinuousIcebergEnumerator enumerator = - createEnumerator(enumeratorContext, scanContext, splitPlanner); - - Collection pendingSplitsEmpty = - enumerator.snapshotState(1).pendingSplits(); - Assert.assertEquals(0, pendingSplitsEmpty.size()); - - // make one split available and trigger the periodic discovery - List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); - splitPlanner.addSplits(splits); - enumeratorContext.triggerAllActions(); - - Collection pendingSplits = enumerator.snapshotState(2).pendingSplits(); - Assert.assertEquals(1, pendingSplits.size()); - IcebergSourceSplitState pendingSplit = pendingSplits.iterator().next(); - Assert.assertEquals(splits.get(0).splitId(), pendingSplit.split().splitId()); - Assert.assertEquals(IcebergSourceSplitStatus.UNASSIGNED, pendingSplit.status()); - } - - @Test - public void testDiscoverWhenReaderRegistered() throws Exception { - TestingSplitEnumeratorContext enumeratorContext = - new TestingSplitEnumeratorContext<>(4); - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 0); - ContinuousIcebergEnumerator enumerator = - createEnumerator(enumeratorContext, scanContext, splitPlanner); - - // register one reader, and let it request a split - enumeratorContext.registerReader(2, "localhost"); - enumerator.addReader(2); - enumerator.handleSourceEvent(2, new SplitRequestEvent()); - - // make one split available and trigger the periodic discovery - List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); - splitPlanner.addSplits(splits); - enumeratorContext.triggerAllActions(); - - Assert.assertTrue(enumerator.snapshotState(1).pendingSplits().isEmpty()); - Assertions.assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) - .contains(splits.get(0)); - } - - @Test - public void testRequestingReaderUnavailableWhenSplitDiscovered() throws Exception { - TestingSplitEnumeratorContext enumeratorContext = - new TestingSplitEnumeratorContext<>(4); - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 0); - ContinuousIcebergEnumerator enumerator = - createEnumerator(enumeratorContext, scanContext, splitPlanner); - - // register one reader, and let it request a split - enumeratorContext.registerReader(2, "localhost"); - enumerator.addReader(2); - enumerator.handleSourceEvent(2, new SplitRequestEvent()); - - // remove the reader (like in a failure) - enumeratorContext.registeredReaders().remove(2); - - // make one split available and trigger the periodic discovery - List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); - Assert.assertEquals(1, splits.size()); - splitPlanner.addSplits(splits); - enumeratorContext.triggerAllActions(); - - Assert.assertFalse(enumeratorContext.getSplitAssignments().containsKey(2)); - List pendingSplitIds = - enumerator.snapshotState(1).pendingSplits().stream() - .map(IcebergSourceSplitState::split) - .map(IcebergSourceSplit::splitId) - .collect(Collectors.toList()); - Assert.assertEquals(splits.size(), pendingSplitIds.size()); - Assert.assertEquals(splits.get(0).splitId(), pendingSplitIds.get(0)); - - // register the reader again, and let it request a split - enumeratorContext.registerReader(2, "localhost"); - enumerator.addReader(2); - enumerator.handleSourceEvent(2, new SplitRequestEvent()); - - Assert.assertTrue(enumerator.snapshotState(2).pendingSplits().isEmpty()); - Assertions.assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) - .contains(splits.get(0)); - } - - @Test - public void testThrottlingDiscovery() throws Exception { - // create 10 splits - List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 10, 1); - - TestingSplitEnumeratorContext enumeratorContext = - new TestingSplitEnumeratorContext<>(4); - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) - // discover one snapshot at a time - .maxPlanningSnapshotCount(1) - .build(); - ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 0); - ContinuousIcebergEnumerator enumerator = - createEnumerator(enumeratorContext, scanContext, splitPlanner); - - // register reader-2, and let it request a split - enumeratorContext.registerReader(2, "localhost"); - enumerator.addReader(2); - enumerator.handleSourceEvent(2, new SplitRequestEvent()); - - // add splits[0] to the planner for next discovery - splitPlanner.addSplits(Collections.singletonList(splits.get(0))); - enumeratorContext.triggerAllActions(); - - // because discovered split was assigned to reader, pending splits should be empty - Assert.assertEquals(0, enumerator.snapshotState(1).pendingSplits().size()); - // split assignment to reader-2 should contain splits[0, 1) - Assert.assertEquals( - splits.subList(0, 1), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); - - // add the remaining 9 splits (one for every snapshot) - // run discovery cycles while reader-2 still processing the splits[0] - for (int i = 1; i < 10; ++i) { - splitPlanner.addSplits(Collections.singletonList(splits.get(i))); - enumeratorContext.triggerAllActions(); - } - - // can only discover up to 3 snapshots/splits - Assert.assertEquals(3, enumerator.snapshotState(2).pendingSplits().size()); - // split assignment to reader-2 should be splits[0, 1) - Assert.assertEquals( - splits.subList(0, 1), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); - - // now reader-2 finished splits[0] - enumerator.handleSourceEvent( - 2, new SplitRequestEvent(Collections.singletonList(splits.get(0).splitId()))); - enumeratorContext.triggerAllActions(); - // still have 3 pending splits. After assigned splits[1] to reader-2, one more split was - // discovered and added. - Assert.assertEquals(3, enumerator.snapshotState(3).pendingSplits().size()); - // split assignment to reader-2 should be splits[0, 2) - Assert.assertEquals( - splits.subList(0, 2), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); - - // run 3 more split discovery cycles - for (int i = 0; i < 3; ++i) { - enumeratorContext.triggerAllActions(); - } - - // no more splits are discovered due to throttling - Assert.assertEquals(3, enumerator.snapshotState(4).pendingSplits().size()); - // split assignment to reader-2 should still be splits[0, 2) - Assert.assertEquals( - splits.subList(0, 2), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); - - // now reader-2 finished splits[1] - enumerator.handleSourceEvent( - 2, new SplitRequestEvent(Collections.singletonList(splits.get(1).splitId()))); - enumeratorContext.triggerAllActions(); - // still have 3 pending splits. After assigned new splits[2] to reader-2, one more split was - // discovered and added. - Assert.assertEquals(3, enumerator.snapshotState(5).pendingSplits().size()); - // split assignment to reader-2 should be splits[0, 3) - Assert.assertEquals( - splits.subList(0, 3), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); - } - - @Test - public void testTransientPlanningErrorsWithSuccessfulRetry() throws Exception { - TestingSplitEnumeratorContext enumeratorContext = - new TestingSplitEnumeratorContext<>(4); - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) - .maxPlanningSnapshotCount(1) - .maxAllowedPlanningFailures(2) - .build(); - ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 1); - ContinuousIcebergEnumerator enumerator = - createEnumerator(enumeratorContext, scanContext, splitPlanner); - - // Make one split available and trigger the periodic discovery - List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); - splitPlanner.addSplits(splits); - - // Trigger a planning and check that no splits returned due to the planning error - enumeratorContext.triggerAllActions(); - Assert.assertEquals(0, enumerator.snapshotState(2).pendingSplits().size()); - - // Second scan planning should succeed and discover the expected splits - enumeratorContext.triggerAllActions(); - Collection pendingSplits = enumerator.snapshotState(3).pendingSplits(); - Assert.assertEquals(1, pendingSplits.size()); - IcebergSourceSplitState pendingSplit = pendingSplits.iterator().next(); - Assert.assertEquals(splits.get(0).splitId(), pendingSplit.split().splitId()); - Assert.assertEquals(IcebergSourceSplitStatus.UNASSIGNED, pendingSplit.status()); - } - - @Test - public void testOverMaxAllowedPlanningErrors() throws Exception { - TestingSplitEnumeratorContext enumeratorContext = - new TestingSplitEnumeratorContext<>(4); - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) - .maxPlanningSnapshotCount(1) - .maxAllowedPlanningFailures(1) - .build(); - ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 2); - createEnumerator(enumeratorContext, scanContext, splitPlanner); - - // Make one split available and trigger the periodic discovery - List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); - splitPlanner.addSplits(splits); - - // Check that the scheduler response ignores the current error and continues to run until the - // failure limit is reached - enumeratorContext.triggerAllActions(); - Assert.assertFalse( - enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).isDone()); - - // Check that the task has failed with the expected exception after the failure limit is reached - enumeratorContext.triggerAllActions(); - Assert.assertTrue( - enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).isDone()); - Assertions.assertThatThrownBy( - () -> enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).get()) - .hasCauseInstanceOf(RuntimeException.class) - .hasMessageContaining("Failed to discover new split"); - } - - @Test - public void testPlanningIgnoringErrors() throws Exception { - int expectedFailures = 3; - TestingSplitEnumeratorContext enumeratorContext = - new TestingSplitEnumeratorContext<>(4); - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) - .maxPlanningSnapshotCount(1) - .maxAllowedPlanningFailures(-1) - .build(); - ManualContinuousSplitPlanner splitPlanner = - new ManualContinuousSplitPlanner(scanContext, expectedFailures); - ContinuousIcebergEnumerator enumerator = - createEnumerator(enumeratorContext, scanContext, splitPlanner); - - // Make one split available and trigger the periodic discovery - List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); - splitPlanner.addSplits(splits); - - Collection pendingSplits; - // Can not discover the new split with planning failures - for (int i = 0; i < expectedFailures; ++i) { - enumeratorContext.triggerAllActions(); - pendingSplits = enumerator.snapshotState(i).pendingSplits(); - Assert.assertEquals(0, pendingSplits.size()); - } - - // Discovered the new split after a successful scan planning - enumeratorContext.triggerAllActions(); - pendingSplits = enumerator.snapshotState(expectedFailures + 1).pendingSplits(); - Assert.assertEquals(1, pendingSplits.size()); - IcebergSourceSplitState pendingSplit = pendingSplits.iterator().next(); - Assert.assertEquals(splits.get(0).splitId(), pendingSplit.split().splitId()); - Assert.assertEquals(IcebergSourceSplitStatus.UNASSIGNED, pendingSplit.status()); - } - - private static ContinuousIcebergEnumerator createEnumerator( - SplitEnumeratorContext context, - ScanContext scanContext, - ContinuousSplitPlanner splitPlanner) { - - ContinuousIcebergEnumerator enumerator = - new ContinuousIcebergEnumerator( - context, - new DefaultSplitAssigner(null, Collections.emptyList()), - scanContext, - splitPlanner, - null); - enumerator.start(); - return enumerator; - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java deleted file mode 100644 index bb747caae589..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java +++ /dev/null @@ -1,688 +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.iceberg.flink.source.enumerator; - -import java.io.IOException; -import java.util.List; -import java.util.Set; -import java.util.concurrent.atomic.AtomicLong; -import java.util.stream.Collectors; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.HadoopTableResource; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.source.ScanContext; -import org.apache.iceberg.flink.source.StreamingStartingStrategy; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.rules.TestName; - -public class TestContinuousSplitPlannerImpl { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - private static final FileFormat fileFormat = FileFormat.PARQUET; - private static final AtomicLong randomSeed = new AtomicLong(); - - @Rule - public final HadoopTableResource tableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); - - @Rule public TestName testName = new TestName(); - - private GenericAppenderHelper dataAppender; - private DataFile dataFile1; - private Snapshot snapshot1; - private DataFile dataFile2; - private Snapshot snapshot2; - - @Before - public void before() throws IOException { - dataAppender = new GenericAppenderHelper(tableResource.table(), fileFormat, TEMPORARY_FOLDER); - } - - private void appendTwoSnapshots() throws IOException { - // snapshot1 - List batch1 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); - dataFile1 = dataAppender.writeFile(null, batch1); - dataAppender.appendToTable(dataFile1); - snapshot1 = tableResource.table().currentSnapshot(); - - // snapshot2 - List batch2 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 1L); - dataFile2 = dataAppender.writeFile(null, batch2); - dataAppender.appendToTable(dataFile2); - snapshot2 = tableResource.table().currentSnapshot(); - } - - /** @return the last enumerated snapshot id */ - private CycleResult verifyOneCycle( - ContinuousSplitPlannerImpl splitPlanner, IcebergEnumeratorPosition lastPosition) - throws Exception { - List batch = - RandomGenericData.generate(TestFixtures.SCHEMA, 2, randomSeed.incrementAndGet()); - DataFile dataFile = dataAppender.writeFile(null, batch); - dataAppender.appendToTable(dataFile); - Snapshot snapshot = tableResource.table().currentSnapshot(); - - ContinuousEnumerationResult result = splitPlanner.planSplits(lastPosition); - Assert.assertEquals(lastPosition.snapshotId(), result.fromPosition().snapshotId()); - Assert.assertEquals( - lastPosition.snapshotTimestampMs(), result.fromPosition().snapshotTimestampMs()); - Assert.assertEquals(snapshot.snapshotId(), result.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot.timestampMillis(), result.toPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(1, result.splits().size()); - IcebergSourceSplit split = Iterables.getOnlyElement(result.splits()); - Assert.assertEquals(1, split.task().files().size()); - Assert.assertEquals( - dataFile.path().toString(), - Iterables.getOnlyElement(split.task().files()).file().path().toString()); - return new CycleResult(result.toPosition(), split); - } - - @Test - public void testTableScanThenIncrementalWithEmptyTable() throws Exception { - ScanContext scanContext = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult emptyTableInitialDiscoveryResult = splitPlanner.planSplits(null); - Assert.assertTrue(emptyTableInitialDiscoveryResult.splits().isEmpty()); - Assert.assertNull(emptyTableInitialDiscoveryResult.fromPosition()); - Assert.assertTrue(emptyTableInitialDiscoveryResult.toPosition().isEmpty()); - Assert.assertNull(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()); - - ContinuousEnumerationResult emptyTableSecondDiscoveryResult = - splitPlanner.planSplits(emptyTableInitialDiscoveryResult.toPosition()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.splits().isEmpty()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.fromPosition().isEmpty()); - Assert.assertNull(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.toPosition().isEmpty()); - Assert.assertNull(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()); - - // next 3 snapshots - IcebergEnumeratorPosition lastPosition = emptyTableSecondDiscoveryResult.toPosition(); - for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; - } - } - - @Test - public void testTableScanThenIncrementalWithNonEmptyTable() throws Exception { - appendTwoSnapshots(); - - ScanContext scanContext = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertNull(initialResult.fromPosition()); - Assert.assertEquals( - snapshot2.snapshotId(), initialResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot2.timestampMillis(), initialResult.toPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(1, initialResult.splits().size()); - IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); - Assert.assertEquals(2, split.task().files().size()); - Set discoveredFiles = - split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) - .collect(Collectors.toSet()); - Set expectedFiles = - ImmutableSet.of(dataFile1.path().toString(), dataFile2.path().toString()); - Assert.assertEquals(expectedFiles, discoveredFiles); - - IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); - for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; - } - } - - @Test - public void testIncrementalFromLatestSnapshotWithEmptyTable() throws Exception { - ScanContext scanContext = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) - .splitSize(1L) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult emptyTableInitialDiscoveryResult = splitPlanner.planSplits(null); - Assert.assertTrue(emptyTableInitialDiscoveryResult.splits().isEmpty()); - Assert.assertNull(emptyTableInitialDiscoveryResult.fromPosition()); - Assert.assertTrue(emptyTableInitialDiscoveryResult.toPosition().isEmpty()); - Assert.assertNull(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()); - - ContinuousEnumerationResult emptyTableSecondDiscoveryResult = - splitPlanner.planSplits(emptyTableInitialDiscoveryResult.toPosition()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.splits().isEmpty()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.fromPosition().isEmpty()); - Assert.assertNull(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.toPosition().isEmpty()); - Assert.assertNull(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()); - - // latest mode should discover both snapshots, as latest position is marked by when job starts - appendTwoSnapshots(); - ContinuousEnumerationResult afterTwoSnapshotsAppended = - splitPlanner.planSplits(emptyTableSecondDiscoveryResult.toPosition()); - Assert.assertEquals(2, afterTwoSnapshotsAppended.splits().size()); - - // next 3 snapshots - IcebergEnumeratorPosition lastPosition = afterTwoSnapshotsAppended.toPosition(); - for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; - } - } - - @Test - public void testIncrementalFromLatestSnapshotWithNonEmptyTable() throws Exception { - appendTwoSnapshots(); - - ScanContext scanContext = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertNull(initialResult.fromPosition()); - // For inclusive behavior, the initial result should point to snapshot1 - // Then the next incremental scan shall discover files from latest snapshot2 (inclusive) - Assert.assertEquals( - snapshot1.snapshotId(), initialResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot1.timestampMillis(), initialResult.toPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(0, initialResult.splits().size()); - - ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); - Assert.assertEquals( - snapshot1.snapshotId(), secondResult.fromPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot1.timestampMillis(), secondResult.fromPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(snapshot2.snapshotId(), secondResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot2.timestampMillis(), secondResult.toPosition().snapshotTimestampMs().longValue()); - IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); - Assert.assertEquals(1, split.task().files().size()); - Set discoveredFiles = - split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) - .collect(Collectors.toSet()); - // should discover dataFile2 appended in snapshot2 - Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); - Assert.assertEquals(expectedFiles, discoveredFiles); - - IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); - for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; - } - } - - @Test - public void testIncrementalFromEarliestSnapshotWithEmptyTable() throws Exception { - ScanContext scanContext = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult emptyTableInitialDiscoveryResult = splitPlanner.planSplits(null); - Assert.assertTrue(emptyTableInitialDiscoveryResult.splits().isEmpty()); - Assert.assertNull(emptyTableInitialDiscoveryResult.fromPosition()); - Assert.assertNull(emptyTableInitialDiscoveryResult.toPosition().snapshotId()); - Assert.assertNull(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()); - - ContinuousEnumerationResult emptyTableSecondDiscoveryResult = - splitPlanner.planSplits(emptyTableInitialDiscoveryResult.toPosition()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.splits().isEmpty()); - Assert.assertNull(emptyTableSecondDiscoveryResult.fromPosition().snapshotId()); - Assert.assertNull(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()); - Assert.assertNull(emptyTableSecondDiscoveryResult.toPosition().snapshotId()); - Assert.assertNull(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()); - - // next 3 snapshots - IcebergEnumeratorPosition lastPosition = emptyTableSecondDiscoveryResult.toPosition(); - for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; - } - } - - @Test - public void testIncrementalFromEarliestSnapshotWithNonEmptyTable() throws Exception { - appendTwoSnapshots(); - - ScanContext scanContext = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertNull(initialResult.fromPosition()); - // For inclusive behavior, the initial result should point to snapshot1's parent, - // which leads to null snapshotId and snapshotTimestampMs. - Assert.assertNull(initialResult.toPosition().snapshotId()); - Assert.assertNull(initialResult.toPosition().snapshotTimestampMs()); - Assert.assertEquals(0, initialResult.splits().size()); - - ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); - Assert.assertNull(secondResult.fromPosition().snapshotId()); - Assert.assertNull(secondResult.fromPosition().snapshotTimestampMs()); - Assert.assertEquals(snapshot2.snapshotId(), secondResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot2.timestampMillis(), secondResult.toPosition().snapshotTimestampMs().longValue()); - IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); - Assert.assertEquals(2, split.task().files().size()); - Set discoveredFiles = - split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) - .collect(Collectors.toSet()); - // should discover files appended in both snapshot1 and snapshot2 - Set expectedFiles = - ImmutableSet.of(dataFile1.path().toString(), dataFile2.path().toString()); - Assert.assertEquals(expectedFiles, discoveredFiles); - - IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); - for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; - } - } - - @Test - public void testIncrementalFromSnapshotIdWithEmptyTable() { - ScanContext scanContextWithInvalidSnapshotId = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) - .startSnapshotId(1L) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl( - tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); - Assertions.assertThatThrownBy(() -> splitPlanner.planSplits(null)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Start snapshot id not found in history: 1"); - } - - @Test - public void testIncrementalFromSnapshotIdWithInvalidIds() throws Exception { - appendTwoSnapshots(); - - // find an invalid snapshotId - long invalidSnapshotId = 0L; - while (invalidSnapshotId == snapshot1.snapshotId() - || invalidSnapshotId == snapshot2.snapshotId()) { - invalidSnapshotId++; - } - - ScanContext scanContextWithInvalidSnapshotId = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) - .startSnapshotId(invalidSnapshotId) - .build(); - - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl( - tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); - Assertions.assertThatThrownBy(() -> splitPlanner.planSplits(null)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Start snapshot id not found in history: " + invalidSnapshotId); - } - - @Test - public void testIncrementalFromSnapshotId() throws Exception { - appendTwoSnapshots(); - - ScanContext scanContext = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) - .startSnapshotId(snapshot2.snapshotId()) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertNull(initialResult.fromPosition()); - // For inclusive behavior of snapshot2, the initial result should point to snapshot1 (as - // snapshot2's parent) - Assert.assertEquals( - snapshot1.snapshotId(), initialResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot1.timestampMillis(), initialResult.toPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(0, initialResult.splits().size()); - - ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); - Assert.assertEquals( - snapshot1.snapshotId(), secondResult.fromPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot1.timestampMillis(), secondResult.fromPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(snapshot2.snapshotId(), secondResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot2.timestampMillis(), secondResult.toPosition().snapshotTimestampMs().longValue()); - IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); - Assert.assertEquals(1, split.task().files().size()); - Set discoveredFiles = - split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) - .collect(Collectors.toSet()); - // should discover dataFile2 appended in snapshot2 - Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); - Assert.assertEquals(expectedFiles, discoveredFiles); - - IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); - for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; - } - } - - @Test - public void testIncrementalFromSnapshotTimestampWithEmptyTable() { - ScanContext scanContextWithInvalidSnapshotId = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) - .startSnapshotTimestamp(1L) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl( - tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); - Assertions.assertThatThrownBy(() -> splitPlanner.planSplits(null)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot find a snapshot after: 1"); - } - - @Test - public void testIncrementalFromSnapshotTimestampWithInvalidIds() throws Exception { - appendTwoSnapshots(); - - long invalidSnapshotTimestampMs = snapshot2.timestampMillis() + 1000L; - - ScanContext scanContextWithInvalidSnapshotId = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) - .startSnapshotTimestamp(invalidSnapshotTimestampMs) - .build(); - - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl( - tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); - Assertions.assertThatThrownBy(() -> splitPlanner.planSplits(null)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessageStartingWith("Cannot find a snapshot after: "); - } - - @Test - public void testIncrementalFromSnapshotTimestamp() throws Exception { - appendTwoSnapshots(); - - ScanContext scanContext = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) - .startSnapshotTimestamp(snapshot2.timestampMillis()) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertNull(initialResult.fromPosition()); - // For inclusive behavior, the initial result should point to snapshot1 (as snapshot2's parent). - Assert.assertEquals( - snapshot1.snapshotId(), initialResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot1.timestampMillis(), initialResult.toPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(0, initialResult.splits().size()); - - ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); - Assert.assertEquals( - snapshot1.snapshotId(), secondResult.fromPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot1.timestampMillis(), secondResult.fromPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(snapshot2.snapshotId(), secondResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot2.timestampMillis(), secondResult.toPosition().snapshotTimestampMs().longValue()); - IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); - Assert.assertEquals(1, split.task().files().size()); - Set discoveredFiles = - split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) - .collect(Collectors.toSet()); - // should discover dataFile2 appended in snapshot2 - Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); - Assert.assertEquals(expectedFiles, discoveredFiles); - - IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); - for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; - } - } - - @Test - public void testMaxPlanningSnapshotCount() throws Exception { - appendTwoSnapshots(); - // append 3 more snapshots - for (int i = 2; i < 5; ++i) { - appendSnapshot(i, 2); - } - - ScanContext scanContext = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) - // limit to 1 snapshot per discovery - .maxPlanningSnapshotCount(1) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertNull(initialResult.fromPosition()); - // For inclusive behavior, the initial result should point to snapshot1's parent, - // which leads to null snapshotId and snapshotTimestampMs. - Assert.assertNull(initialResult.toPosition().snapshotId()); - Assert.assertNull(initialResult.toPosition().snapshotTimestampMs()); - Assert.assertEquals(0, initialResult.splits().size()); - - ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); - // should discover dataFile1 appended in snapshot1 - verifyMaxPlanningSnapshotCountResult( - secondResult, null, snapshot1, ImmutableSet.of(dataFile1.path().toString())); - - ContinuousEnumerationResult thirdResult = splitPlanner.planSplits(secondResult.toPosition()); - // should discover dataFile2 appended in snapshot2 - verifyMaxPlanningSnapshotCountResult( - thirdResult, snapshot1, snapshot2, ImmutableSet.of(dataFile2.path().toString())); - } - - @Test - public void testTableScanNoStats() throws Exception { - appendTwoSnapshots(); - - ScanContext scanContext = - ScanContext.builder() - .includeColumnStats(false) - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertEquals(1, initialResult.splits().size()); - IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); - Assert.assertEquals(2, split.task().files().size()); - verifyStatCount(split, 0); - - IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); - for (int i = 0; i < 3; ++i) { - CycleResult result = verifyOneCycle(splitPlanner, lastPosition); - verifyStatCount(result.split, 0); - lastPosition = result.lastPosition; - } - } - - @Test - public void testTableScanAllStats() throws Exception { - appendTwoSnapshots(); - - ScanContext scanContext = - ScanContext.builder() - .includeColumnStats(true) - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertEquals(1, initialResult.splits().size()); - IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); - Assert.assertEquals(2, split.task().files().size()); - verifyStatCount(split, 3); - - IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); - for (int i = 0; i < 3; ++i) { - CycleResult result = verifyOneCycle(splitPlanner, lastPosition); - verifyStatCount(result.split, 3); - lastPosition = result.lastPosition; - } - } - - @Test - public void testTableScanSingleStat() throws Exception { - appendTwoSnapshots(); - - ScanContext scanContext = - ScanContext.builder() - .includeColumnStats(ImmutableSet.of("data")) - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertEquals(1, initialResult.splits().size()); - IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); - Assert.assertEquals(2, split.task().files().size()); - verifyStatCount(split, 1); - - IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); - for (int i = 0; i < 3; ++i) { - CycleResult result = verifyOneCycle(splitPlanner, lastPosition); - verifyStatCount(result.split, 1); - lastPosition = result.lastPosition; - } - } - - private void verifyStatCount(IcebergSourceSplit split, int expected) { - if (expected == 0) { - split - .task() - .files() - .forEach( - f -> { - Assert.assertNull(f.file().valueCounts()); - Assert.assertNull(f.file().columnSizes()); - Assert.assertNull(f.file().lowerBounds()); - Assert.assertNull(f.file().upperBounds()); - Assert.assertNull(f.file().nanValueCounts()); - Assert.assertNull(f.file().nullValueCounts()); - }); - } else { - split - .task() - .files() - .forEach( - f -> { - Assert.assertEquals(expected, f.file().valueCounts().size()); - Assert.assertEquals(expected, f.file().columnSizes().size()); - Assert.assertEquals(expected, f.file().lowerBounds().size()); - Assert.assertEquals(expected, f.file().upperBounds().size()); - Assert.assertEquals(expected, f.file().nullValueCounts().size()); - // The nanValue is not counted for long and string fields - Assert.assertEquals(0, f.file().nanValueCounts().size()); - }); - } - } - - private void verifyMaxPlanningSnapshotCountResult( - ContinuousEnumerationResult result, - Snapshot fromSnapshotExclusive, - Snapshot toSnapshotInclusive, - Set expectedFiles) { - if (fromSnapshotExclusive == null) { - Assert.assertNull(result.fromPosition().snapshotId()); - Assert.assertNull(result.fromPosition().snapshotTimestampMs()); - } else { - Assert.assertEquals( - fromSnapshotExclusive.snapshotId(), result.fromPosition().snapshotId().longValue()); - Assert.assertEquals( - fromSnapshotExclusive.timestampMillis(), - result.fromPosition().snapshotTimestampMs().longValue()); - } - Assert.assertEquals( - toSnapshotInclusive.snapshotId(), result.toPosition().snapshotId().longValue()); - Assert.assertEquals( - toSnapshotInclusive.timestampMillis(), - result.toPosition().snapshotTimestampMs().longValue()); - // should only have one split with one data file, because split discover is limited to - // one snapshot and each snapshot has only one data file appended. - IcebergSourceSplit split = Iterables.getOnlyElement(result.splits()); - Assert.assertEquals(1, split.task().files().size()); - Set discoveredFiles = - split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) - .collect(Collectors.toSet()); - Assert.assertEquals(expectedFiles, discoveredFiles); - } - - private Snapshot appendSnapshot(long seed, int numRecords) throws Exception { - List batch = RandomGenericData.generate(TestFixtures.SCHEMA, numRecords, seed); - DataFile dataFile = dataAppender.writeFile(null, batch); - dataAppender.appendToTable(dataFile); - return tableResource.table().currentSnapshot(); - } - - private static class CycleResult { - IcebergEnumeratorPosition lastPosition; - IcebergSourceSplit split; - - CycleResult(IcebergEnumeratorPosition lastPosition, IcebergSourceSplit split) { - this.lastPosition = lastPosition; - this.split = split; - } - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java deleted file mode 100644 index 2c94f215908a..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java +++ /dev/null @@ -1,205 +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.iceberg.flink.source.enumerator; - -import java.io.IOException; -import java.util.List; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.HadoopTableResource; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.source.ScanContext; -import org.apache.iceberg.flink.source.StreamingStartingStrategy; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.RuleChain; -import org.junit.rules.TemporaryFolder; -import org.junit.rules.TestRule; - -public class TestContinuousSplitPlannerImplStartStrategy { - private static final FileFormat FILE_FORMAT = FileFormat.PARQUET; - - public final TemporaryFolder temporaryFolder = new TemporaryFolder(); - public final HadoopTableResource tableResource = - new HadoopTableResource( - temporaryFolder, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); - @Rule public final TestRule chain = RuleChain.outerRule(temporaryFolder).around(tableResource); - - private GenericAppenderHelper dataAppender; - private Snapshot snapshot1; - private Snapshot snapshot2; - private Snapshot snapshot3; - - @Before - public void before() throws IOException { - dataAppender = new GenericAppenderHelper(tableResource.table(), FILE_FORMAT, temporaryFolder); - } - - private void appendThreeSnapshots() throws IOException { - List batch1 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); - dataAppender.appendToTable(batch1); - snapshot1 = tableResource.table().currentSnapshot(); - - List batch2 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 1L); - dataAppender.appendToTable(batch2); - snapshot2 = tableResource.table().currentSnapshot(); - - List batch3 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 2L); - dataAppender.appendToTable(batch3); - snapshot3 = tableResource.table().currentSnapshot(); - } - - @Test - public void testTableScanThenIncrementalStrategy() throws IOException { - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - - // empty table - Assert.assertFalse( - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).isPresent()); - - appendThreeSnapshots(); - Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); - Assert.assertEquals(snapshot3.snapshotId(), startSnapshot.snapshotId()); - } - - @Test - public void testForLatestSnapshotStrategy() throws IOException { - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) - .build(); - - // empty table - Assert.assertFalse( - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).isPresent()); - - appendThreeSnapshots(); - Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); - Assert.assertEquals(snapshot3.snapshotId(), startSnapshot.snapshotId()); - } - - @Test - public void testForEarliestSnapshotStrategy() throws IOException { - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) - .build(); - - // empty table - Assert.assertFalse( - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).isPresent()); - - appendThreeSnapshots(); - Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); - Assert.assertEquals(snapshot1.snapshotId(), startSnapshot.snapshotId()); - } - - @Test - public void testForSpecificSnapshotIdStrategy() throws IOException { - ScanContext scanContextInvalidSnapshotId = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) - .startSnapshotId(1L) - .build(); - - // empty table - Assertions.assertThatThrownBy( - () -> - ContinuousSplitPlannerImpl.startSnapshot( - tableResource.table(), scanContextInvalidSnapshotId)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Start snapshot id not found in history: 1"); - - appendThreeSnapshots(); - - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) - .startSnapshotId(snapshot2.snapshotId()) - .build(); - - Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); - Assert.assertEquals(snapshot2.snapshotId(), startSnapshot.snapshotId()); - } - - @Test - public void testForSpecificSnapshotTimestampStrategySnapshot2() throws IOException { - ScanContext scanContextInvalidSnapshotTimestamp = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) - .startSnapshotTimestamp(1L) - .build(); - - // empty table - Assertions.assertThatThrownBy( - () -> - ContinuousSplitPlannerImpl.startSnapshot( - tableResource.table(), scanContextInvalidSnapshotTimestamp)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessageStartingWith("Cannot find a snapshot after: "); - - appendThreeSnapshots(); - - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) - .startSnapshotTimestamp(snapshot2.timestampMillis()) - .build(); - - Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); - Assert.assertEquals(snapshot2.snapshotId(), startSnapshot.snapshotId()); - } - - @Test - public void testForSpecificSnapshotTimestampStrategySnapshot2Minus1() throws IOException { - appendThreeSnapshots(); - - ScanContext config = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) - .startSnapshotTimestamp(snapshot2.timestampMillis() - 1L) - .build(); - - Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), config).get(); - Assert.assertEquals(snapshot2.snapshotId(), startSnapshot.snapshotId()); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java deleted file mode 100644 index e2be0b4b0398..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java +++ /dev/null @@ -1,134 +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.iceberg.flink.source.enumerator; - -import org.junit.Assert; -import org.junit.Test; - -public class TestEnumerationHistory { - private static final int MAX_HISTORY_SIZE = 3; - private static final int FEW_PENDING_SPLITS = 2; - private static final int TOO_MANY_PENDING_SPLITS = 100; - - @Test - public void testEmptyHistory() { - EnumerationHistory history = new EnumerationHistory(MAX_HISTORY_SIZE); - int[] expectedHistorySnapshot = new int[0]; - testHistory(history, expectedHistorySnapshot); - } - - @Test - public void testNotFullHistory() { - EnumerationHistory history = new EnumerationHistory(3); - history.add(1); - history.add(2); - int[] expectedHistorySnapshot = {1, 2}; - testHistory(history, expectedHistorySnapshot); - } - - @Test - public void testExactFullHistory() { - EnumerationHistory history = new EnumerationHistory(3); - history.add(1); - history.add(2); - history.add(3); - int[] expectedHistorySnapshot = {1, 2, 3}; - testHistory(history, expectedHistorySnapshot); - } - - @Test - public void testOneMoreThanFullHistory() { - EnumerationHistory history = new EnumerationHistory(3); - history.add(1); - history.add(2); - history.add(3); - history.add(4); - int[] expectedHistorySnapshot = {2, 3, 4}; - testHistory(history, expectedHistorySnapshot); - } - - @Test - public void testTwoMoreThanFullHistory() { - EnumerationHistory history = new EnumerationHistory(3); - history.add(1); - history.add(2); - history.add(3); - history.add(4); - history.add(5); - int[] expectedHistorySnapshot = {3, 4, 5}; - testHistory(history, expectedHistorySnapshot); - } - - @Test - public void testThreeMoreThanFullHistory() { - EnumerationHistory history = new EnumerationHistory(3); - history.add(1); - history.add(2); - history.add(3); - history.add(4); - history.add(5); - history.add(6); - int[] expectedHistorySnapshot = {4, 5, 6}; - testHistory(history, expectedHistorySnapshot); - } - - private void testHistory(EnumerationHistory history, int[] expectedHistorySnapshot) { - Assert.assertFalse(history.shouldPauseSplitDiscovery(FEW_PENDING_SPLITS)); - if (history.hasFullHistory()) { - // throttle because pending split count is more than the sum of enumeration history - Assert.assertTrue(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)); - } else { - // skipped throttling check because there is not enough history - Assert.assertFalse(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)); - } - - int[] historySnapshot = history.snapshot(); - Assert.assertArrayEquals(expectedHistorySnapshot, historySnapshot); - - EnumerationHistory restoredHistory = new EnumerationHistory(MAX_HISTORY_SIZE); - restoredHistory.restore(historySnapshot); - - Assert.assertFalse(history.shouldPauseSplitDiscovery(FEW_PENDING_SPLITS)); - if (history.hasFullHistory()) { - // throttle because pending split count is more than the sum of enumeration history - Assert.assertTrue(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)); - } else { - // skipped throttling check because there is not enough history - Assert.assertFalse(history.shouldPauseSplitDiscovery(30)); - } - } - - @Test - public void testRestoreDifferentSize() { - EnumerationHistory history = new EnumerationHistory(3); - history.add(1); - history.add(2); - history.add(3); - int[] historySnapshot = history.snapshot(); - - EnumerationHistory smallerHistory = new EnumerationHistory(2); - smallerHistory.restore(historySnapshot); - int[] expectedRestoredHistorySnapshot = {2, 3}; - Assert.assertArrayEquals(expectedRestoredHistorySnapshot, smallerHistory.snapshot()); - - EnumerationHistory largerHisotry = new EnumerationHistory(4); - largerHisotry.restore(historySnapshot); - Assert.assertArrayEquals(historySnapshot, largerHisotry.snapshot()); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java deleted file mode 100644 index 1d12d9f66a8a..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java +++ /dev/null @@ -1,145 +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.iceberg.flink.source.enumerator; - -import java.io.IOException; -import java.util.Collection; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import org.apache.iceberg.flink.source.SplitHelpers; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestIcebergEnumeratorStateSerializer { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - private final IcebergEnumeratorStateSerializer serializer = - new IcebergEnumeratorStateSerializer(true); - - protected final int version; - - @Parameterized.Parameters(name = "version={0}") - public static Object[] parameters() { - return new Object[] {1, 2}; - } - - public TestIcebergEnumeratorStateSerializer(int version) { - this.version = version; - } - - @Test - public void testEmptySnapshotIdAndPendingSplits() throws Exception { - IcebergEnumeratorState enumeratorState = new IcebergEnumeratorState(Collections.emptyList()); - testSerializer(enumeratorState); - } - - @Test - public void testSomeSnapshotIdAndEmptyPendingSplits() throws Exception { - IcebergEnumeratorPosition position = - IcebergEnumeratorPosition.of(1L, System.currentTimeMillis()); - - IcebergEnumeratorState enumeratorState = - new IcebergEnumeratorState(position, Collections.emptyList()); - testSerializer(enumeratorState); - } - - @Test - public void testSomeSnapshotIdAndPendingSplits() throws Exception { - IcebergEnumeratorPosition position = - IcebergEnumeratorPosition.of(2L, System.currentTimeMillis()); - List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 3, 1); - Collection pendingSplits = Lists.newArrayList(); - pendingSplits.add( - new IcebergSourceSplitState(splits.get(0), IcebergSourceSplitStatus.UNASSIGNED)); - pendingSplits.add( - new IcebergSourceSplitState(splits.get(1), IcebergSourceSplitStatus.ASSIGNED)); - pendingSplits.add( - new IcebergSourceSplitState(splits.get(2), IcebergSourceSplitStatus.COMPLETED)); - - IcebergEnumeratorState enumeratorState = new IcebergEnumeratorState(position, pendingSplits); - testSerializer(enumeratorState); - } - - @Test - public void testEnumerationSplitCountHistory() throws Exception { - if (version == 2) { - IcebergEnumeratorPosition position = - IcebergEnumeratorPosition.of(2L, System.currentTimeMillis()); - List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 3, 1); - Collection pendingSplits = Lists.newArrayList(); - pendingSplits.add( - new IcebergSourceSplitState(splits.get(0), IcebergSourceSplitStatus.UNASSIGNED)); - pendingSplits.add( - new IcebergSourceSplitState(splits.get(1), IcebergSourceSplitStatus.ASSIGNED)); - pendingSplits.add( - new IcebergSourceSplitState(splits.get(2), IcebergSourceSplitStatus.COMPLETED)); - int[] enumerationSplitCountHistory = {1, 2, 3}; - - IcebergEnumeratorState enumeratorState = - new IcebergEnumeratorState(position, pendingSplits, enumerationSplitCountHistory); - testSerializer(enumeratorState); - } - } - - private void testSerializer(IcebergEnumeratorState enumeratorState) throws IOException { - byte[] result; - if (version == 1) { - result = serializer.serializeV1(enumeratorState); - } else { - result = serializer.serialize(enumeratorState); - } - - IcebergEnumeratorState deserialized = serializer.deserialize(version, result); - assertEnumeratorStateEquals(enumeratorState, deserialized); - } - - private void assertEnumeratorStateEquals( - IcebergEnumeratorState expected, IcebergEnumeratorState actual) { - Assert.assertEquals(expected.lastEnumeratedPosition(), actual.lastEnumeratedPosition()); - - Assert.assertEquals(expected.pendingSplits().size(), actual.pendingSplits().size()); - Iterator expectedIterator = expected.pendingSplits().iterator(); - Iterator actualIterator = actual.pendingSplits().iterator(); - for (int i = 0; i < expected.pendingSplits().size(); ++i) { - IcebergSourceSplitState expectedSplitState = expectedIterator.next(); - IcebergSourceSplitState actualSplitState = actualIterator.next(); - Assert.assertEquals(expectedSplitState.split().splitId(), actualSplitState.split().splitId()); - Assert.assertEquals( - expectedSplitState.split().fileOffset(), actualSplitState.split().fileOffset()); - Assert.assertEquals( - expectedSplitState.split().recordOffset(), actualSplitState.split().recordOffset()); - Assert.assertEquals(expectedSplitState.status(), actualSplitState.status()); - } - - Assert.assertArrayEquals( - expected.enumerationSplitCountHistory(), actual.enumerationSplitCountHistory()); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java deleted file mode 100644 index 0d33e4ed08ad..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java +++ /dev/null @@ -1,219 +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.iceberg.flink.source.reader; - -import java.io.IOException; -import java.util.List; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.GenericAppenderFactory; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.io.CloseableIterator; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public abstract class ReaderFunctionTestBase { - - @Parameterized.Parameters(name = "fileFormat={0}") - public static Object[][] parameters() { - return new Object[][] { - new Object[] {FileFormat.AVRO}, - new Object[] {FileFormat.ORC}, - new Object[] {FileFormat.PARQUET} - }; - } - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - protected abstract ReaderFunction readerFunction(); - - protected abstract void assertRecords(List expected, List actual, Schema schema); - - private final FileFormat fileFormat; - private final GenericAppenderFactory appenderFactory; - - public ReaderFunctionTestBase(FileFormat fileFormat) { - this.fileFormat = fileFormat; - this.appenderFactory = new GenericAppenderFactory(TestFixtures.SCHEMA); - } - - private void assertRecordsAndPosition( - List expectedRecords, - int expectedFileOffset, - long startRecordOffset, - RecordsWithSplitIds> batch) { - batch.nextSplit(); - List actualRecords = Lists.newArrayList(); - long recordOffset = startRecordOffset; - RecordAndPosition recordAndPosition; - while ((recordAndPosition = batch.nextRecordFromSplit()) != null) { - actualRecords.add(recordAndPosition.record()); - Assert.assertEquals( - "expected file offset", expectedFileOffset, recordAndPosition.fileOffset()); - Assert.assertEquals( - "expected record offset", recordOffset, recordAndPosition.recordOffset() - 1); - recordOffset++; - } - - Assert.assertEquals("expected record count", expectedRecords.size(), actualRecords.size()); - assertRecords(expectedRecords, actualRecords, TestFixtures.SCHEMA); - } - - @Test - public void testNoCheckpointedPosition() throws IOException { - List> recordBatchList = - ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); - CombinedScanTask combinedScanTask = - ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); - IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask); - CloseableIterator>> reader = - readerFunction().apply(split); - - RecordsWithSplitIds> batch0 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(0), 0, 0L, batch0); - batch0.recycle(); - - RecordsWithSplitIds> batch1 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(1), 1, 0L, batch1); - batch1.recycle(); - - RecordsWithSplitIds> batch2 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); - batch2.recycle(); - } - - @Test - public void testCheckpointedPositionBeforeFirstFile() throws IOException { - List> recordBatchList = - ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); - CombinedScanTask combinedScanTask = - ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); - IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 0, 0L); - CloseableIterator>> reader = - readerFunction().apply(split); - - RecordsWithSplitIds> batch0 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(0), 0, 0L, batch0); - batch0.recycle(); - - RecordsWithSplitIds> batch1 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(1), 1, 0L, batch1); - batch1.recycle(); - - RecordsWithSplitIds> batch2 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); - batch2.recycle(); - } - - @Test - public void testCheckpointedPositionMiddleFirstFile() throws IOException { - List> recordBatchList = - ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); - CombinedScanTask combinedScanTask = - ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); - IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 0, 1L); - CloseableIterator>> reader = - readerFunction().apply(split); - - RecordsWithSplitIds> batch0 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(0).subList(1, 2), 0, 1L, batch0); - batch0.recycle(); - - RecordsWithSplitIds> batch1 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(1), 1, 0L, batch1); - batch1.recycle(); - - RecordsWithSplitIds> batch2 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); - batch2.recycle(); - } - - @Test - public void testCheckpointedPositionAfterFirstFile() throws IOException { - List> recordBatchList = - ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); - CombinedScanTask combinedScanTask = - ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); - IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 0, 2L); - CloseableIterator>> reader = - readerFunction().apply(split); - - RecordsWithSplitIds> batch1 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(1), 1, 0L, batch1); - batch1.recycle(); - - RecordsWithSplitIds> batch2 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); - batch2.recycle(); - } - - @Test - public void testCheckpointedPositionBeforeSecondFile() throws IOException { - List> recordBatchList = - ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); - CombinedScanTask combinedScanTask = - ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); - IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 1, 0L); - CloseableIterator>> reader = - readerFunction().apply(split); - - RecordsWithSplitIds> batch1 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(1), 1, 0L, batch1); - batch1.recycle(); - - RecordsWithSplitIds> batch2 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); - batch2.recycle(); - } - - @Test - public void testCheckpointedPositionMidSecondFile() throws IOException { - List> recordBatchList = - ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); - CombinedScanTask combinedScanTask = - ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); - IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 1, 1L); - CloseableIterator>> reader = - readerFunction().apply(split); - - RecordsWithSplitIds> batch1 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(1).subList(1, 2), 1, 1L, batch1); - batch1.recycle(); - - RecordsWithSplitIds> batch2 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); - batch2.recycle(); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java deleted file mode 100644 index 2a2503ef2478..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java +++ /dev/null @@ -1,125 +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.iceberg.flink.source.reader; - -import java.io.File; -import java.io.IOException; -import java.util.Collections; -import java.util.List; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.BaseCombinedScanTask; -import org.apache.iceberg.BaseFileScanTask; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFiles; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.Files; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.PartitionSpecParser; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SchemaParser; -import org.apache.iceberg.data.GenericAppenderFactory; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.encryption.PlaintextEncryptionManager; -import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.expressions.ResidualEvaluator; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.source.DataIterator; -import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; -import org.apache.iceberg.hadoop.HadoopFileIO; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.rules.TemporaryFolder; - -public class ReaderUtil { - - private ReaderUtil() {} - - public static FileScanTask createFileTask( - List records, - File file, - FileFormat fileFormat, - FileAppenderFactory appenderFactory) - throws IOException { - FileAppender appender = - appenderFactory.newAppender(Files.localOutput(file), fileFormat); - try { - appender.addAll(records); - } finally { - appender.close(); - } - - DataFile dataFile = - DataFiles.builder(PartitionSpec.unpartitioned()) - .withRecordCount(records.size()) - .withFileSizeInBytes(file.length()) - .withPath(file.toString()) - .withFormat(fileFormat) - .withMetrics(appender.metrics()) - .build(); - - ResidualEvaluator residuals = ResidualEvaluator.unpartitioned(Expressions.alwaysTrue()); - return new BaseFileScanTask( - dataFile, - null, - SchemaParser.toJson(TestFixtures.SCHEMA), - PartitionSpecParser.toJson(PartitionSpec.unpartitioned()), - residuals); - } - - public static DataIterator createDataIterator(CombinedScanTask combinedTask) { - return new DataIterator<>( - new RowDataFileScanTaskReader( - TestFixtures.SCHEMA, TestFixtures.SCHEMA, null, true, Collections.emptyList()), - combinedTask, - new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), - new PlaintextEncryptionManager()); - } - - public static List> createRecordBatchList( - Schema schema, int listSize, int batchCount) { - return createRecordBatchList(0L, schema, listSize, batchCount); - } - - public static List> createRecordBatchList( - long seed, Schema schema, int listSize, int batchCount) { - List records = RandomGenericData.generate(schema, listSize * batchCount, seed); - return Lists.partition(records, batchCount); - } - - public static CombinedScanTask createCombinedScanTask( - List> recordBatchList, - TemporaryFolder temporaryFolder, - FileFormat fileFormat, - GenericAppenderFactory appenderFactory) - throws IOException { - List fileTasks = Lists.newArrayListWithCapacity(recordBatchList.size()); - for (List recordBatch : recordBatchList) { - FileScanTask fileTask = - ReaderUtil.createFileTask( - recordBatch, temporaryFolder.newFile(), fileFormat, appenderFactory); - fileTasks.add(fileTask); - } - - return new BaseCombinedScanTask(fileTasks); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java deleted file mode 100644 index 644ac2bad6b8..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java +++ /dev/null @@ -1,68 +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.iceberg.flink.source.reader; - -import java.util.concurrent.atomic.AtomicBoolean; -import org.junit.Assert; -import org.junit.Test; - -public class TestArrayBatchRecords { - - @Test - public void testFullRange() { - String[] elements = new String[] {"0", "1", "2", "3"}; - testArray(elements, elements.length, 2, 119); - } - - @Test - public void testSubRange() { - String[] elements = new String[] {"0", "1", "2", "3"}; - testArray(elements, 2, 0, 0); - } - - private void testArray( - String[] elements, int numberOfRecords, int fileOffset, long startingRecordOffset) { - String splitId = "iceberg_split_1"; - AtomicBoolean recycled = new AtomicBoolean(); - - ArrayBatchRecords recordsWithSplitIds = - ArrayBatchRecords.forRecords( - splitId, - ignored -> recycled.set(true), - elements, - numberOfRecords, - fileOffset, - startingRecordOffset); - - Assert.assertEquals(splitId, recordsWithSplitIds.nextSplit()); - - for (int i = 0; i < numberOfRecords; i++) { - RecordAndPosition recAndPos = recordsWithSplitIds.nextRecordFromSplit(); - Assert.assertEquals(elements[i], recAndPos.record()); - Assert.assertEquals(fileOffset, recAndPos.fileOffset()); - // recordOffset points to the position after this one - Assert.assertEquals(startingRecordOffset + i + 1, recAndPos.recordOffset()); - } - - Assert.assertNull(recordsWithSplitIds.nextRecordFromSplit()); - Assert.assertNull(recordsWithSplitIds.nextSplit()); - recordsWithSplitIds.recycle(); - Assert.assertTrue(recycled.get()); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java deleted file mode 100644 index f964a7707689..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java +++ /dev/null @@ -1,358 +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.iceberg.flink.source.reader; - -import java.util.Arrays; -import java.util.List; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.flink.connector.base.source.reader.SourceReaderOptions; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.BaseCombinedScanTask; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.data.GenericAppenderFactory; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.flink.source.DataIterator; -import org.apache.iceberg.io.CloseableIterator; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class TestArrayPoolDataIteratorBatcherRowData { - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - private static final FileFormat fileFormat = FileFormat.PARQUET; - - private final GenericAppenderFactory appenderFactory; - private final DataIteratorBatcher batcher; - - public TestArrayPoolDataIteratorBatcherRowData() { - Configuration config = new Configuration(); - // set array pool size to 1 - config.set(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY, 1); - // set batch array size to 2 - config.set(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 2); - this.batcher = - new ArrayPoolDataIteratorBatcher<>(config, new RowDataRecordFactory(TestFixtures.ROW_TYPE)); - this.appenderFactory = new GenericAppenderFactory(TestFixtures.SCHEMA); - } - - /** Read a CombinedScanTask that contains a single file with less than a full batch of records */ - @Test - public void testSingleFileLessThanOneFullBatch() throws Exception { - List records = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 1); - FileScanTask fileTask = - ReaderUtil.createFileTask(records, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); - CombinedScanTask combinedTask = new BaseCombinedScanTask(fileTask); - DataIterator dataIterator = ReaderUtil.createDataIterator(combinedTask); - String splitId = "someSplitId"; - CloseableIterator>> recordBatchIterator = - batcher.batch(splitId, dataIterator); - - ArrayBatchRecords batch = (ArrayBatchRecords) recordBatchIterator.next(); - Assert.assertTrue(batch.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch.records().length); - // assert actual number of records in the array - Assert.assertEquals(1, batch.numberOfRecords()); - - RecordAndPosition recordAndPosition = batch.nextRecordFromSplit(); - - /////////////////////////////// - // assert first record - - Assert.assertEquals(0, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(1, recordAndPosition.recordOffset()); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(0), recordAndPosition.record()); - - Assert.assertNull(batch.nextRecordFromSplit()); - Assert.assertNull(batch.nextSplit()); - batch.recycle(); - - // assert end of input - Assert.assertFalse(recordBatchIterator.hasNext()); - } - - /** - * Read a CombinedScanTask that contains a single file with multiple batches. - * - *

    Insert 5 records in a single file that should result in 3 batches - */ - @Test - public void testSingleFileWithMultipleBatches() throws Exception { - List records = RandomGenericData.generate(TestFixtures.SCHEMA, 5, 1); - FileScanTask fileTask = - ReaderUtil.createFileTask(records, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); - CombinedScanTask combinedTask = new BaseCombinedScanTask(fileTask); - DataIterator dataIterator = ReaderUtil.createDataIterator(combinedTask); - String splitId = "someSplitId"; - CloseableIterator>> recordBatchIterator = - batcher.batch(splitId, dataIterator); - - /////////////////////////////// - // assert first batch with full batch of 2 records - - ArrayBatchRecords batch0 = (ArrayBatchRecords) recordBatchIterator.next(); - Assert.assertTrue(batch0.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch0.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch0.records().length); - // assert actual number of records in the array - Assert.assertEquals(2, batch0.numberOfRecords()); - - RecordAndPosition recordAndPosition; - - // assert first record - recordAndPosition = batch0.nextRecordFromSplit(); - Assert.assertEquals(0, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(1, recordAndPosition.recordOffset()); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(0), recordAndPosition.record()); - - // assert second record - recordAndPosition = batch0.nextRecordFromSplit(); - Assert.assertEquals(0, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(2, recordAndPosition.recordOffset()); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(1), recordAndPosition.record()); - - Assert.assertNull(batch0.nextRecordFromSplit()); - Assert.assertNull(batch0.nextSplit()); - batch0.recycle(); - - /////////////////////////////// - // assert second batch with full batch of 2 records - - ArrayBatchRecords batch1 = (ArrayBatchRecords) recordBatchIterator.next(); - // assert array is reused - Assert.assertSame(batch0.records(), batch1.records()); - Assert.assertTrue(batch1.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch1.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch1.records().length); - // assert actual number of records in the array - Assert.assertEquals(2, batch1.numberOfRecords()); - - // assert third record - recordAndPosition = batch1.nextRecordFromSplit(); - Assert.assertEquals(0, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(3, recordAndPosition.recordOffset()); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(2), recordAndPosition.record()); - - // assert fourth record - recordAndPosition = batch1.nextRecordFromSplit(); - Assert.assertEquals(0, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(4, recordAndPosition.recordOffset()); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(3), recordAndPosition.record()); - - Assert.assertNull(batch1.nextRecordFromSplit()); - Assert.assertNull(batch1.nextSplit()); - batch1.recycle(); - - /////////////////////////////// - // assert third batch with partial batch of 1 record - - ArrayBatchRecords batch2 = (ArrayBatchRecords) recordBatchIterator.next(); - // assert array is reused - Assert.assertSame(batch0.records(), batch2.records()); - Assert.assertTrue(batch2.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch2.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch2.records().length); - // assert actual number of records in the array - Assert.assertEquals(1, batch2.numberOfRecords()); - - // assert fifth record - recordAndPosition = batch2.nextRecordFromSplit(); - Assert.assertEquals(0, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(5, recordAndPosition.recordOffset()); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(4), recordAndPosition.record()); - - Assert.assertNull(batch2.nextRecordFromSplit()); - Assert.assertNull(batch2.nextSplit()); - batch2.recycle(); - - // assert end of input - Assert.assertFalse(recordBatchIterator.hasNext()); - } - - /** - * Read a CombinedScanTask that contains with multiple files. - * - *

    In this test, we also seek the iterator to starting position (1, 1). - */ - @Test - public void testMultipleFilesWithSeekPosition() throws Exception { - List records0 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 1); - FileScanTask fileTask0 = - ReaderUtil.createFileTask( - records0, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); - List records1 = RandomGenericData.generate(TestFixtures.SCHEMA, 4, 2); - FileScanTask fileTask1 = - ReaderUtil.createFileTask( - records1, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); - List records2 = RandomGenericData.generate(TestFixtures.SCHEMA, 3, 3); - FileScanTask fileTask2 = - ReaderUtil.createFileTask( - records2, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); - CombinedScanTask combinedTask = - new BaseCombinedScanTask(Arrays.asList(fileTask0, fileTask1, fileTask2)); - - DataIterator dataIterator = ReaderUtil.createDataIterator(combinedTask); - // seek to file1 and after record 1 - dataIterator.seek(1, 1); - - String splitId = "someSplitId"; - CloseableIterator>> recordBatchIterator = - batcher.batch(splitId, dataIterator); - - /////////////////////////////// - // file0 is skipped by seek - - /////////////////////////////// - // file1 has 4 records. because the seek position, first record is skipped. - // we should read 3 remaining records in 2 batches: - // batch10 with 2 records and batch11 with 1 records. - - // assert first batch from file1 with full batch of 2 records - - // variable naming convention: batch - ArrayBatchRecords batch10 = (ArrayBatchRecords) recordBatchIterator.next(); - Assert.assertTrue(batch10.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch10.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch10.records().length); - // assert actual number of records in the array - Assert.assertEquals(2, batch10.numberOfRecords()); - - RecordAndPosition recordAndPosition; - - recordAndPosition = batch10.nextRecordFromSplit(); - Assert.assertEquals(1, recordAndPosition.fileOffset()); - // seek should skip the first record in file1. starting from the second record - Assert.assertEquals(2, recordAndPosition.recordOffset()); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records1.get(1), recordAndPosition.record()); - - recordAndPosition = batch10.nextRecordFromSplit(); - Assert.assertEquals(1, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(3, recordAndPosition.recordOffset()); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records1.get(2), recordAndPosition.record()); - - Assert.assertNull(batch10.nextRecordFromSplit()); - Assert.assertNull(batch10.nextSplit()); - batch10.recycle(); - - // assert second batch from file1 with partial batch of 1 record - - // variable naming convention: batch__ - ArrayBatchRecords batch11 = (ArrayBatchRecords) recordBatchIterator.next(); - // assert array is reused - Assert.assertSame(batch10.records(), batch11.records()); - Assert.assertTrue(batch11.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch11.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch11.records().length); - // assert actual number of records in the array - Assert.assertEquals(1, batch11.numberOfRecords()); - - recordAndPosition = batch11.nextRecordFromSplit(); - Assert.assertEquals(1, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(4, recordAndPosition.recordOffset()); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records1.get(3), recordAndPosition.record()); - - Assert.assertNull(batch11.nextRecordFromSplit()); - Assert.assertNull(batch11.nextSplit()); - batch11.recycle(); - - /////////////////////////////// - // file2 has 3 records. - // we should read 3 records in 2 batches: - // batch20 with 2 records and batch21 with 1 records - - // assert first batch from file2 with full batch of 2 records - - // variable naming convention: batch__ - ArrayBatchRecords batch20 = (ArrayBatchRecords) recordBatchIterator.next(); - // assert array is reused - Assert.assertSame(batch10.records(), batch20.records()); - Assert.assertTrue(batch20.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch20.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch20.records().length); - // assert actual number of records in the array - Assert.assertEquals(2, batch20.numberOfRecords()); - - recordAndPosition = batch20.nextRecordFromSplit(); - Assert.assertEquals(2, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(1, recordAndPosition.recordOffset()); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records2.get(0), recordAndPosition.record()); - - recordAndPosition = batch20.nextRecordFromSplit(); - Assert.assertEquals(2, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(2, recordAndPosition.recordOffset()); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records2.get(1), recordAndPosition.record()); - - Assert.assertNull(batch20.nextRecordFromSplit()); - Assert.assertNull(batch20.nextSplit()); - batch20.recycle(); - - /////////////////////////////// - // assert second batch from file2 with partial batch of 1 record - - // variable naming convention: batch__ - ArrayBatchRecords batch21 = (ArrayBatchRecords) recordBatchIterator.next(); - // assert array is reused - Assert.assertSame(batch10.records(), batch21.records()); - Assert.assertTrue(batch21.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch21.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch21.records().length); - // assert actual number of records in the array - Assert.assertEquals(1, batch21.numberOfRecords()); - - recordAndPosition = batch21.nextRecordFromSplit(); - Assert.assertEquals(2, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(3, recordAndPosition.recordOffset()); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records2.get(2), recordAndPosition.record()); - - Assert.assertNull(batch21.nextRecordFromSplit()); - Assert.assertNull(batch21.nextSplit()); - batch21.recycle(); - - // assert end of input - Assert.assertFalse(recordBatchIterator.hasNext()); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java deleted file mode 100644 index afe8a5d0152c..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java +++ /dev/null @@ -1,178 +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.iceberg.flink.source.reader; - -import static org.apache.iceberg.types.Types.NestedField.required; - -import java.io.IOException; -import java.time.LocalDateTime; -import java.time.OffsetDateTime; -import java.time.ZoneOffset; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.concurrent.TimeUnit; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.GenericAppenderFactory; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.HadoopTableResource; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestColumnStatsWatermarkExtractor { - public static final Schema SCHEMA = - new Schema( - required(1, "timestamp_column", Types.TimestampType.withoutZone()), - required(2, "timestamptz_column", Types.TimestampType.withZone()), - required(3, "long_column", Types.LongType.get()), - required(4, "string_column", Types.StringType.get())); - - private static final GenericAppenderFactory APPENDER_FACTORY = new GenericAppenderFactory(SCHEMA); - - private static final List> TEST_RECORDS = - ImmutableList.of( - RandomGenericData.generate(SCHEMA, 3, 2L), RandomGenericData.generate(SCHEMA, 3, 19L)); - - private static final List> MIN_VALUES = - ImmutableList.of(Maps.newHashMapWithExpectedSize(3), Maps.newHashMapWithExpectedSize(3)); - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @Rule - public final HadoopTableResource sourceTableResource = - new HadoopTableResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, SCHEMA); - - private final String columnName; - - @BeforeClass - public static void updateMinValue() { - for (int i = 0; i < TEST_RECORDS.size(); ++i) { - for (Record r : TEST_RECORDS.get(i)) { - Map minValues = MIN_VALUES.get(i); - - LocalDateTime localDateTime = (LocalDateTime) r.get(0); - minValues.merge( - "timestamp_column", localDateTime.toInstant(ZoneOffset.UTC).toEpochMilli(), Math::min); - - OffsetDateTime offsetDateTime = (OffsetDateTime) r.get(1); - minValues.merge("timestamptz_column", offsetDateTime.toInstant().toEpochMilli(), Math::min); - - minValues.merge("long_column", (Long) r.get(2), Math::min); - } - } - } - - @Parameterized.Parameters(name = "{0}") - public static Collection data() { - return ImmutableList.of( - new Object[] {"timestamp_column"}, - new Object[] {"timestamptz_column"}, - new Object[] {"long_column"}); - } - - public TestColumnStatsWatermarkExtractor(String columnName) { - this.columnName = columnName; - } - - @Test - public void testSingle() throws IOException { - ColumnStatsWatermarkExtractor extractor = - new ColumnStatsWatermarkExtractor(SCHEMA, columnName, TimeUnit.MILLISECONDS); - - Assert.assertEquals( - MIN_VALUES.get(0).get(columnName).longValue(), extractor.extractWatermark(split(0))); - } - - @Test - public void testTimeUnit() throws IOException { - Assume.assumeTrue("Run only for long column", columnName.equals("long_column")); - ColumnStatsWatermarkExtractor extractor = - new ColumnStatsWatermarkExtractor(SCHEMA, columnName, TimeUnit.MICROSECONDS); - - Assert.assertEquals( - MIN_VALUES.get(0).get(columnName).longValue() / 1000L, - extractor.extractWatermark(split(0))); - } - - @Test - public void testMultipleFiles() throws IOException { - Assume.assumeTrue("Run only for the timestamp column", columnName.equals("timestamp_column")); - IcebergSourceSplit combinedSplit = - IcebergSourceSplit.fromCombinedScanTask( - ReaderUtil.createCombinedScanTask( - TEST_RECORDS, TEMPORARY_FOLDER, FileFormat.PARQUET, APPENDER_FACTORY)); - - ColumnStatsWatermarkExtractor extractor = - new ColumnStatsWatermarkExtractor(SCHEMA, columnName, null); - - Assert.assertEquals( - MIN_VALUES.get(0).get(columnName).longValue(), extractor.extractWatermark(split(0))); - Assert.assertEquals( - MIN_VALUES.get(1).get(columnName).longValue(), extractor.extractWatermark(split(1))); - Assert.assertEquals( - Math.min(MIN_VALUES.get(0).get(columnName), MIN_VALUES.get(1).get(columnName)), - extractor.extractWatermark(combinedSplit)); - } - - @Test - public void testWrongColumn() { - Assume.assumeTrue("Run only for string column", columnName.equals("string_column")); - Assertions.assertThatThrownBy(() -> new ColumnStatsWatermarkExtractor(SCHEMA, columnName, null)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining( - "Found STRING, expected a LONG or TIMESTAMP column for watermark generation."); - } - - @Test - public void testEmptyStatistics() throws IOException { - Assume.assumeTrue("Run only for timestamp column", columnName.equals("timestamp_column")); - - // Create an extractor for a column we do not have statistics - ColumnStatsWatermarkExtractor extractor = - new ColumnStatsWatermarkExtractor(10, "missing_field"); - Assertions.assertThatThrownBy(() -> extractor.extractWatermark(split(0))) - .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining("Missing statistics for column"); - } - - private IcebergSourceSplit split(int id) throws IOException { - return IcebergSourceSplit.fromCombinedScanTask( - ReaderUtil.createCombinedScanTask( - ImmutableList.of(TEST_RECORDS.get(id)), - TEMPORARY_FOLDER, - FileFormat.PARQUET, - APPENDER_FACTORY)); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java deleted file mode 100644 index f19d57083b89..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java +++ /dev/null @@ -1,184 +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.iceberg.flink.source.reader; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import org.apache.flink.api.connector.source.SourceReaderContext; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.connector.testutils.source.reader.TestingReaderContext; -import org.apache.flink.connector.testutils.source.reader.TestingReaderOutput; -import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.data.GenericAppenderFactory; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.encryption.PlaintextEncryptionManager; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.SerializableComparator; -import org.apache.iceberg.hadoop.HadoopFileIO; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class TestIcebergSourceReader { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - private final GenericAppenderFactory appenderFactory; - - public TestIcebergSourceReader() { - this.appenderFactory = new GenericAppenderFactory(TestFixtures.SCHEMA); - } - - @Test - public void testReaderMetrics() throws Exception { - TestingReaderOutput readerOutput = new TestingReaderOutput<>(); - TestingMetricGroup metricGroup = new TestingMetricGroup(); - TestingReaderContext readerContext = new TestingReaderContext(new Configuration(), metricGroup); - IcebergSourceReader reader = createReader(metricGroup, readerContext, null); - reader.start(); - - testOneSplitFetcher(reader, readerOutput, metricGroup, 1); - testOneSplitFetcher(reader, readerOutput, metricGroup, 2); - } - - @Test - public void testReaderOrder() throws Exception { - // Create 2 splits - List> recordBatchList1 = - ReaderUtil.createRecordBatchList(0L, TestFixtures.SCHEMA, 1, 1); - CombinedScanTask task1 = - ReaderUtil.createCombinedScanTask( - recordBatchList1, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); - - List> recordBatchList2 = - ReaderUtil.createRecordBatchList(1L, TestFixtures.SCHEMA, 1, 1); - CombinedScanTask task2 = - ReaderUtil.createCombinedScanTask( - recordBatchList2, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); - - // Sort the splits in one way - List rowDataList1 = - read( - Arrays.asList( - IcebergSourceSplit.fromCombinedScanTask(task1), - IcebergSourceSplit.fromCombinedScanTask(task2)), - 2); - - // Reverse the splits - List rowDataList2 = - read( - Arrays.asList( - IcebergSourceSplit.fromCombinedScanTask(task2), - IcebergSourceSplit.fromCombinedScanTask(task1)), - 2); - - // Check that the order of the elements is not changed - Assert.assertEquals(rowDataList1.get(0), rowDataList2.get(0)); - Assert.assertEquals(rowDataList1.get(1), rowDataList2.get(1)); - } - - private List read(List splits, long expected) throws Exception { - TestingMetricGroup metricGroup = new TestingMetricGroup(); - TestingReaderContext readerContext = new TestingReaderContext(new Configuration(), metricGroup); - // Using IdBasedComparator, so we can have a deterministic order of the splits - IcebergSourceReader reader = createReader(metricGroup, readerContext, new IdBasedComparator()); - reader.start(); - - reader.addSplits(splits); - TestingReaderOutput readerOutput = new TestingReaderOutput<>(); - while (readerOutput.getEmittedRecords().size() < expected) { - reader.pollNext(readerOutput); - } - - reader.pollNext(readerOutput); - - Assert.assertEquals(expected, readerOutput.getEmittedRecords().size()); - return readerOutput.getEmittedRecords(); - } - - private void testOneSplitFetcher( - IcebergSourceReader reader, - TestingReaderOutput readerOutput, - TestingMetricGroup metricGroup, - int expectedCount) - throws Exception { - long seed = expectedCount; - // Each split should contain only one file with one record - List> recordBatchList = - ReaderUtil.createRecordBatchList(seed, TestFixtures.SCHEMA, 1, 1); - CombinedScanTask task = - ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); - IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(task); - reader.addSplits(Collections.singletonList(split)); - - while (readerOutput.getEmittedRecords().size() < expectedCount) { - reader.pollNext(readerOutput); - } - - Assert.assertEquals(expectedCount, readerOutput.getEmittedRecords().size()); - TestHelpers.assertRowData( - TestFixtures.SCHEMA, - recordBatchList.get(0).get(0), - readerOutput.getEmittedRecords().get(expectedCount - 1)); - Assert.assertEquals(expectedCount, metricGroup.counters().get("assignedSplits").getCount()); - - // One more poll will get null record batch. - // That will finish the split and cause split fetcher to be closed due to idleness. - // Then next split will create a new split reader. - reader.pollNext(readerOutput); - } - - private IcebergSourceReader createReader( - MetricGroup metricGroup, - SourceReaderContext readerContext, - SerializableComparator splitComparator) { - IcebergSourceReaderMetrics readerMetrics = - new IcebergSourceReaderMetrics(metricGroup, "db.tbl"); - RowDataReaderFunction readerFunction = - new RowDataReaderFunction( - new Configuration(), - TestFixtures.SCHEMA, - TestFixtures.SCHEMA, - null, - true, - new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), - new PlaintextEncryptionManager(), - Collections.emptyList()); - return new IcebergSourceReader<>( - SerializableRecordEmitter.defaultEmitter(), - readerMetrics, - readerFunction, - splitComparator, - readerContext); - } - - private static class IdBasedComparator implements SerializableComparator { - @Override - public int compare(IcebergSourceSplit o1, IcebergSourceSplit o2) { - return o1.splitId().compareTo(o2.splitId()); - } - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java deleted file mode 100644 index d063ad7f4a80..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java +++ /dev/null @@ -1,74 +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.iceberg.flink.source.reader; - -import java.util.Collections; -import java.util.List; -import java.util.stream.Collectors; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.conversion.DataStructureConverter; -import org.apache.flink.table.data.conversion.DataStructureConverters; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.utils.TypeConversions; -import org.apache.flink.types.Row; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.encryption.PlaintextEncryptionManager; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.hadoop.HadoopFileIO; - -public class TestRowDataReaderFunction extends ReaderFunctionTestBase { - - protected static final RowType rowType = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); - private static final DataStructureConverter rowDataConverter = - DataStructureConverters.getConverter(TypeConversions.fromLogicalToDataType(rowType)); - - public TestRowDataReaderFunction(FileFormat fileFormat) { - super(fileFormat); - } - - @Override - protected ReaderFunction readerFunction() { - return new RowDataReaderFunction( - new Configuration(), - TestFixtures.SCHEMA, - TestFixtures.SCHEMA, - null, - true, - new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), - new PlaintextEncryptionManager(), - Collections.emptyList()); - } - - @Override - protected void assertRecords(List expected, List actual, Schema schema) { - List rows = toRows(actual); - TestHelpers.assertRecords(rows, expected, TestFixtures.SCHEMA); - } - - private List toRows(List actual) { - return actual.stream() - .map(rowData -> (Row) rowDataConverter.toExternal(rowData)) - .collect(Collectors.toList()); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java deleted file mode 100644 index 290628c5fc90..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java +++ /dev/null @@ -1,102 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.source.reader; - -import java.util.Map; -import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Gauge; -import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.metrics.SimpleCounter; -import org.apache.flink.metrics.groups.OperatorIOMetricGroup; -import org.apache.flink.metrics.groups.SourceReaderMetricGroup; -import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; - -class TestingMetricGroup extends UnregisteredMetricsGroup implements SourceReaderMetricGroup { - private final Map counters; - - TestingMetricGroup() { - this.counters = Maps.newHashMap(); - } - - /** Pass along the reference to share the map for child metric groups. */ - private TestingMetricGroup(Map counters) { - this.counters = counters; - } - - Map counters() { - return counters; - } - - @Override - public Counter counter(String name) { - Counter counter = new SimpleCounter(); - counters.put(name, counter); - return counter; - } - - @Override - public MetricGroup addGroup(String name) { - return new TestingMetricGroup(counters); - } - - @Override - public MetricGroup addGroup(String key, String value) { - return new TestingMetricGroup(counters); - } - - @Override - public OperatorIOMetricGroup getIOMetricGroup() { - return new TestingOperatorIOMetricGroup(); - } - - @Override - public Counter getNumRecordsInErrorsCounter() { - return new SimpleCounter(); - } - - @Override - public void setPendingBytesGauge(Gauge pendingBytesGauge) {} - - @Override - public void setPendingRecordsGauge(Gauge pendingRecordsGauge) {} - - private static class TestingOperatorIOMetricGroup extends UnregisteredMetricsGroup - implements OperatorIOMetricGroup { - @Override - public Counter getNumRecordsInCounter() { - return new SimpleCounter(); - } - - @Override - public Counter getNumRecordsOutCounter() { - return new SimpleCounter(); - } - - @Override - public Counter getNumBytesInCounter() { - return new SimpleCounter(); - } - - @Override - public Counter getNumBytesOutCounter() { - return new SimpleCounter(); - } - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java deleted file mode 100644 index cd778309f90d..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java +++ /dev/null @@ -1,162 +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.iceberg.flink.source.split; - -import java.util.List; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.stream.Collectors; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.flink.source.SplitHelpers; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class TestIcebergSourceSplitSerializer { - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - private final IcebergSourceSplitSerializer serializer = new IcebergSourceSplitSerializer(true); - - @Test - public void testLatestVersion() throws Exception { - serializeAndDeserialize(1, 1); - serializeAndDeserialize(10, 2); - } - - private void serializeAndDeserialize(int splitCount, int filesPerSplit) throws Exception { - final List splits = - SplitHelpers.createSplitsFromTransientHadoopTable( - TEMPORARY_FOLDER, splitCount, filesPerSplit); - for (IcebergSourceSplit split : splits) { - byte[] result = serializer.serialize(split); - IcebergSourceSplit deserialized = serializer.deserialize(serializer.getVersion(), result); - assertSplitEquals(split, deserialized); - - byte[] cachedResult = serializer.serialize(split); - Assert.assertSame(result, cachedResult); - IcebergSourceSplit deserialized2 = - serializer.deserialize(serializer.getVersion(), cachedResult); - assertSplitEquals(split, deserialized2); - - split.updatePosition(0, 100); - byte[] resultAfterUpdatePosition = serializer.serialize(split); - // after position change, serialized bytes should have changed - Assert.assertNotSame(cachedResult, resultAfterUpdatePosition); - IcebergSourceSplit deserialized3 = - serializer.deserialize(serializer.getVersion(), resultAfterUpdatePosition); - assertSplitEquals(split, deserialized3); - } - } - - @Test - public void testV1() throws Exception { - serializeAndDeserializeV1(1, 1); - serializeAndDeserializeV1(10, 2); - } - - private void serializeAndDeserializeV1(int splitCount, int filesPerSplit) throws Exception { - final List splits = - SplitHelpers.createSplitsFromTransientHadoopTable( - TEMPORARY_FOLDER, splitCount, filesPerSplit); - for (IcebergSourceSplit split : splits) { - byte[] result = split.serializeV1(); - IcebergSourceSplit deserialized = IcebergSourceSplit.deserializeV1(result); - assertSplitEquals(split, deserialized); - } - } - - @Test - public void testV2() throws Exception { - serializeAndDeserializeV2(1, 1); - serializeAndDeserializeV2(10, 2); - } - - private void serializeAndDeserializeV2(int splitCount, int filesPerSplit) throws Exception { - final List splits = - SplitHelpers.createSplitsFromTransientHadoopTable( - TEMPORARY_FOLDER, splitCount, filesPerSplit); - for (IcebergSourceSplit split : splits) { - byte[] result = split.serializeV2(); - IcebergSourceSplit deserialized = IcebergSourceSplit.deserializeV2(result, true); - assertSplitEquals(split, deserialized); - } - } - - @Test - public void testDeserializeV1() throws Exception { - final List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); - for (IcebergSourceSplit split : splits) { - byte[] result = split.serializeV1(); - IcebergSourceSplit deserialized = serializer.deserialize(1, result); - assertSplitEquals(split, deserialized); - } - } - - @Test - public void testCheckpointedPosition() throws Exception { - final AtomicInteger index = new AtomicInteger(); - final List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 10, 2).stream() - .map( - split -> { - IcebergSourceSplit result; - if (index.get() % 2 == 0) { - result = IcebergSourceSplit.fromCombinedScanTask(split.task(), 1, 1); - } else { - result = split; - } - index.incrementAndGet(); - return result; - }) - .collect(Collectors.toList()); - - for (IcebergSourceSplit split : splits) { - byte[] result = serializer.serialize(split); - IcebergSourceSplit deserialized = serializer.deserialize(serializer.getVersion(), result); - assertSplitEquals(split, deserialized); - - byte[] cachedResult = serializer.serialize(split); - Assert.assertSame(result, cachedResult); - IcebergSourceSplit deserialized2 = - serializer.deserialize(serializer.getVersion(), cachedResult); - assertSplitEquals(split, deserialized2); - } - } - - private void assertSplitEquals(IcebergSourceSplit expected, IcebergSourceSplit actual) { - List expectedTasks = Lists.newArrayList(expected.task().tasks().iterator()); - List actualTasks = Lists.newArrayList(actual.task().tasks().iterator()); - Assert.assertEquals(expectedTasks.size(), actualTasks.size()); - for (int i = 0; i < expectedTasks.size(); ++i) { - FileScanTask expectedTask = expectedTasks.get(i); - FileScanTask actualTask = actualTasks.get(i); - Assert.assertEquals(expectedTask.file().path(), actualTask.file().path()); - Assert.assertEquals(expectedTask.sizeBytes(), actualTask.sizeBytes()); - Assert.assertEquals(expectedTask.filesCount(), actualTask.filesCount()); - Assert.assertEquals(expectedTask.start(), actualTask.start()); - Assert.assertEquals(expectedTask.length(), actualTask.length()); - } - - Assert.assertEquals(expected.fileOffset(), actual.fileOffset()); - Assert.assertEquals(expected.recordOffset(), actual.recordOffset()); - } -} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java deleted file mode 100644 index caacbd4b5a30..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java +++ /dev/null @@ -1,31 +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.iceberg.flink.util; - -import org.junit.Assert; -import org.junit.Test; - -public class TestFlinkPackage { - - /** This unit test would need to be adjusted as new Flink version is supported. */ - @Test - public void testVersion() { - Assert.assertEquals("1.15.0", FlinkPackage.version()); - } -} diff --git a/flink/v1.15/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink/v1.15/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory deleted file mode 100644 index 47a3c94aa991..000000000000 --- a/flink/v1.15/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ /dev/null @@ -1,16 +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. - -org.apache.iceberg.flink.source.BoundedTableFactory From 22b95dc70eb46e82a0d51fd4cfb71ad4cbdf946a Mon Sep 17 00:00:00 2001 From: Rodrigo Meneses Date: Tue, 5 Dec 2023 08:54:05 -0800 Subject: [PATCH 21/47] Make Flink 1.18 to work --- .github/workflows/flink-ci.yml | 2 +- dev/stage-binaries.sh | 2 +- flink/build.gradle | 8 ++--- flink/v1.18/build.gradle | 36 +++++++++---------- .../org/apache/iceberg/flink/TestHelpers.java | 25 ++++++++++++- .../iceberg/flink/util/TestFlinkPackage.java | 2 +- gradle.properties | 4 +-- gradle/libs.versions.toml | 24 ++++++------- settings.gradle | 18 +++++----- 9 files changed, 72 insertions(+), 49 deletions(-) diff --git a/.github/workflows/flink-ci.yml b/.github/workflows/flink-ci.yml index 30fe6c7723bc..702ae9bc898d 100644 --- a/.github/workflows/flink-ci.yml +++ b/.github/workflows/flink-ci.yml @@ -60,7 +60,7 @@ jobs: strategy: matrix: jvm: [8, 11] - flink: ['1.15', '1.16', '1.17'] + flink: ['1.16', '1.17', '1.18'] env: SPARK_LOCAL_IP: localhost steps: diff --git a/dev/stage-binaries.sh b/dev/stage-binaries.sh index 8b7b3d30aa26..5509a2ea3223 100755 --- a/dev/stage-binaries.sh +++ b/dev/stage-binaries.sh @@ -19,7 +19,7 @@ # SCALA_VERSION=2.12 -FLINK_VERSIONS=1.15,1.16,1.17 +FLINK_VERSIONS=1.16,1.17,1.18 SPARK_VERSIONS=3.2,3.3,3.4,3.5 HIVE_VERSIONS=2,3 diff --git a/flink/build.gradle b/flink/build.gradle index 33c2be904546..a33fc84e5727 100644 --- a/flink/build.gradle +++ b/flink/build.gradle @@ -19,10 +19,6 @@ def flinkVersions = (System.getProperty("flinkVersions") != null ? System.getProperty("flinkVersions") : System.getProperty("defaultFlinkVersions")).split(",") -if (flinkVersions.contains("1.15")) { - apply from: file("$projectDir/v1.15/build.gradle") -} - if (flinkVersions.contains("1.16")) { apply from: file("$projectDir/v1.16/build.gradle") } @@ -30,3 +26,7 @@ if (flinkVersions.contains("1.16")) { if (flinkVersions.contains("1.17")) { apply from: file("$projectDir/v1.17/build.gradle") } + +if (flinkVersions.contains("1.18")) { + apply from: file("$projectDir/v1.18/build.gradle") +} diff --git a/flink/v1.18/build.gradle b/flink/v1.18/build.gradle index 2e081b760878..55578d3b117d 100644 --- a/flink/v1.18/build.gradle +++ b/flink/v1.18/build.gradle @@ -17,7 +17,7 @@ * under the License. */ -String flinkMajorVersion = '1.17' +String flinkMajorVersion = '1.18' String scalaVersion = System.getProperty("scalaVersion") != null ? System.getProperty("scalaVersion") : System.getProperty("defaultScalaVersion") project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { @@ -32,15 +32,15 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { implementation project(':iceberg-parquet') implementation project(':iceberg-hive-metastore') - compileOnly libs.flink117.avro + compileOnly libs.flink118.avro // for dropwizard histogram metrics implementation - compileOnly libs.flink117.metrics.dropwizard - compileOnly libs.flink117.streaming.java - compileOnly "${libs.flink117.streaming.java.get().module}:${libs.flink117.streaming.java.get().getVersion()}:tests" - compileOnly libs.flink117.table.api.java.bridge - compileOnly "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink117.get()}" - compileOnly libs.flink117.connector.base - compileOnly libs.flink117.connector.files + compileOnly libs.flink118.metrics.dropwizard + compileOnly libs.flink118.streaming.java + compileOnly "${libs.flink118.streaming.java.get().module}:${libs.flink118.streaming.java.get().getVersion()}:tests" + compileOnly libs.flink118.table.api.java.bridge + compileOnly "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink118.get()}" + compileOnly libs.flink118.connector.base + compileOnly libs.flink118.connector.files compileOnly libs.hadoop2.hdfs compileOnly libs.hadoop2.common @@ -65,13 +65,13 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { exclude group: 'org.apache.hive', module: 'hive-storage-api' } - testImplementation libs.flink117.connector.test.utils - testImplementation libs.flink117.core - testImplementation libs.flink117.runtime - testImplementation(libs.flink117.test.utilsjunit) { + testImplementation libs.flink118.connector.test.utils + testImplementation libs.flink118.core + testImplementation libs.flink118.runtime + testImplementation(libs.flink118.test.utilsjunit) { exclude group: 'junit' } - testImplementation(libs.flink117.test.utils) { + testImplementation(libs.flink118.test.utils) { exclude group: "org.apache.curator", module: 'curator-test' exclude group: 'junit' } @@ -164,7 +164,7 @@ project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") { } // for dropwizard histogram metrics implementation - implementation libs.flink117.metrics.dropwizard + implementation libs.flink118.metrics.dropwizard // for integration testing with the flink-runtime-jar // all of those dependencies are required because the integration test extends FlinkTestBase @@ -174,13 +174,13 @@ project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") { integrationImplementation project(path: ":iceberg-flink:iceberg-flink-${flinkMajorVersion}", configuration: "testArtifacts") integrationImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') integrationImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') - integrationImplementation(libs.flink117.test.utils) { + integrationImplementation(libs.flink118.test.utils) { exclude group: "org.apache.curator", module: 'curator-test' exclude group: 'junit' } - integrationImplementation libs.flink117.table.api.java.bridge - integrationImplementation "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink117.get()}" + integrationImplementation libs.flink118.table.api.java.bridge + integrationImplementation "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink118.get()}" integrationImplementation libs.hadoop2.common integrationImplementation libs.hadoop2.hdfs diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java index 3e574b841c8d..7d3777935ca8 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java @@ -416,7 +416,7 @@ private static void assertAvroEquals( Collection expectedArrayData = (Collection) expected; ArrayData actualArrayData; try { - actualArrayData = (ArrayData) actual; + actualArrayData = convertToArray(actual); } catch (ClassCastException e) { actualArrayData = new GenericArrayData((Object[]) actual); } @@ -462,6 +462,29 @@ private static void assertAvroEquals( } } + private static GenericArrayData convertToArray(Object actual) { + Class arrayClass = actual.getClass(); + if (Object[].class.equals(arrayClass)) { + return new GenericArrayData((Object[]) actual); + } else if (int[].class.equals(arrayClass)) { + return new GenericArrayData((int[]) actual); + } else if (long[].class.equals(arrayClass)) { + return new GenericArrayData((long[]) actual); + } else if (float[].class.equals(arrayClass)) { + return new GenericArrayData((float[]) actual); + } else if (double[].class.equals(arrayClass)) { + return new GenericArrayData((double[]) actual); + } else if (short[].class.equals(arrayClass)) { + return new GenericArrayData((short[]) actual); + } else if (byte[].class.equals(arrayClass)) { + return new GenericArrayData((byte[]) actual); + } else if (boolean[].class.equals(arrayClass)) { + return new GenericArrayData((boolean[]) actual); + } else { + throw new IllegalArgumentException("Unsupported type " + arrayClass); + } + } + private static void assertArrayValues( Type type, LogicalType logicalType, Collection expectedArray, ArrayData actualArray) { List expectedElements = Lists.newArrayList(expectedArray); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java index cf244f410288..a805d160d809 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java @@ -26,6 +26,6 @@ public class TestFlinkPackage { /** This unit test would need to be adjusted as new Flink version is supported. */ @Test public void testVersion() { - Assert.assertEquals("1.17.1", FlinkPackage.version()); + Assert.assertEquals("1.18.0", FlinkPackage.version()); } } diff --git a/gradle.properties b/gradle.properties index 1bce26270354..84644da7c4ba 100644 --- a/gradle.properties +++ b/gradle.properties @@ -16,8 +16,8 @@ jmhOutputPath=build/reports/jmh/human-readable-output.txt jmhJsonOutputPath=build/reports/jmh/results.json jmhIncludeRegex=.* -systemProp.defaultFlinkVersions=1.17 -systemProp.knownFlinkVersions=1.15,1.16,1.17 +systemProp.defaultFlinkVersions=1.18 +systemProp.knownFlinkVersions=1.16,1.17,1.18 systemProp.defaultHiveVersions=2 systemProp.knownHiveVersions=2,3 systemProp.defaultSparkVersions=3.5 diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index f5c391fa95b5..29d5610f3c32 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -36,9 +36,9 @@ delta-core = "2.2.0" esotericsoftware-kryo = "4.0.2" errorprone-annotations = "2.23.0" findbugs-jsr305 = "3.0.2" -flink115 = { strictly = "[1.15, 1.16[", prefer = "1.15.0"} # see rich version usage explanation above flink116 = { strictly = "[1.16, 1.17[", prefer = "1.16.2"} flink117 = { strictly = "[1.17, 1.18[", prefer = "1.17.1"} +flink118 = { strictly = "[1.18, 1.19[", prefer = "1.18.0"} google-libraries-bom = "26.27.0" guava = "32.1.3-jre" hadoop2 = "2.7.3" @@ -104,12 +104,6 @@ calcite-druid = { module = "org.apache.calcite:calcite-druid", version.ref = "ca delta-standalone = { module = "io.delta:delta-standalone_2.12", version.ref = "delta-standalone" } errorprone-annotations = { module = "com.google.errorprone:error_prone_annotations", version.ref = "errorprone-annotations" } findbugs-jsr305 = { module = "com.google.code.findbugs:jsr305", version.ref = "findbugs-jsr305" } -flink115-avro = { module = "org.apache.flink:flink-avro", version.ref = "flink115" } -flink115-connector-base = { module = "org.apache.flink:flink-connector-base", version.ref = "flink115" } -flink115-connector-files = { module = "org.apache.flink:flink-connector-files", version.ref = "flink115" } -flink115-metrics-dropwizard = { module = "org.apache.flink:flink-metrics-dropwizard", version.ref = "flink115" } -flink115-streaming-java = { module = "org.apache.flink:flink-streaming-java", version.ref = "flink115" } -flink115-table-api-java-bridge = { module = "org.apache.flink:flink-table-api-java-bridge", version.ref = "flink115" } flink116-avro = { module = "org.apache.flink:flink-avro", version.ref = "flink116" } flink116-connector-base = { module = "org.apache.flink:flink-connector-base", version.ref = "flink116" } flink116-connector-files = { module = "org.apache.flink:flink-connector-files", version.ref = "flink116" } @@ -122,6 +116,12 @@ flink117-connector-files = { module = "org.apache.flink:flink-connector-files", flink117-metrics-dropwizard = { module = "org.apache.flink:flink-metrics-dropwizard", version.ref = "flink117" } flink117-streaming-java = { module = "org.apache.flink:flink-streaming-java", version.ref = "flink117" } flink117-table-api-java-bridge = { module = "org.apache.flink:flink-table-api-java-bridge", version.ref = "flink117" } +flink118-avro = { module = "org.apache.flink:flink-avro", version.ref = "flink118" } +flink118-connector-base = { module = "org.apache.flink:flink-connector-base", version.ref = "flink118" } +flink118-connector-files = { module = "org.apache.flink:flink-connector-files", version.ref = "flink118" } +flink118-metrics-dropwizard = { module = "org.apache.flink:flink-metrics-dropwizard", version.ref = "flink118" } +flink118-streaming-java = { module = "org.apache.flink:flink-streaming-java", version.ref = "flink118" } +flink118-table-api-java-bridge = { module = "org.apache.flink:flink-table-api-java-bridge", version.ref = "flink118" } google-libraries-bom = { module = "com.google.cloud:libraries-bom", version.ref = "google-libraries-bom" } guava-guava = { module = "com.google.guava:guava", version.ref = "guava" } hadoop2-client = { module = "org.apache.hadoop:hadoop-client", version.ref = "hadoop2" } @@ -171,11 +171,6 @@ assertj-core = { module = "org.assertj:assertj-core", version.ref = "assertj-cor awaitility = { module = "org.awaitility:awaitility", version.ref = "awaitility" } delta-core = { module = "io.delta:delta-core_2.12", version.ref = "delta-core" } esotericsoftware-kryo = { module = "com.esotericsoftware:kryo", version.ref = "esotericsoftware-kryo" } -flink115-connector-test-utils = { module = "org.apache.flink:flink-connector-test-utils", version.ref = "flink115" } -flink115-core = { module = "org.apache.flink:flink-core", version.ref = "flink115" } -flink115-runtime = { module = "org.apache.flink:flink-runtime", version.ref = "flink115" } -flink115-test-utils = { module = "org.apache.flink:flink-test-utils", version.ref = "flink115" } -flink115-test-utilsjunit = { module = "org.apache.flink:flink-test-utils-junit", version.ref = "flink115" } flink116-connector-test-utils = { module = "org.apache.flink:flink-connector-test-utils", version.ref = "flink116" } flink116-core = { module = "org.apache.flink:flink-core", version.ref = "flink116" } flink116-runtime = { module = "org.apache.flink:flink-runtime", version.ref = "flink116" } @@ -186,6 +181,11 @@ flink117-core = { module = "org.apache.flink:flink-core", version.ref = "flink11 flink117-runtime = { module = "org.apache.flink:flink-runtime", version.ref = "flink117" } flink117-test-utils = { module = "org.apache.flink:flink-test-utils", version.ref = "flink117" } flink117-test-utilsjunit = { module = "org.apache.flink:flink-test-utils-junit", version.ref = "flink117" } +flink118-connector-test-utils = { module = "org.apache.flink:flink-connector-test-utils", version.ref = "flink118" } +flink118-core = { module = "org.apache.flink:flink-core", version.ref = "flink118" } +flink118-runtime = { module = "org.apache.flink:flink-runtime", version.ref = "flink118" } +flink118-test-utils = { module = "org.apache.flink:flink-test-utils", version.ref = "flink118" } +flink118-test-utilsjunit = { module = "org.apache.flink:flink-test-utils-junit", version.ref = "flink118" } guava-testlib = { module = "com.google.guava:guava-testlib", version.ref = "guava" } jackson-annotations = { module = "com.fasterxml.jackson.core:jackson-annotations", version.ref = "jackson-annotations" } jackson-dataformat-xml = { module = "com.fasterxml.jackson.dataformat:jackson-dataformat-xml", version.ref = "jackson-dataformat-xml" } diff --git a/settings.gradle b/settings.gradle index 9ab130413077..a487fe1ffdac 100644 --- a/settings.gradle +++ b/settings.gradle @@ -106,15 +106,6 @@ if (!flinkVersions.isEmpty()) { project(':flink').name = 'iceberg-flink' } -if (flinkVersions.contains("1.15")) { - include ":iceberg-flink:flink-1.15" - include ":iceberg-flink:flink-runtime-1.15" - project(":iceberg-flink:flink-1.15").projectDir = file('flink/v1.15/flink') - project(":iceberg-flink:flink-1.15").name = "iceberg-flink-1.15" - project(":iceberg-flink:flink-runtime-1.15").projectDir = file('flink/v1.15/flink-runtime') - project(":iceberg-flink:flink-runtime-1.15").name = "iceberg-flink-runtime-1.15" -} - if (flinkVersions.contains("1.16")) { include ":iceberg-flink:flink-1.16" include ":iceberg-flink:flink-runtime-1.16" @@ -133,6 +124,15 @@ if (flinkVersions.contains("1.17")) { project(":iceberg-flink:flink-runtime-1.17").name = "iceberg-flink-runtime-1.17" } +if (flinkVersions.contains("1.18")) { + include ":iceberg-flink:flink-1.18" + include ":iceberg-flink:flink-runtime-1.18" + project(":iceberg-flink:flink-1.18").projectDir = file('flink/v1.18/flink') + project(":iceberg-flink:flink-1.18").name = "iceberg-flink-1.18" + project(":iceberg-flink:flink-runtime-1.18").projectDir = file('flink/v1.18/flink-runtime') + project(":iceberg-flink:flink-runtime-1.18").name = "iceberg-flink-runtime-1.18" +} + if (sparkVersions.contains("3.2")) { include ":iceberg-spark:spark-3.2_${scalaVersion}" include ":iceberg-spark:spark-extensions-3.2_${scalaVersion}" From b79a8ffce3a95e5e67a4f926ee63d165153e1c54 Mon Sep 17 00:00:00 2001 From: HonahX <140284484+HonahX@users.noreply.github.com> Date: Thu, 7 Dec 2023 11:33:20 -0800 Subject: [PATCH 22/47] Delta: Fix integration tests and Create DataFile by partition values instead of path (#8398) --- .github/workflows/delta-conversion-ci.yml | 4 +- build.gradle | 17 +-- .../delta/TestSnapshotDeltaLakeTable.java | 114 ++++++++---------- .../BaseSnapshotDeltaLakeTableAction.java | 8 +- gradle/libs.versions.toml | 4 +- 5 files changed, 68 insertions(+), 79 deletions(-) diff --git a/.github/workflows/delta-conversion-ci.yml b/.github/workflows/delta-conversion-ci.yml index 5261b8217624..6fd97e662ab0 100644 --- a/.github/workflows/delta-conversion-ci.yml +++ b/.github/workflows/delta-conversion-ci.yml @@ -74,7 +74,7 @@ jobs: key: ${{ runner.os }}-gradle-${{ hashFiles('**/*.gradle*', '**/gradle-wrapper.properties') }} restore-keys: ${{ runner.os }}-gradle- - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - - run: ./gradlew -DsparkVersions=3.3 -DscalaVersion=2.12 -DhiveVersions= -DflinkVersions= :iceberg-delta-lake:check -Pquick=true -x javadoc + - run: ./gradlew -DsparkVersions=3.5 -DscalaVersion=2.12 -DhiveVersions= -DflinkVersions= :iceberg-delta-lake:check -Pquick=true -x javadoc - uses: actions/upload-artifact@v3 if: failure() with: @@ -103,7 +103,7 @@ jobs: key: ${{ runner.os }}-gradle-${{ hashFiles('**/*.gradle*', '**/gradle-wrapper.properties') }} restore-keys: ${{ runner.os }}-gradle- - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - - run: ./gradlew -DsparkVersions=3.3 -DscalaVersion=2.13 -DhiveVersions= -DflinkVersions= :iceberg-delta-lake:check -Pquick=true -x javadoc + - run: ./gradlew -DsparkVersions=3.5 -DscalaVersion=2.13 -DhiveVersions= -DflinkVersions= :iceberg-delta-lake:check -Pquick=true -x javadoc - uses: actions/upload-artifact@v3 if: failure() with: diff --git a/build.gradle b/build.gradle index 94996a41a648..3f76cbea02bf 100644 --- a/build.gradle +++ b/build.gradle @@ -577,11 +577,11 @@ project(':iceberg-delta-lake') { exclude group: 'com.google.code.gson', module: 'gson' } - // The newest version of delta-core uses Spark 3.3.*. Since its only for test, we do + // The newest version of delta-core uses Spark 3.5.*. Since its only for test, we do // not need to include older version of delta-core - if (sparkVersions.contains("3.3")) { - integrationImplementation "io.delta:delta-core_${scalaVersion}:${libs.versions.delta.core.get()}" - integrationImplementation project(path: ":iceberg-spark:iceberg-spark-3.3_${scalaVersion}") + if (sparkVersions.contains("3.5")) { + integrationImplementation "io.delta:delta-spark_${scalaVersion}:${libs.versions.delta.spark.get()}" + integrationImplementation project(path: ":iceberg-spark:iceberg-spark-3.5_${scalaVersion}") integrationImplementation(libs.hadoop2.minicluster) { exclude group: 'org.apache.avro', module: 'avro' // to make sure netty libs only come from project(':iceberg-arrow') @@ -590,7 +590,7 @@ project(':iceberg-delta-lake') { } integrationImplementation project(path: ':iceberg-hive-metastore') integrationImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') - integrationImplementation("org.apache.spark:spark-hive_${scalaVersion}:${libs.versions.spark.hive33.get()}") { + integrationImplementation("org.apache.spark:spark-hive_${scalaVersion}:${libs.versions.spark.hive35.get()}") { exclude group: 'org.apache.avro', module: 'avro' exclude group: 'org.apache.arrow' exclude group: 'org.apache.parquet' @@ -602,9 +602,9 @@ project(':iceberg-delta-lake') { } } - // The newest version of delta-core uses Spark 3.3.*. The integration test should only be built - // if iceberg-spark-3.3 is available - if (sparkVersions.contains("3.3")) { + // The newest version of delta-core uses Spark 3.5.*. The integration test should only be built + // if iceberg-spark-3.5 is available + if (sparkVersions.contains("3.5")) { sourceSets { integration { java.srcDir "$projectDir/src/integration/java" @@ -615,6 +615,7 @@ project(':iceberg-delta-lake') { } task integrationTest(type: Test) { + useJUnitPlatform() testClassesDirs = sourceSets.integration.output.classesDirs classpath = sourceSets.integration.runtimeClasspath jvmArgs += project.property('extraJvmArgs') diff --git a/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java index 52966a484e68..cebbea65f57f 100644 --- a/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java +++ b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java @@ -18,8 +18,10 @@ */ package org.apache.iceberg.delta; +import static org.apache.spark.sql.functions.col; import static org.apache.spark.sql.functions.current_date; import static org.apache.spark.sql.functions.date_add; +import static org.apache.spark.sql.functions.date_format; import static org.apache.spark.sql.functions.expr; import io.delta.standalone.DeltaLog; @@ -34,7 +36,6 @@ import java.io.IOException; import java.net.URI; import java.nio.file.Files; -import java.nio.file.Path; import java.nio.file.Paths; import java.sql.Timestamp; import java.util.Iterator; @@ -42,7 +43,6 @@ import java.util.Map; import java.util.Random; import java.util.stream.Collectors; -import java.util.stream.Stream; import org.apache.commons.codec.DecoderException; import org.apache.commons.codec.net.URLCodec; import org.apache.iceberg.Snapshot; @@ -62,10 +62,8 @@ import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.Arguments; -import org.junit.jupiter.params.provider.MethodSource; public class TestSnapshotDeltaLakeTable extends SparkDeltaLakeSnapshotTestBase { private static final String SNAPSHOT_SOURCE_PROP = "snapshot_source"; @@ -74,31 +72,22 @@ public class TestSnapshotDeltaLakeTable extends SparkDeltaLakeSnapshotTestBase { private static final String NAMESPACE = "delta_conversion_test"; private static final String defaultSparkCatalog = "spark_catalog"; private static final String icebergCatalogName = "iceberg_hive"; + private static final Map config = + ImmutableMap.of( + "type", "hive", + "default-namespace", "default", + "parquet-enabled", "true", + "cache-enabled", + "false" // Spark will delete tables using v1, leaving the cache out of sync + ); private static Dataset typeTestDataFrame; private static Dataset nestedDataFrame; - static Stream parameters() { - return Stream.of( - Arguments.of( - icebergCatalogName, - SparkCatalog.class.getName(), - ImmutableMap.of( - "type", - "hive", - "default-namespace", - "default", - "parquet-enabled", - "true", - "cache-enabled", - "false" // Spark will delete tables using v1, leaving the cache out of sync - ))); - } - - @TempDir private Path temp; + @TempDir private File tempA; + @TempDir private File tempB; - public TestSnapshotDeltaLakeTable( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); + public TestSnapshotDeltaLakeTable() { + super(icebergCatalogName, SparkCatalog.class.getName(), config); spark.conf().set("spark.sql.catalog." + defaultSparkCatalog, DeltaCatalog.class.getName()); } @@ -115,7 +104,8 @@ public static void beforeClass() { .withColumn("doubleCol", expr("CAST(longCol AS DOUBLE)")) .withColumn("dateCol", date_add(current_date(), 1)) .withColumn("timestampCol", expr("TO_TIMESTAMP(dateCol)")) - .withColumn("stringCol", expr("CAST(timestampCol AS STRING)")) + .withColumn("timestampStrCol", expr("CAST(timestampCol AS STRING)")) + .withColumn("stringCol", date_format(col("timestampCol"), "yyyy/M/d")) .withColumn("booleanCol", expr("longCol > 5")) .withColumn("binaryCol", expr("CAST(longCol AS BINARY)")) .withColumn("byteCol", expr("CAST(longCol AS BYTE)")) @@ -160,11 +150,10 @@ public static void afterClass() { spark.sql(String.format("DROP DATABASE IF EXISTS %s CASCADE", NAMESPACE)); } - @ParameterizedTest(name = "Catalog Name {0} - Options {2}") - @MethodSource("parameters") + @Test public void testBasicSnapshotPartitioned() { String partitionedIdentifier = destName(defaultSparkCatalog, "partitioned_table"); - String partitionedLocation = temp.toFile().toURI().toString(); + String partitionedLocation = tempA.toURI().toString(); writeDeltaTable(nestedDataFrame, partitionedIdentifier, partitionedLocation, "id"); spark.sql("DELETE FROM " + partitionedIdentifier + " WHERE id=3"); @@ -182,13 +171,12 @@ public void testBasicSnapshotPartitioned() { checkIcebergTableLocation(newTableIdentifier, partitionedLocation); } - @ParameterizedTest(name = "Catalog Name {0} - Options {2}") - @MethodSource("parameters") + @Test public void testBasicSnapshotUnpartitioned() { String unpartitionedIdentifier = destName(defaultSparkCatalog, "unpartitioned_table"); - String unpartitionedLocation = temp.toFile().toURI().toString(); + String unpartitionedLocation = tempA.toURI().toString(); - writeDeltaTable(nestedDataFrame, unpartitionedIdentifier, unpartitionedLocation, null); + writeDeltaTable(nestedDataFrame, unpartitionedIdentifier, unpartitionedLocation); spark.sql("DELETE FROM " + unpartitionedIdentifier + " WHERE id=3"); spark.sql("UPDATE " + unpartitionedIdentifier + " SET id=3 WHERE id=1"); @@ -204,12 +192,11 @@ public void testBasicSnapshotUnpartitioned() { checkIcebergTableLocation(newTableIdentifier, unpartitionedLocation); } - @ParameterizedTest(name = "Catalog Name {0} - Options {2}") - @MethodSource("parameters") + @Test public void testSnapshotWithNewLocation() { String partitionedIdentifier = destName(defaultSparkCatalog, "partitioned_table"); - String partitionedLocation = temp.toFile().toURI().toString(); - String newIcebergTableLocation = temp.toFile().toURI().toString(); + String partitionedLocation = tempA.toURI().toString(); + String newIcebergTableLocation = tempB.toURI().toString(); writeDeltaTable(nestedDataFrame, partitionedIdentifier, partitionedLocation, "id"); spark.sql("DELETE FROM " + partitionedIdentifier + " WHERE id=3"); @@ -228,13 +215,12 @@ public void testSnapshotWithNewLocation() { checkIcebergTableLocation(newTableIdentifier, newIcebergTableLocation); } - @ParameterizedTest(name = "Catalog Name {0} - Options {2}") - @MethodSource("parameters") + @Test public void testSnapshotWithAdditionalProperties() { String unpartitionedIdentifier = destName(defaultSparkCatalog, "unpartitioned_table"); - String unpartitionedLocation = temp.toFile().toURI().toString(); + String unpartitionedLocation = tempA.toURI().toString(); - writeDeltaTable(nestedDataFrame, unpartitionedIdentifier, unpartitionedLocation, null); + writeDeltaTable(nestedDataFrame, unpartitionedIdentifier, unpartitionedLocation); spark.sql("DELETE FROM " + unpartitionedIdentifier + " WHERE id=3"); spark.sql("UPDATE " + unpartitionedIdentifier + " SET id=3 WHERE id=1"); @@ -267,20 +253,18 @@ public void testSnapshotWithAdditionalProperties() { unpartitionedLocation); } - @ParameterizedTest(name = "Catalog Name {0} - Options {2}") - @MethodSource("parameters") + @Test public void testSnapshotTableWithExternalDataFiles() { String unpartitionedIdentifier = destName(defaultSparkCatalog, "unpartitioned_table"); String externalDataFilesIdentifier = destName(defaultSparkCatalog, "external_data_files_table"); - String unpartitionedLocation = temp.toFile().toURI().toString(); - String externalDataFilesTableLocation = temp.toFile().toURI().toString(); + String unpartitionedLocation = tempA.toURI().toString(); + String externalDataFilesTableLocation = tempB.toURI().toString(); - writeDeltaTable(nestedDataFrame, unpartitionedIdentifier, unpartitionedLocation, null); + writeDeltaTable(nestedDataFrame, unpartitionedIdentifier, unpartitionedLocation); spark.sql("DELETE FROM " + unpartitionedIdentifier + " WHERE id=3"); spark.sql("UPDATE " + unpartitionedIdentifier + " SET id=3 WHERE id=1"); - writeDeltaTable( - nestedDataFrame, externalDataFilesIdentifier, externalDataFilesTableLocation, null); + writeDeltaTable(nestedDataFrame, externalDataFilesIdentifier, externalDataFilesTableLocation); // Add parquet files to default.external_data_files_table. The newly added parquet files // are not at the same location as the table. addExternalDatafiles(externalDataFilesTableLocation, unpartitionedLocation); @@ -297,13 +281,19 @@ public void testSnapshotTableWithExternalDataFiles() { checkDataFilePathsIntegrity(newTableIdentifier, externalDataFilesTableLocation); } - @ParameterizedTest(name = "Catalog Name {0} - Options {2}") - @MethodSource("parameters") + @Test public void testSnapshotSupportedTypes() { String typeTestIdentifier = destName(defaultSparkCatalog, "type_test_table"); - String typeTestTableLocation = temp.toFile().toURI().toString(); + String typeTestTableLocation = tempA.toURI().toString(); - writeDeltaTable(typeTestDataFrame, typeTestIdentifier, typeTestTableLocation, "stringCol"); + writeDeltaTable( + typeTestDataFrame, + typeTestIdentifier, + typeTestTableLocation, + "stringCol", + "timestampStrCol", + "booleanCol", + "longCol"); String newTableIdentifier = destName(icebergCatalogName, "iceberg_type_test_table"); SnapshotDeltaLakeTable.Result result = DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable( @@ -316,13 +306,12 @@ public void testSnapshotSupportedTypes() { checkIcebergTableProperties(newTableIdentifier, ImmutableMap.of(), typeTestTableLocation); } - @ParameterizedTest(name = "Catalog Name {0} - Options {2}") - @MethodSource("parameters") + @Test public void testSnapshotVacuumTable() throws IOException { String vacuumTestIdentifier = destName(defaultSparkCatalog, "vacuum_test_table"); - String vacuumTestTableLocation = temp.toFile().toURI().toString(); + String vacuumTestTableLocation = tempA.toURI().toString(); - writeDeltaTable(nestedDataFrame, vacuumTestIdentifier, vacuumTestTableLocation, null); + writeDeltaTable(nestedDataFrame, vacuumTestIdentifier, vacuumTestTableLocation); Random random = new Random(); for (int i = 0; i < 13; i++) { spark.sql( @@ -352,11 +341,10 @@ public void testSnapshotVacuumTable() throws IOException { checkIcebergTableLocation(newTableIdentifier, vacuumTestTableLocation); } - @ParameterizedTest(name = "Catalog Name {0} - Options {2}") - @MethodSource("parameters") + @Test public void testSnapshotLogCleanTable() throws IOException { String logCleanTestIdentifier = destName(defaultSparkCatalog, "log_clean_test_table"); - String logCleanTestTableLocation = temp.toFile().toURI().toString(); + String logCleanTestTableLocation = tempA.toURI().toString(); writeDeltaTable(nestedDataFrame, logCleanTestIdentifier, logCleanTestTableLocation, "id"); Random random = new Random(); @@ -549,14 +537,14 @@ private static String getFullFilePath(String path, String tableRoot) { } private void writeDeltaTable( - Dataset df, String identifier, String path, String partitionColumn) { + Dataset df, String identifier, String path, String... partitionColumns) { spark.sql(String.format("DROP TABLE IF EXISTS %s", identifier)); - if (partitionColumn != null) { + if (partitionColumns.length > 0) { df.write() .format("delta") .mode(SaveMode.Append) .option("path", path) - .partitionBy(partitionColumn) + .partitionBy(partitionColumns) .saveAsTable(identifier); } else { df.write().format("delta").mode(SaveMode.Append).option("path", path).saveAsTable(identifier); diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java index eb2414ac8276..afe43f6bb165 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java @@ -386,18 +386,18 @@ private DataFile buildDataFileFromAction(Action action, Table table) { nameMappingString != null ? NameMappingParser.fromJson(nameMappingString) : null; Metrics metrics = getMetricsForFile(file, format, metricsConfig, nameMapping); - String partition = + List partitionValueList = spec.fields().stream() .map(PartitionField::name) - .map(name -> String.format("%s=%s", name, partitionValues.get(name))) - .collect(Collectors.joining("/")); + .map(partitionValues::get) + .collect(Collectors.toList()); return DataFiles.builder(spec) .withPath(fullFilePath) .withFormat(format) .withFileSizeInBytes(fileSize) .withMetrics(metrics) - .withPartitionPath(partition) + .withPartitionValues(partitionValueList) .build(); } diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 29d5610f3c32..f4f50754095e 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -32,7 +32,7 @@ azuresdk-bom = "1.2.18" caffeine = "2.9.3" calcite = "1.10.0" delta-standalone = "0.6.0" -delta-core = "2.2.0" +delta-spark = "3.0.0" esotericsoftware-kryo = "4.0.2" errorprone-annotations = "2.23.0" findbugs-jsr305 = "3.0.2" @@ -169,7 +169,7 @@ snowflake-jdbc = { module = "net.snowflake:snowflake-jdbc", version.ref = "snowf # test libraries assertj-core = { module = "org.assertj:assertj-core", version.ref = "assertj-core" } awaitility = { module = "org.awaitility:awaitility", version.ref = "awaitility" } -delta-core = { module = "io.delta:delta-core_2.12", version.ref = "delta-core" } +delta-spark = { module = "io.delta:delta-spark_2.12", version.ref = "delta-spark" } esotericsoftware-kryo = { module = "com.esotericsoftware:kryo", version.ref = "esotericsoftware-kryo" } flink116-connector-test-utils = { module = "org.apache.flink:flink-connector-test-utils", version.ref = "flink116" } flink116-core = { module = "org.apache.flink:flink-core", version.ref = "flink116" } From 263b530502e5597b19b6b5e282917af8eede7600 Mon Sep 17 00:00:00 2001 From: Pucheng Yang <8072956+puchengy@users.noreply.github.com> Date: Thu, 7 Dec 2023 13:34:27 -0800 Subject: [PATCH 23/47] Spark 3.5: Support Specifying spec_id in RewriteManifestProcedure (#9242) --- .../TestRewriteManifestsProcedure.java | 35 +++++++++++++++++++ .../procedures/RewriteManifestsProcedure.java | 8 ++++- 2 files changed, 42 insertions(+), 1 deletion(-) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java index dc22fc4b704c..8e64eaad44e7 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java @@ -331,4 +331,39 @@ public void testReplacePartitionField() { row(1, Timestamp.valueOf("2022-01-01 10:00:00"), Date.valueOf("2022-01-01"))), sql("SELECT * FROM %s WHERE ts < current_timestamp()", tableName)); } + + @Test + public void testWriteManifestWithSpecId() { + sql( + "CREATE TABLE %s (id int, dt string, hr string) USING iceberg PARTITIONED BY (dt)", + tableName); + sql("ALTER TABLE %s SET TBLPROPERTIES ('commit.manifest-merge.enabled' = 'false')", tableName); + + sql("INSERT INTO %s VALUES (1, '2024-01-01', '00')", tableName); + sql("INSERT INTO %s VALUES (2, '2024-01-01', '00')", tableName); + assertEquals( + "Should have 2 manifests and their partition spec id should be 0", + ImmutableList.of(row(0), row(0)), + sql("SELECT partition_spec_id FROM %s.manifests order by 1 asc", tableName)); + + sql("ALTER TABLE %s ADD PARTITION FIELD hr", tableName); + sql("INSERT INTO %s VALUES (3, '2024-01-01', '00')", tableName); + assertEquals( + "Should have 3 manifests and their partition spec id should be 0 and 1", + ImmutableList.of(row(0), row(0), row(1)), + sql("SELECT partition_spec_id FROM %s.manifests order by 1 asc", tableName)); + + List output = sql("CALL %s.system.rewrite_manifests('%s')", catalogName, tableIdent); + assertEquals("Nothing should be rewritten", ImmutableList.of(row(0, 0)), output); + + output = + sql( + "CALL %s.system.rewrite_manifests(table => '%s', spec_id => 0)", + catalogName, tableIdent); + assertEquals("There should be 2 manifests rewriten", ImmutableList.of(row(2, 1)), output); + assertEquals( + "Should have 2 manifests and their partition spec id should be 0 and 1", + ImmutableList.of(row(0), row(1)), + sql("SELECT partition_spec_id FROM %s.manifests order by 1 asc", tableName)); + } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java index c8becc7e5a0f..e59077ae3da9 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java @@ -46,7 +46,8 @@ class RewriteManifestsProcedure extends BaseProcedure { private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[] { ProcedureParameter.required("table", DataTypes.StringType), - ProcedureParameter.optional("use_caching", DataTypes.BooleanType) + ProcedureParameter.optional("use_caching", DataTypes.BooleanType), + ProcedureParameter.optional("spec_id", DataTypes.IntegerType) }; // counts are not nullable since the action result is never null @@ -85,6 +86,7 @@ public StructType outputType() { public InternalRow[] call(InternalRow args) { Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name()); Boolean useCaching = args.isNullAt(1) ? null : args.getBoolean(1); + Integer specId = args.isNullAt(2) ? null : args.getInt(2); return modifyIcebergTable( tableIdent, @@ -95,6 +97,10 @@ public InternalRow[] call(InternalRow args) { action.option(RewriteManifestsSparkAction.USE_CACHING, useCaching.toString()); } + if (specId != null) { + action.specId(specId); + } + RewriteManifests.Result result = action.execute(); return toOutputRows(result); From feeaa8c73034f60f1ab55d0db763fe13fa3c229f Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Fri, 8 Dec 2023 01:22:36 -0800 Subject: [PATCH 24/47] Spark 3.5: Rework DeleteFileIndexBenchmark (#9165) --- .../apache/iceberg/FileGenerationUtil.java | 191 ++++++++++++++++++ spark/v3.5/build.gradle | 1 + .../iceberg/DeleteFileIndexBenchmark.java | 91 +-------- 3 files changed, 201 insertions(+), 82 deletions(-) create mode 100644 core/src/test/java/org/apache/iceberg/FileGenerationUtil.java diff --git a/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java b/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java new file mode 100644 index 000000000000..98a6eafaf8f6 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java @@ -0,0 +1,191 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import java.nio.ByteBuffer; +import java.util.Map; +import java.util.Random; +import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.iceberg.io.DeleteSchemaUtil; +import org.apache.iceberg.io.LocationProvider; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Types; + +public class FileGenerationUtil { + + private FileGenerationUtil() {} + + public static DataFile generateDataFile(Table table, StructLike partition) { + Schema schema = table.schema(); + PartitionSpec spec = table.spec(); + LocationProvider locations = table.locationProvider(); + String path = locations.newDataLocation(spec, partition, generateFileName()); + long fileSize = generateFileSize(); + Metrics metrics = generateRandomMetrics(schema); + return DataFiles.builder(spec) + .withPath(path) + .withPartition(partition) + .withFileSizeInBytes(fileSize) + .withFormat(FileFormat.PARQUET) + .withMetrics(metrics) + .build(); + } + + public static DeleteFile generatePositionDeleteFile(Table table, StructLike partition) { + PartitionSpec spec = table.spec(); + LocationProvider locations = table.locationProvider(); + String path = locations.newDataLocation(spec, partition, generateFileName()); + long fileSize = generateFileSize(); + Metrics metrics = generatePositionDeleteMetrics(); + return FileMetadata.deleteFileBuilder(table.spec()) + .ofPositionDeletes() + .withPath(path) + .withPartition(partition) + .withFileSizeInBytes(fileSize) + .withFormat(FileFormat.PARQUET) + .withMetrics(metrics) + .build(); + } + + public static DeleteFile generatePositionDeleteFile(Table table, DataFile dataFile) { + PartitionSpec spec = table.spec(); + StructLike partition = dataFile.partition(); + LocationProvider locations = table.locationProvider(); + String path = locations.newDataLocation(spec, partition, generateFileName()); + long fileSize = generateFileSize(); + Metrics metrics = generatePositionDeleteMetrics(dataFile); + return FileMetadata.deleteFileBuilder(table.spec()) + .ofPositionDeletes() + .withPath(path) + .withPartition(partition) + .withFileSizeInBytes(fileSize) + .withFormat(FileFormat.PARQUET) + .withMetrics(metrics) + .build(); + } + + // mimics the behavior of OutputFileFactory + public static String generateFileName() { + int partitionId = random().nextInt(100_000); + int taskId = random().nextInt(100); + UUID operationId = UUID.randomUUID(); + int fileCount = random().nextInt(1_000); + return String.format("%d-%d-%s-%d.parquet", partitionId, taskId, operationId, fileCount); + } + + public static Metrics generateRandomMetrics(Schema schema) { + long rowCount = generateRowCount(); + Map columnSizes = Maps.newHashMap(); + Map valueCounts = Maps.newHashMap(); + Map nullValueCounts = Maps.newHashMap(); + Map nanValueCounts = Maps.newHashMap(); + Map lowerBounds = Maps.newHashMap(); + Map upperBounds = Maps.newHashMap(); + + for (Types.NestedField column : schema.columns()) { + int fieldId = column.fieldId(); + columnSizes.put(fieldId, generateColumnSize()); + valueCounts.put(fieldId, generateValueCount()); + nullValueCounts.put(fieldId, (long) random().nextInt(5)); + nanValueCounts.put(fieldId, (long) random().nextInt(5)); + byte[] lower = new byte[16]; + random().nextBytes(lower); + lowerBounds.put(fieldId, ByteBuffer.wrap(lower)); + byte[] upper = new byte[16]; + random().nextBytes(upper); + upperBounds.put(fieldId, ByteBuffer.wrap(upper)); + } + + return new Metrics( + rowCount, + columnSizes, + valueCounts, + nullValueCounts, + nanValueCounts, + lowerBounds, + upperBounds); + } + + private static Metrics generatePositionDeleteMetrics(DataFile dataFile) { + long rowCount = generateRowCount(); + Map columnSizes = Maps.newHashMap(); + Map lowerBounds = Maps.newHashMap(); + Map upperBounds = Maps.newHashMap(); + + for (Types.NestedField column : DeleteSchemaUtil.pathPosSchema().columns()) { + int fieldId = column.fieldId(); + columnSizes.put(fieldId, generateColumnSize()); + if (fieldId == MetadataColumns.DELETE_FILE_PATH.fieldId()) { + ByteBuffer bound = Conversions.toByteBuffer(Types.StringType.get(), dataFile.path()); + lowerBounds.put(fieldId, bound); + upperBounds.put(fieldId, bound); + } + } + + return new Metrics( + rowCount, + columnSizes, + null /* no value counts */, + null /* no NULL counts */, + null /* no NaN counts */, + lowerBounds, + upperBounds); + } + + private static Metrics generatePositionDeleteMetrics() { + long rowCount = generateRowCount(); + Map columnSizes = Maps.newHashMap(); + + for (Types.NestedField column : DeleteSchemaUtil.pathPosSchema().columns()) { + int fieldId = column.fieldId(); + columnSizes.put(fieldId, generateColumnSize()); + } + + return new Metrics( + rowCount, + columnSizes, + null /* no value counts */, + null /* no NULL counts */, + null /* no NaN counts */, + null /* no lower bounds */, + null /* no upper bounds */); + } + + private static long generateRowCount() { + return 100_000L + random().nextInt(1000); + } + + private static long generateColumnSize() { + return 1_000_000L + random().nextInt(100_000); + } + + private static long generateValueCount() { + return 100_000L + random().nextInt(100); + } + + private static long generateFileSize() { + return random().nextInt(50_000); + } + + private static Random random() { + return ThreadLocalRandom.current(); + } +} diff --git a/spark/v3.5/build.gradle b/spark/v3.5/build.gradle index 58edd4145602..2c58281904c9 100644 --- a/spark/v3.5/build.gradle +++ b/spark/v3.5/build.gradle @@ -158,6 +158,7 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer testImplementation project(path: ':iceberg-parquet') testImplementation project(path: ':iceberg-hive-metastore') testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') + testImplementation project(path: ':iceberg-core', configuration: 'testArtifacts') testImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') testImplementation project(path: ":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}", configuration: 'testArtifacts') diff --git a/spark/v3.5/spark-extensions/src/jmh/java/org/apache/iceberg/DeleteFileIndexBenchmark.java b/spark/v3.5/spark-extensions/src/jmh/java/org/apache/iceberg/DeleteFileIndexBenchmark.java index aa310d8f7fc1..73bbbd85235d 100644 --- a/spark/v3.5/spark-extensions/src/jmh/java/org/apache/iceberg/DeleteFileIndexBenchmark.java +++ b/spark/v3.5/spark-extensions/src/jmh/java/org/apache/iceberg/DeleteFileIndexBenchmark.java @@ -18,8 +18,6 @@ */ package org.apache.iceberg; -import static org.apache.spark.sql.functions.lit; - import com.google.errorprone.annotations.FormatMethod; import com.google.errorprone.annotations.FormatString; import java.io.IOException; @@ -29,25 +27,15 @@ import java.util.concurrent.TimeUnit; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.Spark3Util; -import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.spark.SparkSessionCatalog; -import org.apache.iceberg.spark.data.RandomData; import org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions; import org.apache.iceberg.util.ThreadPools; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.catalyst.parser.ParseException; -import org.apache.spark.sql.types.StructType; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; @@ -83,10 +71,8 @@ public class DeleteFileIndexBenchmark { private static final String PARTITION_COLUMN = "ss_ticket_number"; private static final int NUM_PARTITIONS = 50; - private static final int NUM_REAL_DATA_FILES_PER_PARTITION = 25; - private static final int NUM_REPLICA_DATA_FILES_PER_PARTITION = 50_000; + private static final int NUM_DATA_FILES_PER_PARTITION = 50_000; private static final int NUM_DELETE_FILES_PER_PARTITION = 100; - private static final int NUM_ROWS_PER_DATA_FILE = 500; private final Configuration hadoopConf = new Configuration(); private SparkSession spark; @@ -148,85 +134,26 @@ private DeleteFileIndex buildDeletes() { .build(); } - private DataFile loadAddedDataFile() { - table.refresh(); - - Iterable addedDataFiles = table.currentSnapshot().addedDataFiles(table.io()); - return Iterables.getOnlyElement(addedDataFiles); - } - - private DeleteFile loadAddedDeleteFile() { - table.refresh(); - - Iterable addedDeleteFiles = table.currentSnapshot().addedDeleteFiles(table.io()); - return Iterables.getOnlyElement(addedDeleteFiles); - } - - private void initDataAndDeletes() throws NoSuchTableException { - Schema schema = table.schema(); - PartitionSpec spec = table.spec(); - LocationProvider locations = table.locationProvider(); - + private void initDataAndDeletes() { for (int partitionOrdinal = 0; partitionOrdinal < NUM_PARTITIONS; partitionOrdinal++) { - Dataset inputDF = - randomDataDF(schema, NUM_ROWS_PER_DATA_FILE) - .drop(PARTITION_COLUMN) - .withColumn(PARTITION_COLUMN, lit(partitionOrdinal)); - - for (int fileOrdinal = 0; fileOrdinal < NUM_REAL_DATA_FILES_PER_PARTITION; fileOrdinal++) { - appendAsFile(inputDF); - } + StructLike partition = TestHelpers.Row.of(partitionOrdinal); - DataFile dataFile = loadAddedDataFile(); - - sql( - "DELETE FROM %s WHERE ss_item_sk IS NULL AND %s = %d", - TABLE_NAME, PARTITION_COLUMN, partitionOrdinal); - - DeleteFile deleteFile = loadAddedDeleteFile(); - - AppendFiles append = table.newFastAppend(); + RowDelta rowDelta = table.newRowDelta(); - for (int fileOrdinal = 0; fileOrdinal < NUM_REPLICA_DATA_FILES_PER_PARTITION; fileOrdinal++) { - String replicaFileName = UUID.randomUUID() + "-replica.parquet"; - DataFile replicaDataFile = - DataFiles.builder(spec) - .copy(dataFile) - .withPath(locations.newDataLocation(spec, dataFile.partition(), replicaFileName)) - .build(); - append.appendFile(replicaDataFile); + for (int fileOrdinal = 0; fileOrdinal < NUM_DATA_FILES_PER_PARTITION; fileOrdinal++) { + DataFile dataFile = FileGenerationUtil.generateDataFile(table, partition); + rowDelta.addRows(dataFile); } - append.commit(); - - RowDelta rowDelta = table.newRowDelta(); - for (int fileOrdinal = 0; fileOrdinal < NUM_DELETE_FILES_PER_PARTITION; fileOrdinal++) { - String replicaFileName = UUID.randomUUID() + "-replica.parquet"; - DeleteFile replicaDeleteFile = - FileMetadata.deleteFileBuilder(spec) - .copy(deleteFile) - .withPath(locations.newDataLocation(spec, deleteFile.partition(), replicaFileName)) - .build(); - rowDelta.addDeletes(replicaDeleteFile); + DeleteFile deleteFile = FileGenerationUtil.generatePositionDeleteFile(table, partition); + rowDelta.addDeletes(deleteFile); } rowDelta.commit(); } } - private void appendAsFile(Dataset df) throws NoSuchTableException { - df.coalesce(1).writeTo(TABLE_NAME).append(); - } - - private Dataset randomDataDF(Schema schema, int numRows) { - Iterable rows = RandomData.generateSpark(schema, numRows, 0); - JavaSparkContext context = JavaSparkContext.fromSparkContext(spark.sparkContext()); - JavaRDD rowRDD = context.parallelize(Lists.newArrayList(rows)); - StructType rowSparkType = SparkSchemaUtil.convert(schema); - return spark.internalCreateDataFrame(JavaRDD.toRDD(rowRDD), rowSparkType, false); - } - private void setupSpark() { this.spark = SparkSession.builder() From 3f5f4d924be94c6f06793a3667c8da95c52c4064 Mon Sep 17 00:00:00 2001 From: Pucheng Yang <8072956+puchengy@users.noreply.github.com> Date: Fri, 8 Dec 2023 07:20:26 -0800 Subject: [PATCH 25/47] Spark 3.2, 3.3, 3.4: Support specifying spec_id in RewriteManifestProcedure (#9243)(#9242) --- .../TestRewriteManifestsProcedure.java | 35 +++++++++++++++++++ .../procedures/RewriteManifestsProcedure.java | 8 ++++- .../TestRewriteManifestsProcedure.java | 35 +++++++++++++++++++ .../procedures/RewriteManifestsProcedure.java | 8 ++++- .../TestRewriteManifestsProcedure.java | 35 +++++++++++++++++++ .../procedures/RewriteManifestsProcedure.java | 8 ++++- 6 files changed, 126 insertions(+), 3 deletions(-) diff --git a/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java b/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java index 40625b5e3450..2675c1010baa 100644 --- a/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java +++ b/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java @@ -339,4 +339,39 @@ public void testReplacePartitionField() { row(1, Timestamp.valueOf("2022-01-01 10:00:00"), Date.valueOf("2022-01-01"))), sql("SELECT * FROM %s WHERE ts < current_timestamp()", tableName)); } + + @Test + public void testWriteManifestWithSpecId() { + sql( + "CREATE TABLE %s (id int, dt string, hr string) USING iceberg PARTITIONED BY (dt)", + tableName); + sql("ALTER TABLE %s SET TBLPROPERTIES ('commit.manifest-merge.enabled' = 'false')", tableName); + + sql("INSERT INTO %s VALUES (1, '2024-01-01', '00')", tableName); + sql("INSERT INTO %s VALUES (2, '2024-01-01', '00')", tableName); + assertEquals( + "Should have 2 manifests and their partition spec id should be 0", + ImmutableList.of(row(0), row(0)), + sql("SELECT partition_spec_id FROM %s.manifests order by 1 asc", tableName)); + + sql("ALTER TABLE %s ADD PARTITION FIELD hr", tableName); + sql("INSERT INTO %s VALUES (3, '2024-01-01', '00')", tableName); + assertEquals( + "Should have 3 manifests and their partition spec id should be 0 and 1", + ImmutableList.of(row(0), row(0), row(1)), + sql("SELECT partition_spec_id FROM %s.manifests order by 1 asc", tableName)); + + List output = sql("CALL %s.system.rewrite_manifests('%s')", catalogName, tableIdent); + assertEquals("Nothing should be rewritten", ImmutableList.of(row(0, 0)), output); + + output = + sql( + "CALL %s.system.rewrite_manifests(table => '%s', spec_id => 0)", + catalogName, tableIdent); + assertEquals("There should be 2 manifests rewriten", ImmutableList.of(row(2, 1)), output); + assertEquals( + "Should have 2 manifests and their partition spec id should be 0 and 1", + ImmutableList.of(row(0), row(1)), + sql("SELECT partition_spec_id FROM %s.manifests order by 1 asc", tableName)); + } } diff --git a/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java b/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java index c8becc7e5a0f..e59077ae3da9 100644 --- a/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java +++ b/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java @@ -46,7 +46,8 @@ class RewriteManifestsProcedure extends BaseProcedure { private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[] { ProcedureParameter.required("table", DataTypes.StringType), - ProcedureParameter.optional("use_caching", DataTypes.BooleanType) + ProcedureParameter.optional("use_caching", DataTypes.BooleanType), + ProcedureParameter.optional("spec_id", DataTypes.IntegerType) }; // counts are not nullable since the action result is never null @@ -85,6 +86,7 @@ public StructType outputType() { public InternalRow[] call(InternalRow args) { Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name()); Boolean useCaching = args.isNullAt(1) ? null : args.getBoolean(1); + Integer specId = args.isNullAt(2) ? null : args.getInt(2); return modifyIcebergTable( tableIdent, @@ -95,6 +97,10 @@ public InternalRow[] call(InternalRow args) { action.option(RewriteManifestsSparkAction.USE_CACHING, useCaching.toString()); } + if (specId != null) { + action.specId(specId); + } + RewriteManifests.Result result = action.execute(); return toOutputRows(result); diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java index 40625b5e3450..2675c1010baa 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java @@ -339,4 +339,39 @@ public void testReplacePartitionField() { row(1, Timestamp.valueOf("2022-01-01 10:00:00"), Date.valueOf("2022-01-01"))), sql("SELECT * FROM %s WHERE ts < current_timestamp()", tableName)); } + + @Test + public void testWriteManifestWithSpecId() { + sql( + "CREATE TABLE %s (id int, dt string, hr string) USING iceberg PARTITIONED BY (dt)", + tableName); + sql("ALTER TABLE %s SET TBLPROPERTIES ('commit.manifest-merge.enabled' = 'false')", tableName); + + sql("INSERT INTO %s VALUES (1, '2024-01-01', '00')", tableName); + sql("INSERT INTO %s VALUES (2, '2024-01-01', '00')", tableName); + assertEquals( + "Should have 2 manifests and their partition spec id should be 0", + ImmutableList.of(row(0), row(0)), + sql("SELECT partition_spec_id FROM %s.manifests order by 1 asc", tableName)); + + sql("ALTER TABLE %s ADD PARTITION FIELD hr", tableName); + sql("INSERT INTO %s VALUES (3, '2024-01-01', '00')", tableName); + assertEquals( + "Should have 3 manifests and their partition spec id should be 0 and 1", + ImmutableList.of(row(0), row(0), row(1)), + sql("SELECT partition_spec_id FROM %s.manifests order by 1 asc", tableName)); + + List output = sql("CALL %s.system.rewrite_manifests('%s')", catalogName, tableIdent); + assertEquals("Nothing should be rewritten", ImmutableList.of(row(0, 0)), output); + + output = + sql( + "CALL %s.system.rewrite_manifests(table => '%s', spec_id => 0)", + catalogName, tableIdent); + assertEquals("There should be 2 manifests rewriten", ImmutableList.of(row(2, 1)), output); + assertEquals( + "Should have 2 manifests and their partition spec id should be 0 and 1", + ImmutableList.of(row(0), row(1)), + sql("SELECT partition_spec_id FROM %s.manifests order by 1 asc", tableName)); + } } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java index c8becc7e5a0f..e59077ae3da9 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java @@ -46,7 +46,8 @@ class RewriteManifestsProcedure extends BaseProcedure { private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[] { ProcedureParameter.required("table", DataTypes.StringType), - ProcedureParameter.optional("use_caching", DataTypes.BooleanType) + ProcedureParameter.optional("use_caching", DataTypes.BooleanType), + ProcedureParameter.optional("spec_id", DataTypes.IntegerType) }; // counts are not nullable since the action result is never null @@ -85,6 +86,7 @@ public StructType outputType() { public InternalRow[] call(InternalRow args) { Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name()); Boolean useCaching = args.isNullAt(1) ? null : args.getBoolean(1); + Integer specId = args.isNullAt(2) ? null : args.getInt(2); return modifyIcebergTable( tableIdent, @@ -95,6 +97,10 @@ public InternalRow[] call(InternalRow args) { action.option(RewriteManifestsSparkAction.USE_CACHING, useCaching.toString()); } + if (specId != null) { + action.specId(specId); + } + RewriteManifests.Result result = action.execute(); return toOutputRows(result); diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java index dc22fc4b704c..8e64eaad44e7 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java @@ -331,4 +331,39 @@ public void testReplacePartitionField() { row(1, Timestamp.valueOf("2022-01-01 10:00:00"), Date.valueOf("2022-01-01"))), sql("SELECT * FROM %s WHERE ts < current_timestamp()", tableName)); } + + @Test + public void testWriteManifestWithSpecId() { + sql( + "CREATE TABLE %s (id int, dt string, hr string) USING iceberg PARTITIONED BY (dt)", + tableName); + sql("ALTER TABLE %s SET TBLPROPERTIES ('commit.manifest-merge.enabled' = 'false')", tableName); + + sql("INSERT INTO %s VALUES (1, '2024-01-01', '00')", tableName); + sql("INSERT INTO %s VALUES (2, '2024-01-01', '00')", tableName); + assertEquals( + "Should have 2 manifests and their partition spec id should be 0", + ImmutableList.of(row(0), row(0)), + sql("SELECT partition_spec_id FROM %s.manifests order by 1 asc", tableName)); + + sql("ALTER TABLE %s ADD PARTITION FIELD hr", tableName); + sql("INSERT INTO %s VALUES (3, '2024-01-01', '00')", tableName); + assertEquals( + "Should have 3 manifests and their partition spec id should be 0 and 1", + ImmutableList.of(row(0), row(0), row(1)), + sql("SELECT partition_spec_id FROM %s.manifests order by 1 asc", tableName)); + + List output = sql("CALL %s.system.rewrite_manifests('%s')", catalogName, tableIdent); + assertEquals("Nothing should be rewritten", ImmutableList.of(row(0, 0)), output); + + output = + sql( + "CALL %s.system.rewrite_manifests(table => '%s', spec_id => 0)", + catalogName, tableIdent); + assertEquals("There should be 2 manifests rewriten", ImmutableList.of(row(2, 1)), output); + assertEquals( + "Should have 2 manifests and their partition spec id should be 0 and 1", + ImmutableList.of(row(0), row(1)), + sql("SELECT partition_spec_id FROM %s.manifests order by 1 asc", tableName)); + } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java index c8becc7e5a0f..e59077ae3da9 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java @@ -46,7 +46,8 @@ class RewriteManifestsProcedure extends BaseProcedure { private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[] { ProcedureParameter.required("table", DataTypes.StringType), - ProcedureParameter.optional("use_caching", DataTypes.BooleanType) + ProcedureParameter.optional("use_caching", DataTypes.BooleanType), + ProcedureParameter.optional("spec_id", DataTypes.IntegerType) }; // counts are not nullable since the action result is never null @@ -85,6 +86,7 @@ public StructType outputType() { public InternalRow[] call(InternalRow args) { Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name()); Boolean useCaching = args.isNullAt(1) ? null : args.getBoolean(1); + Integer specId = args.isNullAt(2) ? null : args.getInt(2); return modifyIcebergTable( tableIdent, @@ -95,6 +97,10 @@ public InternalRow[] call(InternalRow args) { action.option(RewriteManifestsSparkAction.USE_CACHING, useCaching.toString()); } + if (specId != null) { + action.specId(specId); + } + RewriteManifests.Result result = action.execute(); return toOutputRows(result); From 504c13428cd4ad4d343c644736b957d7f2653b37 Mon Sep 17 00:00:00 2001 From: bknbkn <67318028+bknbkn@users.noreply.github.com> Date: Sat, 9 Dec 2023 01:18:56 +0800 Subject: [PATCH 26/47] Spark 3.5: Fix testReplacePartitionField for rewriting manifests (#9250) --- .../TestRewriteManifestsProcedure.java | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java index 8e64eaad44e7..b230a6912984 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java @@ -316,20 +316,27 @@ public void testReplacePartitionField() { sql( "INSERT INTO %s VALUES (1, CAST('2022-01-01 10:00:00' AS TIMESTAMP), CAST('2022-01-01' AS DATE))", tableName); + sql( + "INSERT INTO %s VALUES (2, CAST('2022-01-01 11:00:00' AS TIMESTAMP), CAST('2022-01-01' AS DATE))", + tableName); assertEquals( "Should have expected rows", ImmutableList.of( - row(1, Timestamp.valueOf("2022-01-01 10:00:00"), Date.valueOf("2022-01-01"))), - sql("SELECT * FROM %s WHERE ts < current_timestamp()", tableName)); + row(1, Timestamp.valueOf("2022-01-01 10:00:00"), Date.valueOf("2022-01-01")), + row(2, Timestamp.valueOf("2022-01-01 11:00:00"), Date.valueOf("2022-01-01"))), + sql("SELECT * FROM %s WHERE ts < current_timestamp() order by 1 asc", tableName)); - sql("CALL %s.system.rewrite_manifests(table => '%s')", catalogName, tableName); + List output = + sql("CALL %s.system.rewrite_manifests(table => '%s')", catalogName, tableName); + assertEquals("Procedure output must match", ImmutableList.of(row(2, 1)), output); assertEquals( "Should have expected rows", ImmutableList.of( - row(1, Timestamp.valueOf("2022-01-01 10:00:00"), Date.valueOf("2022-01-01"))), - sql("SELECT * FROM %s WHERE ts < current_timestamp()", tableName)); + row(1, Timestamp.valueOf("2022-01-01 10:00:00"), Date.valueOf("2022-01-01")), + row(2, Timestamp.valueOf("2022-01-01 11:00:00"), Date.valueOf("2022-01-01"))), + sql("SELECT * FROM %s WHERE ts < current_timestamp() order by 1 asc", tableName)); } @Test From 62a23a37758792b31e2590bd7d1aa46da1074512 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Fri, 8 Dec 2023 12:52:29 -0800 Subject: [PATCH 27/47] Core: Fix null partitions in PartitionSet (#9248) --- .../org/apache/iceberg/util/PartitionSet.java | 4 +- .../apache/iceberg/util/TestPartitionMap.java | 2 + .../apache/iceberg/util/TestPartitionSet.java | 83 +++++++++++++++++++ 3 files changed, 87 insertions(+), 2 deletions(-) create mode 100644 core/src/test/java/org/apache/iceberg/util/TestPartitionSet.java diff --git a/core/src/main/java/org/apache/iceberg/util/PartitionSet.java b/core/src/main/java/org/apache/iceberg/util/PartitionSet.java index 893c3207071d..eff37fa5a9e8 100644 --- a/core/src/main/java/org/apache/iceberg/util/PartitionSet.java +++ b/core/src/main/java/org/apache/iceberg/util/PartitionSet.java @@ -63,7 +63,7 @@ public boolean contains(Object o) { if (o instanceof Pair) { Object first = ((Pair) o).first(); Object second = ((Pair) o).second(); - if (first instanceof Integer && second instanceof StructLike) { + if (first instanceof Integer && (second == null || second instanceof StructLike)) { return contains((Integer) first, (StructLike) second); } } @@ -98,7 +98,7 @@ public boolean remove(Object o) { if (o instanceof Pair) { Object first = ((Pair) o).first(); Object second = ((Pair) o).second(); - if (first instanceof Integer && second instanceof StructLike) { + if (first instanceof Integer && (second == null || second instanceof StructLike)) { return remove((Integer) first, (StructLike) second); } } diff --git a/core/src/test/java/org/apache/iceberg/util/TestPartitionMap.java b/core/src/test/java/org/apache/iceberg/util/TestPartitionMap.java index e528a1e70ee6..268f7eada88d 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestPartitionMap.java +++ b/core/src/test/java/org/apache/iceberg/util/TestPartitionMap.java @@ -274,9 +274,11 @@ public void testKeyAndEntrySetEquality() { map1.put(BY_DATA_SPEC.specId(), Row.of("aaa"), "v1"); map1.put(BY_DATA_SPEC.specId(), Row.of("bbb"), "v2"); + map1.put(UNPARTITIONED_SPEC.specId(), null, "v3"); map2.put(BY_DATA_SPEC.specId(), CustomRow.of("aaa"), "v1"); map2.put(BY_DATA_SPEC.specId(), CustomRow.of("bbb"), "v2"); + map2.put(UNPARTITIONED_SPEC.specId(), null, "v3"); assertThat(map1.keySet()).isEqualTo(map2.keySet()); assertThat(map1.entrySet()).isEqualTo(map2.entrySet()); diff --git a/core/src/test/java/org/apache/iceberg/util/TestPartitionSet.java b/core/src/test/java/org/apache/iceberg/util/TestPartitionSet.java new file mode 100644 index 000000000000..533e5904237b --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/util/TestPartitionSet.java @@ -0,0 +1,83 @@ +/* + * 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.iceberg.util; + +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Map; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.TestHelpers.CustomRow; +import org.apache.iceberg.TestHelpers.Row; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestPartitionSet { + private static final Schema SCHEMA = + new Schema( + required(1, "id", Types.IntegerType.get()), + required(2, "data", Types.StringType.get()), + required(3, "category", Types.StringType.get())); + private static final PartitionSpec UNPARTITIONED_SPEC = PartitionSpec.unpartitioned(); + private static final PartitionSpec BY_DATA_SPEC = + PartitionSpec.builderFor(SCHEMA).identity("data").withSpecId(1).build(); + private static final PartitionSpec BY_DATA_CATEGORY_BUCKET_SPEC = + PartitionSpec.builderFor(SCHEMA).identity("data").bucket("category", 8).withSpecId(3).build(); + private static final Map SPECS = + ImmutableMap.of( + UNPARTITIONED_SPEC.specId(), + UNPARTITIONED_SPEC, + BY_DATA_SPEC.specId(), + BY_DATA_SPEC, + BY_DATA_CATEGORY_BUCKET_SPEC.specId(), + BY_DATA_CATEGORY_BUCKET_SPEC); + + @Test + public void testGet() { + PartitionSet set = PartitionSet.create(SPECS); + set.add(BY_DATA_SPEC.specId(), Row.of("a")); + set.add(UNPARTITIONED_SPEC.specId(), null); + set.add(UNPARTITIONED_SPEC.specId(), Row.of()); + set.add(BY_DATA_CATEGORY_BUCKET_SPEC.specId(), CustomRow.of("a", 1)); + + assertThat(set).hasSize(4); + assertThat(set.contains(BY_DATA_SPEC.specId(), CustomRow.of("a"))).isTrue(); + assertThat(set.contains(UNPARTITIONED_SPEC.specId(), null)).isTrue(); + assertThat(set.contains(UNPARTITIONED_SPEC.specId(), CustomRow.of())).isTrue(); + assertThat(set.contains(BY_DATA_CATEGORY_BUCKET_SPEC.specId(), Row.of("a", 1))).isTrue(); + } + + @Test + public void testRemove() { + PartitionSet set = PartitionSet.create(SPECS); + set.add(BY_DATA_SPEC.specId(), Row.of("a")); + set.add(UNPARTITIONED_SPEC.specId(), null); + set.add(UNPARTITIONED_SPEC.specId(), Row.of()); + set.add(BY_DATA_CATEGORY_BUCKET_SPEC.specId(), CustomRow.of("a", 1)); + + assertThat(set).hasSize(4); + assertThat(set.remove(BY_DATA_SPEC.specId(), CustomRow.of("a"))).isTrue(); + assertThat(set.remove(UNPARTITIONED_SPEC.specId(), null)).isTrue(); + assertThat(set.remove(UNPARTITIONED_SPEC.specId(), CustomRow.of())).isTrue(); + assertThat(set.remove(BY_DATA_CATEGORY_BUCKET_SPEC.specId(), Row.of("a", 1))).isTrue(); + assertThat(set).isEmpty(); + } +} From beb41b649c47ec8bf677127b3cba1bd9690e1aac Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Fri, 8 Dec 2023 12:57:44 -0800 Subject: [PATCH 28/47] Flink: switch to use SortKey for data statistics (#9212) --- .../flink/sink/shuffle/DataStatistics.java | 12 +- .../sink/shuffle/DataStatisticsOperator.java | 25 +- .../sink/shuffle/DataStatisticsUtil.java | 1 - .../flink/sink/shuffle/MapDataStatistics.java | 21 +- .../shuffle/MapDataStatisticsSerializer.java | 56 +-- .../flink/sink/shuffle/SortKeySerializer.java | 353 ++++++++++++++++++ .../shuffle/TestAggregatedStatistics.java | 48 +-- .../TestAggregatedStatisticsTracker.java | 97 ++--- .../TestDataStatisticsCoordinator.java | 98 ++--- ...TestDataStatisticsCoordinatorProvider.java | 160 ++++---- .../shuffle/TestDataStatisticsOperator.java | 154 ++++---- .../sink/shuffle/TestMapDataStatistics.java | 90 +++++ .../shuffle/TestSortKeySerializerBase.java | 65 ++++ .../TestSortKeySerializerNestedStruct.java | 55 +++ .../TestSortKeySerializerPrimitives.java | 57 +++ 15 files changed, 978 insertions(+), 314 deletions(-) create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java index 28a05201c02f..9d7cf179ab1c 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java @@ -19,7 +19,7 @@ package org.apache.iceberg.flink.sink.shuffle; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.data.RowData; +import org.apache.iceberg.SortKey; /** * DataStatistics defines the interface to collect data distribution information. @@ -29,7 +29,7 @@ * (sketching) can be used. */ @Internal -interface DataStatistics { +interface DataStatistics, S> { /** * Check if data statistics contains any statistics information. @@ -38,12 +38,8 @@ interface DataStatistics { */ boolean isEmpty(); - /** - * Add data key to data statistics. - * - * @param key generate from data by applying key selector - */ - void add(RowData key); + /** Add row sortKey to data statistics. */ + void add(SortKey sortKey); /** * Merge current statistics with other statistics. diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java index d00d5d2e5aa9..5157a37cf2cd 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java @@ -22,7 +22,6 @@ import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.runtime.operators.coordination.OperatorEvent; import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; import org.apache.flink.runtime.operators.coordination.OperatorEventHandler; @@ -32,6 +31,12 @@ import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -45,11 +50,12 @@ class DataStatisticsOperator, S> extends AbstractStreamOperator> implements OneInputStreamOperator>, OperatorEventHandler { + private static final long serialVersionUID = 1L; private final String operatorName; - // keySelector will be used to generate key from data for collecting data statistics - private final KeySelector keySelector; + private final RowDataWrapper rowDataWrapper; + private final SortKey sortKey; private final OperatorEventGateway operatorEventGateway; private final TypeSerializer> statisticsSerializer; private transient volatile DataStatistics localStatistics; @@ -58,11 +64,13 @@ class DataStatisticsOperator, S> DataStatisticsOperator( String operatorName, - KeySelector keySelector, + Schema schema, + SortOrder sortOrder, OperatorEventGateway operatorEventGateway, TypeSerializer> statisticsSerializer) { this.operatorName = operatorName; - this.keySelector = keySelector; + this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); + this.sortKey = new SortKey(schema, sortOrder); this.operatorEventGateway = operatorEventGateway; this.statisticsSerializer = statisticsSerializer; } @@ -126,10 +134,11 @@ public void handleOperatorEvent(OperatorEvent event) { } @Override - public void processElement(StreamRecord streamRecord) throws Exception { + public void processElement(StreamRecord streamRecord) { RowData record = streamRecord.getValue(); - RowData key = keySelector.getKey(record); - localStatistics.add(key); + StructLike struct = rowDataWrapper.wrap(record); + sortKey.wrap(struct); + localStatistics.add(sortKey); output.collect(new StreamRecord<>(DataStatisticsOrRecord.fromRecord(record))); } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java index 2737b1346f0f..8716cb872d0e 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java @@ -76,7 +76,6 @@ static , S> byte[] serializeAggregatedStatistics( return bytes.toByteArray(); } - @SuppressWarnings("unchecked") static , S> AggregatedStatistics deserializeAggregatedStatistics( byte[] bytes, TypeSerializer> statisticsSerializer) diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java index 246b56526fd6..0ffffd9cf49f 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java @@ -20,20 +20,20 @@ import java.util.Map; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.data.RowData; +import org.apache.iceberg.SortKey; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.collect.Maps; /** MapDataStatistics uses map to count key frequency */ @Internal -class MapDataStatistics implements DataStatistics> { - private final Map statistics; +class MapDataStatistics implements DataStatistics> { + private final Map statistics; MapDataStatistics() { this.statistics = Maps.newHashMap(); } - MapDataStatistics(Map statistics) { + MapDataStatistics(Map statistics) { this.statistics = statistics; } @@ -43,9 +43,14 @@ public boolean isEmpty() { } @Override - public void add(RowData key) { - // increase count of occurrence by one in the dataStatistics map - statistics.merge(key, 1L, Long::sum); + public void add(SortKey sortKey) { + if (statistics.containsKey(sortKey)) { + statistics.merge(sortKey, 1L, Long::sum); + } else { + // clone the sort key before adding to map because input sortKey object can be reused + SortKey copiedKey = sortKey.copy(); + statistics.put(copiedKey, 1L); + } } @Override @@ -54,7 +59,7 @@ public void merge(MapDataStatistics otherStatistics) { } @Override - public Map statistics() { + public Map statistics() { return statistics; } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java index 6d07637b29b3..b6cccd0566fc 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java @@ -29,22 +29,22 @@ import org.apache.flink.api.common.typeutils.base.MapSerializer; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.table.data.RowData; import org.apache.flink.util.Preconditions; +import org.apache.iceberg.SortKey; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @Internal class MapDataStatisticsSerializer - extends TypeSerializer>> { - private final MapSerializer mapSerializer; + extends TypeSerializer>> { + private final MapSerializer mapSerializer; - static TypeSerializer>> fromKeySerializer( - TypeSerializer keySerializer) { + static MapDataStatisticsSerializer fromSortKeySerializer( + TypeSerializer sortKeySerializer) { return new MapDataStatisticsSerializer( - new MapSerializer<>(keySerializer, LongSerializer.INSTANCE)); + new MapSerializer<>(sortKeySerializer, LongSerializer.INSTANCE)); } - MapDataStatisticsSerializer(MapSerializer mapSerializer) { + MapDataStatisticsSerializer(MapSerializer mapSerializer) { this.mapSerializer = mapSerializer; } @@ -55,28 +55,28 @@ public boolean isImmutableType() { @SuppressWarnings("ReferenceEquality") @Override - public TypeSerializer>> duplicate() { - MapSerializer duplicateMapSerializer = - (MapSerializer) mapSerializer.duplicate(); + public TypeSerializer>> duplicate() { + MapSerializer duplicateMapSerializer = + (MapSerializer) mapSerializer.duplicate(); return (duplicateMapSerializer == mapSerializer) ? this : new MapDataStatisticsSerializer(duplicateMapSerializer); } @Override - public DataStatistics> createInstance() { + public MapDataStatistics createInstance() { return new MapDataStatistics(); } @Override - public DataStatistics> copy(DataStatistics obj) { + public MapDataStatistics copy(DataStatistics> obj) { Preconditions.checkArgument( obj instanceof MapDataStatistics, "Invalid data statistics type: " + obj.getClass()); MapDataStatistics from = (MapDataStatistics) obj; - TypeSerializer keySerializer = mapSerializer.getKeySerializer(); - Map newMap = Maps.newHashMapWithExpectedSize(from.statistics().size()); - for (Map.Entry entry : from.statistics().entrySet()) { - RowData newKey = keySerializer.copy(entry.getKey()); + TypeSerializer keySerializer = mapSerializer.getKeySerializer(); + Map newMap = Maps.newHashMapWithExpectedSize(from.statistics().size()); + for (Map.Entry entry : from.statistics().entrySet()) { + SortKey newKey = keySerializer.copy(entry.getKey()); // no need to copy value since it is just a Long newMap.put(newKey, entry.getValue()); } @@ -85,8 +85,9 @@ public DataStatistics> copy(DataStatistics } @Override - public DataStatistics> copy( - DataStatistics from, DataStatistics reuse) { + public DataStatistics> copy( + DataStatistics> from, + DataStatistics> reuse) { // not much benefit to reuse return copy(from); } @@ -97,7 +98,9 @@ public int getLength() { } @Override - public void serialize(DataStatistics obj, DataOutputView target) throws IOException { + public void serialize( + DataStatistics> obj, DataOutputView target) + throws IOException { Preconditions.checkArgument( obj instanceof MapDataStatistics, "Invalid data statistics type: " + obj.getClass()); MapDataStatistics mapStatistics = (MapDataStatistics) obj; @@ -105,14 +108,15 @@ public void serialize(DataStatistics obj, DataOutputView target) throws IOExcept } @Override - public DataStatistics> deserialize(DataInputView source) + public DataStatistics> deserialize(DataInputView source) throws IOException { return new MapDataStatistics(mapSerializer.deserialize(source)); } @Override - public DataStatistics> deserialize( - DataStatistics reuse, DataInputView source) throws IOException { + public DataStatistics> deserialize( + DataStatistics> reuse, DataInputView source) + throws IOException { // not much benefit to reuse return deserialize(source); } @@ -138,14 +142,14 @@ public int hashCode() { } @Override - public TypeSerializerSnapshot>> + public TypeSerializerSnapshot>> snapshotConfiguration() { return new MapDataStatisticsSerializerSnapshot(this); } public static class MapDataStatisticsSerializerSnapshot extends CompositeTypeSerializerSnapshot< - DataStatistics>, MapDataStatisticsSerializer> { + DataStatistics>, MapDataStatisticsSerializer> { private static final int CURRENT_VERSION = 1; // constructors need to public. Otherwise, Flink state restore would complain @@ -175,8 +179,8 @@ protected TypeSerializer[] getNestedSerializers( protected MapDataStatisticsSerializer createOuterSerializerWithNestedSerializers( TypeSerializer[] nestedSerializers) { @SuppressWarnings("unchecked") - MapSerializer mapSerializer = - (MapSerializer) nestedSerializers[0]; + MapSerializer mapSerializer = + (MapSerializer) nestedSerializers[0]; return new MapDataStatisticsSerializer(mapSerializer); } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java new file mode 100644 index 000000000000..d03409f2a430 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java @@ -0,0 +1,353 @@ +/* + * 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.iceberg.flink.sink.shuffle; + +import java.io.IOException; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Objects; +import java.util.UUID; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.StringUtils; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.SortField; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderParser; +import org.apache.iceberg.types.CheckCompatibility; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +class SortKeySerializer extends TypeSerializer { + private final Schema schema; + private final SortOrder sortOrder; + private final int size; + private final Types.NestedField[] transformedFields; + + private transient SortKey sortKey; + + SortKeySerializer(Schema schema, SortOrder sortOrder) { + this.schema = schema; + this.sortOrder = sortOrder; + this.size = sortOrder.fields().size(); + + this.transformedFields = new Types.NestedField[size]; + for (int i = 0; i < size; ++i) { + SortField sortField = sortOrder.fields().get(i); + Types.NestedField sourceField = schema.findField(sortField.sourceId()); + Type resultType = sortField.transform().getResultType(sourceField.type()); + Types.NestedField transformedField = + Types.NestedField.of( + sourceField.fieldId(), + sourceField.isOptional(), + sourceField.name(), + resultType, + sourceField.doc()); + transformedFields[i] = transformedField; + } + } + + private SortKey lazySortKey() { + if (sortKey == null) { + this.sortKey = new SortKey(schema, sortOrder); + } + + return sortKey; + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer duplicate() { + return new SortKeySerializer(schema, sortOrder); + } + + @Override + public SortKey createInstance() { + return new SortKey(schema, sortOrder); + } + + @Override + public SortKey copy(SortKey from) { + return from.copy(); + } + + @Override + public SortKey copy(SortKey from, SortKey reuse) { + // no benefit of reuse + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(SortKey record, DataOutputView target) throws IOException { + Preconditions.checkArgument( + record.size() == size, + "Invalid size of the sort key object: %s. Expected %s", + record.size(), + size); + for (int i = 0; i < size; ++i) { + int fieldId = transformedFields[i].fieldId(); + Type.TypeID typeId = transformedFields[i].type().typeId(); + switch (typeId) { + case BOOLEAN: + target.writeBoolean(record.get(i, Boolean.class)); + break; + case INTEGER: + case DATE: + target.writeInt(record.get(i, Integer.class)); + break; + case LONG: + case TIME: + case TIMESTAMP: + target.writeLong(record.get(i, Long.class)); + break; + case FLOAT: + target.writeFloat(record.get(i, Float.class)); + break; + case DOUBLE: + target.writeDouble(record.get(i, Double.class)); + break; + case STRING: + target.writeUTF(record.get(i, CharSequence.class).toString()); + break; + case UUID: + UUID uuid = record.get(i, UUID.class); + target.writeLong(uuid.getMostSignificantBits()); + target.writeLong(uuid.getLeastSignificantBits()); + break; + case FIXED: + case BINARY: + byte[] bytes = record.get(i, ByteBuffer.class).array(); + target.writeInt(bytes.length); + target.write(bytes); + break; + case DECIMAL: + BigDecimal decimal = record.get(i, BigDecimal.class); + byte[] decimalBytes = decimal.unscaledValue().toByteArray(); + target.writeInt(decimalBytes.length); + target.write(decimalBytes); + target.writeInt(decimal.scale()); + break; + case STRUCT: + case MAP: + case LIST: + default: + // SortKey transformation is a flattened struct without list and map + throw new UnsupportedOperationException( + String.format("Field %d has unsupported field type: %s", fieldId, typeId)); + } + } + } + + @Override + public SortKey deserialize(DataInputView source) throws IOException { + // copying is a little faster than constructing a new SortKey object + SortKey deserialized = lazySortKey().copy(); + deserialize(deserialized, source); + return deserialized; + } + + @Override + public SortKey deserialize(SortKey reuse, DataInputView source) throws IOException { + Preconditions.checkArgument( + reuse.size() == size, + "Invalid size of the sort key object: %s. Expected %s", + reuse.size(), + size); + for (int i = 0; i < size; ++i) { + int fieldId = transformedFields[i].fieldId(); + Type.TypeID typeId = transformedFields[i].type().typeId(); + switch (typeId) { + case BOOLEAN: + reuse.set(i, source.readBoolean()); + break; + case INTEGER: + case DATE: + reuse.set(i, source.readInt()); + break; + case LONG: + case TIME: + case TIMESTAMP: + reuse.set(i, source.readLong()); + break; + case FLOAT: + reuse.set(i, source.readFloat()); + break; + case DOUBLE: + reuse.set(i, source.readDouble()); + break; + case STRING: + reuse.set(i, source.readUTF()); + break; + case UUID: + long mostSignificantBits = source.readLong(); + long leastSignificantBits = source.readLong(); + reuse.set(i, new UUID(mostSignificantBits, leastSignificantBits)); + break; + case FIXED: + case BINARY: + byte[] bytes = new byte[source.readInt()]; + source.read(bytes); + reuse.set(i, ByteBuffer.wrap(bytes)); + break; + case DECIMAL: + byte[] unscaledBytes = new byte[source.readInt()]; + source.read(unscaledBytes); + int scale = source.readInt(); + BigDecimal decimal = new BigDecimal(new BigInteger(unscaledBytes), scale); + reuse.set(i, decimal); + break; + case STRUCT: + case MAP: + case LIST: + default: + // SortKey transformation is a flattened struct without list and map + throw new UnsupportedOperationException( + String.format("Field %d has unsupported field type: %s", fieldId, typeId)); + } + } + + return reuse; + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + // no optimization here + serialize(deserialize(source), target); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof SortKeySerializer)) { + return false; + } + + SortKeySerializer other = (SortKeySerializer) obj; + return Objects.equals(schema.asStruct(), other.schema.asStruct()) + && Objects.equals(sortOrder, other.sortOrder); + } + + @Override + public int hashCode() { + return schema.asStruct().hashCode() * 31 + sortOrder.hashCode(); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new SortKeySerializerSnapshot(schema, sortOrder); + } + + public static class SortKeySerializerSnapshot implements TypeSerializerSnapshot { + private static final int CURRENT_VERSION = 1; + + private Schema schema; + private SortOrder sortOrder; + + @SuppressWarnings({"checkstyle:RedundantModifier", "WeakerAccess"}) + public SortKeySerializerSnapshot() { + // this constructor is used when restoring from a checkpoint. + } + + // constructors need to public. Otherwise, Flink state restore would complain + // "The class has no (implicit) public nullary constructor". + @SuppressWarnings("checkstyle:RedundantModifier") + public SortKeySerializerSnapshot(Schema schema, SortOrder sortOrder) { + this.schema = schema; + this.sortOrder = sortOrder; + } + + @Override + public int getCurrentVersion() { + return CURRENT_VERSION; + } + + @Override + public void writeSnapshot(DataOutputView out) throws IOException { + Preconditions.checkState(schema != null, "Invalid schema: null"); + Preconditions.checkState(sortOrder != null, "Invalid sort order: null"); + + StringUtils.writeString(SchemaParser.toJson(schema), out); + StringUtils.writeString(SortOrderParser.toJson(sortOrder), out); + } + + @Override + public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) + throws IOException { + if (readVersion == 1) { + readV1(in); + } else { + throw new IllegalArgumentException("Unknown read version: " + readVersion); + } + } + + @Override + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + TypeSerializer newSerializer) { + if (!(newSerializer instanceof SortKeySerializer)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + + SortKeySerializer newAvroSerializer = (SortKeySerializer) newSerializer; + return resolveSchemaCompatibility(newAvroSerializer.schema, schema); + } + + @Override + public TypeSerializer restoreSerializer() { + Preconditions.checkState(schema != null, "Invalid schema: null"); + Preconditions.checkState(sortOrder != null, "Invalid sort order: null"); + return new SortKeySerializer(schema, sortOrder); + } + + private void readV1(DataInputView in) throws IOException { + String schemaJson = StringUtils.readString(in); + String sortOrderJson = StringUtils.readString(in); + this.schema = SchemaParser.fromJson(schemaJson); + this.sortOrder = SortOrderParser.fromJson(sortOrderJson).bind(schema); + } + + @VisibleForTesting + static TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + Schema readSchema, Schema writeSchema) { + List compatibilityErrors = + CheckCompatibility.writeCompatibilityErrors(readSchema, writeSchema); + if (compatibilityErrors.isEmpty()) { + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } + + return TypeSerializerSchemaCompatibility.incompatible(); + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java index dd7fcafe5336..890cc361b246 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java @@ -21,41 +21,43 @@ import static org.assertj.core.api.Assertions.assertThat; import java.util.Map; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.VarCharType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.types.Types; import org.junit.Test; public class TestAggregatedStatistics { + private final Schema schema = + new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); + private final SortKey sortKey = new SortKey(schema, sortOrder); + private final MapDataStatisticsSerializer statisticsSerializer = + MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); @Test public void mergeDataStatisticTest() { - GenericRowData rowDataA = GenericRowData.of(StringData.fromString("a")); - GenericRowData rowDataB = GenericRowData.of(StringData.fromString("b")); + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); - AggregatedStatistics> aggregatedStatistics = - new AggregatedStatistics<>( - 1, - MapDataStatisticsSerializer.fromKeySerializer( - new RowDataSerializer(RowType.of(new VarCharType())))); + AggregatedStatistics> aggregatedStatistics = + new AggregatedStatistics<>(1, statisticsSerializer); MapDataStatistics mapDataStatistics1 = new MapDataStatistics(); - mapDataStatistics1.add(rowDataA); - mapDataStatistics1.add(rowDataA); - mapDataStatistics1.add(rowDataB); + mapDataStatistics1.add(keyA); + mapDataStatistics1.add(keyA); + mapDataStatistics1.add(keyB); aggregatedStatistics.mergeDataStatistic("testOperator", 1, mapDataStatistics1); MapDataStatistics mapDataStatistics2 = new MapDataStatistics(); - mapDataStatistics2.add(rowDataA); + mapDataStatistics2.add(keyA); aggregatedStatistics.mergeDataStatistic("testOperator", 1, mapDataStatistics2); - assertThat(aggregatedStatistics.dataStatistics().statistics().get(rowDataA)) + assertThat(aggregatedStatistics.dataStatistics().statistics().get(keyA)) .isEqualTo( - mapDataStatistics1.statistics().get(rowDataA) - + mapDataStatistics2.statistics().get(rowDataA)); - assertThat(aggregatedStatistics.dataStatistics().statistics().get(rowDataB)) + mapDataStatistics1.statistics().get(keyA) + mapDataStatistics2.statistics().get(keyA)); + assertThat(aggregatedStatistics.dataStatistics().statistics().get(keyB)) .isEqualTo( - mapDataStatistics1.statistics().get(rowDataB) - + mapDataStatistics2.statistics().getOrDefault(rowDataB, 0L)); + mapDataStatistics1.statistics().get(keyB) + + mapDataStatistics2.statistics().getOrDefault(keyB, 0L)); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java index 48e4e4d8f966..4c64ce522201 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java @@ -21,32 +21,33 @@ import static org.assertj.core.api.Assertions.assertThat; import java.util.Map; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.binary.BinaryRowData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.VarCharType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.types.Types; import org.junit.Before; import org.junit.Test; public class TestAggregatedStatisticsTracker { private static final int NUM_SUBTASKS = 2; - private final RowType rowType = RowType.of(new VarCharType()); - // When coordinator handles events from operator, DataStatisticsUtil#deserializeDataStatistics - // deserializes bytes into BinaryRowData - private final BinaryRowData binaryRowDataA = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("a"))); - private final BinaryRowData binaryRowDataB = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("b"))); - private final TypeSerializer rowSerializer = new RowDataSerializer(rowType); - private final TypeSerializer>> - statisticsSerializer = MapDataStatisticsSerializer.fromKeySerializer(rowSerializer); - private AggregatedStatisticsTracker> + + private final Schema schema = + new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); + private final SortKey sortKey = new SortKey(schema, sortOrder); + private final MapDataStatisticsSerializer statisticsSerializer = + MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); + private final SortKey keyA = sortKey.copy(); + private final SortKey keyB = sortKey.copy(); + + private AggregatedStatisticsTracker> aggregatedStatisticsTracker; + public TestAggregatedStatisticsTracker() { + keyA.set(0, "a"); + keyB.set(0, "b"); + } + @Before public void before() throws Exception { aggregatedStatisticsTracker = @@ -56,8 +57,8 @@ public void before() throws Exception { @Test public void receiveNewerDataStatisticEvent() { MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(binaryRowDataA); - DataStatisticsEvent> + checkpoint1Subtask0DataStatistic.add(keyA); + DataStatisticsEvent> checkpoint1Subtask0DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); assertThat( @@ -67,8 +68,8 @@ public void receiveNewerDataStatisticEvent() { assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()).isEqualTo(1); MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(binaryRowDataA); - DataStatisticsEvent> + checkpoint2Subtask0DataStatistic.add(keyA); + DataStatisticsEvent> checkpoint2Subtask0DataStatisticEvent = DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); assertThat( @@ -82,10 +83,10 @@ public void receiveNewerDataStatisticEvent() { @Test public void receiveOlderDataStatisticEventTest() { MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(binaryRowDataA); - checkpoint2Subtask0DataStatistic.add(binaryRowDataB); - checkpoint2Subtask0DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> + checkpoint2Subtask0DataStatistic.add(keyA); + checkpoint2Subtask0DataStatistic.add(keyB); + checkpoint2Subtask0DataStatistic.add(keyB); + DataStatisticsEvent> checkpoint3Subtask0DataStatisticEvent = DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); assertThat( @@ -94,8 +95,8 @@ public void receiveOlderDataStatisticEventTest() { .isNull(); MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - checkpoint1Subtask1DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> + checkpoint1Subtask1DataStatistic.add(keyB); + DataStatisticsEvent> checkpoint1Subtask1DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); // Receive event from old checkpoint, aggregatedStatisticsAggregatorTracker won't return @@ -110,10 +111,10 @@ public void receiveOlderDataStatisticEventTest() { @Test public void receiveCompletedDataStatisticEvent() { MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(binaryRowDataA); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> + checkpoint1Subtask0DataStatistic.add(keyA); + checkpoint1Subtask0DataStatistic.add(keyB); + checkpoint1Subtask0DataStatistic.add(keyB); + DataStatisticsEvent> checkpoint1Subtask0DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); assertThat( @@ -122,14 +123,14 @@ public void receiveCompletedDataStatisticEvent() { .isNull(); MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - checkpoint1Subtask1DataStatistic.add(binaryRowDataA); - checkpoint1Subtask1DataStatistic.add(binaryRowDataA); - checkpoint1Subtask1DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> + checkpoint1Subtask1DataStatistic.add(keyA); + checkpoint1Subtask1DataStatistic.add(keyA); + checkpoint1Subtask1DataStatistic.add(keyB); + DataStatisticsEvent> checkpoint1Subtask1DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); // Receive data statistics from all subtasks at checkpoint 1 - AggregatedStatistics> completedStatistics = + AggregatedStatistics> completedStatistics = aggregatedStatisticsTracker.updateAndCheckCompletion( 1, checkpoint1Subtask1DataStatisticEvent); @@ -137,20 +138,20 @@ public void receiveCompletedDataStatisticEvent() { assertThat(completedStatistics.checkpointId()).isEqualTo(1); MapDataStatistics globalDataStatistics = (MapDataStatistics) completedStatistics.dataStatistics(); - assertThat((long) globalDataStatistics.statistics().get(binaryRowDataA)) + assertThat((long) globalDataStatistics.statistics().get(keyA)) .isEqualTo( - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataA) - + checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataA)); - assertThat((long) globalDataStatistics.statistics().get(binaryRowDataB)) + checkpoint1Subtask0DataStatistic.statistics().get(keyA) + + checkpoint1Subtask1DataStatistic.statistics().get(keyA)); + assertThat((long) globalDataStatistics.statistics().get(keyB)) .isEqualTo( - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataB) - + checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataB)); + checkpoint1Subtask0DataStatistic.statistics().get(keyB) + + checkpoint1Subtask1DataStatistic.statistics().get(keyB)); assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()) .isEqualTo(completedStatistics.checkpointId() + 1); MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(binaryRowDataA); - DataStatisticsEvent> + checkpoint2Subtask0DataStatistic.add(keyA); + DataStatisticsEvent> checkpoint2Subtask0DataStatisticEvent = DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); assertThat( @@ -160,8 +161,8 @@ public void receiveCompletedDataStatisticEvent() { assertThat(completedStatistics.checkpointId()).isEqualTo(1); MapDataStatistics checkpoint2Subtask1DataStatistic = new MapDataStatistics(); - checkpoint2Subtask1DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> + checkpoint2Subtask1DataStatistic.add(keyB); + DataStatisticsEvent> checkpoint2Subtask1DataStatisticEvent = DataStatisticsEvent.create(2, checkpoint2Subtask1DataStatistic, statisticsSerializer); // Receive data statistics from all subtasks at checkpoint 2 diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java index 9ec2606e101d..3df714059c37 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java @@ -24,19 +24,15 @@ import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; -import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.binary.BinaryRowData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.VarCharType; import org.apache.flink.util.ExceptionUtils; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; import org.junit.Before; import org.junit.Test; @@ -44,20 +40,21 @@ public class TestDataStatisticsCoordinator { private static final String OPERATOR_NAME = "TestCoordinator"; private static final OperatorID TEST_OPERATOR_ID = new OperatorID(1234L, 5678L); private static final int NUM_SUBTASKS = 2; - private TypeSerializer>> - statisticsSerializer; + + private final Schema schema = + new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); + private final SortKey sortKey = new SortKey(schema, sortOrder); + private final MapDataStatisticsSerializer statisticsSerializer = + MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); private EventReceivingTasks receivingTasks; - private DataStatisticsCoordinator> + private DataStatisticsCoordinator> dataStatisticsCoordinator; @Before public void before() throws Exception { receivingTasks = EventReceivingTasks.createForRunningTasks(); - statisticsSerializer = - MapDataStatisticsSerializer.fromKeySerializer( - new RowDataSerializer(RowType.of(new VarCharType()))); - dataStatisticsCoordinator = new DataStatisticsCoordinator<>( OPERATOR_NAME, @@ -93,59 +90,66 @@ public void testThrowExceptionWhenNotStarted() { @Test public void testDataStatisticsEventHandling() throws Exception { tasksReady(); - // When coordinator handles events from operator, DataStatisticsUtil#deserializeDataStatistics - // deserializes bytes into BinaryRowData - RowType rowType = RowType.of(new VarCharType()); - BinaryRowData binaryRowDataA = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("a"))); - BinaryRowData binaryRowDataB = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("b"))); - BinaryRowData binaryRowDataC = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("c"))); + SortKey key = sortKey.copy(); MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(binaryRowDataA); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - checkpoint1Subtask0DataStatistic.add(binaryRowDataC); - checkpoint1Subtask0DataStatistic.add(binaryRowDataC); - checkpoint1Subtask0DataStatistic.add(binaryRowDataC); - DataStatisticsEvent> + key.set(0, "a"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "b"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "b"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask0DataStatistic.add(key); + + DataStatisticsEvent> checkpoint1Subtask0DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); + MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - checkpoint1Subtask1DataStatistic.add(binaryRowDataA); - checkpoint1Subtask1DataStatistic.add(binaryRowDataB); - checkpoint1Subtask1DataStatistic.add(binaryRowDataC); - checkpoint1Subtask1DataStatistic.add(binaryRowDataC); - DataStatisticsEvent> + key.set(0, "a"); + checkpoint1Subtask1DataStatistic.add(key); + key.set(0, "b"); + checkpoint1Subtask1DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask1DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask1DataStatistic.add(key); + + DataStatisticsEvent> checkpoint1Subtask1DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); + // Handle events from operators for checkpoint 1 dataStatisticsCoordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0DataStatisticEvent); dataStatisticsCoordinator.handleEventFromOperator(1, 0, checkpoint1Subtask1DataStatisticEvent); waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + // Verify global data statistics is the aggregation of all subtasks data statistics + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); + SortKey keyC = sortKey.copy(); + keyC.set(0, "c"); MapDataStatistics globalDataStatistics = (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); assertThat(globalDataStatistics.statistics()) .containsExactlyInAnyOrderEntriesOf( ImmutableMap.of( - binaryRowDataA, - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataA) - + (long) checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataA), - binaryRowDataB, - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataB) - + (long) checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataB), - binaryRowDataC, - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataC) - + (long) checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataC))); + keyA, 2L, + keyB, 3L, + keyC, 5L)); } static void setAllTasksReady( int subtasks, - DataStatisticsCoordinator> dataStatisticsCoordinator, + DataStatisticsCoordinator> dataStatisticsCoordinator, EventReceivingTasks receivingTasks) { for (int i = 0; i < subtasks; i++) { dataStatisticsCoordinator.executionAttemptReady( @@ -154,7 +158,7 @@ static void setAllTasksReady( } static void waitForCoordinatorToProcessActions( - DataStatisticsCoordinator> coordinator) { + DataStatisticsCoordinator> coordinator) { CompletableFuture future = new CompletableFuture<>(); coordinator.callInCoordinatorThread( () -> { diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java index cb9d3f48ffeb..5e0a752be506 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java @@ -23,18 +23,14 @@ import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; -import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; import org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.binary.BinaryRowData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.VarCharType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.types.Types; import org.junit.Before; import org.junit.Test; @@ -42,16 +38,18 @@ public class TestDataStatisticsCoordinatorProvider { private static final OperatorID OPERATOR_ID = new OperatorID(); private static final int NUM_SUBTASKS = 1; - private DataStatisticsCoordinatorProvider> provider; + private final Schema schema = + new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); + private final SortKey sortKey = new SortKey(schema, sortOrder); + private final MapDataStatisticsSerializer statisticsSerializer = + MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); + + private DataStatisticsCoordinatorProvider> provider; private EventReceivingTasks receivingTasks; - private TypeSerializer>> - statisticsSerializer; @Before public void before() { - statisticsSerializer = - MapDataStatisticsSerializer.fromKeySerializer( - new RowDataSerializer(RowType.of(new VarCharType()))); provider = new DataStatisticsCoordinatorProvider<>( "DataStatisticsCoordinatorProvider", OPERATOR_ID, statisticsSerializer); @@ -61,84 +59,82 @@ public void before() { @Test @SuppressWarnings("unchecked") public void testCheckpointAndReset() throws Exception { - RowType rowType = RowType.of(new VarCharType()); - // When coordinator handles events from operator, DataStatisticsUtil#deserializeDataStatistics - // deserializes bytes into BinaryRowData - BinaryRowData binaryRowDataA = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("a"))); - BinaryRowData binaryRowDataB = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("b"))); - BinaryRowData binaryRowDataC = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("c"))); - BinaryRowData binaryRowDataD = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("d"))); - BinaryRowData binaryRowDataE = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("e"))); + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); + SortKey keyC = sortKey.copy(); + keyC.set(0, "c"); + SortKey keyD = sortKey.copy(); + keyD.set(0, "c"); + SortKey keyE = sortKey.copy(); + keyE.set(0, "c"); - RecreateOnResetOperatorCoordinator coordinator = + try (RecreateOnResetOperatorCoordinator coordinator = (RecreateOnResetOperatorCoordinator) - provider.create(new MockOperatorCoordinatorContext(OPERATOR_ID, NUM_SUBTASKS)); - DataStatisticsCoordinator> dataStatisticsCoordinator = - (DataStatisticsCoordinator>) - coordinator.getInternalCoordinator(); + provider.create(new MockOperatorCoordinatorContext(OPERATOR_ID, NUM_SUBTASKS))) { + DataStatisticsCoordinator> dataStatisticsCoordinator = + (DataStatisticsCoordinator>) + coordinator.getInternalCoordinator(); - // Start the coordinator - coordinator.start(); - TestDataStatisticsCoordinator.setAllTasksReady( - NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); - MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(binaryRowDataA); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - checkpoint1Subtask0DataStatistic.add(binaryRowDataC); - DataStatisticsEvent> - checkpoint1Subtask0DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); + // Start the coordinator + coordinator.start(); + TestDataStatisticsCoordinator.setAllTasksReady( + NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); + MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); + checkpoint1Subtask0DataStatistic.add(keyA); + checkpoint1Subtask0DataStatistic.add(keyB); + checkpoint1Subtask0DataStatistic.add(keyC); + DataStatisticsEvent> + checkpoint1Subtask0DataStatisticEvent = + DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); - // Handle events from operators for checkpoint 1 - coordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0DataStatisticEvent); - TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); - // Verify checkpoint 1 global data statistics - MapDataStatistics checkpoint1GlobalDataStatistics = - (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(checkpoint1GlobalDataStatistics.statistics()) - .isEqualTo(checkpoint1Subtask0DataStatistic.statistics()); - byte[] checkpoint1Bytes = waitForCheckpoint(1L, dataStatisticsCoordinator); + // Handle events from operators for checkpoint 1 + coordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0DataStatisticEvent); + TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + // Verify checkpoint 1 global data statistics + MapDataStatistics checkpoint1GlobalDataStatistics = + (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); + assertThat(checkpoint1GlobalDataStatistics.statistics()) + .isEqualTo(checkpoint1Subtask0DataStatistic.statistics()); + byte[] checkpoint1Bytes = waitForCheckpoint(1L, dataStatisticsCoordinator); - MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(binaryRowDataD); - checkpoint2Subtask0DataStatistic.add(binaryRowDataE); - checkpoint2Subtask0DataStatistic.add(binaryRowDataE); - DataStatisticsEvent> - checkpoint2Subtask0DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); - // Handle events from operators for checkpoint 2 - coordinator.handleEventFromOperator(0, 0, checkpoint2Subtask0DataStatisticEvent); - TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); - // Verify checkpoint 2 global data statistics - MapDataStatistics checkpoint2GlobalDataStatistics = - (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(checkpoint2GlobalDataStatistics.statistics()) - .isEqualTo(checkpoint2Subtask0DataStatistic.statistics()); - waitForCheckpoint(2L, dataStatisticsCoordinator); + MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); + checkpoint2Subtask0DataStatistic.add(keyD); + checkpoint2Subtask0DataStatistic.add(keyE); + checkpoint2Subtask0DataStatistic.add(keyE); + DataStatisticsEvent> + checkpoint2Subtask0DataStatisticEvent = + DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); + // Handle events from operators for checkpoint 2 + coordinator.handleEventFromOperator(0, 0, checkpoint2Subtask0DataStatisticEvent); + TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + // Verify checkpoint 2 global data statistics + MapDataStatistics checkpoint2GlobalDataStatistics = + (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); + assertThat(checkpoint2GlobalDataStatistics.statistics()) + .isEqualTo(checkpoint2Subtask0DataStatistic.statistics()); + waitForCheckpoint(2L, dataStatisticsCoordinator); - // Reset coordinator to checkpoint 1 - coordinator.resetToCheckpoint(1L, checkpoint1Bytes); - DataStatisticsCoordinator> - restoredDataStatisticsCoordinator = - (DataStatisticsCoordinator>) - coordinator.getInternalCoordinator(); - assertThat(dataStatisticsCoordinator).isNotEqualTo(restoredDataStatisticsCoordinator); - // Verify restored data statistics - MapDataStatistics restoredAggregateDataStatistics = - (MapDataStatistics) - restoredDataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(restoredAggregateDataStatistics.statistics()) - .isEqualTo(checkpoint1GlobalDataStatistics.statistics()); + // Reset coordinator to checkpoint 1 + coordinator.resetToCheckpoint(1L, checkpoint1Bytes); + DataStatisticsCoordinator> + restoredDataStatisticsCoordinator = + (DataStatisticsCoordinator>) + coordinator.getInternalCoordinator(); + assertThat(dataStatisticsCoordinator).isNotEqualTo(restoredDataStatisticsCoordinator); + // Verify restored data statistics + MapDataStatistics restoredAggregateDataStatistics = + (MapDataStatistics) + restoredDataStatisticsCoordinator.completedStatistics().dataStatistics(); + assertThat(restoredAggregateDataStatistics.statistics()) + .isEqualTo(checkpoint1GlobalDataStatistics.statistics()); + } } private byte[] waitForCheckpoint( long checkpointId, - DataStatisticsCoordinator> coordinator) + DataStatisticsCoordinator> coordinator) throws InterruptedException, ExecutionException { CompletableFuture future = new CompletableFuture<>(); coordinator.checkpointCoordinator(checkpointId, future); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java index 880cb3d5516f..0e99a2d74ccb 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java @@ -27,7 +27,6 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.state.OperatorStateStore; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.CloseableRegistry; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; @@ -50,33 +49,37 @@ import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.binary.BinaryRowData; import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.VarCharType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; import org.junit.After; import org.junit.Before; import org.junit.Test; public class TestDataStatisticsOperator { - private final RowType rowType = RowType.of(new VarCharType()); + private final Schema schema = + new Schema( + Types.NestedField.optional(1, "id", Types.StringType.get()), + Types.NestedField.optional(2, "number", Types.IntegerType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("id").build(); + private final SortKey sortKey = new SortKey(schema, sortOrder); + private final RowType rowType = RowType.of(new VarCharType(), new IntType()); private final TypeSerializer rowSerializer = new RowDataSerializer(rowType); - private final GenericRowData genericRowDataA = GenericRowData.of(StringData.fromString("a")); - private final GenericRowData genericRowDataB = GenericRowData.of(StringData.fromString("b")); - // When operator hands events from coordinator, DataStatisticsUtil#deserializeDataStatistics - // deserializes bytes into BinaryRowData - private final BinaryRowData binaryRowDataA = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("a"))); - private final BinaryRowData binaryRowDataB = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("b"))); - private final BinaryRowData binaryRowDataC = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("c"))); - private final TypeSerializer>> - statisticsSerializer = MapDataStatisticsSerializer.fromKeySerializer(rowSerializer); - private DataStatisticsOperator> operator; + private final TypeSerializer>> + statisticsSerializer = + MapDataStatisticsSerializer.fromSortKeySerializer( + new SortKeySerializer(schema, sortOrder)); + + private DataStatisticsOperator> operator; private Environment getTestingEnvironment() { return new StreamMockEnvironment( @@ -99,20 +102,10 @@ public void before() throws Exception { new MockOutput<>(Lists.newArrayList())); } - private DataStatisticsOperator> createOperator() { + private DataStatisticsOperator> createOperator() { MockOperatorEventGateway mockGateway = new MockOperatorEventGateway(); - KeySelector keySelector = - new KeySelector() { - private static final long serialVersionUID = 7662520075515707428L; - - @Override - public RowData getKey(RowData value) { - return value; - } - }; - return new DataStatisticsOperator<>( - "testOperator", keySelector, mockGateway, statisticsSerializer); + "testOperator", schema, sortOrder, mockGateway, statisticsSerializer); } @After @@ -123,20 +116,26 @@ public void clean() throws Exception { @Test public void testProcessElement() throws Exception { try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> testHarness = createHarness(this.operator)) { StateInitializationContext stateContext = getStateContext(); operator.initializeState(stateContext); - operator.processElement(new StreamRecord<>(genericRowDataA)); - operator.processElement(new StreamRecord<>(genericRowDataA)); - operator.processElement(new StreamRecord<>(genericRowDataB)); + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 5))); + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 3))); + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 1))); assertThat(operator.localDataStatistics()).isInstanceOf(MapDataStatistics.class); + + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); + Map expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L); + MapDataStatistics mapDataStatistics = (MapDataStatistics) operator.localDataStatistics(); - Map statsMap = mapDataStatistics.statistics(); + Map statsMap = mapDataStatistics.statistics(); assertThat(statsMap).hasSize(2); - assertThat(statsMap) - .containsExactlyInAnyOrderEntriesOf( - ImmutableMap.of(genericRowDataA, 2L, genericRowDataB, 1L)); + assertThat(statsMap).containsExactlyInAnyOrderEntriesOf(expectedMap); + testHarness.endInput(); } } @@ -144,11 +143,14 @@ public void testProcessElement() throws Exception { @Test public void testOperatorOutput() throws Exception { try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> testHarness = createHarness(this.operator)) { - testHarness.processElement(new StreamRecord<>(genericRowDataA)); - testHarness.processElement(new StreamRecord<>(genericRowDataB)); - testHarness.processElement(new StreamRecord<>(genericRowDataB)); + testHarness.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 2))); + testHarness.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 3))); + testHarness.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 1))); List recordsOutput = testHarness.extractOutputValues().stream() @@ -157,7 +159,10 @@ public void testOperatorOutput() throws Exception { .collect(Collectors.toList()); assertThat(recordsOutput) .containsExactlyInAnyOrderElementsOf( - ImmutableList.of(genericRowDataA, genericRowDataB, genericRowDataB)); + ImmutableList.of( + GenericRowData.of(StringData.fromString("a"), 2), + GenericRowData.of(StringData.fromString("b"), 3), + GenericRowData.of(StringData.fromString("b"), 1))); } } @@ -165,36 +170,61 @@ public void testOperatorOutput() throws Exception { public void testRestoreState() throws Exception { OperatorSubtaskState snapshot; try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> testHarness1 = createHarness(this.operator)) { - DataStatistics> mapDataStatistics = - new MapDataStatistics(); - mapDataStatistics.add(binaryRowDataA); - mapDataStatistics.add(binaryRowDataA); - mapDataStatistics.add(binaryRowDataB); - mapDataStatistics.add(binaryRowDataC); - operator.handleOperatorEvent( - DataStatisticsEvent.create(0, mapDataStatistics, statisticsSerializer)); + MapDataStatistics mapDataStatistics = new MapDataStatistics(); + + SortKey key = sortKey.copy(); + key.set(0, "a"); + mapDataStatistics.add(key); + key.set(0, "a"); + mapDataStatistics.add(key); + key.set(0, "b"); + mapDataStatistics.add(key); + key.set(0, "c"); + mapDataStatistics.add(key); + + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); + SortKey keyC = sortKey.copy(); + keyC.set(0, "c"); + Map expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L, keyC, 1L); + + DataStatisticsEvent> event = + DataStatisticsEvent.create(0, mapDataStatistics, statisticsSerializer); + operator.handleOperatorEvent(event); assertThat(operator.globalDataStatistics()).isInstanceOf(MapDataStatistics.class); assertThat(operator.globalDataStatistics().statistics()) - .containsExactlyInAnyOrderEntriesOf( - ImmutableMap.of(binaryRowDataA, 2L, binaryRowDataB, 1L, binaryRowDataC, 1L)); + .containsExactlyInAnyOrderEntriesOf(expectedMap); snapshot = testHarness1.snapshot(1L, 0); } // Use the snapshot to initialize state for another new operator and then verify that the global // statistics for the new operator is same as before - DataStatisticsOperator> restoredOperator = + DataStatisticsOperator> restoredOperator = createOperator(); try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> testHarness2 = new OneInputStreamOperatorTestHarness<>(restoredOperator, 2, 2, 1)) { testHarness2.setup(); testHarness2.initializeState(snapshot); assertThat(restoredOperator.globalDataStatistics()).isInstanceOf(MapDataStatistics.class); - assertThat(restoredOperator.globalDataStatistics().statistics()) - .containsExactlyInAnyOrderEntriesOf( - ImmutableMap.of(binaryRowDataA, 2L, binaryRowDataB, 1L, binaryRowDataC, 1L)); + + // restored RowData is BinaryRowData. convert to GenericRowData for comparison + Map restoredStatistics = Maps.newHashMap(); + restoredStatistics.putAll(restoredOperator.globalDataStatistics().statistics()); + + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); + SortKey keyC = sortKey.copy(); + keyC.set(0, "c"); + Map expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L, keyC, 1L); + + assertThat(restoredStatistics).containsExactlyInAnyOrderEntriesOf(expectedMap); } } @@ -209,18 +239,16 @@ private StateInitializationContext getStateContext() throws Exception { } private OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> createHarness( - final DataStatisticsOperator> + final DataStatisticsOperator> dataStatisticsOperator) throws Exception { OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> harness = new OneInputStreamOperatorTestHarness<>(dataStatisticsOperator, 1, 1, 0); - harness.setup( - new DataStatisticsOrRecordSerializer<>( - MapDataStatisticsSerializer.fromKeySerializer(rowSerializer), rowSerializer)); + harness.setup(new DataStatisticsOrRecordSerializer<>(statisticsSerializer, rowSerializer)); harness.open(); return harness; } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java new file mode 100644 index 000000000000..a07808e935d9 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java @@ -0,0 +1,90 @@ +/* + * 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.iceberg.flink.sink.shuffle; + +import java.util.Map; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +public class TestMapDataStatistics { + private final SortOrder sortOrder = SortOrder.builderFor(TestFixtures.SCHEMA).asc("data").build(); + private final SortKey sortKey = new SortKey(TestFixtures.SCHEMA, sortOrder); + private final RowType rowType = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); + private final RowDataWrapper rowWrapper = + new RowDataWrapper(rowType, TestFixtures.SCHEMA.asStruct()); + + @Test + public void testAddsAndGet() { + MapDataStatistics dataStatistics = new MapDataStatistics(); + + GenericRowData reusedRow = + GenericRowData.of(StringData.fromString("a"), 1, StringData.fromString("2023-06-20")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("b")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("c")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("b")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("a")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("b")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + Map actual = dataStatistics.statistics(); + + rowWrapper.wrap( + GenericRowData.of(StringData.fromString("a"), 1, StringData.fromString("2023-06-20"))); + sortKey.wrap(rowWrapper); + SortKey keyA = sortKey.copy(); + + rowWrapper.wrap( + GenericRowData.of(StringData.fromString("b"), 1, StringData.fromString("2023-06-20"))); + sortKey.wrap(rowWrapper); + SortKey keyB = sortKey.copy(); + + rowWrapper.wrap( + GenericRowData.of(StringData.fromString("c"), 1, StringData.fromString("2023-06-20"))); + sortKey.wrap(rowWrapper); + SortKey keyC = sortKey.copy(); + + Map expected = ImmutableMap.of(keyA, 2L, keyB, 3L, keyC, 1L); + Assertions.assertThat(actual).isEqualTo(expected); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java new file mode 100644 index 000000000000..c7fea015142c --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java @@ -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. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.table.data.GenericRowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; + +public abstract class TestSortKeySerializerBase extends SerializerTestBase { + + protected abstract Schema schema(); + + protected abstract SortOrder sortOrder(); + + protected abstract GenericRowData rowData(); + + @Override + protected TypeSerializer createSerializer() { + return new SortKeySerializer(schema(), sortOrder()); + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return SortKey.class; + } + + @Override + protected SortKey[] getTestData() { + return new SortKey[] {sortKey()}; + } + + private SortKey sortKey() { + RowDataWrapper rowDataWrapper = + new RowDataWrapper(FlinkSchemaUtil.convert(schema()), schema().asStruct()); + SortKey sortKey = new SortKey(schema(), sortOrder()); + sortKey.wrap(rowDataWrapper.wrap(rowData())); + return sortKey; + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java new file mode 100644 index 000000000000..0000688a8b55 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java @@ -0,0 +1,55 @@ +/* + * 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.iceberg.flink.sink.shuffle; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortDirection; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.flink.DataGenerator; +import org.apache.iceberg.flink.DataGenerators; + +public class TestSortKeySerializerNestedStruct extends TestSortKeySerializerBase { + private final DataGenerator generator = new DataGenerators.StructOfStruct(); + + @Override + protected Schema schema() { + return generator.icebergSchema(); + } + + @Override + protected SortOrder sortOrder() { + return SortOrder.builderFor(schema()) + .asc("row_id") + .sortBy( + Expressions.bucket("struct_of_struct.id", 4), SortDirection.DESC, NullOrder.NULLS_LAST) + .sortBy( + Expressions.truncate("struct_of_struct.person_struct.name", 16), + SortDirection.ASC, + NullOrder.NULLS_FIRST) + .build(); + } + + @Override + protected GenericRowData rowData() { + return generator.generateFlinkRowData(); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java new file mode 100644 index 000000000000..291302aef486 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortDirection; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.flink.DataGenerator; +import org.apache.iceberg.flink.DataGenerators; + +public class TestSortKeySerializerPrimitives extends TestSortKeySerializerBase { + private final DataGenerator generator = new DataGenerators.Primitives(); + + @Override + protected Schema schema() { + return generator.icebergSchema(); + } + + @Override + protected SortOrder sortOrder() { + return SortOrder.builderFor(schema()) + .asc("boolean_field") + .sortBy(Expressions.bucket("int_field", 4), SortDirection.DESC, NullOrder.NULLS_LAST) + .sortBy(Expressions.truncate("string_field", 2), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy(Expressions.bucket("uuid_field", 16), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy(Expressions.hour("ts_with_zone_field"), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy(Expressions.day("ts_without_zone_field"), SortDirection.ASC, NullOrder.NULLS_FIRST) + // can not test HeapByteBuffer due to equality test inside SerializerTestBase + // .sortBy(Expressions.truncate("binary_field", 2), SortDirection.ASC, + // NullOrder.NULLS_FIRST) + .build(); + } + + @Override + protected GenericRowData rowData() { + return generator.generateFlinkRowData(); + } +} From 4d0b69beba104e6912f6e6850189121fcd23ef8a Mon Sep 17 00:00:00 2001 From: Mason Chen Date: Sat, 9 Dec 2023 00:06:32 -0800 Subject: [PATCH 29/47] Flink: Fix IcebergSource tableloader lifecycle management in batch mode (#9173) --- .../iceberg/flink/source/IcebergSource.java | 58 +++++++------------ .../ContinuousSplitPlannerImpl.java | 4 +- 2 files changed, 24 insertions(+), 38 deletions(-) diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java index 179253cb3a18..a7ce2db61ffb 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -81,16 +81,18 @@ public class IcebergSource implements Source { private static final Logger LOG = LoggerFactory.getLogger(IcebergSource.class); + // This table loader can be closed, and it is only safe to use this instance for resource + // independent information (e.g. a table name). Copies of this are required to avoid lifecycle + // management conflicts with the user provided table loader. e.g. a copy of this is required for + // split planning, which uses the underlying io, and should be closed after split planning is + // complete. private final TableLoader tableLoader; private final ScanContext scanContext; private final ReaderFunction readerFunction; private final SplitAssignerFactory assignerFactory; private final SerializableComparator splitComparator; private final SerializableRecordEmitter emitter; - - // Can't use SerializableTable as enumerator needs a regular table - // that can discover table changes - private transient Table table; + private final String tableName; IcebergSource( TableLoader tableLoader, @@ -100,17 +102,21 @@ public class IcebergSource implements Source splitComparator, Table table, SerializableRecordEmitter emitter) { + Preconditions.checkNotNull(tableLoader, "tableLoader is required."); + Preconditions.checkNotNull(readerFunction, "readerFunction is required."); + Preconditions.checkNotNull(assignerFactory, "assignerFactory is required."); + Preconditions.checkNotNull(table, "table is required."); this.tableLoader = tableLoader; this.scanContext = scanContext; this.readerFunction = readerFunction; this.assignerFactory = assignerFactory; this.splitComparator = splitComparator; - this.table = table; this.emitter = emitter; + this.tableName = table.name(); } String name() { - return "IcebergSource-" + lazyTable().name(); + return "IcebergSource-" + tableName; } private String planningThreadName() { @@ -120,38 +126,26 @@ private String planningThreadName() { // a public API like the protected method "OperatorCoordinator.Context getCoordinatorContext()" // from SourceCoordinatorContext implementation. For now,

  • - is used as // the unique thread pool name. - return lazyTable().name() + "-" + UUID.randomUUID(); + return tableName + "-" + UUID.randomUUID(); } private List planSplitsForBatch(String threadName) { ExecutorService workerPool = ThreadPools.newWorkerPool(threadName, scanContext.planParallelism()); - try { + try (TableLoader loader = tableLoader.clone()) { + loader.open(); List splits = - FlinkSplitPlanner.planIcebergSourceSplits(lazyTable(), scanContext, workerPool); + FlinkSplitPlanner.planIcebergSourceSplits(loader.loadTable(), scanContext, workerPool); LOG.info( - "Discovered {} splits from table {} during job initialization", - splits.size(), - lazyTable().name()); + "Discovered {} splits from table {} during job initialization", splits.size(), tableName); return splits; + } catch (IOException e) { + throw new UncheckedIOException("Failed to close table loader", e); } finally { workerPool.shutdown(); } } - private Table lazyTable() { - if (table == null) { - tableLoader.open(); - try (TableLoader loader = tableLoader) { - this.table = loader.loadTable(); - } catch (IOException e) { - throw new UncheckedIOException("Failed to close table loader", e); - } - } - - return table; - } - @Override public Boundedness getBoundedness() { return scanContext.isStreaming() ? Boundedness.CONTINUOUS_UNBOUNDED : Boundedness.BOUNDED; @@ -160,7 +154,7 @@ public Boundedness getBoundedness() { @Override public SourceReader createReader(SourceReaderContext readerContext) { IcebergSourceReaderMetrics metrics = - new IcebergSourceReaderMetrics(readerContext.metricGroup(), lazyTable().name()); + new IcebergSourceReaderMetrics(readerContext.metricGroup(), tableName); return new IcebergSourceReader<>( emitter, metrics, readerFunction, splitComparator, readerContext); } @@ -197,13 +191,12 @@ private SplitEnumerator createEnumer LOG.info( "Iceberg source restored {} splits from state for table {}", enumState.pendingSplits().size(), - lazyTable().name()); + tableName); assigner = assignerFactory.createAssigner(enumState.pendingSplits()); } - if (scanContext.isStreaming()) { ContinuousSplitPlanner splitPlanner = - new ContinuousSplitPlannerImpl(tableLoader.clone(), scanContext, planningThreadName()); + new ContinuousSplitPlannerImpl(tableLoader, scanContext, planningThreadName()); return new ContinuousIcebergEnumerator( enumContext, assigner, scanContext, splitPlanner, enumState); } else { @@ -537,7 +530,6 @@ public IcebergSource build() { } } - checkRequired(); // Since builder already load the table, pass it to the source to avoid double loading return new IcebergSource<>( tableLoader, @@ -548,11 +540,5 @@ public IcebergSource build() { table, emitter); } - - private void checkRequired() { - Preconditions.checkNotNull(tableLoader, "tableLoader is required."); - Preconditions.checkNotNull(splitAssignerFactory, "assignerFactory is required."); - Preconditions.checkNotNull(readerFunction, "readerFunction is required."); - } } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java index f0d8ca8d7057..450b649253a4 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java @@ -56,9 +56,9 @@ public class ContinuousSplitPlannerImpl implements ContinuousSplitPlanner { */ public ContinuousSplitPlannerImpl( TableLoader tableLoader, ScanContext scanContext, String threadName) { - this.tableLoader = tableLoader; + this.tableLoader = tableLoader.clone(); this.tableLoader.open(); - this.table = tableLoader.loadTable(); + this.table = this.tableLoader.loadTable(); this.scanContext = scanContext; this.isSharedPool = threadName == null; this.workerPool = From 21522697c42cee3bedd758aa399b9530b001e30a Mon Sep 17 00:00:00 2001 From: Steven Wu Date: Fri, 8 Dec 2023 13:04:53 -0800 Subject: [PATCH 30/47] Flink: backport PR #9212 to 1.16 for switching to SortKey for data statistics --- .../flink/sink/shuffle/DataStatistics.java | 12 +- .../shuffle/DataStatisticsCoordinator.java | 3 +- .../sink/shuffle/DataStatisticsOperator.java | 25 +- .../sink/shuffle/DataStatisticsUtil.java | 1 - .../flink/sink/shuffle/MapDataStatistics.java | 21 +- .../shuffle/MapDataStatisticsSerializer.java | 56 +-- .../flink/sink/shuffle/SortKeySerializer.java | 353 ++++++++++++++++++ .../shuffle/TestAggregatedStatistics.java | 48 +-- .../TestAggregatedStatisticsTracker.java | 97 ++--- .../TestDataStatisticsCoordinator.java | 98 ++--- ...TestDataStatisticsCoordinatorProvider.java | 160 ++++---- .../shuffle/TestDataStatisticsOperator.java | 154 ++++---- .../sink/shuffle/TestMapDataStatistics.java | 90 +++++ .../shuffle/TestSortKeySerializerBase.java | 65 ++++ .../TestSortKeySerializerNestedStruct.java | 55 +++ .../TestSortKeySerializerPrimitives.java | 57 +++ 16 files changed, 980 insertions(+), 315 deletions(-) create mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java create mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java create mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java create mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java create mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java index 28a05201c02f..9d7cf179ab1c 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java @@ -19,7 +19,7 @@ package org.apache.iceberg.flink.sink.shuffle; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.data.RowData; +import org.apache.iceberg.SortKey; /** * DataStatistics defines the interface to collect data distribution information. @@ -29,7 +29,7 @@ * (sketching) can be used. */ @Internal -interface DataStatistics { +interface DataStatistics, S> { /** * Check if data statistics contains any statistics information. @@ -38,12 +38,8 @@ interface DataStatistics { */ boolean isEmpty(); - /** - * Add data key to data statistics. - * - * @param key generate from data by applying key selector - */ - void add(RowData key); + /** Add row sortKey to data statistics. */ + void add(SortKey sortKey); /** * Merge current statistics with other statistics. diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java index fcfd79884265..c8ac79c61bf6 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java @@ -172,6 +172,7 @@ private void handleDataStatisticRequest(int subtask, DataStatisticsEvent e } } + @SuppressWarnings("FutureReturnValueIgnored") private void sendDataStatisticsToSubtasks( long checkpointId, DataStatistics globalDataStatistics) { callInCoordinatorThread( @@ -339,7 +340,7 @@ private void unregisterSubtaskGateway(int subtaskIndex, int attemptNumber) { private OperatorCoordinator.SubtaskGateway getSubtaskGateway(int subtaskIndex) { Preconditions.checkState( - gateways[subtaskIndex].size() > 0, + !gateways[subtaskIndex].isEmpty(), "Coordinator of %s subtask %d is not ready yet to receive events", operatorName, subtaskIndex); diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java index d00d5d2e5aa9..5157a37cf2cd 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java @@ -22,7 +22,6 @@ import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.runtime.operators.coordination.OperatorEvent; import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; import org.apache.flink.runtime.operators.coordination.OperatorEventHandler; @@ -32,6 +31,12 @@ import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -45,11 +50,12 @@ class DataStatisticsOperator, S> extends AbstractStreamOperator> implements OneInputStreamOperator>, OperatorEventHandler { + private static final long serialVersionUID = 1L; private final String operatorName; - // keySelector will be used to generate key from data for collecting data statistics - private final KeySelector keySelector; + private final RowDataWrapper rowDataWrapper; + private final SortKey sortKey; private final OperatorEventGateway operatorEventGateway; private final TypeSerializer> statisticsSerializer; private transient volatile DataStatistics localStatistics; @@ -58,11 +64,13 @@ class DataStatisticsOperator, S> DataStatisticsOperator( String operatorName, - KeySelector keySelector, + Schema schema, + SortOrder sortOrder, OperatorEventGateway operatorEventGateway, TypeSerializer> statisticsSerializer) { this.operatorName = operatorName; - this.keySelector = keySelector; + this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); + this.sortKey = new SortKey(schema, sortOrder); this.operatorEventGateway = operatorEventGateway; this.statisticsSerializer = statisticsSerializer; } @@ -126,10 +134,11 @@ public void handleOperatorEvent(OperatorEvent event) { } @Override - public void processElement(StreamRecord streamRecord) throws Exception { + public void processElement(StreamRecord streamRecord) { RowData record = streamRecord.getValue(); - RowData key = keySelector.getKey(record); - localStatistics.add(key); + StructLike struct = rowDataWrapper.wrap(record); + sortKey.wrap(struct); + localStatistics.add(sortKey); output.collect(new StreamRecord<>(DataStatisticsOrRecord.fromRecord(record))); } diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java index 2737b1346f0f..8716cb872d0e 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java @@ -76,7 +76,6 @@ static , S> byte[] serializeAggregatedStatistics( return bytes.toByteArray(); } - @SuppressWarnings("unchecked") static , S> AggregatedStatistics deserializeAggregatedStatistics( byte[] bytes, TypeSerializer> statisticsSerializer) diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java index 246b56526fd6..0ffffd9cf49f 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java @@ -20,20 +20,20 @@ import java.util.Map; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.data.RowData; +import org.apache.iceberg.SortKey; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.collect.Maps; /** MapDataStatistics uses map to count key frequency */ @Internal -class MapDataStatistics implements DataStatistics> { - private final Map statistics; +class MapDataStatistics implements DataStatistics> { + private final Map statistics; MapDataStatistics() { this.statistics = Maps.newHashMap(); } - MapDataStatistics(Map statistics) { + MapDataStatistics(Map statistics) { this.statistics = statistics; } @@ -43,9 +43,14 @@ public boolean isEmpty() { } @Override - public void add(RowData key) { - // increase count of occurrence by one in the dataStatistics map - statistics.merge(key, 1L, Long::sum); + public void add(SortKey sortKey) { + if (statistics.containsKey(sortKey)) { + statistics.merge(sortKey, 1L, Long::sum); + } else { + // clone the sort key before adding to map because input sortKey object can be reused + SortKey copiedKey = sortKey.copy(); + statistics.put(copiedKey, 1L); + } } @Override @@ -54,7 +59,7 @@ public void merge(MapDataStatistics otherStatistics) { } @Override - public Map statistics() { + public Map statistics() { return statistics; } diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java index 6d07637b29b3..b6cccd0566fc 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java @@ -29,22 +29,22 @@ import org.apache.flink.api.common.typeutils.base.MapSerializer; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.table.data.RowData; import org.apache.flink.util.Preconditions; +import org.apache.iceberg.SortKey; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @Internal class MapDataStatisticsSerializer - extends TypeSerializer>> { - private final MapSerializer mapSerializer; + extends TypeSerializer>> { + private final MapSerializer mapSerializer; - static TypeSerializer>> fromKeySerializer( - TypeSerializer keySerializer) { + static MapDataStatisticsSerializer fromSortKeySerializer( + TypeSerializer sortKeySerializer) { return new MapDataStatisticsSerializer( - new MapSerializer<>(keySerializer, LongSerializer.INSTANCE)); + new MapSerializer<>(sortKeySerializer, LongSerializer.INSTANCE)); } - MapDataStatisticsSerializer(MapSerializer mapSerializer) { + MapDataStatisticsSerializer(MapSerializer mapSerializer) { this.mapSerializer = mapSerializer; } @@ -55,28 +55,28 @@ public boolean isImmutableType() { @SuppressWarnings("ReferenceEquality") @Override - public TypeSerializer>> duplicate() { - MapSerializer duplicateMapSerializer = - (MapSerializer) mapSerializer.duplicate(); + public TypeSerializer>> duplicate() { + MapSerializer duplicateMapSerializer = + (MapSerializer) mapSerializer.duplicate(); return (duplicateMapSerializer == mapSerializer) ? this : new MapDataStatisticsSerializer(duplicateMapSerializer); } @Override - public DataStatistics> createInstance() { + public MapDataStatistics createInstance() { return new MapDataStatistics(); } @Override - public DataStatistics> copy(DataStatistics obj) { + public MapDataStatistics copy(DataStatistics> obj) { Preconditions.checkArgument( obj instanceof MapDataStatistics, "Invalid data statistics type: " + obj.getClass()); MapDataStatistics from = (MapDataStatistics) obj; - TypeSerializer keySerializer = mapSerializer.getKeySerializer(); - Map newMap = Maps.newHashMapWithExpectedSize(from.statistics().size()); - for (Map.Entry entry : from.statistics().entrySet()) { - RowData newKey = keySerializer.copy(entry.getKey()); + TypeSerializer keySerializer = mapSerializer.getKeySerializer(); + Map newMap = Maps.newHashMapWithExpectedSize(from.statistics().size()); + for (Map.Entry entry : from.statistics().entrySet()) { + SortKey newKey = keySerializer.copy(entry.getKey()); // no need to copy value since it is just a Long newMap.put(newKey, entry.getValue()); } @@ -85,8 +85,9 @@ public DataStatistics> copy(DataStatistics } @Override - public DataStatistics> copy( - DataStatistics from, DataStatistics reuse) { + public DataStatistics> copy( + DataStatistics> from, + DataStatistics> reuse) { // not much benefit to reuse return copy(from); } @@ -97,7 +98,9 @@ public int getLength() { } @Override - public void serialize(DataStatistics obj, DataOutputView target) throws IOException { + public void serialize( + DataStatistics> obj, DataOutputView target) + throws IOException { Preconditions.checkArgument( obj instanceof MapDataStatistics, "Invalid data statistics type: " + obj.getClass()); MapDataStatistics mapStatistics = (MapDataStatistics) obj; @@ -105,14 +108,15 @@ public void serialize(DataStatistics obj, DataOutputView target) throws IOExcept } @Override - public DataStatistics> deserialize(DataInputView source) + public DataStatistics> deserialize(DataInputView source) throws IOException { return new MapDataStatistics(mapSerializer.deserialize(source)); } @Override - public DataStatistics> deserialize( - DataStatistics reuse, DataInputView source) throws IOException { + public DataStatistics> deserialize( + DataStatistics> reuse, DataInputView source) + throws IOException { // not much benefit to reuse return deserialize(source); } @@ -138,14 +142,14 @@ public int hashCode() { } @Override - public TypeSerializerSnapshot>> + public TypeSerializerSnapshot>> snapshotConfiguration() { return new MapDataStatisticsSerializerSnapshot(this); } public static class MapDataStatisticsSerializerSnapshot extends CompositeTypeSerializerSnapshot< - DataStatistics>, MapDataStatisticsSerializer> { + DataStatistics>, MapDataStatisticsSerializer> { private static final int CURRENT_VERSION = 1; // constructors need to public. Otherwise, Flink state restore would complain @@ -175,8 +179,8 @@ protected TypeSerializer[] getNestedSerializers( protected MapDataStatisticsSerializer createOuterSerializerWithNestedSerializers( TypeSerializer[] nestedSerializers) { @SuppressWarnings("unchecked") - MapSerializer mapSerializer = - (MapSerializer) nestedSerializers[0]; + MapSerializer mapSerializer = + (MapSerializer) nestedSerializers[0]; return new MapDataStatisticsSerializer(mapSerializer); } } diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java new file mode 100644 index 000000000000..d03409f2a430 --- /dev/null +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java @@ -0,0 +1,353 @@ +/* + * 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.iceberg.flink.sink.shuffle; + +import java.io.IOException; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Objects; +import java.util.UUID; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.StringUtils; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.SortField; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderParser; +import org.apache.iceberg.types.CheckCompatibility; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +class SortKeySerializer extends TypeSerializer { + private final Schema schema; + private final SortOrder sortOrder; + private final int size; + private final Types.NestedField[] transformedFields; + + private transient SortKey sortKey; + + SortKeySerializer(Schema schema, SortOrder sortOrder) { + this.schema = schema; + this.sortOrder = sortOrder; + this.size = sortOrder.fields().size(); + + this.transformedFields = new Types.NestedField[size]; + for (int i = 0; i < size; ++i) { + SortField sortField = sortOrder.fields().get(i); + Types.NestedField sourceField = schema.findField(sortField.sourceId()); + Type resultType = sortField.transform().getResultType(sourceField.type()); + Types.NestedField transformedField = + Types.NestedField.of( + sourceField.fieldId(), + sourceField.isOptional(), + sourceField.name(), + resultType, + sourceField.doc()); + transformedFields[i] = transformedField; + } + } + + private SortKey lazySortKey() { + if (sortKey == null) { + this.sortKey = new SortKey(schema, sortOrder); + } + + return sortKey; + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer duplicate() { + return new SortKeySerializer(schema, sortOrder); + } + + @Override + public SortKey createInstance() { + return new SortKey(schema, sortOrder); + } + + @Override + public SortKey copy(SortKey from) { + return from.copy(); + } + + @Override + public SortKey copy(SortKey from, SortKey reuse) { + // no benefit of reuse + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(SortKey record, DataOutputView target) throws IOException { + Preconditions.checkArgument( + record.size() == size, + "Invalid size of the sort key object: %s. Expected %s", + record.size(), + size); + for (int i = 0; i < size; ++i) { + int fieldId = transformedFields[i].fieldId(); + Type.TypeID typeId = transformedFields[i].type().typeId(); + switch (typeId) { + case BOOLEAN: + target.writeBoolean(record.get(i, Boolean.class)); + break; + case INTEGER: + case DATE: + target.writeInt(record.get(i, Integer.class)); + break; + case LONG: + case TIME: + case TIMESTAMP: + target.writeLong(record.get(i, Long.class)); + break; + case FLOAT: + target.writeFloat(record.get(i, Float.class)); + break; + case DOUBLE: + target.writeDouble(record.get(i, Double.class)); + break; + case STRING: + target.writeUTF(record.get(i, CharSequence.class).toString()); + break; + case UUID: + UUID uuid = record.get(i, UUID.class); + target.writeLong(uuid.getMostSignificantBits()); + target.writeLong(uuid.getLeastSignificantBits()); + break; + case FIXED: + case BINARY: + byte[] bytes = record.get(i, ByteBuffer.class).array(); + target.writeInt(bytes.length); + target.write(bytes); + break; + case DECIMAL: + BigDecimal decimal = record.get(i, BigDecimal.class); + byte[] decimalBytes = decimal.unscaledValue().toByteArray(); + target.writeInt(decimalBytes.length); + target.write(decimalBytes); + target.writeInt(decimal.scale()); + break; + case STRUCT: + case MAP: + case LIST: + default: + // SortKey transformation is a flattened struct without list and map + throw new UnsupportedOperationException( + String.format("Field %d has unsupported field type: %s", fieldId, typeId)); + } + } + } + + @Override + public SortKey deserialize(DataInputView source) throws IOException { + // copying is a little faster than constructing a new SortKey object + SortKey deserialized = lazySortKey().copy(); + deserialize(deserialized, source); + return deserialized; + } + + @Override + public SortKey deserialize(SortKey reuse, DataInputView source) throws IOException { + Preconditions.checkArgument( + reuse.size() == size, + "Invalid size of the sort key object: %s. Expected %s", + reuse.size(), + size); + for (int i = 0; i < size; ++i) { + int fieldId = transformedFields[i].fieldId(); + Type.TypeID typeId = transformedFields[i].type().typeId(); + switch (typeId) { + case BOOLEAN: + reuse.set(i, source.readBoolean()); + break; + case INTEGER: + case DATE: + reuse.set(i, source.readInt()); + break; + case LONG: + case TIME: + case TIMESTAMP: + reuse.set(i, source.readLong()); + break; + case FLOAT: + reuse.set(i, source.readFloat()); + break; + case DOUBLE: + reuse.set(i, source.readDouble()); + break; + case STRING: + reuse.set(i, source.readUTF()); + break; + case UUID: + long mostSignificantBits = source.readLong(); + long leastSignificantBits = source.readLong(); + reuse.set(i, new UUID(mostSignificantBits, leastSignificantBits)); + break; + case FIXED: + case BINARY: + byte[] bytes = new byte[source.readInt()]; + source.read(bytes); + reuse.set(i, ByteBuffer.wrap(bytes)); + break; + case DECIMAL: + byte[] unscaledBytes = new byte[source.readInt()]; + source.read(unscaledBytes); + int scale = source.readInt(); + BigDecimal decimal = new BigDecimal(new BigInteger(unscaledBytes), scale); + reuse.set(i, decimal); + break; + case STRUCT: + case MAP: + case LIST: + default: + // SortKey transformation is a flattened struct without list and map + throw new UnsupportedOperationException( + String.format("Field %d has unsupported field type: %s", fieldId, typeId)); + } + } + + return reuse; + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + // no optimization here + serialize(deserialize(source), target); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof SortKeySerializer)) { + return false; + } + + SortKeySerializer other = (SortKeySerializer) obj; + return Objects.equals(schema.asStruct(), other.schema.asStruct()) + && Objects.equals(sortOrder, other.sortOrder); + } + + @Override + public int hashCode() { + return schema.asStruct().hashCode() * 31 + sortOrder.hashCode(); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new SortKeySerializerSnapshot(schema, sortOrder); + } + + public static class SortKeySerializerSnapshot implements TypeSerializerSnapshot { + private static final int CURRENT_VERSION = 1; + + private Schema schema; + private SortOrder sortOrder; + + @SuppressWarnings({"checkstyle:RedundantModifier", "WeakerAccess"}) + public SortKeySerializerSnapshot() { + // this constructor is used when restoring from a checkpoint. + } + + // constructors need to public. Otherwise, Flink state restore would complain + // "The class has no (implicit) public nullary constructor". + @SuppressWarnings("checkstyle:RedundantModifier") + public SortKeySerializerSnapshot(Schema schema, SortOrder sortOrder) { + this.schema = schema; + this.sortOrder = sortOrder; + } + + @Override + public int getCurrentVersion() { + return CURRENT_VERSION; + } + + @Override + public void writeSnapshot(DataOutputView out) throws IOException { + Preconditions.checkState(schema != null, "Invalid schema: null"); + Preconditions.checkState(sortOrder != null, "Invalid sort order: null"); + + StringUtils.writeString(SchemaParser.toJson(schema), out); + StringUtils.writeString(SortOrderParser.toJson(sortOrder), out); + } + + @Override + public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) + throws IOException { + if (readVersion == 1) { + readV1(in); + } else { + throw new IllegalArgumentException("Unknown read version: " + readVersion); + } + } + + @Override + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + TypeSerializer newSerializer) { + if (!(newSerializer instanceof SortKeySerializer)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + + SortKeySerializer newAvroSerializer = (SortKeySerializer) newSerializer; + return resolveSchemaCompatibility(newAvroSerializer.schema, schema); + } + + @Override + public TypeSerializer restoreSerializer() { + Preconditions.checkState(schema != null, "Invalid schema: null"); + Preconditions.checkState(sortOrder != null, "Invalid sort order: null"); + return new SortKeySerializer(schema, sortOrder); + } + + private void readV1(DataInputView in) throws IOException { + String schemaJson = StringUtils.readString(in); + String sortOrderJson = StringUtils.readString(in); + this.schema = SchemaParser.fromJson(schemaJson); + this.sortOrder = SortOrderParser.fromJson(sortOrderJson).bind(schema); + } + + @VisibleForTesting + static TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + Schema readSchema, Schema writeSchema) { + List compatibilityErrors = + CheckCompatibility.writeCompatibilityErrors(readSchema, writeSchema); + if (compatibilityErrors.isEmpty()) { + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } + + return TypeSerializerSchemaCompatibility.incompatible(); + } + } +} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java index dd7fcafe5336..890cc361b246 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java @@ -21,41 +21,43 @@ import static org.assertj.core.api.Assertions.assertThat; import java.util.Map; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.VarCharType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.types.Types; import org.junit.Test; public class TestAggregatedStatistics { + private final Schema schema = + new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); + private final SortKey sortKey = new SortKey(schema, sortOrder); + private final MapDataStatisticsSerializer statisticsSerializer = + MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); @Test public void mergeDataStatisticTest() { - GenericRowData rowDataA = GenericRowData.of(StringData.fromString("a")); - GenericRowData rowDataB = GenericRowData.of(StringData.fromString("b")); + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); - AggregatedStatistics> aggregatedStatistics = - new AggregatedStatistics<>( - 1, - MapDataStatisticsSerializer.fromKeySerializer( - new RowDataSerializer(RowType.of(new VarCharType())))); + AggregatedStatistics> aggregatedStatistics = + new AggregatedStatistics<>(1, statisticsSerializer); MapDataStatistics mapDataStatistics1 = new MapDataStatistics(); - mapDataStatistics1.add(rowDataA); - mapDataStatistics1.add(rowDataA); - mapDataStatistics1.add(rowDataB); + mapDataStatistics1.add(keyA); + mapDataStatistics1.add(keyA); + mapDataStatistics1.add(keyB); aggregatedStatistics.mergeDataStatistic("testOperator", 1, mapDataStatistics1); MapDataStatistics mapDataStatistics2 = new MapDataStatistics(); - mapDataStatistics2.add(rowDataA); + mapDataStatistics2.add(keyA); aggregatedStatistics.mergeDataStatistic("testOperator", 1, mapDataStatistics2); - assertThat(aggregatedStatistics.dataStatistics().statistics().get(rowDataA)) + assertThat(aggregatedStatistics.dataStatistics().statistics().get(keyA)) .isEqualTo( - mapDataStatistics1.statistics().get(rowDataA) - + mapDataStatistics2.statistics().get(rowDataA)); - assertThat(aggregatedStatistics.dataStatistics().statistics().get(rowDataB)) + mapDataStatistics1.statistics().get(keyA) + mapDataStatistics2.statistics().get(keyA)); + assertThat(aggregatedStatistics.dataStatistics().statistics().get(keyB)) .isEqualTo( - mapDataStatistics1.statistics().get(rowDataB) - + mapDataStatistics2.statistics().getOrDefault(rowDataB, 0L)); + mapDataStatistics1.statistics().get(keyB) + + mapDataStatistics2.statistics().getOrDefault(keyB, 0L)); } } diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java index 48e4e4d8f966..4c64ce522201 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java @@ -21,32 +21,33 @@ import static org.assertj.core.api.Assertions.assertThat; import java.util.Map; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.binary.BinaryRowData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.VarCharType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.types.Types; import org.junit.Before; import org.junit.Test; public class TestAggregatedStatisticsTracker { private static final int NUM_SUBTASKS = 2; - private final RowType rowType = RowType.of(new VarCharType()); - // When coordinator handles events from operator, DataStatisticsUtil#deserializeDataStatistics - // deserializes bytes into BinaryRowData - private final BinaryRowData binaryRowDataA = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("a"))); - private final BinaryRowData binaryRowDataB = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("b"))); - private final TypeSerializer rowSerializer = new RowDataSerializer(rowType); - private final TypeSerializer>> - statisticsSerializer = MapDataStatisticsSerializer.fromKeySerializer(rowSerializer); - private AggregatedStatisticsTracker> + + private final Schema schema = + new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); + private final SortKey sortKey = new SortKey(schema, sortOrder); + private final MapDataStatisticsSerializer statisticsSerializer = + MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); + private final SortKey keyA = sortKey.copy(); + private final SortKey keyB = sortKey.copy(); + + private AggregatedStatisticsTracker> aggregatedStatisticsTracker; + public TestAggregatedStatisticsTracker() { + keyA.set(0, "a"); + keyB.set(0, "b"); + } + @Before public void before() throws Exception { aggregatedStatisticsTracker = @@ -56,8 +57,8 @@ public void before() throws Exception { @Test public void receiveNewerDataStatisticEvent() { MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(binaryRowDataA); - DataStatisticsEvent> + checkpoint1Subtask0DataStatistic.add(keyA); + DataStatisticsEvent> checkpoint1Subtask0DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); assertThat( @@ -67,8 +68,8 @@ public void receiveNewerDataStatisticEvent() { assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()).isEqualTo(1); MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(binaryRowDataA); - DataStatisticsEvent> + checkpoint2Subtask0DataStatistic.add(keyA); + DataStatisticsEvent> checkpoint2Subtask0DataStatisticEvent = DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); assertThat( @@ -82,10 +83,10 @@ public void receiveNewerDataStatisticEvent() { @Test public void receiveOlderDataStatisticEventTest() { MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(binaryRowDataA); - checkpoint2Subtask0DataStatistic.add(binaryRowDataB); - checkpoint2Subtask0DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> + checkpoint2Subtask0DataStatistic.add(keyA); + checkpoint2Subtask0DataStatistic.add(keyB); + checkpoint2Subtask0DataStatistic.add(keyB); + DataStatisticsEvent> checkpoint3Subtask0DataStatisticEvent = DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); assertThat( @@ -94,8 +95,8 @@ public void receiveOlderDataStatisticEventTest() { .isNull(); MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - checkpoint1Subtask1DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> + checkpoint1Subtask1DataStatistic.add(keyB); + DataStatisticsEvent> checkpoint1Subtask1DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); // Receive event from old checkpoint, aggregatedStatisticsAggregatorTracker won't return @@ -110,10 +111,10 @@ public void receiveOlderDataStatisticEventTest() { @Test public void receiveCompletedDataStatisticEvent() { MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(binaryRowDataA); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> + checkpoint1Subtask0DataStatistic.add(keyA); + checkpoint1Subtask0DataStatistic.add(keyB); + checkpoint1Subtask0DataStatistic.add(keyB); + DataStatisticsEvent> checkpoint1Subtask0DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); assertThat( @@ -122,14 +123,14 @@ public void receiveCompletedDataStatisticEvent() { .isNull(); MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - checkpoint1Subtask1DataStatistic.add(binaryRowDataA); - checkpoint1Subtask1DataStatistic.add(binaryRowDataA); - checkpoint1Subtask1DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> + checkpoint1Subtask1DataStatistic.add(keyA); + checkpoint1Subtask1DataStatistic.add(keyA); + checkpoint1Subtask1DataStatistic.add(keyB); + DataStatisticsEvent> checkpoint1Subtask1DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); // Receive data statistics from all subtasks at checkpoint 1 - AggregatedStatistics> completedStatistics = + AggregatedStatistics> completedStatistics = aggregatedStatisticsTracker.updateAndCheckCompletion( 1, checkpoint1Subtask1DataStatisticEvent); @@ -137,20 +138,20 @@ public void receiveCompletedDataStatisticEvent() { assertThat(completedStatistics.checkpointId()).isEqualTo(1); MapDataStatistics globalDataStatistics = (MapDataStatistics) completedStatistics.dataStatistics(); - assertThat((long) globalDataStatistics.statistics().get(binaryRowDataA)) + assertThat((long) globalDataStatistics.statistics().get(keyA)) .isEqualTo( - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataA) - + checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataA)); - assertThat((long) globalDataStatistics.statistics().get(binaryRowDataB)) + checkpoint1Subtask0DataStatistic.statistics().get(keyA) + + checkpoint1Subtask1DataStatistic.statistics().get(keyA)); + assertThat((long) globalDataStatistics.statistics().get(keyB)) .isEqualTo( - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataB) - + checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataB)); + checkpoint1Subtask0DataStatistic.statistics().get(keyB) + + checkpoint1Subtask1DataStatistic.statistics().get(keyB)); assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()) .isEqualTo(completedStatistics.checkpointId() + 1); MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(binaryRowDataA); - DataStatisticsEvent> + checkpoint2Subtask0DataStatistic.add(keyA); + DataStatisticsEvent> checkpoint2Subtask0DataStatisticEvent = DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); assertThat( @@ -160,8 +161,8 @@ public void receiveCompletedDataStatisticEvent() { assertThat(completedStatistics.checkpointId()).isEqualTo(1); MapDataStatistics checkpoint2Subtask1DataStatistic = new MapDataStatistics(); - checkpoint2Subtask1DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> + checkpoint2Subtask1DataStatistic.add(keyB); + DataStatisticsEvent> checkpoint2Subtask1DataStatisticEvent = DataStatisticsEvent.create(2, checkpoint2Subtask1DataStatistic, statisticsSerializer); // Receive data statistics from all subtasks at checkpoint 2 diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java index 9ec2606e101d..3df714059c37 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java @@ -24,19 +24,15 @@ import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; -import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.binary.BinaryRowData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.VarCharType; import org.apache.flink.util.ExceptionUtils; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; import org.junit.Before; import org.junit.Test; @@ -44,20 +40,21 @@ public class TestDataStatisticsCoordinator { private static final String OPERATOR_NAME = "TestCoordinator"; private static final OperatorID TEST_OPERATOR_ID = new OperatorID(1234L, 5678L); private static final int NUM_SUBTASKS = 2; - private TypeSerializer>> - statisticsSerializer; + + private final Schema schema = + new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); + private final SortKey sortKey = new SortKey(schema, sortOrder); + private final MapDataStatisticsSerializer statisticsSerializer = + MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); private EventReceivingTasks receivingTasks; - private DataStatisticsCoordinator> + private DataStatisticsCoordinator> dataStatisticsCoordinator; @Before public void before() throws Exception { receivingTasks = EventReceivingTasks.createForRunningTasks(); - statisticsSerializer = - MapDataStatisticsSerializer.fromKeySerializer( - new RowDataSerializer(RowType.of(new VarCharType()))); - dataStatisticsCoordinator = new DataStatisticsCoordinator<>( OPERATOR_NAME, @@ -93,59 +90,66 @@ public void testThrowExceptionWhenNotStarted() { @Test public void testDataStatisticsEventHandling() throws Exception { tasksReady(); - // When coordinator handles events from operator, DataStatisticsUtil#deserializeDataStatistics - // deserializes bytes into BinaryRowData - RowType rowType = RowType.of(new VarCharType()); - BinaryRowData binaryRowDataA = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("a"))); - BinaryRowData binaryRowDataB = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("b"))); - BinaryRowData binaryRowDataC = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("c"))); + SortKey key = sortKey.copy(); MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(binaryRowDataA); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - checkpoint1Subtask0DataStatistic.add(binaryRowDataC); - checkpoint1Subtask0DataStatistic.add(binaryRowDataC); - checkpoint1Subtask0DataStatistic.add(binaryRowDataC); - DataStatisticsEvent> + key.set(0, "a"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "b"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "b"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask0DataStatistic.add(key); + + DataStatisticsEvent> checkpoint1Subtask0DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); + MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - checkpoint1Subtask1DataStatistic.add(binaryRowDataA); - checkpoint1Subtask1DataStatistic.add(binaryRowDataB); - checkpoint1Subtask1DataStatistic.add(binaryRowDataC); - checkpoint1Subtask1DataStatistic.add(binaryRowDataC); - DataStatisticsEvent> + key.set(0, "a"); + checkpoint1Subtask1DataStatistic.add(key); + key.set(0, "b"); + checkpoint1Subtask1DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask1DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask1DataStatistic.add(key); + + DataStatisticsEvent> checkpoint1Subtask1DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); + // Handle events from operators for checkpoint 1 dataStatisticsCoordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0DataStatisticEvent); dataStatisticsCoordinator.handleEventFromOperator(1, 0, checkpoint1Subtask1DataStatisticEvent); waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + // Verify global data statistics is the aggregation of all subtasks data statistics + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); + SortKey keyC = sortKey.copy(); + keyC.set(0, "c"); MapDataStatistics globalDataStatistics = (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); assertThat(globalDataStatistics.statistics()) .containsExactlyInAnyOrderEntriesOf( ImmutableMap.of( - binaryRowDataA, - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataA) - + (long) checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataA), - binaryRowDataB, - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataB) - + (long) checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataB), - binaryRowDataC, - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataC) - + (long) checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataC))); + keyA, 2L, + keyB, 3L, + keyC, 5L)); } static void setAllTasksReady( int subtasks, - DataStatisticsCoordinator> dataStatisticsCoordinator, + DataStatisticsCoordinator> dataStatisticsCoordinator, EventReceivingTasks receivingTasks) { for (int i = 0; i < subtasks; i++) { dataStatisticsCoordinator.executionAttemptReady( @@ -154,7 +158,7 @@ static void setAllTasksReady( } static void waitForCoordinatorToProcessActions( - DataStatisticsCoordinator> coordinator) { + DataStatisticsCoordinator> coordinator) { CompletableFuture future = new CompletableFuture<>(); coordinator.callInCoordinatorThread( () -> { diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java index cb9d3f48ffeb..5e0a752be506 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java @@ -23,18 +23,14 @@ import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; -import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; import org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.binary.BinaryRowData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.VarCharType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.types.Types; import org.junit.Before; import org.junit.Test; @@ -42,16 +38,18 @@ public class TestDataStatisticsCoordinatorProvider { private static final OperatorID OPERATOR_ID = new OperatorID(); private static final int NUM_SUBTASKS = 1; - private DataStatisticsCoordinatorProvider> provider; + private final Schema schema = + new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); + private final SortKey sortKey = new SortKey(schema, sortOrder); + private final MapDataStatisticsSerializer statisticsSerializer = + MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); + + private DataStatisticsCoordinatorProvider> provider; private EventReceivingTasks receivingTasks; - private TypeSerializer>> - statisticsSerializer; @Before public void before() { - statisticsSerializer = - MapDataStatisticsSerializer.fromKeySerializer( - new RowDataSerializer(RowType.of(new VarCharType()))); provider = new DataStatisticsCoordinatorProvider<>( "DataStatisticsCoordinatorProvider", OPERATOR_ID, statisticsSerializer); @@ -61,84 +59,82 @@ public void before() { @Test @SuppressWarnings("unchecked") public void testCheckpointAndReset() throws Exception { - RowType rowType = RowType.of(new VarCharType()); - // When coordinator handles events from operator, DataStatisticsUtil#deserializeDataStatistics - // deserializes bytes into BinaryRowData - BinaryRowData binaryRowDataA = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("a"))); - BinaryRowData binaryRowDataB = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("b"))); - BinaryRowData binaryRowDataC = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("c"))); - BinaryRowData binaryRowDataD = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("d"))); - BinaryRowData binaryRowDataE = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("e"))); + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); + SortKey keyC = sortKey.copy(); + keyC.set(0, "c"); + SortKey keyD = sortKey.copy(); + keyD.set(0, "c"); + SortKey keyE = sortKey.copy(); + keyE.set(0, "c"); - RecreateOnResetOperatorCoordinator coordinator = + try (RecreateOnResetOperatorCoordinator coordinator = (RecreateOnResetOperatorCoordinator) - provider.create(new MockOperatorCoordinatorContext(OPERATOR_ID, NUM_SUBTASKS)); - DataStatisticsCoordinator> dataStatisticsCoordinator = - (DataStatisticsCoordinator>) - coordinator.getInternalCoordinator(); + provider.create(new MockOperatorCoordinatorContext(OPERATOR_ID, NUM_SUBTASKS))) { + DataStatisticsCoordinator> dataStatisticsCoordinator = + (DataStatisticsCoordinator>) + coordinator.getInternalCoordinator(); - // Start the coordinator - coordinator.start(); - TestDataStatisticsCoordinator.setAllTasksReady( - NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); - MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(binaryRowDataA); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - checkpoint1Subtask0DataStatistic.add(binaryRowDataC); - DataStatisticsEvent> - checkpoint1Subtask0DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); + // Start the coordinator + coordinator.start(); + TestDataStatisticsCoordinator.setAllTasksReady( + NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); + MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); + checkpoint1Subtask0DataStatistic.add(keyA); + checkpoint1Subtask0DataStatistic.add(keyB); + checkpoint1Subtask0DataStatistic.add(keyC); + DataStatisticsEvent> + checkpoint1Subtask0DataStatisticEvent = + DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); - // Handle events from operators for checkpoint 1 - coordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0DataStatisticEvent); - TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); - // Verify checkpoint 1 global data statistics - MapDataStatistics checkpoint1GlobalDataStatistics = - (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(checkpoint1GlobalDataStatistics.statistics()) - .isEqualTo(checkpoint1Subtask0DataStatistic.statistics()); - byte[] checkpoint1Bytes = waitForCheckpoint(1L, dataStatisticsCoordinator); + // Handle events from operators for checkpoint 1 + coordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0DataStatisticEvent); + TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + // Verify checkpoint 1 global data statistics + MapDataStatistics checkpoint1GlobalDataStatistics = + (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); + assertThat(checkpoint1GlobalDataStatistics.statistics()) + .isEqualTo(checkpoint1Subtask0DataStatistic.statistics()); + byte[] checkpoint1Bytes = waitForCheckpoint(1L, dataStatisticsCoordinator); - MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(binaryRowDataD); - checkpoint2Subtask0DataStatistic.add(binaryRowDataE); - checkpoint2Subtask0DataStatistic.add(binaryRowDataE); - DataStatisticsEvent> - checkpoint2Subtask0DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); - // Handle events from operators for checkpoint 2 - coordinator.handleEventFromOperator(0, 0, checkpoint2Subtask0DataStatisticEvent); - TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); - // Verify checkpoint 2 global data statistics - MapDataStatistics checkpoint2GlobalDataStatistics = - (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(checkpoint2GlobalDataStatistics.statistics()) - .isEqualTo(checkpoint2Subtask0DataStatistic.statistics()); - waitForCheckpoint(2L, dataStatisticsCoordinator); + MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); + checkpoint2Subtask0DataStatistic.add(keyD); + checkpoint2Subtask0DataStatistic.add(keyE); + checkpoint2Subtask0DataStatistic.add(keyE); + DataStatisticsEvent> + checkpoint2Subtask0DataStatisticEvent = + DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); + // Handle events from operators for checkpoint 2 + coordinator.handleEventFromOperator(0, 0, checkpoint2Subtask0DataStatisticEvent); + TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + // Verify checkpoint 2 global data statistics + MapDataStatistics checkpoint2GlobalDataStatistics = + (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); + assertThat(checkpoint2GlobalDataStatistics.statistics()) + .isEqualTo(checkpoint2Subtask0DataStatistic.statistics()); + waitForCheckpoint(2L, dataStatisticsCoordinator); - // Reset coordinator to checkpoint 1 - coordinator.resetToCheckpoint(1L, checkpoint1Bytes); - DataStatisticsCoordinator> - restoredDataStatisticsCoordinator = - (DataStatisticsCoordinator>) - coordinator.getInternalCoordinator(); - assertThat(dataStatisticsCoordinator).isNotEqualTo(restoredDataStatisticsCoordinator); - // Verify restored data statistics - MapDataStatistics restoredAggregateDataStatistics = - (MapDataStatistics) - restoredDataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(restoredAggregateDataStatistics.statistics()) - .isEqualTo(checkpoint1GlobalDataStatistics.statistics()); + // Reset coordinator to checkpoint 1 + coordinator.resetToCheckpoint(1L, checkpoint1Bytes); + DataStatisticsCoordinator> + restoredDataStatisticsCoordinator = + (DataStatisticsCoordinator>) + coordinator.getInternalCoordinator(); + assertThat(dataStatisticsCoordinator).isNotEqualTo(restoredDataStatisticsCoordinator); + // Verify restored data statistics + MapDataStatistics restoredAggregateDataStatistics = + (MapDataStatistics) + restoredDataStatisticsCoordinator.completedStatistics().dataStatistics(); + assertThat(restoredAggregateDataStatistics.statistics()) + .isEqualTo(checkpoint1GlobalDataStatistics.statistics()); + } } private byte[] waitForCheckpoint( long checkpointId, - DataStatisticsCoordinator> coordinator) + DataStatisticsCoordinator> coordinator) throws InterruptedException, ExecutionException { CompletableFuture future = new CompletableFuture<>(); coordinator.checkpointCoordinator(checkpointId, future); diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java index 880cb3d5516f..0e99a2d74ccb 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java @@ -27,7 +27,6 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.state.OperatorStateStore; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.CloseableRegistry; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; @@ -50,33 +49,37 @@ import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.binary.BinaryRowData; import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.VarCharType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; import org.junit.After; import org.junit.Before; import org.junit.Test; public class TestDataStatisticsOperator { - private final RowType rowType = RowType.of(new VarCharType()); + private final Schema schema = + new Schema( + Types.NestedField.optional(1, "id", Types.StringType.get()), + Types.NestedField.optional(2, "number", Types.IntegerType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("id").build(); + private final SortKey sortKey = new SortKey(schema, sortOrder); + private final RowType rowType = RowType.of(new VarCharType(), new IntType()); private final TypeSerializer rowSerializer = new RowDataSerializer(rowType); - private final GenericRowData genericRowDataA = GenericRowData.of(StringData.fromString("a")); - private final GenericRowData genericRowDataB = GenericRowData.of(StringData.fromString("b")); - // When operator hands events from coordinator, DataStatisticsUtil#deserializeDataStatistics - // deserializes bytes into BinaryRowData - private final BinaryRowData binaryRowDataA = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("a"))); - private final BinaryRowData binaryRowDataB = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("b"))); - private final BinaryRowData binaryRowDataC = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("c"))); - private final TypeSerializer>> - statisticsSerializer = MapDataStatisticsSerializer.fromKeySerializer(rowSerializer); - private DataStatisticsOperator> operator; + private final TypeSerializer>> + statisticsSerializer = + MapDataStatisticsSerializer.fromSortKeySerializer( + new SortKeySerializer(schema, sortOrder)); + + private DataStatisticsOperator> operator; private Environment getTestingEnvironment() { return new StreamMockEnvironment( @@ -99,20 +102,10 @@ public void before() throws Exception { new MockOutput<>(Lists.newArrayList())); } - private DataStatisticsOperator> createOperator() { + private DataStatisticsOperator> createOperator() { MockOperatorEventGateway mockGateway = new MockOperatorEventGateway(); - KeySelector keySelector = - new KeySelector() { - private static final long serialVersionUID = 7662520075515707428L; - - @Override - public RowData getKey(RowData value) { - return value; - } - }; - return new DataStatisticsOperator<>( - "testOperator", keySelector, mockGateway, statisticsSerializer); + "testOperator", schema, sortOrder, mockGateway, statisticsSerializer); } @After @@ -123,20 +116,26 @@ public void clean() throws Exception { @Test public void testProcessElement() throws Exception { try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> testHarness = createHarness(this.operator)) { StateInitializationContext stateContext = getStateContext(); operator.initializeState(stateContext); - operator.processElement(new StreamRecord<>(genericRowDataA)); - operator.processElement(new StreamRecord<>(genericRowDataA)); - operator.processElement(new StreamRecord<>(genericRowDataB)); + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 5))); + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 3))); + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 1))); assertThat(operator.localDataStatistics()).isInstanceOf(MapDataStatistics.class); + + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); + Map expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L); + MapDataStatistics mapDataStatistics = (MapDataStatistics) operator.localDataStatistics(); - Map statsMap = mapDataStatistics.statistics(); + Map statsMap = mapDataStatistics.statistics(); assertThat(statsMap).hasSize(2); - assertThat(statsMap) - .containsExactlyInAnyOrderEntriesOf( - ImmutableMap.of(genericRowDataA, 2L, genericRowDataB, 1L)); + assertThat(statsMap).containsExactlyInAnyOrderEntriesOf(expectedMap); + testHarness.endInput(); } } @@ -144,11 +143,14 @@ public void testProcessElement() throws Exception { @Test public void testOperatorOutput() throws Exception { try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> testHarness = createHarness(this.operator)) { - testHarness.processElement(new StreamRecord<>(genericRowDataA)); - testHarness.processElement(new StreamRecord<>(genericRowDataB)); - testHarness.processElement(new StreamRecord<>(genericRowDataB)); + testHarness.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 2))); + testHarness.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 3))); + testHarness.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 1))); List recordsOutput = testHarness.extractOutputValues().stream() @@ -157,7 +159,10 @@ public void testOperatorOutput() throws Exception { .collect(Collectors.toList()); assertThat(recordsOutput) .containsExactlyInAnyOrderElementsOf( - ImmutableList.of(genericRowDataA, genericRowDataB, genericRowDataB)); + ImmutableList.of( + GenericRowData.of(StringData.fromString("a"), 2), + GenericRowData.of(StringData.fromString("b"), 3), + GenericRowData.of(StringData.fromString("b"), 1))); } } @@ -165,36 +170,61 @@ public void testOperatorOutput() throws Exception { public void testRestoreState() throws Exception { OperatorSubtaskState snapshot; try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> testHarness1 = createHarness(this.operator)) { - DataStatistics> mapDataStatistics = - new MapDataStatistics(); - mapDataStatistics.add(binaryRowDataA); - mapDataStatistics.add(binaryRowDataA); - mapDataStatistics.add(binaryRowDataB); - mapDataStatistics.add(binaryRowDataC); - operator.handleOperatorEvent( - DataStatisticsEvent.create(0, mapDataStatistics, statisticsSerializer)); + MapDataStatistics mapDataStatistics = new MapDataStatistics(); + + SortKey key = sortKey.copy(); + key.set(0, "a"); + mapDataStatistics.add(key); + key.set(0, "a"); + mapDataStatistics.add(key); + key.set(0, "b"); + mapDataStatistics.add(key); + key.set(0, "c"); + mapDataStatistics.add(key); + + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); + SortKey keyC = sortKey.copy(); + keyC.set(0, "c"); + Map expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L, keyC, 1L); + + DataStatisticsEvent> event = + DataStatisticsEvent.create(0, mapDataStatistics, statisticsSerializer); + operator.handleOperatorEvent(event); assertThat(operator.globalDataStatistics()).isInstanceOf(MapDataStatistics.class); assertThat(operator.globalDataStatistics().statistics()) - .containsExactlyInAnyOrderEntriesOf( - ImmutableMap.of(binaryRowDataA, 2L, binaryRowDataB, 1L, binaryRowDataC, 1L)); + .containsExactlyInAnyOrderEntriesOf(expectedMap); snapshot = testHarness1.snapshot(1L, 0); } // Use the snapshot to initialize state for another new operator and then verify that the global // statistics for the new operator is same as before - DataStatisticsOperator> restoredOperator = + DataStatisticsOperator> restoredOperator = createOperator(); try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> testHarness2 = new OneInputStreamOperatorTestHarness<>(restoredOperator, 2, 2, 1)) { testHarness2.setup(); testHarness2.initializeState(snapshot); assertThat(restoredOperator.globalDataStatistics()).isInstanceOf(MapDataStatistics.class); - assertThat(restoredOperator.globalDataStatistics().statistics()) - .containsExactlyInAnyOrderEntriesOf( - ImmutableMap.of(binaryRowDataA, 2L, binaryRowDataB, 1L, binaryRowDataC, 1L)); + + // restored RowData is BinaryRowData. convert to GenericRowData for comparison + Map restoredStatistics = Maps.newHashMap(); + restoredStatistics.putAll(restoredOperator.globalDataStatistics().statistics()); + + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); + SortKey keyC = sortKey.copy(); + keyC.set(0, "c"); + Map expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L, keyC, 1L); + + assertThat(restoredStatistics).containsExactlyInAnyOrderEntriesOf(expectedMap); } } @@ -209,18 +239,16 @@ private StateInitializationContext getStateContext() throws Exception { } private OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> createHarness( - final DataStatisticsOperator> + final DataStatisticsOperator> dataStatisticsOperator) throws Exception { OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> harness = new OneInputStreamOperatorTestHarness<>(dataStatisticsOperator, 1, 1, 0); - harness.setup( - new DataStatisticsOrRecordSerializer<>( - MapDataStatisticsSerializer.fromKeySerializer(rowSerializer), rowSerializer)); + harness.setup(new DataStatisticsOrRecordSerializer<>(statisticsSerializer, rowSerializer)); harness.open(); return harness; } diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java new file mode 100644 index 000000000000..a07808e935d9 --- /dev/null +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java @@ -0,0 +1,90 @@ +/* + * 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.iceberg.flink.sink.shuffle; + +import java.util.Map; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +public class TestMapDataStatistics { + private final SortOrder sortOrder = SortOrder.builderFor(TestFixtures.SCHEMA).asc("data").build(); + private final SortKey sortKey = new SortKey(TestFixtures.SCHEMA, sortOrder); + private final RowType rowType = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); + private final RowDataWrapper rowWrapper = + new RowDataWrapper(rowType, TestFixtures.SCHEMA.asStruct()); + + @Test + public void testAddsAndGet() { + MapDataStatistics dataStatistics = new MapDataStatistics(); + + GenericRowData reusedRow = + GenericRowData.of(StringData.fromString("a"), 1, StringData.fromString("2023-06-20")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("b")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("c")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("b")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("a")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("b")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + Map actual = dataStatistics.statistics(); + + rowWrapper.wrap( + GenericRowData.of(StringData.fromString("a"), 1, StringData.fromString("2023-06-20"))); + sortKey.wrap(rowWrapper); + SortKey keyA = sortKey.copy(); + + rowWrapper.wrap( + GenericRowData.of(StringData.fromString("b"), 1, StringData.fromString("2023-06-20"))); + sortKey.wrap(rowWrapper); + SortKey keyB = sortKey.copy(); + + rowWrapper.wrap( + GenericRowData.of(StringData.fromString("c"), 1, StringData.fromString("2023-06-20"))); + sortKey.wrap(rowWrapper); + SortKey keyC = sortKey.copy(); + + Map expected = ImmutableMap.of(keyA, 2L, keyB, 3L, keyC, 1L); + Assertions.assertThat(actual).isEqualTo(expected); + } +} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java new file mode 100644 index 000000000000..c7fea015142c --- /dev/null +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java @@ -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. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.table.data.GenericRowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; + +public abstract class TestSortKeySerializerBase extends SerializerTestBase { + + protected abstract Schema schema(); + + protected abstract SortOrder sortOrder(); + + protected abstract GenericRowData rowData(); + + @Override + protected TypeSerializer createSerializer() { + return new SortKeySerializer(schema(), sortOrder()); + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return SortKey.class; + } + + @Override + protected SortKey[] getTestData() { + return new SortKey[] {sortKey()}; + } + + private SortKey sortKey() { + RowDataWrapper rowDataWrapper = + new RowDataWrapper(FlinkSchemaUtil.convert(schema()), schema().asStruct()); + SortKey sortKey = new SortKey(schema(), sortOrder()); + sortKey.wrap(rowDataWrapper.wrap(rowData())); + return sortKey; + } +} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java new file mode 100644 index 000000000000..0000688a8b55 --- /dev/null +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java @@ -0,0 +1,55 @@ +/* + * 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.iceberg.flink.sink.shuffle; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortDirection; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.flink.DataGenerator; +import org.apache.iceberg.flink.DataGenerators; + +public class TestSortKeySerializerNestedStruct extends TestSortKeySerializerBase { + private final DataGenerator generator = new DataGenerators.StructOfStruct(); + + @Override + protected Schema schema() { + return generator.icebergSchema(); + } + + @Override + protected SortOrder sortOrder() { + return SortOrder.builderFor(schema()) + .asc("row_id") + .sortBy( + Expressions.bucket("struct_of_struct.id", 4), SortDirection.DESC, NullOrder.NULLS_LAST) + .sortBy( + Expressions.truncate("struct_of_struct.person_struct.name", 16), + SortDirection.ASC, + NullOrder.NULLS_FIRST) + .build(); + } + + @Override + protected GenericRowData rowData() { + return generator.generateFlinkRowData(); + } +} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java new file mode 100644 index 000000000000..291302aef486 --- /dev/null +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortDirection; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.flink.DataGenerator; +import org.apache.iceberg.flink.DataGenerators; + +public class TestSortKeySerializerPrimitives extends TestSortKeySerializerBase { + private final DataGenerator generator = new DataGenerators.Primitives(); + + @Override + protected Schema schema() { + return generator.icebergSchema(); + } + + @Override + protected SortOrder sortOrder() { + return SortOrder.builderFor(schema()) + .asc("boolean_field") + .sortBy(Expressions.bucket("int_field", 4), SortDirection.DESC, NullOrder.NULLS_LAST) + .sortBy(Expressions.truncate("string_field", 2), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy(Expressions.bucket("uuid_field", 16), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy(Expressions.hour("ts_with_zone_field"), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy(Expressions.day("ts_without_zone_field"), SortDirection.ASC, NullOrder.NULLS_FIRST) + // can not test HeapByteBuffer due to equality test inside SerializerTestBase + // .sortBy(Expressions.truncate("binary_field", 2), SortDirection.ASC, + // NullOrder.NULLS_FIRST) + .build(); + } + + @Override + protected GenericRowData rowData() { + return generator.generateFlinkRowData(); + } +} From 2c31acc8aea7b807c91623c21c7fa2d979462021 Mon Sep 17 00:00:00 2001 From: Steven Wu Date: Fri, 8 Dec 2023 13:06:35 -0800 Subject: [PATCH 31/47] Flink: backport PR #9212 to 1.18 for switching to SortKey for data statistics --- .../flink/sink/shuffle/DataStatistics.java | 12 +- .../sink/shuffle/DataStatisticsOperator.java | 25 +- .../sink/shuffle/DataStatisticsUtil.java | 1 - .../flink/sink/shuffle/MapDataStatistics.java | 21 +- .../shuffle/MapDataStatisticsSerializer.java | 56 +-- .../flink/sink/shuffle/SortKeySerializer.java | 353 ++++++++++++++++++ .../shuffle/TestAggregatedStatistics.java | 48 +-- .../TestAggregatedStatisticsTracker.java | 97 ++--- .../TestDataStatisticsCoordinator.java | 98 ++--- ...TestDataStatisticsCoordinatorProvider.java | 160 ++++---- .../shuffle/TestDataStatisticsOperator.java | 154 ++++---- .../sink/shuffle/TestMapDataStatistics.java | 90 +++++ .../shuffle/TestSortKeySerializerBase.java | 65 ++++ .../TestSortKeySerializerNestedStruct.java | 55 +++ .../TestSortKeySerializerPrimitives.java | 57 +++ 15 files changed, 978 insertions(+), 314 deletions(-) create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java index 28a05201c02f..9d7cf179ab1c 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java @@ -19,7 +19,7 @@ package org.apache.iceberg.flink.sink.shuffle; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.data.RowData; +import org.apache.iceberg.SortKey; /** * DataStatistics defines the interface to collect data distribution information. @@ -29,7 +29,7 @@ * (sketching) can be used. */ @Internal -interface DataStatistics { +interface DataStatistics, S> { /** * Check if data statistics contains any statistics information. @@ -38,12 +38,8 @@ interface DataStatistics { */ boolean isEmpty(); - /** - * Add data key to data statistics. - * - * @param key generate from data by applying key selector - */ - void add(RowData key); + /** Add row sortKey to data statistics. */ + void add(SortKey sortKey); /** * Merge current statistics with other statistics. diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java index d00d5d2e5aa9..5157a37cf2cd 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java @@ -22,7 +22,6 @@ import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.runtime.operators.coordination.OperatorEvent; import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; import org.apache.flink.runtime.operators.coordination.OperatorEventHandler; @@ -32,6 +31,12 @@ import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -45,11 +50,12 @@ class DataStatisticsOperator, S> extends AbstractStreamOperator> implements OneInputStreamOperator>, OperatorEventHandler { + private static final long serialVersionUID = 1L; private final String operatorName; - // keySelector will be used to generate key from data for collecting data statistics - private final KeySelector keySelector; + private final RowDataWrapper rowDataWrapper; + private final SortKey sortKey; private final OperatorEventGateway operatorEventGateway; private final TypeSerializer> statisticsSerializer; private transient volatile DataStatistics localStatistics; @@ -58,11 +64,13 @@ class DataStatisticsOperator, S> DataStatisticsOperator( String operatorName, - KeySelector keySelector, + Schema schema, + SortOrder sortOrder, OperatorEventGateway operatorEventGateway, TypeSerializer> statisticsSerializer) { this.operatorName = operatorName; - this.keySelector = keySelector; + this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); + this.sortKey = new SortKey(schema, sortOrder); this.operatorEventGateway = operatorEventGateway; this.statisticsSerializer = statisticsSerializer; } @@ -126,10 +134,11 @@ public void handleOperatorEvent(OperatorEvent event) { } @Override - public void processElement(StreamRecord streamRecord) throws Exception { + public void processElement(StreamRecord streamRecord) { RowData record = streamRecord.getValue(); - RowData key = keySelector.getKey(record); - localStatistics.add(key); + StructLike struct = rowDataWrapper.wrap(record); + sortKey.wrap(struct); + localStatistics.add(sortKey); output.collect(new StreamRecord<>(DataStatisticsOrRecord.fromRecord(record))); } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java index 2737b1346f0f..8716cb872d0e 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java @@ -76,7 +76,6 @@ static , S> byte[] serializeAggregatedStatistics( return bytes.toByteArray(); } - @SuppressWarnings("unchecked") static , S> AggregatedStatistics deserializeAggregatedStatistics( byte[] bytes, TypeSerializer> statisticsSerializer) diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java index 246b56526fd6..0ffffd9cf49f 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java @@ -20,20 +20,20 @@ import java.util.Map; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.data.RowData; +import org.apache.iceberg.SortKey; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.collect.Maps; /** MapDataStatistics uses map to count key frequency */ @Internal -class MapDataStatistics implements DataStatistics> { - private final Map statistics; +class MapDataStatistics implements DataStatistics> { + private final Map statistics; MapDataStatistics() { this.statistics = Maps.newHashMap(); } - MapDataStatistics(Map statistics) { + MapDataStatistics(Map statistics) { this.statistics = statistics; } @@ -43,9 +43,14 @@ public boolean isEmpty() { } @Override - public void add(RowData key) { - // increase count of occurrence by one in the dataStatistics map - statistics.merge(key, 1L, Long::sum); + public void add(SortKey sortKey) { + if (statistics.containsKey(sortKey)) { + statistics.merge(sortKey, 1L, Long::sum); + } else { + // clone the sort key before adding to map because input sortKey object can be reused + SortKey copiedKey = sortKey.copy(); + statistics.put(copiedKey, 1L); + } } @Override @@ -54,7 +59,7 @@ public void merge(MapDataStatistics otherStatistics) { } @Override - public Map statistics() { + public Map statistics() { return statistics; } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java index 6d07637b29b3..b6cccd0566fc 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java @@ -29,22 +29,22 @@ import org.apache.flink.api.common.typeutils.base.MapSerializer; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.table.data.RowData; import org.apache.flink.util.Preconditions; +import org.apache.iceberg.SortKey; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @Internal class MapDataStatisticsSerializer - extends TypeSerializer>> { - private final MapSerializer mapSerializer; + extends TypeSerializer>> { + private final MapSerializer mapSerializer; - static TypeSerializer>> fromKeySerializer( - TypeSerializer keySerializer) { + static MapDataStatisticsSerializer fromSortKeySerializer( + TypeSerializer sortKeySerializer) { return new MapDataStatisticsSerializer( - new MapSerializer<>(keySerializer, LongSerializer.INSTANCE)); + new MapSerializer<>(sortKeySerializer, LongSerializer.INSTANCE)); } - MapDataStatisticsSerializer(MapSerializer mapSerializer) { + MapDataStatisticsSerializer(MapSerializer mapSerializer) { this.mapSerializer = mapSerializer; } @@ -55,28 +55,28 @@ public boolean isImmutableType() { @SuppressWarnings("ReferenceEquality") @Override - public TypeSerializer>> duplicate() { - MapSerializer duplicateMapSerializer = - (MapSerializer) mapSerializer.duplicate(); + public TypeSerializer>> duplicate() { + MapSerializer duplicateMapSerializer = + (MapSerializer) mapSerializer.duplicate(); return (duplicateMapSerializer == mapSerializer) ? this : new MapDataStatisticsSerializer(duplicateMapSerializer); } @Override - public DataStatistics> createInstance() { + public MapDataStatistics createInstance() { return new MapDataStatistics(); } @Override - public DataStatistics> copy(DataStatistics obj) { + public MapDataStatistics copy(DataStatistics> obj) { Preconditions.checkArgument( obj instanceof MapDataStatistics, "Invalid data statistics type: " + obj.getClass()); MapDataStatistics from = (MapDataStatistics) obj; - TypeSerializer keySerializer = mapSerializer.getKeySerializer(); - Map newMap = Maps.newHashMapWithExpectedSize(from.statistics().size()); - for (Map.Entry entry : from.statistics().entrySet()) { - RowData newKey = keySerializer.copy(entry.getKey()); + TypeSerializer keySerializer = mapSerializer.getKeySerializer(); + Map newMap = Maps.newHashMapWithExpectedSize(from.statistics().size()); + for (Map.Entry entry : from.statistics().entrySet()) { + SortKey newKey = keySerializer.copy(entry.getKey()); // no need to copy value since it is just a Long newMap.put(newKey, entry.getValue()); } @@ -85,8 +85,9 @@ public DataStatistics> copy(DataStatistics } @Override - public DataStatistics> copy( - DataStatistics from, DataStatistics reuse) { + public DataStatistics> copy( + DataStatistics> from, + DataStatistics> reuse) { // not much benefit to reuse return copy(from); } @@ -97,7 +98,9 @@ public int getLength() { } @Override - public void serialize(DataStatistics obj, DataOutputView target) throws IOException { + public void serialize( + DataStatistics> obj, DataOutputView target) + throws IOException { Preconditions.checkArgument( obj instanceof MapDataStatistics, "Invalid data statistics type: " + obj.getClass()); MapDataStatistics mapStatistics = (MapDataStatistics) obj; @@ -105,14 +108,15 @@ public void serialize(DataStatistics obj, DataOutputView target) throws IOExcept } @Override - public DataStatistics> deserialize(DataInputView source) + public DataStatistics> deserialize(DataInputView source) throws IOException { return new MapDataStatistics(mapSerializer.deserialize(source)); } @Override - public DataStatistics> deserialize( - DataStatistics reuse, DataInputView source) throws IOException { + public DataStatistics> deserialize( + DataStatistics> reuse, DataInputView source) + throws IOException { // not much benefit to reuse return deserialize(source); } @@ -138,14 +142,14 @@ public int hashCode() { } @Override - public TypeSerializerSnapshot>> + public TypeSerializerSnapshot>> snapshotConfiguration() { return new MapDataStatisticsSerializerSnapshot(this); } public static class MapDataStatisticsSerializerSnapshot extends CompositeTypeSerializerSnapshot< - DataStatistics>, MapDataStatisticsSerializer> { + DataStatistics>, MapDataStatisticsSerializer> { private static final int CURRENT_VERSION = 1; // constructors need to public. Otherwise, Flink state restore would complain @@ -175,8 +179,8 @@ protected TypeSerializer[] getNestedSerializers( protected MapDataStatisticsSerializer createOuterSerializerWithNestedSerializers( TypeSerializer[] nestedSerializers) { @SuppressWarnings("unchecked") - MapSerializer mapSerializer = - (MapSerializer) nestedSerializers[0]; + MapSerializer mapSerializer = + (MapSerializer) nestedSerializers[0]; return new MapDataStatisticsSerializer(mapSerializer); } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java new file mode 100644 index 000000000000..d03409f2a430 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java @@ -0,0 +1,353 @@ +/* + * 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.iceberg.flink.sink.shuffle; + +import java.io.IOException; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Objects; +import java.util.UUID; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.StringUtils; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.SortField; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderParser; +import org.apache.iceberg.types.CheckCompatibility; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +class SortKeySerializer extends TypeSerializer { + private final Schema schema; + private final SortOrder sortOrder; + private final int size; + private final Types.NestedField[] transformedFields; + + private transient SortKey sortKey; + + SortKeySerializer(Schema schema, SortOrder sortOrder) { + this.schema = schema; + this.sortOrder = sortOrder; + this.size = sortOrder.fields().size(); + + this.transformedFields = new Types.NestedField[size]; + for (int i = 0; i < size; ++i) { + SortField sortField = sortOrder.fields().get(i); + Types.NestedField sourceField = schema.findField(sortField.sourceId()); + Type resultType = sortField.transform().getResultType(sourceField.type()); + Types.NestedField transformedField = + Types.NestedField.of( + sourceField.fieldId(), + sourceField.isOptional(), + sourceField.name(), + resultType, + sourceField.doc()); + transformedFields[i] = transformedField; + } + } + + private SortKey lazySortKey() { + if (sortKey == null) { + this.sortKey = new SortKey(schema, sortOrder); + } + + return sortKey; + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer duplicate() { + return new SortKeySerializer(schema, sortOrder); + } + + @Override + public SortKey createInstance() { + return new SortKey(schema, sortOrder); + } + + @Override + public SortKey copy(SortKey from) { + return from.copy(); + } + + @Override + public SortKey copy(SortKey from, SortKey reuse) { + // no benefit of reuse + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(SortKey record, DataOutputView target) throws IOException { + Preconditions.checkArgument( + record.size() == size, + "Invalid size of the sort key object: %s. Expected %s", + record.size(), + size); + for (int i = 0; i < size; ++i) { + int fieldId = transformedFields[i].fieldId(); + Type.TypeID typeId = transformedFields[i].type().typeId(); + switch (typeId) { + case BOOLEAN: + target.writeBoolean(record.get(i, Boolean.class)); + break; + case INTEGER: + case DATE: + target.writeInt(record.get(i, Integer.class)); + break; + case LONG: + case TIME: + case TIMESTAMP: + target.writeLong(record.get(i, Long.class)); + break; + case FLOAT: + target.writeFloat(record.get(i, Float.class)); + break; + case DOUBLE: + target.writeDouble(record.get(i, Double.class)); + break; + case STRING: + target.writeUTF(record.get(i, CharSequence.class).toString()); + break; + case UUID: + UUID uuid = record.get(i, UUID.class); + target.writeLong(uuid.getMostSignificantBits()); + target.writeLong(uuid.getLeastSignificantBits()); + break; + case FIXED: + case BINARY: + byte[] bytes = record.get(i, ByteBuffer.class).array(); + target.writeInt(bytes.length); + target.write(bytes); + break; + case DECIMAL: + BigDecimal decimal = record.get(i, BigDecimal.class); + byte[] decimalBytes = decimal.unscaledValue().toByteArray(); + target.writeInt(decimalBytes.length); + target.write(decimalBytes); + target.writeInt(decimal.scale()); + break; + case STRUCT: + case MAP: + case LIST: + default: + // SortKey transformation is a flattened struct without list and map + throw new UnsupportedOperationException( + String.format("Field %d has unsupported field type: %s", fieldId, typeId)); + } + } + } + + @Override + public SortKey deserialize(DataInputView source) throws IOException { + // copying is a little faster than constructing a new SortKey object + SortKey deserialized = lazySortKey().copy(); + deserialize(deserialized, source); + return deserialized; + } + + @Override + public SortKey deserialize(SortKey reuse, DataInputView source) throws IOException { + Preconditions.checkArgument( + reuse.size() == size, + "Invalid size of the sort key object: %s. Expected %s", + reuse.size(), + size); + for (int i = 0; i < size; ++i) { + int fieldId = transformedFields[i].fieldId(); + Type.TypeID typeId = transformedFields[i].type().typeId(); + switch (typeId) { + case BOOLEAN: + reuse.set(i, source.readBoolean()); + break; + case INTEGER: + case DATE: + reuse.set(i, source.readInt()); + break; + case LONG: + case TIME: + case TIMESTAMP: + reuse.set(i, source.readLong()); + break; + case FLOAT: + reuse.set(i, source.readFloat()); + break; + case DOUBLE: + reuse.set(i, source.readDouble()); + break; + case STRING: + reuse.set(i, source.readUTF()); + break; + case UUID: + long mostSignificantBits = source.readLong(); + long leastSignificantBits = source.readLong(); + reuse.set(i, new UUID(mostSignificantBits, leastSignificantBits)); + break; + case FIXED: + case BINARY: + byte[] bytes = new byte[source.readInt()]; + source.read(bytes); + reuse.set(i, ByteBuffer.wrap(bytes)); + break; + case DECIMAL: + byte[] unscaledBytes = new byte[source.readInt()]; + source.read(unscaledBytes); + int scale = source.readInt(); + BigDecimal decimal = new BigDecimal(new BigInteger(unscaledBytes), scale); + reuse.set(i, decimal); + break; + case STRUCT: + case MAP: + case LIST: + default: + // SortKey transformation is a flattened struct without list and map + throw new UnsupportedOperationException( + String.format("Field %d has unsupported field type: %s", fieldId, typeId)); + } + } + + return reuse; + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + // no optimization here + serialize(deserialize(source), target); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof SortKeySerializer)) { + return false; + } + + SortKeySerializer other = (SortKeySerializer) obj; + return Objects.equals(schema.asStruct(), other.schema.asStruct()) + && Objects.equals(sortOrder, other.sortOrder); + } + + @Override + public int hashCode() { + return schema.asStruct().hashCode() * 31 + sortOrder.hashCode(); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new SortKeySerializerSnapshot(schema, sortOrder); + } + + public static class SortKeySerializerSnapshot implements TypeSerializerSnapshot { + private static final int CURRENT_VERSION = 1; + + private Schema schema; + private SortOrder sortOrder; + + @SuppressWarnings({"checkstyle:RedundantModifier", "WeakerAccess"}) + public SortKeySerializerSnapshot() { + // this constructor is used when restoring from a checkpoint. + } + + // constructors need to public. Otherwise, Flink state restore would complain + // "The class has no (implicit) public nullary constructor". + @SuppressWarnings("checkstyle:RedundantModifier") + public SortKeySerializerSnapshot(Schema schema, SortOrder sortOrder) { + this.schema = schema; + this.sortOrder = sortOrder; + } + + @Override + public int getCurrentVersion() { + return CURRENT_VERSION; + } + + @Override + public void writeSnapshot(DataOutputView out) throws IOException { + Preconditions.checkState(schema != null, "Invalid schema: null"); + Preconditions.checkState(sortOrder != null, "Invalid sort order: null"); + + StringUtils.writeString(SchemaParser.toJson(schema), out); + StringUtils.writeString(SortOrderParser.toJson(sortOrder), out); + } + + @Override + public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) + throws IOException { + if (readVersion == 1) { + readV1(in); + } else { + throw new IllegalArgumentException("Unknown read version: " + readVersion); + } + } + + @Override + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + TypeSerializer newSerializer) { + if (!(newSerializer instanceof SortKeySerializer)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + + SortKeySerializer newAvroSerializer = (SortKeySerializer) newSerializer; + return resolveSchemaCompatibility(newAvroSerializer.schema, schema); + } + + @Override + public TypeSerializer restoreSerializer() { + Preconditions.checkState(schema != null, "Invalid schema: null"); + Preconditions.checkState(sortOrder != null, "Invalid sort order: null"); + return new SortKeySerializer(schema, sortOrder); + } + + private void readV1(DataInputView in) throws IOException { + String schemaJson = StringUtils.readString(in); + String sortOrderJson = StringUtils.readString(in); + this.schema = SchemaParser.fromJson(schemaJson); + this.sortOrder = SortOrderParser.fromJson(sortOrderJson).bind(schema); + } + + @VisibleForTesting + static TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + Schema readSchema, Schema writeSchema) { + List compatibilityErrors = + CheckCompatibility.writeCompatibilityErrors(readSchema, writeSchema); + if (compatibilityErrors.isEmpty()) { + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } + + return TypeSerializerSchemaCompatibility.incompatible(); + } + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java index dd7fcafe5336..890cc361b246 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java @@ -21,41 +21,43 @@ import static org.assertj.core.api.Assertions.assertThat; import java.util.Map; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.VarCharType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.types.Types; import org.junit.Test; public class TestAggregatedStatistics { + private final Schema schema = + new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); + private final SortKey sortKey = new SortKey(schema, sortOrder); + private final MapDataStatisticsSerializer statisticsSerializer = + MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); @Test public void mergeDataStatisticTest() { - GenericRowData rowDataA = GenericRowData.of(StringData.fromString("a")); - GenericRowData rowDataB = GenericRowData.of(StringData.fromString("b")); + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); - AggregatedStatistics> aggregatedStatistics = - new AggregatedStatistics<>( - 1, - MapDataStatisticsSerializer.fromKeySerializer( - new RowDataSerializer(RowType.of(new VarCharType())))); + AggregatedStatistics> aggregatedStatistics = + new AggregatedStatistics<>(1, statisticsSerializer); MapDataStatistics mapDataStatistics1 = new MapDataStatistics(); - mapDataStatistics1.add(rowDataA); - mapDataStatistics1.add(rowDataA); - mapDataStatistics1.add(rowDataB); + mapDataStatistics1.add(keyA); + mapDataStatistics1.add(keyA); + mapDataStatistics1.add(keyB); aggregatedStatistics.mergeDataStatistic("testOperator", 1, mapDataStatistics1); MapDataStatistics mapDataStatistics2 = new MapDataStatistics(); - mapDataStatistics2.add(rowDataA); + mapDataStatistics2.add(keyA); aggregatedStatistics.mergeDataStatistic("testOperator", 1, mapDataStatistics2); - assertThat(aggregatedStatistics.dataStatistics().statistics().get(rowDataA)) + assertThat(aggregatedStatistics.dataStatistics().statistics().get(keyA)) .isEqualTo( - mapDataStatistics1.statistics().get(rowDataA) - + mapDataStatistics2.statistics().get(rowDataA)); - assertThat(aggregatedStatistics.dataStatistics().statistics().get(rowDataB)) + mapDataStatistics1.statistics().get(keyA) + mapDataStatistics2.statistics().get(keyA)); + assertThat(aggregatedStatistics.dataStatistics().statistics().get(keyB)) .isEqualTo( - mapDataStatistics1.statistics().get(rowDataB) - + mapDataStatistics2.statistics().getOrDefault(rowDataB, 0L)); + mapDataStatistics1.statistics().get(keyB) + + mapDataStatistics2.statistics().getOrDefault(keyB, 0L)); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java index 48e4e4d8f966..4c64ce522201 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java @@ -21,32 +21,33 @@ import static org.assertj.core.api.Assertions.assertThat; import java.util.Map; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.binary.BinaryRowData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.VarCharType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.types.Types; import org.junit.Before; import org.junit.Test; public class TestAggregatedStatisticsTracker { private static final int NUM_SUBTASKS = 2; - private final RowType rowType = RowType.of(new VarCharType()); - // When coordinator handles events from operator, DataStatisticsUtil#deserializeDataStatistics - // deserializes bytes into BinaryRowData - private final BinaryRowData binaryRowDataA = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("a"))); - private final BinaryRowData binaryRowDataB = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("b"))); - private final TypeSerializer rowSerializer = new RowDataSerializer(rowType); - private final TypeSerializer>> - statisticsSerializer = MapDataStatisticsSerializer.fromKeySerializer(rowSerializer); - private AggregatedStatisticsTracker> + + private final Schema schema = + new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); + private final SortKey sortKey = new SortKey(schema, sortOrder); + private final MapDataStatisticsSerializer statisticsSerializer = + MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); + private final SortKey keyA = sortKey.copy(); + private final SortKey keyB = sortKey.copy(); + + private AggregatedStatisticsTracker> aggregatedStatisticsTracker; + public TestAggregatedStatisticsTracker() { + keyA.set(0, "a"); + keyB.set(0, "b"); + } + @Before public void before() throws Exception { aggregatedStatisticsTracker = @@ -56,8 +57,8 @@ public void before() throws Exception { @Test public void receiveNewerDataStatisticEvent() { MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(binaryRowDataA); - DataStatisticsEvent> + checkpoint1Subtask0DataStatistic.add(keyA); + DataStatisticsEvent> checkpoint1Subtask0DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); assertThat( @@ -67,8 +68,8 @@ public void receiveNewerDataStatisticEvent() { assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()).isEqualTo(1); MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(binaryRowDataA); - DataStatisticsEvent> + checkpoint2Subtask0DataStatistic.add(keyA); + DataStatisticsEvent> checkpoint2Subtask0DataStatisticEvent = DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); assertThat( @@ -82,10 +83,10 @@ public void receiveNewerDataStatisticEvent() { @Test public void receiveOlderDataStatisticEventTest() { MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(binaryRowDataA); - checkpoint2Subtask0DataStatistic.add(binaryRowDataB); - checkpoint2Subtask0DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> + checkpoint2Subtask0DataStatistic.add(keyA); + checkpoint2Subtask0DataStatistic.add(keyB); + checkpoint2Subtask0DataStatistic.add(keyB); + DataStatisticsEvent> checkpoint3Subtask0DataStatisticEvent = DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); assertThat( @@ -94,8 +95,8 @@ public void receiveOlderDataStatisticEventTest() { .isNull(); MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - checkpoint1Subtask1DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> + checkpoint1Subtask1DataStatistic.add(keyB); + DataStatisticsEvent> checkpoint1Subtask1DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); // Receive event from old checkpoint, aggregatedStatisticsAggregatorTracker won't return @@ -110,10 +111,10 @@ public void receiveOlderDataStatisticEventTest() { @Test public void receiveCompletedDataStatisticEvent() { MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(binaryRowDataA); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> + checkpoint1Subtask0DataStatistic.add(keyA); + checkpoint1Subtask0DataStatistic.add(keyB); + checkpoint1Subtask0DataStatistic.add(keyB); + DataStatisticsEvent> checkpoint1Subtask0DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); assertThat( @@ -122,14 +123,14 @@ public void receiveCompletedDataStatisticEvent() { .isNull(); MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - checkpoint1Subtask1DataStatistic.add(binaryRowDataA); - checkpoint1Subtask1DataStatistic.add(binaryRowDataA); - checkpoint1Subtask1DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> + checkpoint1Subtask1DataStatistic.add(keyA); + checkpoint1Subtask1DataStatistic.add(keyA); + checkpoint1Subtask1DataStatistic.add(keyB); + DataStatisticsEvent> checkpoint1Subtask1DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); // Receive data statistics from all subtasks at checkpoint 1 - AggregatedStatistics> completedStatistics = + AggregatedStatistics> completedStatistics = aggregatedStatisticsTracker.updateAndCheckCompletion( 1, checkpoint1Subtask1DataStatisticEvent); @@ -137,20 +138,20 @@ public void receiveCompletedDataStatisticEvent() { assertThat(completedStatistics.checkpointId()).isEqualTo(1); MapDataStatistics globalDataStatistics = (MapDataStatistics) completedStatistics.dataStatistics(); - assertThat((long) globalDataStatistics.statistics().get(binaryRowDataA)) + assertThat((long) globalDataStatistics.statistics().get(keyA)) .isEqualTo( - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataA) - + checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataA)); - assertThat((long) globalDataStatistics.statistics().get(binaryRowDataB)) + checkpoint1Subtask0DataStatistic.statistics().get(keyA) + + checkpoint1Subtask1DataStatistic.statistics().get(keyA)); + assertThat((long) globalDataStatistics.statistics().get(keyB)) .isEqualTo( - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataB) - + checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataB)); + checkpoint1Subtask0DataStatistic.statistics().get(keyB) + + checkpoint1Subtask1DataStatistic.statistics().get(keyB)); assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()) .isEqualTo(completedStatistics.checkpointId() + 1); MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(binaryRowDataA); - DataStatisticsEvent> + checkpoint2Subtask0DataStatistic.add(keyA); + DataStatisticsEvent> checkpoint2Subtask0DataStatisticEvent = DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); assertThat( @@ -160,8 +161,8 @@ public void receiveCompletedDataStatisticEvent() { assertThat(completedStatistics.checkpointId()).isEqualTo(1); MapDataStatistics checkpoint2Subtask1DataStatistic = new MapDataStatistics(); - checkpoint2Subtask1DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> + checkpoint2Subtask1DataStatistic.add(keyB); + DataStatisticsEvent> checkpoint2Subtask1DataStatisticEvent = DataStatisticsEvent.create(2, checkpoint2Subtask1DataStatistic, statisticsSerializer); // Receive data statistics from all subtasks at checkpoint 2 diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java index 9ec2606e101d..3df714059c37 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java @@ -24,19 +24,15 @@ import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; -import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.binary.BinaryRowData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.VarCharType; import org.apache.flink.util.ExceptionUtils; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; import org.junit.Before; import org.junit.Test; @@ -44,20 +40,21 @@ public class TestDataStatisticsCoordinator { private static final String OPERATOR_NAME = "TestCoordinator"; private static final OperatorID TEST_OPERATOR_ID = new OperatorID(1234L, 5678L); private static final int NUM_SUBTASKS = 2; - private TypeSerializer>> - statisticsSerializer; + + private final Schema schema = + new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); + private final SortKey sortKey = new SortKey(schema, sortOrder); + private final MapDataStatisticsSerializer statisticsSerializer = + MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); private EventReceivingTasks receivingTasks; - private DataStatisticsCoordinator> + private DataStatisticsCoordinator> dataStatisticsCoordinator; @Before public void before() throws Exception { receivingTasks = EventReceivingTasks.createForRunningTasks(); - statisticsSerializer = - MapDataStatisticsSerializer.fromKeySerializer( - new RowDataSerializer(RowType.of(new VarCharType()))); - dataStatisticsCoordinator = new DataStatisticsCoordinator<>( OPERATOR_NAME, @@ -93,59 +90,66 @@ public void testThrowExceptionWhenNotStarted() { @Test public void testDataStatisticsEventHandling() throws Exception { tasksReady(); - // When coordinator handles events from operator, DataStatisticsUtil#deserializeDataStatistics - // deserializes bytes into BinaryRowData - RowType rowType = RowType.of(new VarCharType()); - BinaryRowData binaryRowDataA = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("a"))); - BinaryRowData binaryRowDataB = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("b"))); - BinaryRowData binaryRowDataC = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("c"))); + SortKey key = sortKey.copy(); MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(binaryRowDataA); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - checkpoint1Subtask0DataStatistic.add(binaryRowDataC); - checkpoint1Subtask0DataStatistic.add(binaryRowDataC); - checkpoint1Subtask0DataStatistic.add(binaryRowDataC); - DataStatisticsEvent> + key.set(0, "a"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "b"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "b"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask0DataStatistic.add(key); + + DataStatisticsEvent> checkpoint1Subtask0DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); + MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - checkpoint1Subtask1DataStatistic.add(binaryRowDataA); - checkpoint1Subtask1DataStatistic.add(binaryRowDataB); - checkpoint1Subtask1DataStatistic.add(binaryRowDataC); - checkpoint1Subtask1DataStatistic.add(binaryRowDataC); - DataStatisticsEvent> + key.set(0, "a"); + checkpoint1Subtask1DataStatistic.add(key); + key.set(0, "b"); + checkpoint1Subtask1DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask1DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask1DataStatistic.add(key); + + DataStatisticsEvent> checkpoint1Subtask1DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); + // Handle events from operators for checkpoint 1 dataStatisticsCoordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0DataStatisticEvent); dataStatisticsCoordinator.handleEventFromOperator(1, 0, checkpoint1Subtask1DataStatisticEvent); waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + // Verify global data statistics is the aggregation of all subtasks data statistics + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); + SortKey keyC = sortKey.copy(); + keyC.set(0, "c"); MapDataStatistics globalDataStatistics = (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); assertThat(globalDataStatistics.statistics()) .containsExactlyInAnyOrderEntriesOf( ImmutableMap.of( - binaryRowDataA, - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataA) - + (long) checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataA), - binaryRowDataB, - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataB) - + (long) checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataB), - binaryRowDataC, - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataC) - + (long) checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataC))); + keyA, 2L, + keyB, 3L, + keyC, 5L)); } static void setAllTasksReady( int subtasks, - DataStatisticsCoordinator> dataStatisticsCoordinator, + DataStatisticsCoordinator> dataStatisticsCoordinator, EventReceivingTasks receivingTasks) { for (int i = 0; i < subtasks; i++) { dataStatisticsCoordinator.executionAttemptReady( @@ -154,7 +158,7 @@ static void setAllTasksReady( } static void waitForCoordinatorToProcessActions( - DataStatisticsCoordinator> coordinator) { + DataStatisticsCoordinator> coordinator) { CompletableFuture future = new CompletableFuture<>(); coordinator.callInCoordinatorThread( () -> { diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java index cb9d3f48ffeb..5e0a752be506 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java @@ -23,18 +23,14 @@ import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; -import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; import org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.binary.BinaryRowData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.VarCharType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.types.Types; import org.junit.Before; import org.junit.Test; @@ -42,16 +38,18 @@ public class TestDataStatisticsCoordinatorProvider { private static final OperatorID OPERATOR_ID = new OperatorID(); private static final int NUM_SUBTASKS = 1; - private DataStatisticsCoordinatorProvider> provider; + private final Schema schema = + new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); + private final SortKey sortKey = new SortKey(schema, sortOrder); + private final MapDataStatisticsSerializer statisticsSerializer = + MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); + + private DataStatisticsCoordinatorProvider> provider; private EventReceivingTasks receivingTasks; - private TypeSerializer>> - statisticsSerializer; @Before public void before() { - statisticsSerializer = - MapDataStatisticsSerializer.fromKeySerializer( - new RowDataSerializer(RowType.of(new VarCharType()))); provider = new DataStatisticsCoordinatorProvider<>( "DataStatisticsCoordinatorProvider", OPERATOR_ID, statisticsSerializer); @@ -61,84 +59,82 @@ public void before() { @Test @SuppressWarnings("unchecked") public void testCheckpointAndReset() throws Exception { - RowType rowType = RowType.of(new VarCharType()); - // When coordinator handles events from operator, DataStatisticsUtil#deserializeDataStatistics - // deserializes bytes into BinaryRowData - BinaryRowData binaryRowDataA = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("a"))); - BinaryRowData binaryRowDataB = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("b"))); - BinaryRowData binaryRowDataC = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("c"))); - BinaryRowData binaryRowDataD = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("d"))); - BinaryRowData binaryRowDataE = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("e"))); + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); + SortKey keyC = sortKey.copy(); + keyC.set(0, "c"); + SortKey keyD = sortKey.copy(); + keyD.set(0, "c"); + SortKey keyE = sortKey.copy(); + keyE.set(0, "c"); - RecreateOnResetOperatorCoordinator coordinator = + try (RecreateOnResetOperatorCoordinator coordinator = (RecreateOnResetOperatorCoordinator) - provider.create(new MockOperatorCoordinatorContext(OPERATOR_ID, NUM_SUBTASKS)); - DataStatisticsCoordinator> dataStatisticsCoordinator = - (DataStatisticsCoordinator>) - coordinator.getInternalCoordinator(); + provider.create(new MockOperatorCoordinatorContext(OPERATOR_ID, NUM_SUBTASKS))) { + DataStatisticsCoordinator> dataStatisticsCoordinator = + (DataStatisticsCoordinator>) + coordinator.getInternalCoordinator(); - // Start the coordinator - coordinator.start(); - TestDataStatisticsCoordinator.setAllTasksReady( - NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); - MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(binaryRowDataA); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - checkpoint1Subtask0DataStatistic.add(binaryRowDataC); - DataStatisticsEvent> - checkpoint1Subtask0DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); + // Start the coordinator + coordinator.start(); + TestDataStatisticsCoordinator.setAllTasksReady( + NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); + MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); + checkpoint1Subtask0DataStatistic.add(keyA); + checkpoint1Subtask0DataStatistic.add(keyB); + checkpoint1Subtask0DataStatistic.add(keyC); + DataStatisticsEvent> + checkpoint1Subtask0DataStatisticEvent = + DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); - // Handle events from operators for checkpoint 1 - coordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0DataStatisticEvent); - TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); - // Verify checkpoint 1 global data statistics - MapDataStatistics checkpoint1GlobalDataStatistics = - (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(checkpoint1GlobalDataStatistics.statistics()) - .isEqualTo(checkpoint1Subtask0DataStatistic.statistics()); - byte[] checkpoint1Bytes = waitForCheckpoint(1L, dataStatisticsCoordinator); + // Handle events from operators for checkpoint 1 + coordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0DataStatisticEvent); + TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + // Verify checkpoint 1 global data statistics + MapDataStatistics checkpoint1GlobalDataStatistics = + (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); + assertThat(checkpoint1GlobalDataStatistics.statistics()) + .isEqualTo(checkpoint1Subtask0DataStatistic.statistics()); + byte[] checkpoint1Bytes = waitForCheckpoint(1L, dataStatisticsCoordinator); - MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(binaryRowDataD); - checkpoint2Subtask0DataStatistic.add(binaryRowDataE); - checkpoint2Subtask0DataStatistic.add(binaryRowDataE); - DataStatisticsEvent> - checkpoint2Subtask0DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); - // Handle events from operators for checkpoint 2 - coordinator.handleEventFromOperator(0, 0, checkpoint2Subtask0DataStatisticEvent); - TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); - // Verify checkpoint 2 global data statistics - MapDataStatistics checkpoint2GlobalDataStatistics = - (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(checkpoint2GlobalDataStatistics.statistics()) - .isEqualTo(checkpoint2Subtask0DataStatistic.statistics()); - waitForCheckpoint(2L, dataStatisticsCoordinator); + MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); + checkpoint2Subtask0DataStatistic.add(keyD); + checkpoint2Subtask0DataStatistic.add(keyE); + checkpoint2Subtask0DataStatistic.add(keyE); + DataStatisticsEvent> + checkpoint2Subtask0DataStatisticEvent = + DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); + // Handle events from operators for checkpoint 2 + coordinator.handleEventFromOperator(0, 0, checkpoint2Subtask0DataStatisticEvent); + TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + // Verify checkpoint 2 global data statistics + MapDataStatistics checkpoint2GlobalDataStatistics = + (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); + assertThat(checkpoint2GlobalDataStatistics.statistics()) + .isEqualTo(checkpoint2Subtask0DataStatistic.statistics()); + waitForCheckpoint(2L, dataStatisticsCoordinator); - // Reset coordinator to checkpoint 1 - coordinator.resetToCheckpoint(1L, checkpoint1Bytes); - DataStatisticsCoordinator> - restoredDataStatisticsCoordinator = - (DataStatisticsCoordinator>) - coordinator.getInternalCoordinator(); - assertThat(dataStatisticsCoordinator).isNotEqualTo(restoredDataStatisticsCoordinator); - // Verify restored data statistics - MapDataStatistics restoredAggregateDataStatistics = - (MapDataStatistics) - restoredDataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(restoredAggregateDataStatistics.statistics()) - .isEqualTo(checkpoint1GlobalDataStatistics.statistics()); + // Reset coordinator to checkpoint 1 + coordinator.resetToCheckpoint(1L, checkpoint1Bytes); + DataStatisticsCoordinator> + restoredDataStatisticsCoordinator = + (DataStatisticsCoordinator>) + coordinator.getInternalCoordinator(); + assertThat(dataStatisticsCoordinator).isNotEqualTo(restoredDataStatisticsCoordinator); + // Verify restored data statistics + MapDataStatistics restoredAggregateDataStatistics = + (MapDataStatistics) + restoredDataStatisticsCoordinator.completedStatistics().dataStatistics(); + assertThat(restoredAggregateDataStatistics.statistics()) + .isEqualTo(checkpoint1GlobalDataStatistics.statistics()); + } } private byte[] waitForCheckpoint( long checkpointId, - DataStatisticsCoordinator> coordinator) + DataStatisticsCoordinator> coordinator) throws InterruptedException, ExecutionException { CompletableFuture future = new CompletableFuture<>(); coordinator.checkpointCoordinator(checkpointId, future); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java index 880cb3d5516f..0e99a2d74ccb 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java @@ -27,7 +27,6 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.state.OperatorStateStore; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.CloseableRegistry; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; @@ -50,33 +49,37 @@ import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.binary.BinaryRowData; import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.VarCharType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; import org.junit.After; import org.junit.Before; import org.junit.Test; public class TestDataStatisticsOperator { - private final RowType rowType = RowType.of(new VarCharType()); + private final Schema schema = + new Schema( + Types.NestedField.optional(1, "id", Types.StringType.get()), + Types.NestedField.optional(2, "number", Types.IntegerType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("id").build(); + private final SortKey sortKey = new SortKey(schema, sortOrder); + private final RowType rowType = RowType.of(new VarCharType(), new IntType()); private final TypeSerializer rowSerializer = new RowDataSerializer(rowType); - private final GenericRowData genericRowDataA = GenericRowData.of(StringData.fromString("a")); - private final GenericRowData genericRowDataB = GenericRowData.of(StringData.fromString("b")); - // When operator hands events from coordinator, DataStatisticsUtil#deserializeDataStatistics - // deserializes bytes into BinaryRowData - private final BinaryRowData binaryRowDataA = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("a"))); - private final BinaryRowData binaryRowDataB = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("b"))); - private final BinaryRowData binaryRowDataC = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("c"))); - private final TypeSerializer>> - statisticsSerializer = MapDataStatisticsSerializer.fromKeySerializer(rowSerializer); - private DataStatisticsOperator> operator; + private final TypeSerializer>> + statisticsSerializer = + MapDataStatisticsSerializer.fromSortKeySerializer( + new SortKeySerializer(schema, sortOrder)); + + private DataStatisticsOperator> operator; private Environment getTestingEnvironment() { return new StreamMockEnvironment( @@ -99,20 +102,10 @@ public void before() throws Exception { new MockOutput<>(Lists.newArrayList())); } - private DataStatisticsOperator> createOperator() { + private DataStatisticsOperator> createOperator() { MockOperatorEventGateway mockGateway = new MockOperatorEventGateway(); - KeySelector keySelector = - new KeySelector() { - private static final long serialVersionUID = 7662520075515707428L; - - @Override - public RowData getKey(RowData value) { - return value; - } - }; - return new DataStatisticsOperator<>( - "testOperator", keySelector, mockGateway, statisticsSerializer); + "testOperator", schema, sortOrder, mockGateway, statisticsSerializer); } @After @@ -123,20 +116,26 @@ public void clean() throws Exception { @Test public void testProcessElement() throws Exception { try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> testHarness = createHarness(this.operator)) { StateInitializationContext stateContext = getStateContext(); operator.initializeState(stateContext); - operator.processElement(new StreamRecord<>(genericRowDataA)); - operator.processElement(new StreamRecord<>(genericRowDataA)); - operator.processElement(new StreamRecord<>(genericRowDataB)); + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 5))); + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 3))); + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 1))); assertThat(operator.localDataStatistics()).isInstanceOf(MapDataStatistics.class); + + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); + Map expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L); + MapDataStatistics mapDataStatistics = (MapDataStatistics) operator.localDataStatistics(); - Map statsMap = mapDataStatistics.statistics(); + Map statsMap = mapDataStatistics.statistics(); assertThat(statsMap).hasSize(2); - assertThat(statsMap) - .containsExactlyInAnyOrderEntriesOf( - ImmutableMap.of(genericRowDataA, 2L, genericRowDataB, 1L)); + assertThat(statsMap).containsExactlyInAnyOrderEntriesOf(expectedMap); + testHarness.endInput(); } } @@ -144,11 +143,14 @@ public void testProcessElement() throws Exception { @Test public void testOperatorOutput() throws Exception { try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> testHarness = createHarness(this.operator)) { - testHarness.processElement(new StreamRecord<>(genericRowDataA)); - testHarness.processElement(new StreamRecord<>(genericRowDataB)); - testHarness.processElement(new StreamRecord<>(genericRowDataB)); + testHarness.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 2))); + testHarness.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 3))); + testHarness.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 1))); List recordsOutput = testHarness.extractOutputValues().stream() @@ -157,7 +159,10 @@ public void testOperatorOutput() throws Exception { .collect(Collectors.toList()); assertThat(recordsOutput) .containsExactlyInAnyOrderElementsOf( - ImmutableList.of(genericRowDataA, genericRowDataB, genericRowDataB)); + ImmutableList.of( + GenericRowData.of(StringData.fromString("a"), 2), + GenericRowData.of(StringData.fromString("b"), 3), + GenericRowData.of(StringData.fromString("b"), 1))); } } @@ -165,36 +170,61 @@ public void testOperatorOutput() throws Exception { public void testRestoreState() throws Exception { OperatorSubtaskState snapshot; try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> testHarness1 = createHarness(this.operator)) { - DataStatistics> mapDataStatistics = - new MapDataStatistics(); - mapDataStatistics.add(binaryRowDataA); - mapDataStatistics.add(binaryRowDataA); - mapDataStatistics.add(binaryRowDataB); - mapDataStatistics.add(binaryRowDataC); - operator.handleOperatorEvent( - DataStatisticsEvent.create(0, mapDataStatistics, statisticsSerializer)); + MapDataStatistics mapDataStatistics = new MapDataStatistics(); + + SortKey key = sortKey.copy(); + key.set(0, "a"); + mapDataStatistics.add(key); + key.set(0, "a"); + mapDataStatistics.add(key); + key.set(0, "b"); + mapDataStatistics.add(key); + key.set(0, "c"); + mapDataStatistics.add(key); + + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); + SortKey keyC = sortKey.copy(); + keyC.set(0, "c"); + Map expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L, keyC, 1L); + + DataStatisticsEvent> event = + DataStatisticsEvent.create(0, mapDataStatistics, statisticsSerializer); + operator.handleOperatorEvent(event); assertThat(operator.globalDataStatistics()).isInstanceOf(MapDataStatistics.class); assertThat(operator.globalDataStatistics().statistics()) - .containsExactlyInAnyOrderEntriesOf( - ImmutableMap.of(binaryRowDataA, 2L, binaryRowDataB, 1L, binaryRowDataC, 1L)); + .containsExactlyInAnyOrderEntriesOf(expectedMap); snapshot = testHarness1.snapshot(1L, 0); } // Use the snapshot to initialize state for another new operator and then verify that the global // statistics for the new operator is same as before - DataStatisticsOperator> restoredOperator = + DataStatisticsOperator> restoredOperator = createOperator(); try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> testHarness2 = new OneInputStreamOperatorTestHarness<>(restoredOperator, 2, 2, 1)) { testHarness2.setup(); testHarness2.initializeState(snapshot); assertThat(restoredOperator.globalDataStatistics()).isInstanceOf(MapDataStatistics.class); - assertThat(restoredOperator.globalDataStatistics().statistics()) - .containsExactlyInAnyOrderEntriesOf( - ImmutableMap.of(binaryRowDataA, 2L, binaryRowDataB, 1L, binaryRowDataC, 1L)); + + // restored RowData is BinaryRowData. convert to GenericRowData for comparison + Map restoredStatistics = Maps.newHashMap(); + restoredStatistics.putAll(restoredOperator.globalDataStatistics().statistics()); + + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); + SortKey keyC = sortKey.copy(); + keyC.set(0, "c"); + Map expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L, keyC, 1L); + + assertThat(restoredStatistics).containsExactlyInAnyOrderEntriesOf(expectedMap); } } @@ -209,18 +239,16 @@ private StateInitializationContext getStateContext() throws Exception { } private OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> createHarness( - final DataStatisticsOperator> + final DataStatisticsOperator> dataStatisticsOperator) throws Exception { OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> harness = new OneInputStreamOperatorTestHarness<>(dataStatisticsOperator, 1, 1, 0); - harness.setup( - new DataStatisticsOrRecordSerializer<>( - MapDataStatisticsSerializer.fromKeySerializer(rowSerializer), rowSerializer)); + harness.setup(new DataStatisticsOrRecordSerializer<>(statisticsSerializer, rowSerializer)); harness.open(); return harness; } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java new file mode 100644 index 000000000000..a07808e935d9 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java @@ -0,0 +1,90 @@ +/* + * 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.iceberg.flink.sink.shuffle; + +import java.util.Map; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +public class TestMapDataStatistics { + private final SortOrder sortOrder = SortOrder.builderFor(TestFixtures.SCHEMA).asc("data").build(); + private final SortKey sortKey = new SortKey(TestFixtures.SCHEMA, sortOrder); + private final RowType rowType = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); + private final RowDataWrapper rowWrapper = + new RowDataWrapper(rowType, TestFixtures.SCHEMA.asStruct()); + + @Test + public void testAddsAndGet() { + MapDataStatistics dataStatistics = new MapDataStatistics(); + + GenericRowData reusedRow = + GenericRowData.of(StringData.fromString("a"), 1, StringData.fromString("2023-06-20")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("b")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("c")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("b")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("a")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("b")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + Map actual = dataStatistics.statistics(); + + rowWrapper.wrap( + GenericRowData.of(StringData.fromString("a"), 1, StringData.fromString("2023-06-20"))); + sortKey.wrap(rowWrapper); + SortKey keyA = sortKey.copy(); + + rowWrapper.wrap( + GenericRowData.of(StringData.fromString("b"), 1, StringData.fromString("2023-06-20"))); + sortKey.wrap(rowWrapper); + SortKey keyB = sortKey.copy(); + + rowWrapper.wrap( + GenericRowData.of(StringData.fromString("c"), 1, StringData.fromString("2023-06-20"))); + sortKey.wrap(rowWrapper); + SortKey keyC = sortKey.copy(); + + Map expected = ImmutableMap.of(keyA, 2L, keyB, 3L, keyC, 1L); + Assertions.assertThat(actual).isEqualTo(expected); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java new file mode 100644 index 000000000000..c7fea015142c --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java @@ -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. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.table.data.GenericRowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; + +public abstract class TestSortKeySerializerBase extends SerializerTestBase { + + protected abstract Schema schema(); + + protected abstract SortOrder sortOrder(); + + protected abstract GenericRowData rowData(); + + @Override + protected TypeSerializer createSerializer() { + return new SortKeySerializer(schema(), sortOrder()); + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return SortKey.class; + } + + @Override + protected SortKey[] getTestData() { + return new SortKey[] {sortKey()}; + } + + private SortKey sortKey() { + RowDataWrapper rowDataWrapper = + new RowDataWrapper(FlinkSchemaUtil.convert(schema()), schema().asStruct()); + SortKey sortKey = new SortKey(schema(), sortOrder()); + sortKey.wrap(rowDataWrapper.wrap(rowData())); + return sortKey; + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java new file mode 100644 index 000000000000..0000688a8b55 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java @@ -0,0 +1,55 @@ +/* + * 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.iceberg.flink.sink.shuffle; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortDirection; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.flink.DataGenerator; +import org.apache.iceberg.flink.DataGenerators; + +public class TestSortKeySerializerNestedStruct extends TestSortKeySerializerBase { + private final DataGenerator generator = new DataGenerators.StructOfStruct(); + + @Override + protected Schema schema() { + return generator.icebergSchema(); + } + + @Override + protected SortOrder sortOrder() { + return SortOrder.builderFor(schema()) + .asc("row_id") + .sortBy( + Expressions.bucket("struct_of_struct.id", 4), SortDirection.DESC, NullOrder.NULLS_LAST) + .sortBy( + Expressions.truncate("struct_of_struct.person_struct.name", 16), + SortDirection.ASC, + NullOrder.NULLS_FIRST) + .build(); + } + + @Override + protected GenericRowData rowData() { + return generator.generateFlinkRowData(); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java new file mode 100644 index 000000000000..291302aef486 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortDirection; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.flink.DataGenerator; +import org.apache.iceberg.flink.DataGenerators; + +public class TestSortKeySerializerPrimitives extends TestSortKeySerializerBase { + private final DataGenerator generator = new DataGenerators.Primitives(); + + @Override + protected Schema schema() { + return generator.icebergSchema(); + } + + @Override + protected SortOrder sortOrder() { + return SortOrder.builderFor(schema()) + .asc("boolean_field") + .sortBy(Expressions.bucket("int_field", 4), SortDirection.DESC, NullOrder.NULLS_LAST) + .sortBy(Expressions.truncate("string_field", 2), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy(Expressions.bucket("uuid_field", 16), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy(Expressions.hour("ts_with_zone_field"), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy(Expressions.day("ts_without_zone_field"), SortDirection.ASC, NullOrder.NULLS_FIRST) + // can not test HeapByteBuffer due to equality test inside SerializerTestBase + // .sortBy(Expressions.truncate("binary_field", 2), SortDirection.ASC, + // NullOrder.NULLS_FIRST) + .build(); + } + + @Override + protected GenericRowData rowData() { + return generator.generateFlinkRowData(); + } +} From 5e03d06d27dfb68dc931197533825f7992a411b7 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 10 Dec 2023 11:01:05 +0100 Subject: [PATCH 32/47] Build: Bump actions/setup-python from 4 to 5 (#9266) Bumps [actions/setup-python](https://github.com/actions/setup-python) from 4 to 5. - [Release notes](https://github.com/actions/setup-python/releases) - [Commits](https://github.com/actions/setup-python/compare/v4...v5) --- updated-dependencies: - dependency-name: actions/setup-python dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/open-api.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/open-api.yml b/.github/workflows/open-api.yml index ae18c646e591..397d1bf30e4a 100644 --- a/.github/workflows/open-api.yml +++ b/.github/workflows/open-api.yml @@ -40,7 +40,7 @@ jobs: steps: - uses: actions/checkout@v4 - - uses: actions/setup-python@v4 + - uses: actions/setup-python@v5 with: python-version: 3.9 - name: Install From 1b80537e821b7bf901a5b221fc3bfe53c8d87794 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 10 Dec 2023 11:01:19 +0100 Subject: [PATCH 33/47] Build: Bump actions/labeler from 4 to 5 (#9264) Bumps [actions/labeler](https://github.com/actions/labeler) from 4 to 5. - [Release notes](https://github.com/actions/labeler/releases) - [Commits](https://github.com/actions/labeler/compare/v4...v5) --- updated-dependencies: - dependency-name: actions/labeler dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/labeler.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/labeler.yml b/.github/workflows/labeler.yml index c4cad7aef2dc..7d6cd5bd2177 100644 --- a/.github/workflows/labeler.yml +++ b/.github/workflows/labeler.yml @@ -28,7 +28,7 @@ jobs: triage: runs-on: ubuntu-22.04 steps: - - uses: actions/labeler@v4 + - uses: actions/labeler@v5 with: repo-token: "${{ secrets.GITHUB_TOKEN }}" sync-labels: true From ec92fa33f780d48aa1f0a146428cd26e1c9bdbcc Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 10 Dec 2023 11:01:33 +0100 Subject: [PATCH 34/47] Build: Bump actions/stale from 8.0.0 to 9.0.0 (#9265) Bumps [actions/stale](https://github.com/actions/stale) from 8.0.0 to 9.0.0. - [Release notes](https://github.com/actions/stale/releases) - [Changelog](https://github.com/actions/stale/blob/main/CHANGELOG.md) - [Commits](https://github.com/actions/stale/compare/v8.0.0...v9.0.0) --- updated-dependencies: - dependency-name: actions/stale dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/stale.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/stale.yml b/.github/workflows/stale.yml index 566ae2441ea0..3c98621c2beb 100644 --- a/.github/workflows/stale.yml +++ b/.github/workflows/stale.yml @@ -31,7 +31,7 @@ jobs: if: github.repository_owner == 'apache' runs-on: ubuntu-22.04 steps: - - uses: actions/stale@v8.0.0 + - uses: actions/stale@v9.0.0 with: stale-issue-label: 'stale' exempt-issue-labels: 'not-stale' From 06894dbc5407b59e07aae9c56ebd904bb5388cf5 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 10 Dec 2023 11:04:13 +0100 Subject: [PATCH 35/47] Build: Bump mkdocs-material from 9.4.12 to 9.5.1 (#9256) Bumps [mkdocs-material](https://github.com/squidfunk/mkdocs-material) from 9.4.12 to 9.5.1. - [Release notes](https://github.com/squidfunk/mkdocs-material/releases) - [Changelog](https://github.com/squidfunk/mkdocs-material/blob/master/CHANGELOG) - [Commits](https://github.com/squidfunk/mkdocs-material/compare/9.4.12...9.5.1) --- updated-dependencies: - dependency-name: mkdocs-material dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- site/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/site/requirements.txt b/site/requirements.txt index 33b14fe4f540..5ae04767048e 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.9.2 mkdocs-macros-plugin==1.0.5 -mkdocs-material==9.4.12 +mkdocs-material==9.5.1 mkdocs-material-extensions==1.3 mkdocs-monorepo-plugin==1.0.5 mkdocs-redirects==1.2.1 From d3deeecd8133e8effc0eb59562808a6b7b7279d3 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 10 Dec 2023 11:04:45 +0100 Subject: [PATCH 36/47] Build: Bump net.snowflake:snowflake-jdbc from 3.14.3 to 3.14.4 (#9257) Bumps [net.snowflake:snowflake-jdbc](https://github.com/snowflakedb/snowflake-jdbc) from 3.14.3 to 3.14.4. - [Release notes](https://github.com/snowflakedb/snowflake-jdbc/releases) - [Changelog](https://github.com/snowflakedb/snowflake-jdbc/blob/master/CHANGELOG.rst) - [Commits](https://github.com/snowflakedb/snowflake-jdbc/compare/v3.14.3...v3.14.4) --- updated-dependencies: - dependency-name: net.snowflake:snowflake-jdbc dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index f4f50754095e..d9f7cd2002e0 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -75,7 +75,7 @@ roaringbitmap = "1.0.0" s3mock-junit5 = "2.11.0" scala-collection-compat = "2.11.0" slf4j = "1.7.36" -snowflake-jdbc = "3.14.3" +snowflake-jdbc = "3.14.4" spark-hive32 = "3.2.2" spark-hive33 = "3.3.3" spark-hive34 = "3.4.2" From ce9186f6bc5dc1fdcce96775898e4b3f9e769ded Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 10 Dec 2023 11:05:03 +0100 Subject: [PATCH 37/47] Build: Bump software.amazon.awssdk:bom from 2.21.29 to 2.21.42 (#9259) Bumps software.amazon.awssdk:bom from 2.21.29 to 2.21.42. --- updated-dependencies: - dependency-name: software.amazon.awssdk:bom dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index d9f7cd2002e0..fc2fc602c178 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -27,7 +27,7 @@ arrow = "14.0.1" avro = "1.11.3" assertj-core = "3.24.2" awaitility = "4.2.0" -awssdk-bom = "2.21.29" +awssdk-bom = "2.21.42" azuresdk-bom = "1.2.18" caffeine = "2.9.3" calcite = "1.10.0" From 0331aba1a3ea5853d320b3244bb096fc9eebdc55 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 10 Dec 2023 11:05:20 +0100 Subject: [PATCH 38/47] Build: Bump com.google.cloud:libraries-bom from 26.27.0 to 26.28.0 (#9258) Bumps [com.google.cloud:libraries-bom](https://github.com/googleapis/java-cloud-bom) from 26.27.0 to 26.28.0. - [Release notes](https://github.com/googleapis/java-cloud-bom/releases) - [Changelog](https://github.com/googleapis/java-cloud-bom/blob/main/release-please-config.json) - [Commits](https://github.com/googleapis/java-cloud-bom/compare/v26.27.0...v26.28.0) --- updated-dependencies: - dependency-name: com.google.cloud:libraries-bom dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index fc2fc602c178..1ddaccb7a174 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -39,7 +39,7 @@ findbugs-jsr305 = "3.0.2" flink116 = { strictly = "[1.16, 1.17[", prefer = "1.16.2"} flink117 = { strictly = "[1.17, 1.18[", prefer = "1.17.1"} flink118 = { strictly = "[1.18, 1.19[", prefer = "1.18.0"} -google-libraries-bom = "26.27.0" +google-libraries-bom = "26.28.0" guava = "32.1.3-jre" hadoop2 = "2.7.3" hadoop3-client = "3.3.6" From 7d06af33ff787c86e0edf14856b6873021e8ea45 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Sun, 10 Dec 2023 22:52:07 +0100 Subject: [PATCH 39/47] Core: Improve view/table detection when replacing a table/view (#9012) --- .../iceberg/rest/RESTSessionCatalog.java | 9 ++++++ .../view/BaseMetastoreViewCatalog.java | 29 +++++++++++++++++++ .../apache/iceberg/view/ViewCatalogTests.java | 13 +++------ 3 files changed, 42 insertions(+), 9 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java index 5a55afbfce22..a2c2b7dd7030 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -55,6 +55,7 @@ import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableCommit; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.AlreadyExistsException; import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.exceptions.NoSuchViewException; @@ -702,6 +703,10 @@ public Transaction createTransaction() { @Override public Transaction replaceTransaction() { + if (viewExists(context, ident)) { + throw new AlreadyExistsException("View with same name already exists: %s", ident); + } + LoadTableResponse response = loadInternal(context, ident, snapshotMode); String fullName = fullTableName(ident); @@ -1170,6 +1175,10 @@ public View createOrReplace() { @Override public View replace() { + if (tableExists(context, identifier)) { + throw new AlreadyExistsException("Table with same name already exists: %s", identifier); + } + return replace(loadView()); } diff --git a/core/src/main/java/org/apache/iceberg/view/BaseMetastoreViewCatalog.java b/core/src/main/java/org/apache/iceberg/view/BaseMetastoreViewCatalog.java index 955bfc2bebf1..6e2d6ff5e864 100644 --- a/core/src/main/java/org/apache/iceberg/view/BaseMetastoreViewCatalog.java +++ b/core/src/main/java/org/apache/iceberg/view/BaseMetastoreViewCatalog.java @@ -23,6 +23,7 @@ import org.apache.iceberg.BaseMetastoreCatalog; import org.apache.iceberg.EnvironmentContext; import org.apache.iceberg.Schema; +import org.apache.iceberg.Transaction; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.catalog.ViewCatalog; @@ -182,6 +183,10 @@ private View create(ViewOperations ops) { } private View replace(ViewOperations ops) { + if (tableExists(identifier)) { + throw new AlreadyExistsException("Table with same name already exists: %s", identifier); + } + if (null == ops.current()) { throw new NoSuchViewException("View does not exist: %s", identifier); } @@ -230,4 +235,28 @@ private View replace(ViewOperations ops) { return new BaseView(ops, ViewUtil.fullViewName(name(), identifier)); } } + + @Override + public TableBuilder buildTable(TableIdentifier identifier, Schema schema) { + return new BaseMetastoreViewCatalogTableBuilder(identifier, schema); + } + + /** The purpose of this class is to add view detection when replacing a table */ + protected class BaseMetastoreViewCatalogTableBuilder extends BaseMetastoreCatalogTableBuilder { + private final TableIdentifier identifier; + + public BaseMetastoreViewCatalogTableBuilder(TableIdentifier identifier, Schema schema) { + super(identifier, schema); + this.identifier = identifier; + } + + @Override + public Transaction replaceTransaction() { + if (viewExists(identifier)) { + throw new AlreadyExistsException("View with same name already exists: %s", identifier); + } + + return super.replaceTransaction(); + } + } } diff --git a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java index 8280a200d605..10d2d7039ce8 100644 --- a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java +++ b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java @@ -36,7 +36,6 @@ import org.apache.iceberg.exceptions.AlreadyExistsException; import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.exceptions.NoSuchNamespaceException; -import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.exceptions.NoSuchViewException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.types.Types; @@ -401,16 +400,14 @@ public void replaceTableViaTransactionThatAlreadyExistsAsView() { assertThat(catalog().viewExists(viewIdentifier)).as("View should exist").isTrue(); - // replace transaction requires table existence - // TODO: replace should check whether the table exists as a view assertThatThrownBy( () -> tableCatalog() .buildTable(viewIdentifier, SCHEMA) .replaceTransaction() .commitTransaction()) - .isInstanceOf(NoSuchTableException.class) - .hasMessageStartingWith("Table does not exist: ns.view"); + .isInstanceOf(AlreadyExistsException.class) + .hasMessageStartingWith("View with same name already exists: ns.view"); } @Test @@ -464,8 +461,6 @@ public void replaceViewThatAlreadyExistsAsTable() { assertThat(tableCatalog().tableExists(tableIdentifier)).as("Table should exist").isTrue(); - // replace view requires the view to exist - // TODO: replace should check whether the view exists as a table assertThatThrownBy( () -> catalog() @@ -474,8 +469,8 @@ public void replaceViewThatAlreadyExistsAsTable() { .withDefaultNamespace(tableIdentifier.namespace()) .withQuery("spark", "select * from ns.tbl") .replace()) - .isInstanceOf(NoSuchViewException.class) - .hasMessageStartingWith("View does not exist: ns.table"); + .isInstanceOf(AlreadyExistsException.class) + .hasMessageContaining("Table with same name already exists: ns.table"); } @Test From 4090a8860061f58748e3faa6804094f90d3575f3 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Sun, 10 Dec 2023 22:57:32 +0100 Subject: [PATCH 40/47] Core: Add REST catalog table session cache (#8920) The purpose of caching auth session for tables is mainly so that we can stop refreshing the session when it isn't used anymore --- .../iceberg/rest/RESTSessionCatalog.java | 56 +++++++++++++++---- .../apache/iceberg/rest/TestRESTCatalog.java | 2 +- 2 files changed, 45 insertions(+), 13 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java index a2c2b7dd7030..5f660f0f4fe8 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -92,6 +92,7 @@ import org.apache.iceberg.rest.responses.OAuthTokenResponse; import org.apache.iceberg.rest.responses.UpdateNamespacePropertiesResponse; import org.apache.iceberg.util.EnvironmentUtil; +import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.ThreadPools; import org.apache.iceberg.view.BaseView; @@ -123,6 +124,7 @@ public class RESTSessionCatalog extends BaseViewSessionCatalog private final Function, RESTClient> clientBuilder; private final BiFunction, FileIO> ioBuilder; private Cache sessions = null; + private Cache tableSessions = null; private Cache fileIOCloser; private AuthSession catalogAuth = null; private boolean keepTokenRefreshed = true; @@ -197,6 +199,7 @@ public void initialize(String name, Map unresolved) { Map baseHeaders = configHeaders(mergedProps); this.sessions = newSessionCache(mergedProps); + this.tableSessions = newSessionCache(mergedProps); this.keepTokenRefreshed = PropertyUtil.propertyAsBoolean( mergedProps, @@ -242,7 +245,15 @@ private AuthSession session(SessionContext context) { AuthSession session = sessions.get( context.sessionId(), - id -> newSession(context.credentials(), context.properties(), catalogAuth)); + id -> { + Pair> newSession = + newSession(context.credentials(), context.properties(), catalogAuth); + if (null != newSession) { + return newSession.second().get(); + } + + return null; + }); return session != null ? session : catalogAuth; } @@ -859,7 +870,12 @@ private FileIO tableFileIO(SessionContext context, Map config) { } private AuthSession tableSession(Map tableConf, AuthSession parent) { - AuthSession session = newSession(tableConf, tableConf, parent); + Pair> newSession = newSession(tableConf, tableConf, parent); + if (null == newSession) { + return parent; + } + + AuthSession session = tableSessions.get(newSession.first(), id -> newSession.second().get()); return session != null ? session : parent; } @@ -889,30 +905,46 @@ private static ConfigResponse fetchConfig( return configResponse; } - private AuthSession newSession( + private Pair> newSession( Map credentials, Map properties, AuthSession parent) { if (credentials != null) { // use the bearer token without exchanging if (credentials.containsKey(OAuth2Properties.TOKEN)) { - return AuthSession.fromAccessToken( - client, - tokenRefreshExecutor(), + return Pair.of( credentials.get(OAuth2Properties.TOKEN), - expiresAtMillis(properties), - parent); + () -> + AuthSession.fromAccessToken( + client, + tokenRefreshExecutor(), + credentials.get(OAuth2Properties.TOKEN), + expiresAtMillis(properties), + parent)); } if (credentials.containsKey(OAuth2Properties.CREDENTIAL)) { // fetch a token using the client credentials flow - return AuthSession.fromCredential( - client, tokenRefreshExecutor(), credentials.get(OAuth2Properties.CREDENTIAL), parent); + return Pair.of( + credentials.get(OAuth2Properties.CREDENTIAL), + () -> + AuthSession.fromCredential( + client, + tokenRefreshExecutor(), + credentials.get(OAuth2Properties.CREDENTIAL), + parent)); } for (String tokenType : TOKEN_PREFERENCE_ORDER) { if (credentials.containsKey(tokenType)) { // exchange the token for an access token using the token exchange flow - return AuthSession.fromTokenExchange( - client, tokenRefreshExecutor(), credentials.get(tokenType), tokenType, parent); + return Pair.of( + credentials.get(tokenType), + () -> + AuthSession.fromTokenExchange( + client, + tokenRefreshExecutor(), + credentials.get(tokenType), + tokenType, + parent)); } } } diff --git a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java index 3de9c7b1d3f7..d0634be16c9f 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java @@ -1162,7 +1162,7 @@ public void testTableAuth( // if the table returned a bearer token, there will be no token request if (!tableConfig.containsKey("token")) { // client credentials or token exchange to get a table token - Mockito.verify(adapter, times(2)) + Mockito.verify(adapter, times(1)) .execute( eq(HTTPMethod.POST), eq("v1/oauth/tokens"), From 61cf766c4301b44fd077ee5487b4f605f54905cb Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 11 Dec 2023 08:49:27 +0100 Subject: [PATCH 41/47] Build: Bump datamodel-code-generator from 0.24.2 to 0.25.1 (#9199) Bumps [datamodel-code-generator](https://github.com/koxudaxi/datamodel-code-generator) from 0.24.2 to 0.25.1. - [Release notes](https://github.com/koxudaxi/datamodel-code-generator/releases) - [Commits](https://github.com/koxudaxi/datamodel-code-generator/compare/0.24.2...0.25.1) --- updated-dependencies: - dependency-name: datamodel-code-generator dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- open-api/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/open-api/requirements.txt b/open-api/requirements.txt index c20e40f4ad2a..c7a3b253147e 100644 --- a/open-api/requirements.txt +++ b/open-api/requirements.txt @@ -16,4 +16,4 @@ # under the License. openapi-spec-validator==0.7.1 -datamodel-code-generator==0.25.0 +datamodel-code-generator==0.25.1 From 1b953050958f7df9d299f00f76465e2336336c9f Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Mon, 11 Dec 2023 13:20:04 +0530 Subject: [PATCH 42/47] Revert "Build: Bump actions/labeler from 4 to 5 (#9264)" (#9271) This reverts commit 1b80537e821b7bf901a5b221fc3bfe53c8d87794. --- .github/workflows/labeler.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/labeler.yml b/.github/workflows/labeler.yml index 7d6cd5bd2177..c4cad7aef2dc 100644 --- a/.github/workflows/labeler.yml +++ b/.github/workflows/labeler.yml @@ -28,7 +28,7 @@ jobs: triage: runs-on: ubuntu-22.04 steps: - - uses: actions/labeler@v5 + - uses: actions/labeler@v4 with: repo-token: "${{ secrets.GITHUB_TOKEN }}" sync-labels: true From 492018902cd044adb8bdc312f01d70a4a63cdcc0 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 11 Dec 2023 09:13:52 +0100 Subject: [PATCH 43/47] Open-API: Refactor updates with discriminator (#9240) * Open-API: Refactor updates with discriminator This generates nicer code * Add missing --- open-api/rest-catalog-open-api.py | 37 ++-- open-api/rest-catalog-open-api.yaml | 321 ++++++++++++++++------------ 2 files changed, 205 insertions(+), 153 deletions(-) diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index 6cd60fe9abd0..cc70d6d4cd89 100644 --- a/open-api/rest-catalog-open-api.py +++ b/open-api/rest-catalog-open-api.py @@ -239,25 +239,7 @@ class ViewVersion(BaseModel): class BaseUpdate(BaseModel): - action: Literal[ - 'assign-uuid', - 'upgrade-format-version', - 'add-schema', - 'set-current-schema', - 'add-spec', - 'set-default-spec', - 'add-sort-order', - 'set-default-sort-order', - 'add-snapshot', - 'set-snapshot-ref', - 'remove-snapshots', - 'remove-snapshot-ref', - 'set-location', - 'set-properties', - 'remove-properties', - 'add-view-version', - 'set-current-view-version', - ] + action: str class AssignUUIDUpdate(BaseUpdate): @@ -265,14 +247,17 @@ class AssignUUIDUpdate(BaseUpdate): Assigning a UUID to a table/view should only be done when creating the table/view. It is not safe to re-assign the UUID if a table/view already has a UUID assigned """ + action: Literal['assign-uuid'] uuid: str class UpgradeFormatVersionUpdate(BaseUpdate): + action: Literal['upgrade-format-version'] format_version: int = Field(..., alias='format-version') class SetCurrentSchemaUpdate(BaseUpdate): + action: Literal['set-current-schema'] schema_id: int = Field( ..., alias='schema-id', @@ -281,10 +266,12 @@ class SetCurrentSchemaUpdate(BaseUpdate): class AddPartitionSpecUpdate(BaseUpdate): + action: Literal['add-spec'] spec: PartitionSpec class SetDefaultSpecUpdate(BaseUpdate): + action: Literal['set-default-spec'] spec_id: int = Field( ..., alias='spec-id', @@ -293,10 +280,12 @@ class SetDefaultSpecUpdate(BaseUpdate): class AddSortOrderUpdate(BaseUpdate): + action: Literal['add-sort-order'] sort_order: SortOrder = Field(..., alias='sort-order') class SetDefaultSortOrderUpdate(BaseUpdate): + action: Literal['set-default-sort-order'] sort_order_id: int = Field( ..., alias='sort-order-id', @@ -305,38 +294,47 @@ class SetDefaultSortOrderUpdate(BaseUpdate): class AddSnapshotUpdate(BaseUpdate): + action: Literal['add-snapshot'] snapshot: Snapshot class SetSnapshotRefUpdate(BaseUpdate, SnapshotReference): + action: Literal['set-snapshot-ref'] ref_name: str = Field(..., alias='ref-name') class RemoveSnapshotsUpdate(BaseUpdate): + action: Literal['remove-snapshots'] snapshot_ids: List[int] = Field(..., alias='snapshot-ids') class RemoveSnapshotRefUpdate(BaseUpdate): + action: Literal['remove-snapshot-ref'] ref_name: str = Field(..., alias='ref-name') class SetLocationUpdate(BaseUpdate): + action: Literal['set-location'] location: str class SetPropertiesUpdate(BaseUpdate): + action: Literal['set-properties'] updates: Dict[str, str] class RemovePropertiesUpdate(BaseUpdate): + action: Literal['remove-properties'] removals: List[str] class AddViewVersionUpdate(BaseUpdate): + action: Literal['add-view-version'] view_version: ViewVersion = Field(..., alias='view-version') class SetCurrentViewVersionUpdate(BaseUpdate): + action: Literal['set-current-view-version'] view_version_id: int = Field( ..., alias='view-version-id', @@ -734,6 +732,7 @@ class ViewMetadata(BaseModel): class AddSchemaUpdate(BaseUpdate): + action: Literal['add-schema'] schema_: Schema = Field(..., alias='schema') last_column_id: Optional[int] = Field( None, diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index a9d30ed02c63..0b008ace605d 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -2130,216 +2130,269 @@ components: type: string BaseUpdate: + discriminator: + propertyName: action + mapping: + assign-uuid: '#/components/schemas/AssignUUIDUpdate' + upgrade-format-version: '#/components/schemas/UpgradeFormatVersionUpdate' + add-schema: '#/components/schemas/AddSchemaUpdate' + set-current-schema: '#/components/schemas/SetCurrentSchemaUpdate' + add-spec: '#/components/schemas/AddPartitionSpecUpdate' + set-default-spec: '#/components/schemas/SetDefaultSpecUpdate' + add-sort-order: '#/components/schemas/AddSortOrderUpdate' + set-default-sort-order: '#/components/schemas/SetDefaultSortOrderUpdate' + add-snapshot: '#/components/schemas/AddSnapshotUpdate' + set-snapshot-ref: '#/components/schemas/SetSnapshotRefUpdate' + remove-snapshots: '#/components/schemas/RemoveSnapshotsUpdate' + remove-snapshot-ref: '#/components/schemas/RemoveSnapshotRefUpdate' + set-location: '#/components/schemas/SetLocationUpdate' + set-properties: '#/components/schemas/SetPropertiesUpdate' + remove-properties: '#/components/schemas/RemovePropertiesUpdate' + add-view-version: '#/components/schemas/AddViewVersionUpdate' + set-current-view-version: '#/components/schemas/SetCurrentViewVersionUpdate' type: object required: - action properties: action: type: string - enum: - - assign-uuid - - upgrade-format-version - - add-schema - - set-current-schema - - add-spec - - set-default-spec - - add-sort-order - - set-default-sort-order - - add-snapshot - - set-snapshot-ref - - remove-snapshots - - remove-snapshot-ref - - set-location - - set-properties - - remove-properties - - add-view-version - - set-current-view-version AssignUUIDUpdate: description: Assigning a UUID to a table/view should only be done when creating the table/view. It is not safe to re-assign the UUID if a table/view already has a UUID assigned allOf: - $ref: '#/components/schemas/BaseUpdate' - - type: object - required: - - uuid - properties: - uuid: - type: string + required: + - action + - uuid + properties: + action: + type: string + enum: ["assign-uuid"] + uuid: + type: string UpgradeFormatVersionUpdate: allOf: - $ref: '#/components/schemas/BaseUpdate' - - type: object - required: - - format-version - properties: - format-version: - type: integer + required: + - action + - format-version + properties: + action: + type: string + enum: ["upgrade-format-version"] + format-version: + type: integer AddSchemaUpdate: allOf: - $ref: '#/components/schemas/BaseUpdate' - - type: object - required: - - schema - properties: - schema: - $ref: '#/components/schemas/Schema' - last-column-id: - type: integer - description: The highest assigned column ID for the table. This is used to ensure columns are always assigned an unused ID when evolving schemas. When omitted, it will be computed on the server side. + required: + - action + - schema + properties: + action: + type: string + enum: ["add-schema"] + schema: + $ref: '#/components/schemas/Schema' + last-column-id: + type: integer + description: The highest assigned column ID for the table. This is used to ensure columns are always assigned an unused ID when evolving schemas. When omitted, it will be computed on the server side. SetCurrentSchemaUpdate: allOf: - $ref: '#/components/schemas/BaseUpdate' - - type: object - required: - - schema-id - properties: - schema-id: - type: integer - description: Schema ID to set as current, or -1 to set last added schema + required: + - action + - schema-id + properties: + action: + type: string + enum: ["set-current-schema"] + schema-id: + type: integer + description: Schema ID to set as current, or -1 to set last added schema AddPartitionSpecUpdate: allOf: - $ref: '#/components/schemas/BaseUpdate' - - type: object - required: - - spec - properties: - spec: - $ref: '#/components/schemas/PartitionSpec' + required: + - action + - spec + properties: + action: + type: string + enum: ["add-spec"] + spec: + $ref: '#/components/schemas/PartitionSpec' SetDefaultSpecUpdate: allOf: - $ref: '#/components/schemas/BaseUpdate' - - type: object - required: - - spec-id - properties: - spec-id: - type: integer - description: Partition spec ID to set as the default, or -1 to set last added spec + required: + - action + - spec-id + properties: + action: + type: string + enum: [ "set-default-spec" ] + spec-id: + type: integer + description: Partition spec ID to set as the default, or -1 to set last added spec AddSortOrderUpdate: allOf: - $ref: '#/components/schemas/BaseUpdate' - - type: object - required: - - sort-order - properties: - sort-order: - $ref: '#/components/schemas/SortOrder' + required: + - action + - sort-order + properties: + action: + type: string + enum: [ "add-sort-order" ] + sort-order: + $ref: '#/components/schemas/SortOrder' SetDefaultSortOrderUpdate: allOf: - $ref: '#/components/schemas/BaseUpdate' - - type: object - required: - - sort-order-id - properties: - sort-order-id: - type: integer - description: Sort order ID to set as the default, or -1 to set last added sort order + required: + - action + - sort-order-id + properties: + action: + type: string + enum: [ "set-default-sort-order" ] + sort-order-id: + type: integer + description: Sort order ID to set as the default, or -1 to set last added sort order AddSnapshotUpdate: allOf: - $ref: '#/components/schemas/BaseUpdate' - - type: object - required: - - snapshot - properties: - snapshot: - $ref: '#/components/schemas/Snapshot' + required: + - action + - snapshot + properties: + action: + type: string + enum: [ "add-snapshot" ] + snapshot: + $ref: '#/components/schemas/Snapshot' SetSnapshotRefUpdate: allOf: - $ref: '#/components/schemas/BaseUpdate' - $ref: '#/components/schemas/SnapshotReference' - - type: object - required: - - ref-name - properties: - ref-name: - type: string + required: + - action + - ref-name + properties: + action: + type: string + enum: [ "set-snapshot-ref" ] + ref-name: + type: string RemoveSnapshotsUpdate: allOf: - $ref: '#/components/schemas/BaseUpdate' - - type: object - required: - - snapshot-ids - properties: - snapshot-ids: - type: array - items: - type: integer - format: int64 + required: + - action + - snapshot-ids + properties: + action: + type: string + enum: [ "remove-snapshots" ] + snapshot-ids: + type: array + items: + type: integer + format: int64 RemoveSnapshotRefUpdate: allOf: - $ref: '#/components/schemas/BaseUpdate' - - type: object - required: - - ref-name - properties: - ref-name: - type: string + required: + - action + - ref-name + properties: + action: + type: string + enum: [ "remove-snapshot-ref" ] + ref-name: + type: string SetLocationUpdate: allOf: - $ref: '#/components/schemas/BaseUpdate' - - type: object - required: - - location - properties: - location: - type: string + required: + - action + - location + properties: + action: + type: string + enum: [ "set-location" ] + location: + type: string SetPropertiesUpdate: allOf: - $ref: '#/components/schemas/BaseUpdate' - - type: object - required: - - updates - properties: - updates: - type: object - additionalProperties: - type: string + required: + - action + - updates + properties: + action: + type: string + enum: [ "set-properties" ] + updates: + type: object + additionalProperties: + type: string RemovePropertiesUpdate: allOf: - $ref: '#/components/schemas/BaseUpdate' - - type: object - required: - - removals - properties: - removals: - type: array - items: - type: string + required: + - action + - removals + properties: + action: + type: string + enum: [ "remove-properties" ] + removals: + type: array + items: + type: string AddViewVersionUpdate: allOf: - $ref: '#/components/schemas/BaseUpdate' - - type: object - required: - - view-version - properties: - view-version: - $ref: '#/components/schemas/ViewVersion' + required: + - action + - view-version + properties: + action: + type: string + enum: [ "add-view-version" ] + view-version: + $ref: '#/components/schemas/ViewVersion' SetCurrentViewVersionUpdate: allOf: - $ref: '#/components/schemas/BaseUpdate' - - type: object - required: - - view-version-id - properties: - view-version-id: - type: integer - description: The view version id to set as current, or -1 to set last added view version id + required: + - action + - view-version-id + properties: + action: + type: string + enum: [ "set-current-view-version" ] + view-version-id: + type: integer + description: The view version id to set as current, or -1 to set last added view version id TableUpdate: anyOf: From f21199d002c6ad008a6772783b4fdad622c09b59 Mon Sep 17 00:00:00 2001 From: L S Chetan Rao <48798234+lschetanrao@users.noreply.github.com> Date: Mon, 11 Dec 2023 14:20:47 +0530 Subject: [PATCH 44/47] MR: Migrate tests to JUnit5 (#9241) --- mr/build.gradle | 3 + .../org/apache/iceberg/mr/TestCatalogs.java | 95 ++++----- .../iceberg/mr/hive/HiveIcebergTestUtils.java | 21 +- .../iceberg/mr/hive/TestDeserializer.java | 23 +-- .../mr/hive/TestHiveIcebergFilterFactory.java | 52 ++--- .../hive/TestHiveIcebergOutputCommitter.java | 43 ++-- .../iceberg/mr/hive/TestHiveIcebergSerDe.java | 18 +- .../TestIcebergBinaryObjectInspector.java | 44 +++-- .../TestIcebergDateObjectInspector.java | 34 ++-- .../TestIcebergDecimalObjectInspector.java | 48 ++--- .../TestIcebergFixedObjectInspector.java | 39 ++-- .../TestIcebergObjectInspector.java | 184 +++++++++--------- .../TestIcebergRecordObjectInspector.java | 22 +-- .../TestIcebergTimeObjectInspector.java | 39 ++-- .../TestIcebergTimestampObjectInspector.java | 39 ++-- ...ebergTimestampWithZoneObjectInspector.java | 48 ++--- .../TestIcebergUUIDObjectInspector.java | 39 ++-- 17 files changed, 400 insertions(+), 391 deletions(-) diff --git a/mr/build.gradle b/mr/build.gradle index 848f6dee3c43..37e665d6d2e6 100644 --- a/mr/build.gradle +++ b/mr/build.gradle @@ -23,6 +23,9 @@ project(':iceberg-mr') { exclude group: 'org.apache.parquet', module: 'parquet-hadoop-bundle' } } + test { + useJUnitPlatform() + } dependencies { implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow') diff --git a/mr/src/test/java/org/apache/iceberg/mr/TestCatalogs.java b/mr/src/test/java/org/apache/iceberg/mr/TestCatalogs.java index f849163acc6a..012ad2350224 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/TestCatalogs.java +++ b/mr/src/test/java/org/apache/iceberg/mr/TestCatalogs.java @@ -20,8 +20,10 @@ import static org.apache.iceberg.types.Types.NestedField.required; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; +import java.nio.file.Path; import java.util.Optional; import java.util.Properties; import org.apache.hadoop.conf.Configuration; @@ -39,12 +41,9 @@ import org.apache.iceberg.hadoop.HadoopTables; import org.apache.iceberg.hive.HiveCatalog; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestCatalogs { @@ -54,9 +53,9 @@ public class TestCatalogs { private Configuration conf; - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; - @Before + @BeforeEach public void before() { conf = new Configuration(); } @@ -65,25 +64,25 @@ public void before() { public void testLoadTableFromLocation() throws IOException { conf.set(CatalogUtil.ICEBERG_CATALOG_TYPE, Catalogs.LOCATION); - Assertions.assertThatThrownBy(() -> Catalogs.loadTable(conf)) + assertThatThrownBy(() -> Catalogs.loadTable(conf)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Table location not set"); HadoopTables tables = new HadoopTables(); - Table hadoopTable = tables.create(SCHEMA, temp.newFolder("hadoop_tables").toString()); + Table hadoopTable = tables.create(SCHEMA, temp.resolve("hadoop_tables").toString()); conf.set(InputFormatConfig.TABLE_LOCATION, hadoopTable.location()); - Assert.assertEquals(hadoopTable.location(), Catalogs.loadTable(conf).location()); + assertThat(Catalogs.loadTable(conf).location()).isEqualTo(hadoopTable.location()); } @Test public void testLoadTableFromCatalog() throws IOException { String defaultCatalogName = "default"; - String warehouseLocation = temp.newFolder("hadoop", "warehouse").toString(); + String warehouseLocation = temp.resolve("hadoop").resolve("warehouse").toString(); setCustomCatalogProperties(defaultCatalogName, warehouseLocation); - Assertions.assertThatThrownBy(() -> Catalogs.loadTable(conf)) + assertThatThrownBy(() -> Catalogs.loadTable(conf)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Table identifier not set"); @@ -92,15 +91,15 @@ public void testLoadTableFromCatalog() throws IOException { conf.set(InputFormatConfig.TABLE_IDENTIFIER, "table"); - Assert.assertEquals(hadoopCatalogTable.location(), Catalogs.loadTable(conf).location()); + assertThat(Catalogs.loadTable(conf).location()).isEqualTo(hadoopCatalogTable.location()); } @Test public void testCreateDropTableToLocation() throws IOException { Properties missingSchema = new Properties(); - missingSchema.put("location", temp.newFolder("hadoop_tables").toString()); + missingSchema.put("location", temp.resolve("hadoop_tables").toString()); - Assertions.assertThatThrownBy(() -> Catalogs.createTable(conf, missingSchema)) + assertThatThrownBy(() -> Catalogs.createTable(conf, missingSchema)) .isInstanceOf(NullPointerException.class) .hasMessage("Table schema not set"); @@ -108,12 +107,12 @@ public void testCreateDropTableToLocation() throws IOException { Properties missingLocation = new Properties(); missingLocation.put(InputFormatConfig.TABLE_SCHEMA, SchemaParser.toJson(SCHEMA)); - Assertions.assertThatThrownBy(() -> Catalogs.createTable(conf, missingLocation)) + assertThatThrownBy(() -> Catalogs.createTable(conf, missingLocation)) .isInstanceOf(NullPointerException.class) .hasMessage("Table location not set"); Properties properties = new Properties(); - properties.put("location", temp.getRoot() + "/hadoop_tables"); + properties.put("location", temp.toFile() + "/hadoop_tables"); properties.put(InputFormatConfig.TABLE_SCHEMA, SchemaParser.toJson(SCHEMA)); properties.put(InputFormatConfig.PARTITION_SPEC, PartitionSpecParser.toJson(SPEC)); properties.put("dummy", "test"); @@ -123,20 +122,21 @@ public void testCreateDropTableToLocation() throws IOException { HadoopTables tables = new HadoopTables(); Table table = tables.load(properties.getProperty("location")); - Assert.assertEquals(properties.getProperty("location"), table.location()); - Assert.assertEquals(SchemaParser.toJson(SCHEMA), SchemaParser.toJson(table.schema())); - Assert.assertEquals(PartitionSpecParser.toJson(SPEC), PartitionSpecParser.toJson(table.spec())); + assertThat(table.location()).isEqualTo(properties.getProperty("location")); + assertThat(SchemaParser.toJson(table.schema())).isEqualTo(SchemaParser.toJson(SCHEMA)); + assertThat(PartitionSpecParser.toJson(table.spec())) + .isEqualTo(PartitionSpecParser.toJson(SPEC)); assertThat(table.properties()).containsEntry("dummy", "test"); - Assertions.assertThatThrownBy(() -> Catalogs.dropTable(conf, new Properties())) + assertThatThrownBy(() -> Catalogs.dropTable(conf, new Properties())) .isInstanceOf(NullPointerException.class) .hasMessage("Table location not set"); Properties dropProperties = new Properties(); - dropProperties.put("location", temp.getRoot() + "/hadoop_tables"); + dropProperties.put("location", temp.toFile() + "/hadoop_tables"); Catalogs.dropTable(conf, dropProperties); - Assertions.assertThatThrownBy(() -> Catalogs.loadTable(conf, dropProperties)) + assertThatThrownBy(() -> Catalogs.loadTable(conf, dropProperties)) .isInstanceOf(NoSuchTableException.class) .hasMessage("Table does not exist at location: " + properties.getProperty("location")); } @@ -145,7 +145,7 @@ public void testCreateDropTableToLocation() throws IOException { public void testCreateDropTableToCatalog() throws IOException { TableIdentifier identifier = TableIdentifier.of("test", "table"); String defaultCatalogName = "default"; - String warehouseLocation = temp.newFolder("hadoop", "warehouse").toString(); + String warehouseLocation = temp.resolve("hadoop").resolve("warehouse").toString(); setCustomCatalogProperties(defaultCatalogName, warehouseLocation); @@ -153,14 +153,14 @@ public void testCreateDropTableToCatalog() throws IOException { missingSchema.put("name", identifier.toString()); missingSchema.put(InputFormatConfig.CATALOG_NAME, defaultCatalogName); - Assertions.assertThatThrownBy(() -> Catalogs.createTable(conf, missingSchema)) + assertThatThrownBy(() -> Catalogs.createTable(conf, missingSchema)) .isInstanceOf(NullPointerException.class) .hasMessage("Table schema not set"); Properties missingIdentifier = new Properties(); missingIdentifier.put(InputFormatConfig.TABLE_SCHEMA, SchemaParser.toJson(SCHEMA)); missingIdentifier.put(InputFormatConfig.CATALOG_NAME, defaultCatalogName); - Assertions.assertThatThrownBy(() -> Catalogs.createTable(conf, missingIdentifier)) + assertThatThrownBy(() -> Catalogs.createTable(conf, missingIdentifier)) .isInstanceOf(NullPointerException.class) .hasMessage("Table identifier not set"); @@ -176,11 +176,12 @@ public void testCreateDropTableToCatalog() throws IOException { HadoopCatalog catalog = new CustomHadoopCatalog(conf, warehouseLocation); Table table = catalog.loadTable(identifier); - Assert.assertEquals(SchemaParser.toJson(SCHEMA), SchemaParser.toJson(table.schema())); - Assert.assertEquals(PartitionSpecParser.toJson(SPEC), PartitionSpecParser.toJson(table.spec())); + assertThat(SchemaParser.toJson(table.schema())).isEqualTo(SchemaParser.toJson(SCHEMA)); + assertThat(PartitionSpecParser.toJson(table.spec())) + .isEqualTo(PartitionSpecParser.toJson(SPEC)); assertThat(table.properties()).containsEntry("dummy", "test"); - Assertions.assertThatThrownBy(() -> Catalogs.dropTable(conf, new Properties())) + assertThatThrownBy(() -> Catalogs.dropTable(conf, new Properties())) .isInstanceOf(NullPointerException.class) .hasMessage("Table identifier not set"); @@ -189,7 +190,7 @@ public void testCreateDropTableToCatalog() throws IOException { dropProperties.put(InputFormatConfig.CATALOG_NAME, defaultCatalogName); Catalogs.dropTable(conf, dropProperties); - Assertions.assertThatThrownBy(() -> Catalogs.loadTable(conf, dropProperties)) + assertThatThrownBy(() -> Catalogs.loadTable(conf, dropProperties)) .isInstanceOf(NoSuchTableException.class) .hasMessage("Table does not exist: test.table"); } @@ -198,11 +199,11 @@ public void testCreateDropTableToCatalog() throws IOException { public void testLoadCatalogDefault() { String catalogName = "barCatalog"; Optional defaultCatalog = Catalogs.loadCatalog(conf, catalogName); - Assert.assertTrue(defaultCatalog.isPresent()); - Assertions.assertThat(defaultCatalog.get()).isInstanceOf(HiveCatalog.class); + assertThat(defaultCatalog).isPresent(); + assertThat(defaultCatalog.get()).isInstanceOf(HiveCatalog.class); Properties properties = new Properties(); properties.put(InputFormatConfig.CATALOG_NAME, catalogName); - Assert.assertTrue(Catalogs.hiveCatalog(conf, properties)); + assertThat(Catalogs.hiveCatalog(conf, properties)).isTrue(); } @Test @@ -212,11 +213,11 @@ public void testLoadCatalogHive() { InputFormatConfig.catalogPropertyConfigKey(catalogName, CatalogUtil.ICEBERG_CATALOG_TYPE), CatalogUtil.ICEBERG_CATALOG_TYPE_HIVE); Optional hiveCatalog = Catalogs.loadCatalog(conf, catalogName); - Assert.assertTrue(hiveCatalog.isPresent()); - Assertions.assertThat(hiveCatalog.get()).isInstanceOf(HiveCatalog.class); + assertThat(hiveCatalog).isPresent(); + assertThat(hiveCatalog.get()).isInstanceOf(HiveCatalog.class); Properties properties = new Properties(); properties.put(InputFormatConfig.CATALOG_NAME, catalogName); - Assert.assertTrue(Catalogs.hiveCatalog(conf, properties)); + assertThat(Catalogs.hiveCatalog(conf, properties)).isTrue(); } @Test @@ -230,13 +231,13 @@ public void testLoadCatalogHadoop() { catalogName, CatalogProperties.WAREHOUSE_LOCATION), "/tmp/mylocation"); Optional hadoopCatalog = Catalogs.loadCatalog(conf, catalogName); - Assert.assertTrue(hadoopCatalog.isPresent()); - Assertions.assertThat(hadoopCatalog.get()).isInstanceOf(HadoopCatalog.class); - Assert.assertEquals( - "HadoopCatalog{name=barCatalog, location=/tmp/mylocation}", hadoopCatalog.get().toString()); + assertThat(hadoopCatalog).isPresent(); + assertThat(hadoopCatalog.get()).isInstanceOf(HadoopCatalog.class); + assertThat(hadoopCatalog.get().toString()) + .isEqualTo("HadoopCatalog{name=barCatalog, location=/tmp/mylocation}"); Properties properties = new Properties(); properties.put(InputFormatConfig.CATALOG_NAME, catalogName); - Assert.assertFalse(Catalogs.hiveCatalog(conf, properties)); + assertThat(Catalogs.hiveCatalog(conf, properties)).isFalse(); } @Test @@ -250,16 +251,16 @@ public void testLoadCatalogCustom() { catalogName, CatalogProperties.WAREHOUSE_LOCATION), "/tmp/mylocation"); Optional customHadoopCatalog = Catalogs.loadCatalog(conf, catalogName); - Assert.assertTrue(customHadoopCatalog.isPresent()); - Assertions.assertThat(customHadoopCatalog.get()).isInstanceOf(CustomHadoopCatalog.class); + assertThat(customHadoopCatalog).isPresent(); + assertThat(customHadoopCatalog.get()).isInstanceOf(CustomHadoopCatalog.class); Properties properties = new Properties(); properties.put(InputFormatConfig.CATALOG_NAME, catalogName); - Assert.assertFalse(Catalogs.hiveCatalog(conf, properties)); + assertThat(Catalogs.hiveCatalog(conf, properties)).isFalse(); } @Test public void testLoadCatalogLocation() { - Assert.assertFalse(Catalogs.loadCatalog(conf, Catalogs.ICEBERG_HADOOP_TABLE_NAME).isPresent()); + assertThat(Catalogs.loadCatalog(conf, Catalogs.ICEBERG_HADOOP_TABLE_NAME)).isNotPresent(); } @Test @@ -269,7 +270,7 @@ public void testLoadCatalogUnknown() { InputFormatConfig.catalogPropertyConfigKey(catalogName, CatalogUtil.ICEBERG_CATALOG_TYPE), "fooType"); - Assertions.assertThatThrownBy(() -> Catalogs.loadCatalog(conf, catalogName)) + assertThatThrownBy(() -> Catalogs.loadCatalog(conf, catalogName)) .isInstanceOf(UnsupportedOperationException.class) .hasMessage("Unknown catalog type: fooType"); } diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergTestUtils.java b/mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergTestUtils.java index de5189f7c3ad..43a61d83e434 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergTestUtils.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergTestUtils.java @@ -19,6 +19,7 @@ package org.apache.iceberg.mr.hive; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; @@ -63,7 +64,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.ByteBuffers; -import org.junit.Assert; public class HiveIcebergTestUtils { // TODO: Can this be a constant all around the Iceberg tests? @@ -218,13 +218,10 @@ public static void assertEquals(Record expected, Record actual) { for (int i = 0; i < expected.size(); ++i) { if (expected.get(i) instanceof OffsetDateTime) { // For OffsetDateTime we just compare the actual instant - Assert.assertEquals( - ((OffsetDateTime) expected.get(i)).toInstant(), - ((OffsetDateTime) actual.get(i)).toInstant()); - } else if (expected.get(i) instanceof byte[]) { - Assert.assertArrayEquals((byte[]) expected.get(i), (byte[]) actual.get(i)); + assertThat(((OffsetDateTime) actual.get(i)).toInstant()) + .isEqualTo(((OffsetDateTime) expected.get(i)).toInstant()); } else { - Assert.assertEquals(expected.get(i), actual.get(i)); + assertThat(actual.get(i)).isEqualTo(expected.get(i)); } } } @@ -265,7 +262,7 @@ public static void validateData(List expected, List actual, int sortedExpected.sort(Comparator.comparingLong(record -> (Long) record.get(sortBy))); sortedActual.sort(Comparator.comparingLong(record -> (Long) record.get(sortBy))); - Assert.assertEquals(sortedExpected.size(), sortedActual.size()); + assertThat(sortedActual).hasSameSizeAs(sortedExpected); for (int i = 0; i < sortedExpected.size(); ++i) { assertEquals(sortedExpected.get(i), sortedActual.get(i)); } @@ -288,9 +285,9 @@ public static void validateFiles(Table table, Configuration conf, JobID jobId, i .filter(path -> !path.getFileName().toString().startsWith(".")) .collect(Collectors.toList()); - Assert.assertEquals(dataFileNum, dataFiles.size()); - Assert.assertFalse( - new File(HiveIcebergOutputCommitter.generateJobLocation(table.location(), conf, jobId)) - .exists()); + assertThat(dataFiles).hasSize(dataFileNum); + assertThat( + new File(HiveIcebergOutputCommitter.generateJobLocation(table.location(), conf, jobId))) + .doesNotExist(); } } diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestDeserializer.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestDeserializer.java index 4a5d819279f2..8f58a36d6265 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/TestDeserializer.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestDeserializer.java @@ -19,6 +19,8 @@ package org.apache.iceberg.mr.hive; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; import java.util.Arrays; import java.util.Collections; @@ -35,9 +37,7 @@ import org.apache.iceberg.hive.HiveVersion; import org.apache.iceberg.mr.hive.serde.objectinspector.IcebergObjectInspector; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestDeserializer { private static final Schema CUSTOMER_SCHEMA = @@ -74,7 +74,7 @@ public void testSchemaDeserialize() { Record actual = deserializer.deserialize(new Object[] {new LongWritable(1L), new Text("Bob")}); - Assert.assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); } @Test @@ -92,7 +92,7 @@ public void testStructDeserialize() { Record actual = deserializer.deserialize(new Object[] {new LongWritable(1L), new Text("Bob")}); - Assert.assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); } @Test @@ -127,7 +127,7 @@ public void testMapDeserialize() { Object[] data = new Object[] {map}; Record actual = deserializer.deserialize(data); - Assert.assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); } @Test @@ -155,13 +155,14 @@ public void testListDeserialize() { Object[] data = new Object[] {new Object[] {new LongWritable(1L)}}; Record actual = deserializer.deserialize(data); - Assert.assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); } @Test public void testDeserializeEverySupportedType() { - Assume.assumeFalse( - "No test yet for Hive3 (Date/Timestamp creation)", HiveVersion.min(HiveVersion.HIVE_3)); + assumeThat(HiveVersion.min(HiveVersion.HIVE_3)) + .as("No test yet for Hive3 (Date/Timestamp creation)") + .isFalse(); Deserializer deserializer = new Deserializer.Builder() @@ -196,9 +197,9 @@ public void testNullDeserialize() { Record actual = deserializer.deserialize(nulls); - Assert.assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); // Check null record as well - Assert.assertNull(deserializer.deserialize(null)); + assertThat(deserializer.deserialize(null)).isNull(); } } diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergFilterFactory.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergFilterFactory.java index 05fe0fdea5ae..121e2c8b6d8a 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergFilterFactory.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergFilterFactory.java @@ -18,7 +18,7 @@ */ package org.apache.iceberg.mr.hive; -import static org.junit.Assert.assertEquals; +import static org.assertj.core.api.Assertions.assertThat; import java.math.BigDecimal; import java.sql.Date; @@ -40,7 +40,7 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.util.DateTimeUtil; import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestHiveIcebergFilterFactory { @@ -82,10 +82,10 @@ public void testNotEqualsOperand() { UnboundPredicate childExpressionActual = (UnboundPredicate) actual.child(); UnboundPredicate childExpressionExpected = Expressions.equal("salary", 3000L); - assertEquals(actual.op(), expected.op()); - assertEquals(actual.child().op(), expected.child().op()); - assertEquals(childExpressionActual.ref().name(), childExpressionExpected.ref().name()); - assertEquals(childExpressionActual.literal(), childExpressionExpected.literal()); + assertThat(expected.op()).isEqualTo(actual.op()); + assertThat(expected.child().op()).isEqualTo(actual.child().op()); + assertThat(childExpressionExpected.ref().name()).isEqualTo(childExpressionActual.ref().name()); + assertThat(childExpressionExpected.literal()).isEqualTo(childExpressionActual.literal()); } @Test @@ -98,9 +98,9 @@ public void testLessThanOperand() { UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg); - assertEquals(actual.op(), expected.op()); - assertEquals(actual.literal(), expected.literal()); - assertEquals(actual.ref().name(), expected.ref().name()); + assertThat(expected.op()).isEqualTo(actual.op()); + assertThat(expected.literal()).isEqualTo(actual.literal()); + assertThat(expected.ref().name()).isEqualTo(actual.ref().name()); } @Test @@ -126,9 +126,9 @@ public void testInOperand() { UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg); - assertEquals(actual.op(), expected.op()); - assertEquals(actual.literals(), expected.literals()); - assertEquals(actual.ref().name(), expected.ref().name()); + assertThat(expected.op()).isEqualTo(actual.op()); + assertThat(expected.literals()).isEqualTo(actual.literals()); + assertThat(expected.ref().name()).isEqualTo(actual.ref().name()); } @Test @@ -144,9 +144,9 @@ public void testBetweenOperand() { Expressions.lessThanOrEqual("salary", 3000L)); And actual = (And) HiveIcebergFilterFactory.generateFilterExpression(arg); - assertEquals(actual.op(), expected.op()); - assertEquals(actual.left().op(), expected.left().op()); - assertEquals(actual.right().op(), expected.right().op()); + assertThat(expected.op()).isEqualTo(actual.op()); + assertThat(expected.left().op()).isEqualTo(actual.left().op()); + assertThat(expected.right().op()).isEqualTo(actual.right().op()); } @Test @@ -173,8 +173,8 @@ public void testIsNullOperand() { UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg); - assertEquals(actual.op(), expected.op()); - assertEquals(actual.ref().name(), expected.ref().name()); + assertThat(expected.op()).isEqualTo(actual.op()); + assertThat(expected.ref().name()).isEqualTo(actual.ref().name()); } @Test @@ -193,9 +193,9 @@ public void testAndOperand() { Expressions.and(Expressions.equal("salary", 3000L), Expressions.equal("salary", 4000L)); And actual = (And) HiveIcebergFilterFactory.generateFilterExpression(arg); - assertEquals(actual.op(), expected.op()); - assertEquals(actual.left().op(), expected.left().op()); - assertEquals(actual.right().op(), expected.right().op()); + assertThat(expected.op()).isEqualTo(actual.op()); + assertThat(expected.left().op()).isEqualTo(actual.left().op()); + assertThat(expected.right().op()).isEqualTo(actual.right().op()); } @Test @@ -213,9 +213,9 @@ public void testOrOperand() { (Or) Expressions.or(Expressions.equal("salary", 3000L), Expressions.equal("salary", 4000L)); Or actual = (Or) HiveIcebergFilterFactory.generateFilterExpression(arg); - assertEquals(actual.op(), expected.op()); - assertEquals(actual.left().op(), expected.left().op()); - assertEquals(actual.right().op(), expected.right().op()); + assertThat(expected.op()).isEqualTo(actual.op()); + assertThat(expected.left().op()).isEqualTo(actual.left().op()); + assertThat(expected.right().op()).isEqualTo(actual.right().op()); } @Test @@ -308,9 +308,9 @@ public void testDecimalType() { } private void assertPredicatesMatch(UnboundPredicate expected, UnboundPredicate actual) { - assertEquals(expected.op(), actual.op()); - assertEquals(expected.literal(), actual.literal()); - assertEquals(expected.ref().name(), actual.ref().name()); + assertThat(actual.op()).isEqualTo(expected.op()); + assertThat(actual.literal()).isEqualTo(expected.literal()); + assertThat(actual.ref().name()).isEqualTo(expected.ref().name()); } private static class MockSearchArgument implements SearchArgument { diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergOutputCommitter.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergOutputCommitter.java index 0d9b5a0c1d5e..8b8e209144fa 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergOutputCommitter.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergOutputCommitter.java @@ -20,8 +20,10 @@ import static org.apache.iceberg.mr.hive.HiveIcebergRecordWriter.getWriters; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; import java.io.IOException; +import java.nio.file.Path; import java.util.Collections; import java.util.List; import java.util.Map; @@ -55,10 +57,8 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.util.SerializationUtil; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; import org.mockito.ArgumentCaptor; import org.mockito.Mockito; @@ -80,7 +80,7 @@ public class TestHiveIcebergOutputCommitter { private static final PartitionSpec PARTITIONED_SPEC = PartitionSpec.builderFor(CUSTOMER_SCHEMA).bucket("customer_id", 3).build(); - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; @Test public void testNeedsTaskCommit() { @@ -91,24 +91,25 @@ public void testNeedsTaskCommit() { mapOnlyJobConf.setNumReduceTasks(0); // Map only job should commit map tasks - Assert.assertTrue( - committer.needsTaskCommit(new TaskAttemptContextImpl(mapOnlyJobConf, MAP_TASK_ID))); + assertThat(committer.needsTaskCommit(new TaskAttemptContextImpl(mapOnlyJobConf, MAP_TASK_ID))) + .isTrue(); JobConf mapReduceJobConf = new JobConf(); mapReduceJobConf.setNumMapTasks(10); mapReduceJobConf.setNumReduceTasks(10); // MapReduce job should not commit map tasks, but should commit reduce tasks - Assert.assertFalse( - committer.needsTaskCommit(new TaskAttemptContextImpl(mapReduceJobConf, MAP_TASK_ID))); - Assert.assertTrue( - committer.needsTaskCommit(new TaskAttemptContextImpl(mapReduceJobConf, REDUCE_TASK_ID))); + assertThat(committer.needsTaskCommit(new TaskAttemptContextImpl(mapReduceJobConf, MAP_TASK_ID))) + .isFalse(); + assertThat( + committer.needsTaskCommit(new TaskAttemptContextImpl(mapReduceJobConf, REDUCE_TASK_ID))) + .isTrue(); } @Test public void testSuccessfulUnpartitionedWrite() throws IOException { HiveIcebergOutputCommitter committer = new HiveIcebergOutputCommitter(); - Table table = table(temp.getRoot().getPath(), false); + Table table = table(temp.toFile().getPath(), false); JobConf conf = jobConf(table, 1); List expected = writeRecords(table.name(), 1, 0, true, false, conf); @@ -121,7 +122,7 @@ public void testSuccessfulUnpartitionedWrite() throws IOException { @Test public void testSuccessfulPartitionedWrite() throws IOException { HiveIcebergOutputCommitter committer = new HiveIcebergOutputCommitter(); - Table table = table(temp.getRoot().getPath(), true); + Table table = table(temp.toFile().getPath(), true); JobConf conf = jobConf(table, 1); List expected = writeRecords(table.name(), 1, 0, true, false, conf); committer.commitJob(new JobContextImpl(conf, JOB_ID)); @@ -133,7 +134,7 @@ public void testSuccessfulPartitionedWrite() throws IOException { @Test public void testSuccessfulMultipleTasksUnpartitionedWrite() throws IOException { HiveIcebergOutputCommitter committer = new HiveIcebergOutputCommitter(); - Table table = table(temp.getRoot().getPath(), false); + Table table = table(temp.toFile().getPath(), false); JobConf conf = jobConf(table, 2); List expected = writeRecords(table.name(), 2, 0, true, false, conf); committer.commitJob(new JobContextImpl(conf, JOB_ID)); @@ -145,7 +146,7 @@ public void testSuccessfulMultipleTasksUnpartitionedWrite() throws IOException { @Test public void testSuccessfulMultipleTasksPartitionedWrite() throws IOException { HiveIcebergOutputCommitter committer = new HiveIcebergOutputCommitter(); - Table table = table(temp.getRoot().getPath(), true); + Table table = table(temp.toFile().getPath(), true); JobConf conf = jobConf(table, 2); List expected = writeRecords(table.name(), 2, 0, true, false, conf); committer.commitJob(new JobContextImpl(conf, JOB_ID)); @@ -157,7 +158,7 @@ public void testSuccessfulMultipleTasksPartitionedWrite() throws IOException { @Test public void testRetryTask() throws IOException { HiveIcebergOutputCommitter committer = new HiveIcebergOutputCommitter(); - Table table = table(temp.getRoot().getPath(), false); + Table table = table(temp.toFile().getPath(), false); JobConf conf = jobConf(table, 2); // Write records and abort the tasks @@ -181,7 +182,7 @@ public void testRetryTask() throws IOException { @Test public void testAbortJob() throws IOException { HiveIcebergOutputCommitter committer = new HiveIcebergOutputCommitter(); - Table table = table(temp.getRoot().getPath(), false); + Table table = table(temp.toFile().getPath(), false); JobConf conf = jobConf(table, 1); writeRecords(table.name(), 1, 0, true, false, conf); committer.abortJob(new JobContextImpl(conf, JOB_ID), JobStatus.State.FAILED); @@ -201,7 +202,7 @@ public void writerIsClosedAfterTaskCommitFailure() throws IOException { .when(failingCommitter) .commitTask(argumentCaptor.capture()); - Table table = table(temp.getRoot().getPath(), false); + Table table = table(temp.toFile().getPath(), false); JobConf conf = jobConf(table, 1); Assertions.assertThatThrownBy( @@ -209,14 +210,14 @@ public void writerIsClosedAfterTaskCommitFailure() throws IOException { .isInstanceOf(RuntimeException.class) .hasMessage(exceptionMessage); - Assert.assertEquals(1, argumentCaptor.getAllValues().size()); + assertThat(argumentCaptor.getAllValues()).hasSize(1); TaskAttemptID capturedId = TezUtil.taskAttemptWrapper(argumentCaptor.getValue().getTaskAttemptID()); // writer is still in the map after commitTask failure - Assert.assertNotNull(getWriters(capturedId)); + assertThat(getWriters(capturedId)).isNotNull(); failingCommitter.abortTask(new TaskAttemptContextImpl(conf, capturedId)); // abortTask succeeds and removes writer - Assert.assertNull(getWriters(capturedId)); + assertThat(getWriters(capturedId)).isNull(); } private Table table(String location, boolean partitioned) { diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSerDe.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSerDe.java index 889c441c28fc..919230a9fbd9 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSerDe.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSerDe.java @@ -19,9 +19,11 @@ package org.apache.iceberg.mr.hive; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; +import java.nio.file.Path; import java.util.Properties; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.serde2.SerDeException; @@ -34,22 +36,20 @@ import org.apache.iceberg.mr.hive.serde.objectinspector.IcebergObjectInspector; import org.apache.iceberg.mr.mapred.Container; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestHiveIcebergSerDe { private static final Schema schema = new Schema(required(1, "string_field", Types.StringType.get())); - @Rule public TemporaryFolder tmp = new TemporaryFolder(); + @TempDir private Path tmp; @Test public void testInitialize() throws IOException, SerDeException { - File location = tmp.newFolder(); - Assert.assertTrue(location.delete()); + File location = tmp.toFile(); + assertThat(location.delete()).isTrue(); Configuration conf = new Configuration(); @@ -63,7 +63,7 @@ public void testInitialize() throws IOException, SerDeException { HiveIcebergSerDe serDe = new HiveIcebergSerDe(); serDe.initialize(conf, properties); - Assert.assertEquals(IcebergObjectInspector.create(schema), serDe.getObjectInspector()); + assertThat(serDe.getObjectInspector()).isEqualTo(IcebergObjectInspector.create(schema)); } @Test @@ -74,6 +74,6 @@ public void testDeserialize() { Container container = new Container<>(); container.set(record); - Assert.assertEquals(record, serDe.deserialize(container)); + assertThat(serDe.deserialize(container)).isEqualTo(record); } } diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergBinaryObjectInspector.java b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergBinaryObjectInspector.java index 5db84e5aa4b9..87fe4208c964 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergBinaryObjectInspector.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergBinaryObjectInspector.java @@ -18,14 +18,15 @@ */ package org.apache.iceberg.mr.hive.serde.objectinspector; +import static org.assertj.core.api.Assertions.assertThat; + import java.nio.ByteBuffer; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.BinaryObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; import org.apache.hadoop.io.BytesWritable; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestIcebergBinaryObjectInspector { @@ -33,39 +34,40 @@ public class TestIcebergBinaryObjectInspector { public void testIcebergByteBufferObjectInspector() { BinaryObjectInspector oi = IcebergBinaryObjectInspector.get(); - Assert.assertEquals(ObjectInspector.Category.PRIMITIVE, oi.getCategory()); - Assert.assertEquals( - PrimitiveObjectInspector.PrimitiveCategory.BINARY, oi.getPrimitiveCategory()); + assertThat(oi.getCategory()).isEqualTo(ObjectInspector.Category.PRIMITIVE); + assertThat(oi.getPrimitiveCategory()) + .isEqualTo(PrimitiveObjectInspector.PrimitiveCategory.BINARY); - Assert.assertEquals(TypeInfoFactory.binaryTypeInfo, oi.getTypeInfo()); - Assert.assertEquals(TypeInfoFactory.binaryTypeInfo.getTypeName(), oi.getTypeName()); + assertThat(oi.getTypeInfo()).isEqualTo(TypeInfoFactory.binaryTypeInfo); + assertThat(oi.getTypeName()).isEqualTo(TypeInfoFactory.binaryTypeInfo.getTypeName()); - Assert.assertEquals(byte[].class, oi.getJavaPrimitiveClass()); - Assert.assertEquals(BytesWritable.class, oi.getPrimitiveWritableClass()); + assertThat(oi.getJavaPrimitiveClass()).isEqualTo(byte[].class); + assertThat(oi.getPrimitiveWritableClass()).isEqualTo(BytesWritable.class); - Assert.assertNull(oi.copyObject(null)); - Assert.assertNull(oi.getPrimitiveJavaObject(null)); - Assert.assertNull(oi.getPrimitiveWritableObject(null)); + assertThat(oi.copyObject(null)).isNull(); + assertThat(oi.getPrimitiveJavaObject(null)).isNull(); + assertThat(oi.getPrimitiveWritableObject(null)).isNull(); byte[] bytes = new byte[] {0, 1, 2, 3}; ByteBuffer buffer = ByteBuffer.wrap(bytes); - Assert.assertArrayEquals(bytes, oi.getPrimitiveJavaObject(buffer)); - Assert.assertEquals(new BytesWritable(bytes), oi.getPrimitiveWritableObject(buffer)); + assertThat(oi.getPrimitiveJavaObject(buffer)).isEqualTo(bytes); + assertThat(oi.getPrimitiveWritableObject(buffer)).isEqualTo(new BytesWritable(bytes)); ByteBuffer slice = ByteBuffer.wrap(bytes, 1, 2).slice(); - Assert.assertArrayEquals(new byte[] {1, 2}, oi.getPrimitiveJavaObject(slice)); - Assert.assertEquals(new BytesWritable(new byte[] {1, 2}), oi.getPrimitiveWritableObject(slice)); + assertThat(oi.getPrimitiveJavaObject(slice)).isEqualTo(new byte[] {1, 2}); + assertThat(oi.getPrimitiveWritableObject(slice)) + .isEqualTo(new BytesWritable(new byte[] {1, 2})); slice.position(1); - Assert.assertArrayEquals(new byte[] {2}, oi.getPrimitiveJavaObject(slice)); - Assert.assertEquals(new BytesWritable(new byte[] {2}), oi.getPrimitiveWritableObject(slice)); + assertThat(oi.getPrimitiveJavaObject(slice)).isEqualTo(new byte[] {2}); + assertThat(oi.getPrimitiveWritableObject(slice)).isEqualTo(new BytesWritable(new byte[] {2})); byte[] copy = (byte[]) oi.copyObject(bytes); - Assert.assertArrayEquals(bytes, copy); - Assert.assertNotSame(bytes, copy); + assertThat(copy).isEqualTo(bytes); + assertThat(copy).isNotSameAs(bytes); - Assert.assertFalse(oi.preferWritable()); + assertThat(oi.preferWritable()).isFalse(); } } diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergDateObjectInspector.java b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergDateObjectInspector.java index 73681fec5799..6e03fae861f9 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergDateObjectInspector.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergDateObjectInspector.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.mr.hive.serde.objectinspector; +import static org.assertj.core.api.Assertions.assertThat; + import java.sql.Date; import java.time.LocalDate; import org.apache.hadoop.hive.serde2.io.DateWritable; @@ -25,8 +27,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.DateObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestIcebergDateObjectInspector { @@ -34,30 +35,31 @@ public class TestIcebergDateObjectInspector { public void testIcebergDateObjectInspector() { DateObjectInspector oi = IcebergDateObjectInspector.get(); - Assert.assertEquals(ObjectInspector.Category.PRIMITIVE, oi.getCategory()); - Assert.assertEquals(PrimitiveObjectInspector.PrimitiveCategory.DATE, oi.getPrimitiveCategory()); + assertThat(oi.getCategory()).isEqualTo(ObjectInspector.Category.PRIMITIVE); + assertThat(oi.getPrimitiveCategory()) + .isEqualTo(PrimitiveObjectInspector.PrimitiveCategory.DATE); - Assert.assertEquals(TypeInfoFactory.dateTypeInfo, oi.getTypeInfo()); - Assert.assertEquals(TypeInfoFactory.dateTypeInfo.getTypeName(), oi.getTypeName()); + assertThat(oi.getTypeInfo()).isEqualTo(TypeInfoFactory.dateTypeInfo); + assertThat(oi.getTypeName()).isEqualTo(TypeInfoFactory.dateTypeInfo.getTypeName()); - Assert.assertEquals(Date.class, oi.getJavaPrimitiveClass()); - Assert.assertEquals(DateWritable.class, oi.getPrimitiveWritableClass()); + assertThat(oi.getJavaPrimitiveClass()).isEqualTo(Date.class); + assertThat(oi.getPrimitiveWritableClass()).isEqualTo(DateWritable.class); - Assert.assertNull(oi.copyObject(null)); - Assert.assertNull(oi.getPrimitiveJavaObject(null)); - Assert.assertNull(oi.getPrimitiveWritableObject(null)); + assertThat(oi.copyObject(null)).isNull(); + assertThat(oi.getPrimitiveJavaObject(null)).isNull(); + assertThat(oi.getPrimitiveWritableObject(null)).isNull(); LocalDate local = LocalDate.of(2020, 1, 1); Date date = Date.valueOf("2020-01-01"); - Assert.assertEquals(date, oi.getPrimitiveJavaObject(local)); - Assert.assertEquals(new DateWritable(date), oi.getPrimitiveWritableObject(local)); + assertThat(oi.getPrimitiveJavaObject(local)).isEqualTo(date); + assertThat(oi.getPrimitiveWritableObject(local)).isEqualTo(new DateWritable(date)); Date copy = (Date) oi.copyObject(date); - Assert.assertEquals(date, copy); - Assert.assertNotSame(date, copy); + assertThat(copy).isEqualTo(date); + assertThat(copy).isNotSameAs(date); - Assert.assertFalse(oi.preferWritable()); + assertThat(oi.preferWritable()).isFalse(); } } diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergDecimalObjectInspector.java b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergDecimalObjectInspector.java index 1c4734c77f4c..58d43e3d7047 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergDecimalObjectInspector.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergDecimalObjectInspector.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.mr.hive.serde.objectinspector; +import static org.assertj.core.api.Assertions.assertThat; + import java.math.BigDecimal; import org.apache.hadoop.hive.common.type.HiveDecimal; import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; @@ -26,8 +28,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveDecimalObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestIcebergDecimalObjectInspector { @@ -35,43 +36,44 @@ public class TestIcebergDecimalObjectInspector { public void testCache() { HiveDecimalObjectInspector oi = IcebergDecimalObjectInspector.get(38, 18); - Assert.assertSame(oi, IcebergDecimalObjectInspector.get(38, 18)); - Assert.assertNotSame(oi, IcebergDecimalObjectInspector.get(28, 18)); - Assert.assertNotSame(oi, IcebergDecimalObjectInspector.get(38, 28)); + assertThat(IcebergDecimalObjectInspector.get(38, 18)).isSameAs(oi); + assertThat(IcebergDecimalObjectInspector.get(28, 18)).isNotSameAs(oi); + assertThat(IcebergDecimalObjectInspector.get(38, 28)).isNotSameAs(oi); } @Test public void testIcebergDecimalObjectInspector() { HiveDecimalObjectInspector oi = IcebergDecimalObjectInspector.get(38, 18); - Assert.assertEquals(ObjectInspector.Category.PRIMITIVE, oi.getCategory()); - Assert.assertEquals( - PrimitiveObjectInspector.PrimitiveCategory.DECIMAL, oi.getPrimitiveCategory()); + assertThat(oi.getCategory()).isEqualTo(ObjectInspector.Category.PRIMITIVE); + assertThat(oi.getPrimitiveCategory()) + .isEqualTo(PrimitiveObjectInspector.PrimitiveCategory.DECIMAL); - Assert.assertEquals(new DecimalTypeInfo(38, 18), oi.getTypeInfo()); - Assert.assertEquals(TypeInfoFactory.decimalTypeInfo.getTypeName(), oi.getTypeName()); + assertThat(oi.getTypeInfo()).isEqualTo(new DecimalTypeInfo(38, 18)); + assertThat(oi.getTypeName()) + .isEqualTo(TypeInfoFactory.decimalTypeInfo.getTypeName(), oi.getTypeName()); - Assert.assertEquals(38, oi.precision()); - Assert.assertEquals(18, oi.scale()); + assertThat(oi.precision()).isEqualTo(38); + assertThat(oi.scale()).isEqualTo(18); - Assert.assertEquals(HiveDecimal.class, oi.getJavaPrimitiveClass()); - Assert.assertEquals(HiveDecimalWritable.class, oi.getPrimitiveWritableClass()); + assertThat(oi.getJavaPrimitiveClass()).isEqualTo(HiveDecimal.class); + assertThat(oi.getPrimitiveWritableClass()).isEqualTo(HiveDecimalWritable.class); - Assert.assertNull(oi.copyObject(null)); - Assert.assertNull(oi.getPrimitiveJavaObject(null)); - Assert.assertNull(oi.getPrimitiveWritableObject(null)); + assertThat(oi.copyObject(null)).isNull(); + assertThat(oi.getPrimitiveJavaObject(null)).isNull(); + assertThat(oi.getPrimitiveWritableObject(null)).isNull(); HiveDecimal one = HiveDecimal.create(BigDecimal.ONE); - Assert.assertEquals(one, oi.getPrimitiveJavaObject(BigDecimal.ONE)); - Assert.assertEquals( - new HiveDecimalWritable(one), oi.getPrimitiveWritableObject(BigDecimal.ONE)); + assertThat(oi.getPrimitiveJavaObject(BigDecimal.ONE)).isEqualTo(one); + assertThat(oi.getPrimitiveWritableObject(BigDecimal.ONE)) + .isEqualTo(new HiveDecimalWritable(one)); HiveDecimal copy = (HiveDecimal) oi.copyObject(one); - Assert.assertEquals(one, copy); - Assert.assertNotSame(one, copy); + assertThat(copy).isEqualTo(one); + assertThat(copy).isNotSameAs(one); - Assert.assertFalse(oi.preferWritable()); + assertThat(oi.preferWritable()).isFalse(); } } diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergFixedObjectInspector.java b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergFixedObjectInspector.java index a902d027269d..7a8450f6852f 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergFixedObjectInspector.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergFixedObjectInspector.java @@ -18,12 +18,13 @@ */ package org.apache.iceberg.mr.hive.serde.objectinspector; +import static org.assertj.core.api.Assertions.assertThat; + import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; import org.apache.hadoop.io.BytesWritable; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestIcebergFixedObjectInspector { @@ -31,33 +32,33 @@ public class TestIcebergFixedObjectInspector { public void testIcebergFixedObjectInspector() { IcebergFixedObjectInspector oi = IcebergFixedObjectInspector.get(); - Assert.assertEquals(ObjectInspector.Category.PRIMITIVE, oi.getCategory()); - Assert.assertEquals( - PrimitiveObjectInspector.PrimitiveCategory.BINARY, oi.getPrimitiveCategory()); + assertThat(oi.getCategory()).isEqualTo(ObjectInspector.Category.PRIMITIVE); + assertThat(oi.getPrimitiveCategory()) + .isEqualTo(PrimitiveObjectInspector.PrimitiveCategory.BINARY); - Assert.assertEquals(TypeInfoFactory.binaryTypeInfo, oi.getTypeInfo()); - Assert.assertEquals(TypeInfoFactory.binaryTypeInfo.getTypeName(), oi.getTypeName()); + assertThat(oi.getTypeInfo()).isEqualTo(TypeInfoFactory.binaryTypeInfo); + assertThat(oi.getTypeName()).isEqualTo(TypeInfoFactory.binaryTypeInfo.getTypeName()); - Assert.assertEquals(byte[].class, oi.getJavaPrimitiveClass()); - Assert.assertEquals(BytesWritable.class, oi.getPrimitiveWritableClass()); + assertThat(oi.getJavaPrimitiveClass()).isEqualTo(byte[].class); + assertThat(oi.getPrimitiveWritableClass()).isEqualTo(BytesWritable.class); - Assert.assertNull(oi.copyObject(null)); - Assert.assertNull(oi.getPrimitiveJavaObject(null)); - Assert.assertNull(oi.getPrimitiveWritableObject(null)); - Assert.assertNull(oi.convert(null)); + assertThat(oi.copyObject(null)).isNull(); + assertThat(oi.getPrimitiveJavaObject(null)).isNull(); + assertThat(oi.getPrimitiveWritableObject(null)).isNull(); + assertThat(oi.convert(null)).isNull(); byte[] bytes = new byte[] {0, 1}; BytesWritable bytesWritable = new BytesWritable(bytes); - Assert.assertArrayEquals(bytes, oi.getPrimitiveJavaObject(bytes)); - Assert.assertEquals(bytesWritable, oi.getPrimitiveWritableObject(bytes)); - Assert.assertEquals(bytes, oi.convert(bytes)); + assertThat(oi.getPrimitiveJavaObject(bytes)).isEqualTo(bytes); + assertThat(oi.getPrimitiveWritableObject(bytes)).isEqualTo(bytesWritable); + assertThat(oi.convert(bytes)).isEqualTo(bytes); byte[] copy = (byte[]) oi.copyObject(bytes); - Assert.assertArrayEquals(bytes, copy); - Assert.assertNotSame(bytes, copy); + assertThat(copy).isEqualTo(bytes); + assertThat(copy).isNotSameAs(bytes); - Assert.assertFalse(oi.preferWritable()); + assertThat(oi.preferWritable()).isFalse(); } } diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergObjectInspector.java b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergObjectInspector.java index c4d3cea4b002..c2646376890c 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergObjectInspector.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergObjectInspector.java @@ -19,6 +19,7 @@ package org.apache.iceberg.mr.hive.serde.objectinspector; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; @@ -31,8 +32,7 @@ import org.apache.iceberg.hive.HiveVersion; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestIcebergObjectInspector { @@ -74,165 +74,161 @@ public class TestIcebergObjectInspector { @Test public void testIcebergObjectInspector() { ObjectInspector oi = IcebergObjectInspector.create(schema); - Assert.assertNotNull(oi); - Assert.assertEquals(ObjectInspector.Category.STRUCT, oi.getCategory()); + assertThat(oi).isNotNull(); + assertThat(oi.getCategory()).isEqualTo(ObjectInspector.Category.STRUCT); StructObjectInspector soi = (StructObjectInspector) oi; // binary StructField binaryField = soi.getStructFieldRef("binary_field"); - Assert.assertEquals(1, binaryField.getFieldID()); - Assert.assertEquals("binary_field", binaryField.getFieldName()); - Assert.assertEquals("binary comment", binaryField.getFieldComment()); - Assert.assertEquals(IcebergBinaryObjectInspector.get(), binaryField.getFieldObjectInspector()); + assertThat(binaryField.getFieldID()).isEqualTo(1); + assertThat(binaryField.getFieldName()).isEqualTo("binary_field"); + assertThat(binaryField.getFieldComment()).isEqualTo("binary comment"); + assertThat(binaryField.getFieldObjectInspector()).isEqualTo(IcebergBinaryObjectInspector.get()); // boolean StructField booleanField = soi.getStructFieldRef("boolean_field"); - Assert.assertEquals(2, booleanField.getFieldID()); - Assert.assertEquals("boolean_field", booleanField.getFieldName()); - Assert.assertEquals("boolean comment", booleanField.getFieldComment()); - Assert.assertEquals( - getPrimitiveObjectInspector(boolean.class), booleanField.getFieldObjectInspector()); + assertThat(booleanField.getFieldID()).isEqualTo(2); + assertThat(booleanField.getFieldName()).isEqualTo("boolean_field"); + assertThat(booleanField.getFieldComment()).isEqualTo("boolean comment"); + assertThat(booleanField.getFieldObjectInspector()) + .isEqualTo(getPrimitiveObjectInspector(boolean.class)); // date StructField dateField = soi.getStructFieldRef("date_field"); - Assert.assertEquals(3, dateField.getFieldID()); - Assert.assertEquals("date_field", dateField.getFieldName()); - Assert.assertEquals("date comment", dateField.getFieldComment()); + assertThat(dateField.getFieldID()).isEqualTo(3); + assertThat(dateField.getFieldName()).isEqualTo("date_field"); + assertThat(dateField.getFieldComment()).isEqualTo("date comment"); if (HiveVersion.min(HiveVersion.HIVE_3)) { - Assert.assertEquals( - "org.apache.iceberg.mr.hive.serde.objectinspector.IcebergDateObjectInspectorHive3", - dateField.getFieldObjectInspector().getClass().getName()); + assertThat(dateField.getFieldObjectInspector().getClass().getName()) + .isEqualTo( + "org.apache.iceberg.mr.hive.serde.objectinspector.IcebergDateObjectInspectorHive3"); } else { - Assert.assertEquals( - "org.apache.iceberg.mr.hive.serde.objectinspector.IcebergDateObjectInspector", - dateField.getFieldObjectInspector().getClass().getName()); + assertThat(dateField.getFieldObjectInspector().getClass().getName()) + .isEqualTo("org.apache.iceberg.mr.hive.serde.objectinspector.IcebergDateObjectInspector"); } // decimal StructField decimalField = soi.getStructFieldRef("decimal_field"); - Assert.assertEquals(4, decimalField.getFieldID()); - Assert.assertEquals("decimal_field", decimalField.getFieldName()); - Assert.assertEquals("decimal comment", decimalField.getFieldComment()); - Assert.assertEquals( - IcebergDecimalObjectInspector.get(38, 18), decimalField.getFieldObjectInspector()); + assertThat(decimalField.getFieldID()).isEqualTo(4); + assertThat(decimalField.getFieldName()).isEqualTo("decimal_field"); + assertThat(decimalField.getFieldComment()).isEqualTo("decimal comment"); + assertThat(decimalField.getFieldObjectInspector()) + .isEqualTo(IcebergDecimalObjectInspector.get(38, 18)); // double StructField doubleField = soi.getStructFieldRef("double_field"); - Assert.assertEquals(5, doubleField.getFieldID()); - Assert.assertEquals("double_field", doubleField.getFieldName()); - Assert.assertEquals("double comment", doubleField.getFieldComment()); - Assert.assertEquals( - getPrimitiveObjectInspector(double.class), doubleField.getFieldObjectInspector()); + assertThat(doubleField.getFieldID()).isEqualTo(5); + assertThat(doubleField.getFieldName()).isEqualTo("double_field"); + assertThat(doubleField.getFieldComment()).isEqualTo("double comment"); + assertThat(doubleField.getFieldObjectInspector()) + .isEqualTo(getPrimitiveObjectInspector(double.class)); // fixed StructField fixedField = soi.getStructFieldRef("fixed_field"); - Assert.assertEquals(6, fixedField.getFieldID()); - Assert.assertEquals("fixed_field", fixedField.getFieldName()); - Assert.assertEquals("fixed comment", fixedField.getFieldComment()); - Assert.assertEquals(IcebergFixedObjectInspector.get(), fixedField.getFieldObjectInspector()); + assertThat(fixedField.getFieldID()).isEqualTo(6); + assertThat(fixedField.getFieldName()).isEqualTo("fixed_field"); + assertThat(fixedField.getFieldComment()).isEqualTo("fixed comment"); + assertThat(fixedField.getFieldObjectInspector()).isEqualTo(IcebergFixedObjectInspector.get()); // float StructField floatField = soi.getStructFieldRef("float_field"); - Assert.assertEquals(7, floatField.getFieldID()); - Assert.assertEquals("float_field", floatField.getFieldName()); - Assert.assertEquals("float comment", floatField.getFieldComment()); - Assert.assertEquals( - getPrimitiveObjectInspector(float.class), floatField.getFieldObjectInspector()); + assertThat(floatField.getFieldID()).isEqualTo(7); + assertThat(floatField.getFieldName()).isEqualTo("float_field"); + assertThat(floatField.getFieldComment()).isEqualTo("float comment"); + assertThat(floatField.getFieldObjectInspector()) + .isEqualTo(getPrimitiveObjectInspector(float.class)); // integer StructField integerField = soi.getStructFieldRef("integer_field"); - Assert.assertEquals(8, integerField.getFieldID()); - Assert.assertEquals("integer_field", integerField.getFieldName()); - Assert.assertEquals("integer comment", integerField.getFieldComment()); - Assert.assertEquals( - getPrimitiveObjectInspector(int.class), integerField.getFieldObjectInspector()); + assertThat(integerField.getFieldID()).isEqualTo(8); + assertThat(integerField.getFieldName()).isEqualTo("integer_field"); + assertThat(integerField.getFieldComment()).isEqualTo("integer comment"); + assertThat(integerField.getFieldObjectInspector()) + .isEqualTo(getPrimitiveObjectInspector(int.class)); // long StructField longField = soi.getStructFieldRef("long_field"); - Assert.assertEquals(9, longField.getFieldID()); - Assert.assertEquals("long_field", longField.getFieldName()); - Assert.assertEquals("long comment", longField.getFieldComment()); - Assert.assertEquals( - getPrimitiveObjectInspector(long.class), longField.getFieldObjectInspector()); + assertThat(longField.getFieldID()).isEqualTo(9); + assertThat(longField.getFieldName()).isEqualTo("long_field"); + assertThat(longField.getFieldComment()).isEqualTo("long comment"); + assertThat(longField.getFieldObjectInspector()) + .isEqualTo(getPrimitiveObjectInspector(long.class)); // string StructField stringField = soi.getStructFieldRef("string_field"); - Assert.assertEquals(10, stringField.getFieldID()); - Assert.assertEquals("string_field", stringField.getFieldName()); - Assert.assertEquals("string comment", stringField.getFieldComment()); - Assert.assertEquals( - getPrimitiveObjectInspector(String.class), stringField.getFieldObjectInspector()); + assertThat(stringField.getFieldID()).isEqualTo(10); + assertThat(stringField.getFieldName()).isEqualTo("string_field"); + assertThat(stringField.getFieldComment()).isEqualTo("string comment"); + assertThat(stringField.getFieldObjectInspector()) + .isEqualTo(getPrimitiveObjectInspector(String.class)); // timestamp without tz StructField timestampField = soi.getStructFieldRef("timestamp_field"); - Assert.assertEquals(11, timestampField.getFieldID()); - Assert.assertEquals("timestamp_field", timestampField.getFieldName()); - Assert.assertEquals("timestamp comment", timestampField.getFieldComment()); + assertThat(timestampField.getFieldID()).isEqualTo(11); + assertThat(timestampField.getFieldName()).isEqualTo("timestamp_field"); + assertThat(timestampField.getFieldComment()).isEqualTo("timestamp comment"); if (HiveVersion.min(HiveVersion.HIVE_3)) { - Assert.assertEquals( - "IcebergTimestampObjectInspectorHive3", - timestampField.getFieldObjectInspector().getClass().getSimpleName()); + assertThat(timestampField.getFieldObjectInspector().getClass().getSimpleName()) + .isEqualTo("IcebergTimestampObjectInspectorHive3"); } else { - Assert.assertEquals( - IcebergTimestampObjectInspector.get(), timestampField.getFieldObjectInspector()); + assertThat(timestampField.getFieldObjectInspector()) + .isEqualTo(IcebergTimestampObjectInspector.get()); } // timestamp with tz StructField timestampTzField = soi.getStructFieldRef("timestamptz_field"); - Assert.assertEquals(12, timestampTzField.getFieldID()); - Assert.assertEquals("timestamptz_field", timestampTzField.getFieldName()); - Assert.assertEquals("timestamptz comment", timestampTzField.getFieldComment()); + assertThat(timestampTzField.getFieldID()).isEqualTo(12); + assertThat(timestampTzField.getFieldName()).isEqualTo("timestamptz_field"); + assertThat(timestampTzField.getFieldComment()).isEqualTo("timestamptz comment"); if (HiveVersion.min(HiveVersion.HIVE_3)) { - Assert.assertEquals( - "IcebergTimestampWithZoneObjectInspectorHive3", - timestampTzField.getFieldObjectInspector().getClass().getSimpleName()); + assertThat(timestampTzField.getFieldObjectInspector().getClass().getSimpleName()) + .isEqualTo("IcebergTimestampWithZoneObjectInspectorHive3"); } else { - Assert.assertEquals( - IcebergTimestampWithZoneObjectInspector.get(), - timestampTzField.getFieldObjectInspector()); + assertThat(timestampTzField.getFieldObjectInspector()) + .isEqualTo(IcebergTimestampWithZoneObjectInspector.get()); } // UUID StructField uuidField = soi.getStructFieldRef("uuid_field"); - Assert.assertEquals(13, uuidField.getFieldID()); - Assert.assertEquals("uuid_field", uuidField.getFieldName()); - Assert.assertEquals("uuid comment", uuidField.getFieldComment()); - Assert.assertEquals(IcebergUUIDObjectInspector.get(), uuidField.getFieldObjectInspector()); + assertThat(uuidField.getFieldID()).isEqualTo(13); + assertThat(uuidField.getFieldName()).isEqualTo("uuid_field"); + assertThat(uuidField.getFieldComment()).isEqualTo("uuid comment"); + assertThat(uuidField.getFieldObjectInspector()).isEqualTo(IcebergUUIDObjectInspector.get()); // list StructField listField = soi.getStructFieldRef("list_field"); - Assert.assertEquals(14, listField.getFieldID()); - Assert.assertEquals("list_field", listField.getFieldName()); - Assert.assertEquals("list comment", listField.getFieldComment()); - Assert.assertEquals(getListObjectInspector(String.class), listField.getFieldObjectInspector()); + assertThat(listField.getFieldID()).isEqualTo(14); + assertThat(listField.getFieldName()).isEqualTo("list_field"); + assertThat(listField.getFieldComment()).isEqualTo("list comment"); + assertThat(listField.getFieldObjectInspector()).isEqualTo(getListObjectInspector(String.class)); // map StructField mapField = soi.getStructFieldRef("map_field"); - Assert.assertEquals(16, mapField.getFieldID()); - Assert.assertEquals("map_field", mapField.getFieldName()); - Assert.assertEquals("map comment", mapField.getFieldComment()); - Assert.assertEquals( - getMapObjectInspector(String.class, int.class), mapField.getFieldObjectInspector()); + assertThat(mapField.getFieldID()).isEqualTo(16); + assertThat(mapField.getFieldName()).isEqualTo("map_field"); + assertThat(mapField.getFieldComment()).isEqualTo("map comment"); + assertThat(mapField.getFieldObjectInspector()) + .isEqualTo(getMapObjectInspector(String.class, int.class)); // struct StructField structField = soi.getStructFieldRef("struct_field"); - Assert.assertEquals(19, structField.getFieldID()); - Assert.assertEquals("struct_field", structField.getFieldName()); - Assert.assertEquals("struct comment", structField.getFieldComment()); + assertThat(structField.getFieldID()).isEqualTo(19); + assertThat(structField.getFieldName()).isEqualTo("struct_field"); + assertThat(structField.getFieldComment()).isEqualTo("struct comment"); ObjectInspector expectedObjectInspector = new IcebergRecordObjectInspector( (Types.StructType) schema.findType(19), ImmutableList.of(getPrimitiveObjectInspector(String.class))); - Assert.assertEquals(expectedObjectInspector, structField.getFieldObjectInspector()); + assertThat(structField.getFieldObjectInspector()).isEqualTo(expectedObjectInspector); // time StructField timeField = soi.getStructFieldRef("time_field"); - Assert.assertEquals(21, timeField.getFieldID()); - Assert.assertEquals("time_field", timeField.getFieldName()); - Assert.assertEquals("time comment", timeField.getFieldComment()); - Assert.assertEquals(IcebergTimeObjectInspector.get(), timeField.getFieldObjectInspector()); + assertThat(timeField.getFieldID()).isEqualTo(21); + assertThat(timeField.getFieldName()).isEqualTo("time_field"); + assertThat(timeField.getFieldComment()).isEqualTo("time comment"); + assertThat(timeField.getFieldObjectInspector()).isEqualTo(IcebergTimeObjectInspector.get()); } private static ObjectInspector getPrimitiveObjectInspector(Class clazz) { diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergRecordObjectInspector.java b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergRecordObjectInspector.java index 4ed358c116fb..d5824f8bd7d8 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergRecordObjectInspector.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergRecordObjectInspector.java @@ -19,6 +19,7 @@ package org.apache.iceberg.mr.hive.serde.objectinspector; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; import org.apache.hadoop.hive.serde2.objectinspector.StructField; import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; @@ -27,8 +28,7 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestIcebergRecordObjectInspector { @@ -47,22 +47,22 @@ public void testIcebergRecordObjectInspector() { Record innerRecord = record.get(1, Record.class); StructObjectInspector soi = (StructObjectInspector) IcebergObjectInspector.create(schema); - Assert.assertEquals( - ImmutableList.of(record.get(0), record.get(1)), soi.getStructFieldsDataAsList(record)); + assertThat(soi.getStructFieldsDataAsList(record)) + .isEqualTo(ImmutableList.of(record.get(0), record.get(1))); StructField integerField = soi.getStructFieldRef("integer_field"); - Assert.assertEquals(record.get(0), soi.getStructFieldData(record, integerField)); + assertThat(soi.getStructFieldData(record, integerField)).isEqualTo(record.get(0)); StructField structField = soi.getStructFieldRef("struct_field"); Object innerData = soi.getStructFieldData(record, structField); - Assert.assertEquals(innerRecord, innerData); + assertThat(innerData).isEqualTo(innerRecord); StructObjectInspector innerSoi = (StructObjectInspector) structField.getFieldObjectInspector(); StructField stringField = innerSoi.getStructFieldRef("string_field"); - Assert.assertEquals( - ImmutableList.of(innerRecord.get(0)), innerSoi.getStructFieldsDataAsList(innerRecord)); - Assert.assertEquals(innerRecord.get(0), innerSoi.getStructFieldData(innerData, stringField)); + assertThat(innerSoi.getStructFieldsDataAsList(innerRecord)) + .isEqualTo(ImmutableList.of(innerRecord.get(0))); + assertThat(innerSoi.getStructFieldData(innerData, stringField)).isEqualTo(innerRecord.get(0)); } @Test @@ -76,8 +76,8 @@ public void testIcebergRecordObjectInspectorWithRowNull() { Types.StructType.of( Types.NestedField.required(3, "string_field", Types.StringType.get())))); StructObjectInspector soi = (StructObjectInspector) IcebergObjectInspector.create(schema); - Assert.assertNull(soi.getStructFieldsDataAsList(null)); + assertThat(soi.getStructFieldsDataAsList(null)).isNull(); StructField integerField = soi.getStructFieldRef("integer_field"); - Assert.assertNull(soi.getStructFieldData(null, integerField)); + assertThat(soi.getStructFieldData(null, integerField)).isNull(); } } diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergTimeObjectInspector.java b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergTimeObjectInspector.java index 04c3c710f27d..5af9ba341ebd 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergTimeObjectInspector.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergTimeObjectInspector.java @@ -18,13 +18,14 @@ */ package org.apache.iceberg.mr.hive.serde.objectinspector; +import static org.assertj.core.api.Assertions.assertThat; + import java.time.LocalTime; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; import org.apache.hadoop.io.Text; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestIcebergTimeObjectInspector { @@ -33,34 +34,34 @@ public void testIcebergTimeObjectInspector() { IcebergTimeObjectInspector oi = IcebergTimeObjectInspector.get(); - Assert.assertEquals(ObjectInspector.Category.PRIMITIVE, oi.getCategory()); - Assert.assertEquals( - PrimitiveObjectInspector.PrimitiveCategory.STRING, oi.getPrimitiveCategory()); + assertThat(oi.getCategory()).isEqualTo(ObjectInspector.Category.PRIMITIVE); + assertThat(oi.getPrimitiveCategory()) + .isEqualTo(PrimitiveObjectInspector.PrimitiveCategory.STRING); - Assert.assertEquals(TypeInfoFactory.stringTypeInfo, oi.getTypeInfo()); - Assert.assertEquals(TypeInfoFactory.stringTypeInfo.getTypeName(), oi.getTypeName()); + assertThat(oi.getTypeInfo()).isEqualTo(TypeInfoFactory.stringTypeInfo); + assertThat(oi.getTypeName()).isEqualTo(TypeInfoFactory.stringTypeInfo.getTypeName()); - Assert.assertEquals(String.class, oi.getJavaPrimitiveClass()); - Assert.assertEquals(Text.class, oi.getPrimitiveWritableClass()); + assertThat(oi.getJavaPrimitiveClass()).isEqualTo(String.class); + assertThat(oi.getPrimitiveWritableClass()).isEqualTo(Text.class); - Assert.assertNull(oi.copyObject(null)); - Assert.assertNull(oi.getPrimitiveJavaObject(null)); - Assert.assertNull(oi.getPrimitiveWritableObject(null)); - Assert.assertNull(oi.convert(null)); + assertThat(oi.copyObject(null)).isNull(); + assertThat(oi.getPrimitiveJavaObject(null)).isNull(); + assertThat(oi.getPrimitiveWritableObject(null)).isNull(); + assertThat(oi.convert(null)).isNull(); LocalTime localTime = LocalTime.now(); String time = localTime.toString(); Text text = new Text(time); - Assert.assertEquals(time, oi.getPrimitiveJavaObject(text)); - Assert.assertEquals(text, oi.getPrimitiveWritableObject(time)); - Assert.assertEquals(localTime, oi.convert(time)); + assertThat(oi.getPrimitiveJavaObject(text)).isEqualTo(time); + assertThat(oi.getPrimitiveWritableObject(time)).isEqualTo(text); + assertThat(oi.convert(time)).isEqualTo(localTime); Text copy = (Text) oi.copyObject(text); - Assert.assertEquals(text, copy); - Assert.assertNotSame(text, copy); + assertThat(copy).isEqualTo(text); + assertThat(copy).isNotSameAs(text); - Assert.assertFalse(oi.preferWritable()); + assertThat(oi.preferWritable()).isFalse(); } } diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergTimestampObjectInspector.java b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergTimestampObjectInspector.java index 9205d7c0d7f8..ea40cc20420a 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergTimestampObjectInspector.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergTimestampObjectInspector.java @@ -18,14 +18,15 @@ */ package org.apache.iceberg.mr.hive.serde.objectinspector; +import static org.assertj.core.api.Assertions.assertThat; + import java.sql.Timestamp; import java.time.LocalDateTime; import org.apache.hadoop.hive.serde2.io.TimestampWritable; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestIcebergTimestampObjectInspector { @@ -33,34 +34,34 @@ public class TestIcebergTimestampObjectInspector { public void testIcebergTimestampObjectInspector() { IcebergTimestampObjectInspector oi = IcebergTimestampObjectInspector.get(); - Assert.assertEquals(ObjectInspector.Category.PRIMITIVE, oi.getCategory()); - Assert.assertEquals( - PrimitiveObjectInspector.PrimitiveCategory.TIMESTAMP, oi.getPrimitiveCategory()); + assertThat(oi.getCategory()).isEqualTo(ObjectInspector.Category.PRIMITIVE); + assertThat(oi.getPrimitiveCategory()) + .isEqualTo(PrimitiveObjectInspector.PrimitiveCategory.TIMESTAMP); - Assert.assertEquals(TypeInfoFactory.timestampTypeInfo, oi.getTypeInfo()); - Assert.assertEquals(TypeInfoFactory.timestampTypeInfo.getTypeName(), oi.getTypeName()); + assertThat(oi.getTypeInfo()).isEqualTo(TypeInfoFactory.timestampTypeInfo); + assertThat(oi.getTypeName()).isEqualTo(TypeInfoFactory.timestampTypeInfo.getTypeName()); - Assert.assertEquals(Timestamp.class, oi.getJavaPrimitiveClass()); - Assert.assertEquals(TimestampWritable.class, oi.getPrimitiveWritableClass()); + assertThat(oi.getJavaPrimitiveClass()).isEqualTo(Timestamp.class); + assertThat(oi.getPrimitiveWritableClass()).isEqualTo(TimestampWritable.class); - Assert.assertNull(oi.copyObject(null)); - Assert.assertNull(oi.getPrimitiveJavaObject(null)); - Assert.assertNull(oi.getPrimitiveWritableObject(null)); - Assert.assertNull(oi.convert(null)); + assertThat(oi.copyObject(null)).isNull(); + assertThat(oi.getPrimitiveJavaObject(null)).isNull(); + assertThat(oi.getPrimitiveWritableObject(null)).isNull(); + assertThat(oi.convert(null)).isNull(); LocalDateTime local = LocalDateTime.of(2020, 1, 1, 12, 55, 30, 5560000); Timestamp ts = Timestamp.valueOf(local); - Assert.assertEquals(ts, oi.getPrimitiveJavaObject(local)); - Assert.assertEquals(new TimestampWritable(ts), oi.getPrimitiveWritableObject(local)); + assertThat(oi.getPrimitiveJavaObject(local)).isEqualTo(ts); + assertThat(oi.getPrimitiveWritableObject(local)).isEqualTo(new TimestampWritable(ts)); Timestamp copy = (Timestamp) oi.copyObject(ts); - Assert.assertEquals(ts, copy); - Assert.assertNotSame(ts, copy); + assertThat(copy).isEqualTo(ts); + assertThat(copy).isNotSameAs(ts); - Assert.assertFalse(oi.preferWritable()); + assertThat(oi.preferWritable()).isFalse(); - Assert.assertEquals(local, oi.convert(ts)); + assertThat(oi.convert(ts)).isEqualTo(local); } } diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergTimestampWithZoneObjectInspector.java b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergTimestampWithZoneObjectInspector.java index 20caec44f7bd..1b16e6e02c0e 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergTimestampWithZoneObjectInspector.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergTimestampWithZoneObjectInspector.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.mr.hive.serde.objectinspector; +import static org.assertj.core.api.Assertions.assertThat; + import java.sql.Timestamp; import java.time.LocalDateTime; import java.time.OffsetDateTime; @@ -26,8 +28,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestIcebergTimestampWithZoneObjectInspector { @@ -35,41 +36,40 @@ public class TestIcebergTimestampWithZoneObjectInspector { public void testIcebergTimestampObjectInspectorWithUTCAdjustment() { IcebergTimestampWithZoneObjectInspector oi = IcebergTimestampWithZoneObjectInspector.get(); - Assert.assertEquals(ObjectInspector.Category.PRIMITIVE, oi.getCategory()); - Assert.assertEquals( - PrimitiveObjectInspector.PrimitiveCategory.TIMESTAMP, oi.getPrimitiveCategory()); + assertThat(oi.getCategory()).isEqualTo(ObjectInspector.Category.PRIMITIVE); + assertThat(oi.getPrimitiveCategory()) + .isEqualTo(PrimitiveObjectInspector.PrimitiveCategory.TIMESTAMP); - Assert.assertEquals(TypeInfoFactory.timestampTypeInfo, oi.getTypeInfo()); - Assert.assertEquals(TypeInfoFactory.timestampTypeInfo.getTypeName(), oi.getTypeName()); + assertThat(oi.getTypeInfo()).isEqualTo(TypeInfoFactory.timestampTypeInfo); + assertThat(oi.getTypeName()).isEqualTo(TypeInfoFactory.timestampTypeInfo.getTypeName()); - Assert.assertEquals(Timestamp.class, oi.getJavaPrimitiveClass()); - Assert.assertEquals(TimestampWritable.class, oi.getPrimitiveWritableClass()); + assertThat(oi.getJavaPrimitiveClass()).isEqualTo(Timestamp.class); + assertThat(oi.getPrimitiveWritableClass()).isEqualTo(TimestampWritable.class); - Assert.assertNull(oi.copyObject(null)); - Assert.assertNull(oi.getPrimitiveJavaObject(null)); - Assert.assertNull(oi.getPrimitiveWritableObject(null)); - Assert.assertNull(oi.convert(null)); + assertThat(oi.copyObject(null)).isNull(); + assertThat(oi.getPrimitiveJavaObject(null)).isNull(); + assertThat(oi.getPrimitiveWritableObject(null)).isNull(); + assertThat(oi.convert(null)).isNull(); LocalDateTime local = LocalDateTime.of(2020, 1, 1, 16, 45, 33, 456000); OffsetDateTime offsetDateTime = OffsetDateTime.of(local, ZoneOffset.ofHours(-5)); Timestamp ts = Timestamp.from(offsetDateTime.toInstant()); - Assert.assertEquals(ts, oi.getPrimitiveJavaObject(offsetDateTime)); - Assert.assertEquals(new TimestampWritable(ts), oi.getPrimitiveWritableObject(offsetDateTime)); + assertThat(oi.getPrimitiveJavaObject(offsetDateTime)).isEqualTo(ts); + assertThat(oi.getPrimitiveWritableObject(offsetDateTime)).isEqualTo(new TimestampWritable(ts)); Timestamp copy = (Timestamp) oi.copyObject(ts); - Assert.assertEquals(ts, copy); - Assert.assertNotSame(ts, copy); + assertThat(copy).isEqualTo(ts); + assertThat(copy).isNotSameAs(ts); - Assert.assertFalse(oi.preferWritable()); + assertThat(oi.preferWritable()).isFalse(); - Assert.assertEquals( - OffsetDateTime.ofInstant(local.toInstant(ZoneOffset.ofHours(-5)), ZoneOffset.UTC), - oi.convert(ts)); + assertThat(oi.convert(ts)) + .isEqualTo( + OffsetDateTime.ofInstant(local.toInstant(ZoneOffset.ofHours(-5)), ZoneOffset.UTC)); - Assert.assertEquals( - offsetDateTime.withOffsetSameInstant(ZoneOffset.UTC), - oi.convert(Timestamp.from(offsetDateTime.toInstant()))); + assertThat(offsetDateTime.withOffsetSameInstant(ZoneOffset.UTC)) + .isEqualTo(oi.convert(Timestamp.from(offsetDateTime.toInstant()))); } } diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergUUIDObjectInspector.java b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergUUIDObjectInspector.java index 303cabc1cc15..abc0c01ed8d1 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergUUIDObjectInspector.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergUUIDObjectInspector.java @@ -18,13 +18,14 @@ */ package org.apache.iceberg.mr.hive.serde.objectinspector; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.UUID; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; import org.apache.hadoop.io.Text; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestIcebergUUIDObjectInspector { @@ -32,34 +33,34 @@ public class TestIcebergUUIDObjectInspector { public void testIcebergUUIDObjectInspector() { IcebergUUIDObjectInspector oi = IcebergUUIDObjectInspector.get(); - Assert.assertEquals(ObjectInspector.Category.PRIMITIVE, oi.getCategory()); - Assert.assertEquals( - PrimitiveObjectInspector.PrimitiveCategory.STRING, oi.getPrimitiveCategory()); + assertThat(oi.getCategory()).isEqualTo(ObjectInspector.Category.PRIMITIVE); + assertThat(oi.getPrimitiveCategory()) + .isEqualTo(PrimitiveObjectInspector.PrimitiveCategory.STRING); - Assert.assertEquals(TypeInfoFactory.stringTypeInfo, oi.getTypeInfo()); - Assert.assertEquals(TypeInfoFactory.stringTypeInfo.getTypeName(), oi.getTypeName()); + assertThat(oi.getTypeInfo()).isEqualTo(TypeInfoFactory.stringTypeInfo); + assertThat(oi.getTypeName()).isEqualTo(TypeInfoFactory.stringTypeInfo.getTypeName()); - Assert.assertEquals(String.class, oi.getJavaPrimitiveClass()); - Assert.assertEquals(Text.class, oi.getPrimitiveWritableClass()); + assertThat(oi.getJavaPrimitiveClass()).isEqualTo(String.class); + assertThat(oi.getPrimitiveWritableClass()).isEqualTo(Text.class); - Assert.assertNull(oi.copyObject(null)); - Assert.assertNull(oi.getPrimitiveJavaObject(null)); - Assert.assertNull(oi.getPrimitiveWritableObject(null)); - Assert.assertNull(oi.convert(null)); + assertThat(oi.copyObject(null)).isNull(); + assertThat(oi.getPrimitiveJavaObject(null)).isNull(); + assertThat(oi.getPrimitiveWritableObject(null)).isNull(); + assertThat(oi.convert(null)).isNull(); UUID uuid = UUID.randomUUID(); String uuidStr = uuid.toString(); Text text = new Text(uuidStr); - Assert.assertEquals(uuidStr, oi.getPrimitiveJavaObject(text)); - Assert.assertEquals(text, oi.getPrimitiveWritableObject(uuidStr)); - Assert.assertEquals(uuid, oi.convert(uuidStr)); + assertThat(oi.getPrimitiveJavaObject(text)).isEqualTo(uuidStr); + assertThat(oi.getPrimitiveWritableObject(uuidStr)).isEqualTo(text); + assertThat(oi.convert(uuidStr)).isEqualTo(uuid); Text copy = (Text) oi.copyObject(text); - Assert.assertEquals(text, copy); - Assert.assertNotSame(text, copy); + assertThat(copy).isEqualTo(text); + assertThat(copy).isNotSameAs(text); - Assert.assertFalse(oi.preferWritable()); + assertThat(oi.preferWritable()).isFalse(); } } From 934a3f9df85cd75f397ddb2e6be3b42a68b04dc6 Mon Sep 17 00:00:00 2001 From: Jason Fine Date: Mon, 11 Dec 2023 18:28:02 +0200 Subject: [PATCH 45/47] MergingSnapshotProducer: Change file holder to be generic This will allow re-using it for insert files for specific data versions in future commits --- .../iceberg/MergingSnapshotProducer.java | 38 +++++++++---------- 1 file changed, 19 insertions(+), 19 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index 1dcfa6d3d41d..6143a4ee9097 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -85,7 +85,7 @@ abstract class MergingSnapshotProducer extends SnapshotProducer { // update data private final List newDataFiles = Lists.newArrayList(); private Long newDataFilesDataSequenceNumber; - private final Map> newDeleteFilesBySpec = Maps.newHashMap(); + private final Map>> newDeleteFilesBySpec = Maps.newHashMap(); private final List appendManifests = Lists.newArrayList(); private final List rewrittenAppendManifests = Lists.newArrayList(); private final SnapshotSummary.Builder addedFilesSummary = SnapshotSummary.builder(); @@ -237,22 +237,22 @@ protected void add(DataFile file) { /** Add a delete file to the new snapshot. */ protected void add(DeleteFile file) { Preconditions.checkNotNull(file, "Invalid delete file: null"); - add(new DeleteFileHolder(file)); + addDelete(new FileHolder<>(file)); } /** Add a delete file to the new snapshot. */ protected void add(DeleteFile file, long dataSequenceNumber) { Preconditions.checkNotNull(file, "Invalid delete file: null"); - add(new DeleteFileHolder(file, dataSequenceNumber)); + addDelete(new FileHolder<>(file, dataSequenceNumber)); } - private void add(DeleteFileHolder fileHolder) { - int specId = fileHolder.deleteFile().specId(); + private void addDelete(FileHolder fileHolder) { + int specId = fileHolder.file().specId(); PartitionSpec fileSpec = ops.current().spec(specId); - List deleteFiles = + List> deleteFiles = newDeleteFilesBySpec.computeIfAbsent(specId, s -> Lists.newArrayList()); deleteFiles.add(fileHolder); - addedFilesSummary.addedFile(fileSpec, fileHolder.deleteFile()); + addedFilesSummary.addedFile(fileSpec, fileHolder.file()); hasNewDeleteFiles = true; } @@ -1006,9 +1006,9 @@ private List newDeleteFilesAsManifests() { deleteFiles.forEach( df -> { if (df.dataSequenceNumber() != null) { - writer.add(df.deleteFile(), df.dataSequenceNumber()); + writer.add(df.file(), df.dataSequenceNumber()); } else { - writer.add(df.deleteFile()); + writer.add(df.file()); } }); } finally { @@ -1132,33 +1132,33 @@ protected ManifestReader newManifestReader(ManifestFile manifest) { } } - private static class DeleteFileHolder { - private final DeleteFile deleteFile; + private static class FileHolder>{ + private final T file; private final Long dataSequenceNumber; /** * Wrap a delete file for commit with a given data sequence number * - * @param deleteFile delete file + * @param file content file * @param dataSequenceNumber data sequence number to apply */ - DeleteFileHolder(DeleteFile deleteFile, long dataSequenceNumber) { - this.deleteFile = deleteFile; + FileHolder(T file, long dataSequenceNumber) { + this.file = file; this.dataSequenceNumber = dataSequenceNumber; } /** * Wrap a delete file for commit with the latest sequence number * - * @param deleteFile delete file + * @param file the content fle */ - DeleteFileHolder(DeleteFile deleteFile) { - this.deleteFile = deleteFile; + FileHolder(T file) { + this.file = file; this.dataSequenceNumber = null; } - public DeleteFile deleteFile() { - return deleteFile; + public T file() { + return file; } public Long dataSequenceNumber() { From 01762b1263f10c2ef6b9aae1397c519d6e3f432b Mon Sep 17 00:00:00 2001 From: Jason Fine Date: Mon, 11 Dec 2023 18:37:00 +0200 Subject: [PATCH 46/47] MergingSnapshotProducer: Support adding data files at a specific sequence number --- .../iceberg/MergingSnapshotProducer.java | 42 +++++++++++++++---- 1 file changed, 34 insertions(+), 8 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index 6143a4ee9097..632958e242dd 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -33,6 +33,7 @@ import java.util.Map; import java.util.Objects; import java.util.Set; +import java.util.stream.Collectors; import org.apache.iceberg.events.CreateSnapshotEvent; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.exceptions.ValidationException; @@ -83,7 +84,7 @@ abstract class MergingSnapshotProducer extends SnapshotProducer { private final boolean snapshotIdInheritanceEnabled; // update data - private final List newDataFiles = Lists.newArrayList(); + private final List> newDataFiles = Lists.newArrayList(); private Long newDataFilesDataSequenceNumber; private final Map>> newDeleteFilesBySpec = Maps.newHashMap(); private final List appendManifests = Lists.newArrayList(); @@ -158,7 +159,8 @@ protected Expression rowFilter() { } protected List addedDataFiles() { - return ImmutableList.copyOf(newDataFiles); + return ImmutableList.copyOf( + newDataFiles.stream().map(FileHolder::file).collect(Collectors.toList())); } protected void failAnyDelete() { @@ -228,10 +230,20 @@ protected boolean addsDeleteFiles() { /** Add a data file to the new snapshot. */ protected void add(DataFile file) { Preconditions.checkNotNull(file, "Invalid data file: null"); - setDataSpec(file); - addedFilesSummary.addedFile(dataSpec(), file); + addDataFile(new FileHolder<>(file)); + } + + /** Add a data file to the new snapshot. */ + protected void add(DataFile file, long dataSequenceNumber) { + Preconditions.checkNotNull(file, "Invalid data file: null"); + addDataFile(new FileHolder<>(file, dataSequenceNumber)); + } + + private void addDataFile(FileHolder dataFile) { + setDataSpec(dataFile.file()); + addedFilesSummary.addedFile(dataSpec(), dataFile.file()); hasNewDataFiles = true; - newDataFiles.add(file); + newDataFiles.add(dataFile); } /** Add a delete file to the new snapshot. */ @@ -960,9 +972,23 @@ private List newDataFilesAsManifests() { RollingManifestWriter writer = newRollingManifestWriter(dataSpec()); try { if (newDataFilesDataSequenceNumber == null) { - newDataFiles.forEach(writer::add); + newDataFiles.forEach( + f -> { + if (f.dataSequenceNumber() == null) { + writer.add(f.file()); + } else { + writer.add(f.file(), f.dataSequenceNumber); + } + }); } else { - newDataFiles.forEach(f -> writer.add(f, newDataFilesDataSequenceNumber)); + newDataFiles.forEach( + f -> { + if (f.dataSequenceNumber() == null) { + writer.add(f.file(), newDataFilesDataSequenceNumber); + } else { + writer.add(f.file(), f.dataSequenceNumber); + } + }); } } finally { writer.close(); @@ -1132,7 +1158,7 @@ protected ManifestReader newManifestReader(ManifestFile manifest) { } } - private static class FileHolder>{ + private static class FileHolder> { private final T file; private final Long dataSequenceNumber; From 5357e35065c3bc19d2534997984e3a6450d87631 Mon Sep 17 00:00:00 2001 From: Jason Fine Date: Mon, 11 Dec 2023 18:59:54 +0200 Subject: [PATCH 47/47] Table Operations: Added streaming update operation This operation allows adding multiple consecutive update in a single commit without equality deletes from prior updates affecting inserts that occurred after it. Before this commit you would need to do something like this: ``` for batch in batches: delta = transaction.newRowDelta() delta.add(batch.deletes) delta.add(batch.inserts) delta.commit() transaction.commit() ``` Which produces many manifest files and is very IO intensive. This operation allows: ``` update = table.newStreamingUpdate() for batch, batchIndex in enumerate(batches): update.addUpdate(batch.deleteFiles, batch.dataFiles, batchIndex) update.commit() ``` --- .../org/apache/iceberg/StreamingUpdate.java | 99 +++++++++++++++++++ .../main/java/org/apache/iceberg/Table.java | 11 +++ .../java/org/apache/iceberg/Transaction.java | 11 +++ .../org/apache/iceberg/BaseReadOnlyTable.java | 5 + .../apache/iceberg/BaseStreamingUpdate.java | 97 ++++++++++++++++++ .../java/org/apache/iceberg/BaseTable.java | 5 + .../org/apache/iceberg/BaseTransaction.java | 15 +++ .../org/apache/iceberg/SerializableTable.java | 5 + 8 files changed, 248 insertions(+) create mode 100644 api/src/main/java/org/apache/iceberg/StreamingUpdate.java create mode 100644 core/src/main/java/org/apache/iceberg/BaseStreamingUpdate.java diff --git a/api/src/main/java/org/apache/iceberg/StreamingUpdate.java b/api/src/main/java/org/apache/iceberg/StreamingUpdate.java new file mode 100644 index 000000000000..01f2f4e79a1f --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/StreamingUpdate.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import org.apache.iceberg.exceptions.ValidationException; + +/** + * API for appending sequential updates to a table + * + *

    This API accumulates batches of file additions and deletions by order, produces a new {@link + * Snapshot} of the changes where each batch is added to a new data sequence number, and commits + * that snapshot as the current. + * + *

    When committing, these changes will be applied to the latest table snapshot. Commit conflicts + * will be resolved by applying the changes to the new latest snapshot and reattempting the commit. + * If any of the deleted files are no longer in the latest snapshot when reattempting, the commit + * will throw a {@link ValidationException}. + */ +public interface StreamingUpdate extends SnapshotUpdate { + /** + * Remove a data file from the current table state. + * + *

    This rewrite operation may change the size or layout of the data files. When applicable, it + * is also recommended to discard already deleted records while rewriting data files. However, the + * set of live data records must never change. + * + * @param dataFile a rewritten data file + * @return this for method chaining + */ + default StreamingUpdate deleteFile(DataFile dataFile) { + throw new UnsupportedOperationException( + this.getClass().getName() + " does not implement deleteFile"); + } + + /** + * Remove a delete file from the table state. + * + *

    This rewrite operation may change the size or layout of the delete files. When applicable, + * it is also recommended to discard delete records for files that are no longer part of the table + * state. However, the set of applicable delete records must never change. + * + * @param deleteFile a rewritten delete file + * @return this for method chaining + */ + default StreamingUpdate deleteFile(DeleteFile deleteFile) { + throw new UnsupportedOperationException( + this.getClass().getName() + " does not implement deleteFile"); + } + + /** + * Add a new data file to a specific. All files in this batch will receive the same data sequence + * number. + * + *

    This rewrite operation may change the size or layout of the data files. When applicable, it + * is also recommended to discard already deleted records while rewriting data files. However, the + * set of live data records must never change. + * + * @param dataFile a new data file + * @param batchOrdinal The batch ordinal to associate with this data file + * @return this for method chaining + */ + default StreamingUpdate addFile(DataFile dataFile, int batchOrdinal) { + throw new UnsupportedOperationException( + this.getClass().getName() + " does not implement addFile"); + } + + /** + * Add a new delete file to a specific batch. All files in this batch will receive the same data + * sequence number. + * + *

    This rewrite operation may change the size or layout of the delete files. When applicable, + * it is also recommended to discard delete records for files that are no longer part of the table + * state. However, the set of applicable delete records must never change. + * + * @param deleteFile a new delete file + * @param batchOrdinal The batch ordinal to associate with this data file + * @return this for method chaining + */ + default StreamingUpdate addFile(DeleteFile deleteFile, int batchOrdinal) { + throw new UnsupportedOperationException( + this.getClass().getName() + " does not implement addFile"); + } +} diff --git a/api/src/main/java/org/apache/iceberg/Table.java b/api/src/main/java/org/apache/iceberg/Table.java index 5ab1def51ca0..7683f1d59d84 100644 --- a/api/src/main/java/org/apache/iceberg/Table.java +++ b/api/src/main/java/org/apache/iceberg/Table.java @@ -233,6 +233,17 @@ default AppendFiles newFastAppend() { */ RewriteFiles newRewrite(); + /** + * Create a new {@link StreamingUpdate streaming update API} to append sequential upserts to the + * table. + * + * @return a new {@link StreamingUpdate} + */ + default StreamingUpdate newStreamingUpdate() { + throw new UnsupportedOperationException( + this.getClass().getName() + " doesn't implement newStreamingUpdate()"); + } + /** * Create a new {@link RewriteManifests rewrite manifests API} to replace manifests for this table * and commit. diff --git a/api/src/main/java/org/apache/iceberg/Transaction.java b/api/src/main/java/org/apache/iceberg/Transaction.java index aeec1f589d06..85d0267bd379 100644 --- a/api/src/main/java/org/apache/iceberg/Transaction.java +++ b/api/src/main/java/org/apache/iceberg/Transaction.java @@ -95,6 +95,17 @@ default AppendFiles newFastAppend() { */ RewriteFiles newRewrite(); + /** + * Create a new {@link StreamingUpdate streaming update API} to append sequential upserts to the + * table. + * + * @return a new {@link StreamingUpdate} + */ + default StreamingUpdate newStreamingUpdate() { + throw new UnsupportedOperationException( + this.getClass().getName() + " doesn't implement newStreamingUpdate()"); + } + /** * Create a new {@link RewriteManifests rewrite manifests API} to replace manifests for this * table. diff --git a/core/src/main/java/org/apache/iceberg/BaseReadOnlyTable.java b/core/src/main/java/org/apache/iceberg/BaseReadOnlyTable.java index 4acbf2a16396..f4b015c16053 100644 --- a/core/src/main/java/org/apache/iceberg/BaseReadOnlyTable.java +++ b/core/src/main/java/org/apache/iceberg/BaseReadOnlyTable.java @@ -66,6 +66,11 @@ public RewriteFiles newRewrite() { throw new UnsupportedOperationException("Cannot rewrite in a " + descriptor + " table"); } + @Override + public StreamingUpdate newStreamingUpdate() { + throw new UnsupportedOperationException("Cannot update a " + descriptor + " table"); + } + @Override public RewriteManifests rewriteManifests() { throw new UnsupportedOperationException( diff --git a/core/src/main/java/org/apache/iceberg/BaseStreamingUpdate.java b/core/src/main/java/org/apache/iceberg/BaseStreamingUpdate.java new file mode 100644 index 000000000000..8ef34afb18ec --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/BaseStreamingUpdate.java @@ -0,0 +1,97 @@ +/* + * 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.iceberg; + +import java.util.Comparator; +import java.util.List; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +class BaseStreamingUpdate extends MergingSnapshotProducer + implements StreamingUpdate { + private final List batches = Lists.newArrayList(); + + BaseStreamingUpdate(String tableName, TableOperations ops) { + super(tableName, ops); + } + + @Override + protected BaseStreamingUpdate self() { + return this; + } + + @Override + protected String operation() { + return DataOperations.OVERWRITE; + } + + @Override + public List apply(TableMetadata base, Snapshot snapshot) { + long startingSequenceNumber = base.nextSequenceNumber(); + batches.sort(Comparator.comparingInt(o -> o.ordinal)); + for (Batch batch : batches) { + long dataSequenceNumber = startingSequenceNumber + batch.ordinal + 1; + batch.newDataFiles.forEach(f -> add(f, dataSequenceNumber)); + batch.newDeleteFiles.forEach(f -> add(f, dataSequenceNumber)); + } + return super.apply(base, snapshot); + } + + @Override + public StreamingUpdate addFile(DataFile dataFile, int batchOrdinal) { + return StreamingUpdate.super.addFile(dataFile, batchOrdinal); + } + + @Override + public StreamingUpdate addFile(DeleteFile deleteFile, int batchOrdinal) { + return StreamingUpdate.super.addFile(deleteFile, batchOrdinal); + } + + @Override + public BaseStreamingUpdate toBranch(String branch) { + targetBranch(branch); + return this; + } + + private static class Batch { + private final List newDataFiles = Lists.newArrayList(); + private final List newDeleteFiles = Lists.newArrayList(); + private final int ordinal; + + /** + * Creates a new set of updates to a specific batch + * + * @param ordinal the batch ordinal + */ + Batch(int ordinal) { + this.ordinal = ordinal; + } + + public List getNewDataFiles() { + return newDataFiles; + } + + public List getNewDeleteFiles() { + return newDeleteFiles; + } + + public int getOrdinal() { + return ordinal; + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/BaseTable.java b/core/src/main/java/org/apache/iceberg/BaseTable.java index d4cf1f4b76f4..ccc018768347 100644 --- a/core/src/main/java/org/apache/iceberg/BaseTable.java +++ b/core/src/main/java/org/apache/iceberg/BaseTable.java @@ -190,6 +190,11 @@ public RewriteFiles newRewrite() { return new BaseRewriteFiles(name, ops).reportWith(reporter); } + @Override + public StreamingUpdate newStreamingUpdate() { + return new BaseStreamingUpdate(name, ops).reportWith(reporter); + } + @Override public RewriteManifests rewriteManifests() { return new BaseRewriteManifests(ops).reportWith(reporter); diff --git a/core/src/main/java/org/apache/iceberg/BaseTransaction.java b/core/src/main/java/org/apache/iceberg/BaseTransaction.java index 30103fd87fe2..cf263b755002 100644 --- a/core/src/main/java/org/apache/iceberg/BaseTransaction.java +++ b/core/src/main/java/org/apache/iceberg/BaseTransaction.java @@ -193,6 +193,16 @@ public RewriteFiles newRewrite() { return rewrite; } + @Override + public StreamingUpdate newStreamingUpdate() { + checkLastOperationCommitted("StreamingUpdate"); + StreamingUpdate streamingUpdate = + new BaseStreamingUpdate(tableName, transactionOps).reportWith(reporter); + streamingUpdate.deleteWith(enqueueDelete); + updates.add(streamingUpdate); + return streamingUpdate; + } + @Override public RewriteManifests rewriteManifests() { checkLastOperationCommitted("RewriteManifests"); @@ -703,6 +713,11 @@ public RewriteFiles newRewrite() { return BaseTransaction.this.newRewrite(); } + @Override + public StreamingUpdate newStreamingUpdate() { + return BaseTransaction.this.newStreamingUpdate(); + } + @Override public RewriteManifests rewriteManifests() { return BaseTransaction.this.rewriteManifests(); diff --git a/core/src/main/java/org/apache/iceberg/SerializableTable.java b/core/src/main/java/org/apache/iceberg/SerializableTable.java index 778a993c5144..b1fbed43b626 100644 --- a/core/src/main/java/org/apache/iceberg/SerializableTable.java +++ b/core/src/main/java/org/apache/iceberg/SerializableTable.java @@ -331,6 +331,11 @@ public RewriteFiles newRewrite() { throw new UnsupportedOperationException(errorMsg("newRewrite")); } + @Override + public StreamingUpdate newStreamingUpdate() { + throw new UnsupportedOperationException("newStreamingWrite"); + } + @Override public RewriteManifests rewriteManifests() { throw new UnsupportedOperationException(errorMsg("rewriteManifests"));