diff --git a/.github/workflows/flink-ci.yml b/.github/workflows/flink-ci.yml index 18cf01d06ce7..d3f57a70bc5e 100644 --- a/.github/workflows/flink-ci.yml +++ b/.github/workflows/flink-ci.yml @@ -74,7 +74,7 @@ jobs: strategy: matrix: jvm: [11, 17, 21] - flink: ['1.19', '1.20', '2.0'] + flink: ['1.20', '2.0', '2.1'] env: SPARK_LOCAL_IP: localhost steps: diff --git a/flink/build.gradle b/flink/build.gradle index e315f2558d95..caf2a443ace4 100644 --- a/flink/build.gradle +++ b/flink/build.gradle @@ -19,14 +19,14 @@ def flinkVersions = (System.getProperty("flinkVersions") != null ? System.getProperty("flinkVersions") : System.getProperty("defaultFlinkVersions")).split(",") -if (flinkVersions.contains("1.19")) { - apply from: file("$projectDir/v1.19/build.gradle") -} - if (flinkVersions.contains("1.20")) { apply from: file("$projectDir/v1.20/build.gradle") } if (flinkVersions.contains("2.0")) { apply from: file("$projectDir/v2.0/build.gradle") +} + +if (flinkVersions.contains("2.1")) { + apply from: file("$projectDir/v2.1/build.gradle") } \ No newline at end of file diff --git a/flink/v1.19/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink/v1.19/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory deleted file mode 100644 index 2b6bfa3cd579..000000000000 --- a/flink/v1.19/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.19/build.gradle b/flink/v2.1/build.gradle similarity index 91% rename from flink/v1.19/build.gradle rename to flink/v2.1/build.gradle index 05fdddd63ccf..f24eb22e79e8 100644 --- a/flink/v1.19/build.gradle +++ b/flink/v2.1/build.gradle @@ -17,7 +17,7 @@ * under the License. */ -String flinkMajorVersion = '1.19' +String flinkMajorVersion = '2.1' 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.flink119.avro + compileOnly libs.flink21.avro // for dropwizard histogram metrics implementation - compileOnly libs.flink119.metrics.dropwizard - compileOnly libs.flink119.streaming.java - compileOnly "${libs.flink119.streaming.java.get().module}:${libs.flink119.streaming.java.get().getVersion()}:tests" - compileOnly libs.flink119.table.api.java.bridge - compileOnly "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink119.get()}" - compileOnly libs.flink119.connector.base - compileOnly libs.flink119.connector.files + compileOnly libs.flink21.metrics.dropwizard + compileOnly libs.flink21.streaming.java + compileOnly "${libs.flink21.streaming.java.get().module}:${libs.flink21.streaming.java.get().getVersion()}:tests" + compileOnly libs.flink21.table.api.java.bridge + compileOnly "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink21.get()}" + compileOnly libs.flink21.connector.base + compileOnly libs.flink21.connector.files compileOnly libs.hadoop3.hdfs compileOnly libs.hadoop3.common @@ -71,13 +71,13 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { // for caching in DynamicSink implementation libs.caffeine - testImplementation libs.flink119.connector.test.utils - testImplementation libs.flink119.core - testImplementation libs.flink119.runtime - testImplementation(libs.flink119.test.utilsjunit) { + testImplementation libs.flink21.connector.test.utils + testImplementation libs.flink21.core + testImplementation libs.flink21.runtime + testImplementation(libs.flink21.test.utilsjunit) { exclude group: 'junit' } - testImplementation(libs.flink119.test.utils) { + testImplementation(libs.flink21.test.utils) { exclude group: "org.apache.curator", module: 'curator-test' exclude group: 'junit' } @@ -172,7 +172,7 @@ project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") { } // for dropwizard histogram metrics implementation - implementation libs.flink119.metrics.dropwizard + implementation libs.flink21.metrics.dropwizard // for integration testing with the flink-runtime-jar // all of those dependencies are required because the integration test extends FlinkTestBase @@ -181,13 +181,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.flink119.test.utils) { + integrationImplementation(libs.flink21.test.utils) { exclude group: "org.apache.curator", module: 'curator-test' exclude group: 'junit' } - integrationImplementation libs.flink119.table.api.java.bridge - integrationImplementation "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink119.get()}" + integrationImplementation libs.flink21.table.api.java.bridge + integrationImplementation "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink21.get()}" integrationImplementation libs.hadoop3.common integrationImplementation libs.hadoop3.hdfs diff --git a/flink/v1.19/flink-runtime/LICENSE b/flink/v2.1/flink-runtime/LICENSE similarity index 100% rename from flink/v1.19/flink-runtime/LICENSE rename to flink/v2.1/flink-runtime/LICENSE diff --git a/flink/v1.19/flink-runtime/NOTICE b/flink/v2.1/flink-runtime/NOTICE similarity index 100% rename from flink/v1.19/flink-runtime/NOTICE rename to flink/v2.1/flink-runtime/NOTICE diff --git a/flink/v1.19/flink-runtime/src/integration/java/org/apache/iceberg/flink/TestIcebergConnectorSmoke.java b/flink/v2.1/flink-runtime/src/integration/java/org/apache/iceberg/flink/TestIcebergConnectorSmoke.java similarity index 100% rename from flink/v1.19/flink-runtime/src/integration/java/org/apache/iceberg/flink/TestIcebergConnectorSmoke.java rename to flink/v2.1/flink-runtime/src/integration/java/org/apache/iceberg/flink/TestIcebergConnectorSmoke.java diff --git a/flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordSerializerDeserializerBenchmark.java b/flink/v2.1/flink/src/jmh/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordSerializerDeserializerBenchmark.java similarity index 100% rename from flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordSerializerDeserializerBenchmark.java rename to flink/v2.1/flink/src/jmh/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordSerializerDeserializerBenchmark.java diff --git a/flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java b/flink/v2.1/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java similarity index 100% rename from flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java rename to flink/v2.1/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java diff --git a/flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitionerBenchmark.java b/flink/v2.1/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitionerBenchmark.java similarity index 100% rename from flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitionerBenchmark.java rename to flink/v2.1/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitionerBenchmark.java diff --git a/flink/v2.1/flink/src/main/java/org/apache/flink/table/api/runtime/types/FlinkScalaKryoInstantiator.java b/flink/v2.1/flink/src/main/java/org/apache/flink/table/api/runtime/types/FlinkScalaKryoInstantiator.java new file mode 100644 index 000000000000..7318c147a1b6 --- /dev/null +++ b/flink/v2.1/flink/src/main/java/org/apache/flink/table/api/runtime/types/FlinkScalaKryoInstantiator.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.flink.table.api.runtime.types; + +/** + * Override Flink's internal FlinkScalaKryoInstantiator to avoid loading the Scala extensions for + * the KryoSerializer. This is a workaround until Kryo-related issues with the Scala extensions are + * fixed. See: https://issues.apache.org/jira/browse/FLINK-37546 + */ +public class FlinkScalaKryoInstantiator {} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java similarity index 98% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java index 4c44961df19c..4bb235b811d0 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java @@ -28,7 +28,6 @@ import java.util.Set; import java.util.stream.Collectors; import org.apache.flink.annotation.Internal; -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; @@ -37,7 +36,6 @@ 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.ResolvedCatalogTable; import org.apache.flink.table.catalog.ResolvedSchema; @@ -662,11 +660,16 @@ static CatalogTable toCatalogTableWithProps(Table table, Map pro 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. + // may use DefaultCatalogTable 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( - TableSchema.fromResolvedSchema(resolvedSchema), partitionKeys, props, null); + return CatalogTable.newBuilder() + .schema( + org.apache.flink.table.api.Schema.newBuilder() + .fromResolvedSchema(resolvedSchema) + .build()) + .partitionKeys(partitionKeys) + .options(props) + .build(); } static CatalogTable toCatalogTable(Table table) { diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java similarity index 92% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java index dd065617bd88..33cbc92ddeec 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java @@ -21,9 +21,10 @@ 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 java.util.Set; +import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.runtime.util.HadoopUtils; import org.apache.flink.table.catalog.Catalog; @@ -34,8 +35,7 @@ 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.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.util.PropertyUtil; /** @@ -60,6 +60,8 @@ */ public class FlinkCatalogFactory implements CatalogFactory { + public static final String FACTORY_IDENTIFIER = "iceberg"; + // 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"; @@ -72,8 +74,6 @@ public class FlinkCatalogFactory implements CatalogFactory { public static final String DEFAULT_DATABASE_NAME = "default"; public static final String DEFAULT_CATALOG_NAME = "default_catalog"; 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 @@ -122,21 +122,23 @@ static CatalogLoader createCatalogLoader( } @Override - public Map requiredContext() { - Map context = Maps.newHashMap(); - context.put(TYPE, "iceberg"); - context.put(PROPERTY_VERSION, "1"); - return context; + public String factoryIdentifier() { + return FACTORY_IDENTIFIER; + } + + @Override + public Set> requiredOptions() { + return ImmutableSet.>builder().build(); } @Override - public List supportedProperties() { - return ImmutableList.of("*"); + public Set> optionalOptions() { + return ImmutableSet.>builder().build(); } @Override - public Catalog createCatalog(String name, Map properties) { - return createCatalog(name, properties, clusterHadoopConf()); + public Catalog createCatalog(Context context) { + return createCatalog(context.getName(), context.getOptions(), clusterHadoopConf()); } protected Catalog createCatalog( diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java similarity index 99% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java index 6362bc447634..97e2c70d348e 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java @@ -88,7 +88,7 @@ private FlinkConfigOptions() {} public static final ConfigOption TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE = ConfigOptions.key("table.exec.iceberg.use-flip27-source") .booleanType() - .defaultValue(false) + .defaultValue(true) .withDescription("Use the FLIP-27 based Iceberg source implementation."); public static final ConfigOption TABLE_EXEC_ICEBERG_USE_V2_SINK = diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkCreateTableOptions.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkCreateTableOptions.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkCreateTableOptions.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkCreateTableOptions.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java similarity index 96% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java index 3b9cc0baff11..bd79c1156090 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java @@ -142,17 +142,16 @@ private static TableLoader createTableLoader( mergedProps.forEach(flinkConf::setString); - String catalogName = flinkConf.getString(FlinkCreateTableOptions.CATALOG_NAME); + String catalogName = flinkConf.get(FlinkCreateTableOptions.CATALOG_NAME); Preconditions.checkNotNull( catalogName, "Table property '%s' cannot be null", FlinkCreateTableOptions.CATALOG_NAME.key()); - String catalogDatabase = - flinkConf.getString(FlinkCreateTableOptions.CATALOG_DATABASE, databaseName); + String catalogDatabase = flinkConf.get(FlinkCreateTableOptions.CATALOG_DATABASE, databaseName); Preconditions.checkNotNull(catalogDatabase, "The iceberg database name cannot be null"); - String catalogTable = flinkConf.getString(FlinkCreateTableOptions.CATALOG_TABLE, tableName); + String catalogTable = flinkConf.get(FlinkCreateTableOptions.CATALOG_TABLE, tableName); Preconditions.checkNotNull(catalogTable, "The iceberg table name cannot be null"); org.apache.hadoop.conf.Configuration hadoopConf = FlinkCatalogFactory.clusterHadoopConf(); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkRowData.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkRowData.java similarity index 95% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkRowData.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkRowData.java index 0835795119f8..06c1635312b9 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkRowData.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkRowData.java @@ -37,7 +37,7 @@ public static RowData.FieldGetter createFieldGetter(LogicalType fieldType, int f // RowData.createFieldGetter(..) does not null-check optional / nullable types. Without this // explicit null check, the null flag of BinaryRowData will be ignored and random bytes will // be parsed as actual values. This will produce incorrect writes instead of failing with a - // NullPointerException. + // NullPointerException. See https://issues.apache.org/jira/browse/FLINK-37245 if (!fieldType.isNullable() && rowData.isNullAt(fieldPos)) { return null; } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java similarity index 99% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java index 8e13545cd91a..7f55d4b07bb1 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java @@ -26,11 +26,11 @@ import java.util.function.Function; import java.util.stream.Collectors; 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.catalog.Column; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.catalog.UniqueConstraint; +import org.apache.flink.table.legacy.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; diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java similarity index 65% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java index 218e298c9583..c8c11474177c 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java @@ -21,18 +21,14 @@ 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.catalog.ResolvedSchema; import org.apache.flink.table.catalog.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.table.legacy.api.TableSchema; import org.apache.flink.types.RowKind; import org.apache.iceberg.flink.sink.FlinkSink; import org.apache.iceberg.flink.sink.IcebergSink; @@ -99,64 +95,58 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { .map(UniqueConstraint::getColumns) .orElseGet(ImmutableList::of); - return new DataStreamSinkProvider() { - @Override - public DataStreamSink consumeDataStream( - ProviderContext providerContext, DataStream dataStream) { - if (Boolean.TRUE.equals( - readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_V2_SINK))) { - return IcebergSink.forRowData(dataStream) - .tableLoader(tableLoader) - .resolvedSchema(resolvedSchema) - .equalityFieldColumns(equalityColumns) - .overwrite(overwrite) - .setAll(writeProps) - .flinkConf(readableConfig) - .append(); - } else { - return FlinkSink.forRowData(dataStream) - .tableLoader(tableLoader) - .resolvedSchema(resolvedSchema) - .equalityFieldColumns(equalityColumns) - .overwrite(overwrite) - .setAll(writeProps) - .flinkConf(readableConfig) - .append(); - } - } - }; + return (DataStreamSinkProvider) + (providerContext, dataStream) -> { + if (Boolean.TRUE.equals( + readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_V2_SINK))) { + return IcebergSink.forRowData(dataStream) + .tableLoader(tableLoader) + .resolvedSchema(resolvedSchema) + .equalityFieldColumns(equalityColumns) + .overwrite(overwrite) + .setAll(writeProps) + .flinkConf(readableConfig) + .append(); + } else { + return FlinkSink.forRowData(dataStream) + .tableLoader(tableLoader) + .resolvedSchema(resolvedSchema) + .equalityFieldColumns(equalityColumns) + .overwrite(overwrite) + .setAll(writeProps) + .flinkConf(readableConfig) + .append(); + } + }; } else { List equalityColumns = tableSchema .getPrimaryKey() - .map(org.apache.flink.table.api.constraints.UniqueConstraint::getColumns) + .map(org.apache.flink.table.legacy.api.constraints.UniqueConstraint::getColumns) .orElseGet(ImmutableList::of); - return new DataStreamSinkProvider() { - @Override - public DataStreamSink consumeDataStream( - ProviderContext providerContext, DataStream dataStream) { - if (readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_V2_SINK)) { - return IcebergSink.forRowData(dataStream) - .tableLoader(tableLoader) - .tableSchema(tableSchema) - .equalityFieldColumns(equalityColumns) - .overwrite(overwrite) - .setAll(writeProps) - .flinkConf(readableConfig) - .append(); - } else { - return FlinkSink.forRowData(dataStream) - .tableLoader(tableLoader) - .tableSchema(tableSchema) - .equalityFieldColumns(equalityColumns) - .overwrite(overwrite) - .setAll(writeProps) - .flinkConf(readableConfig) - .append(); - } - } - }; + return (DataStreamSinkProvider) + (providerContext, dataStream) -> { + if (readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_V2_SINK)) { + return IcebergSink.forRowData(dataStream) + .tableLoader(tableLoader) + .tableSchema(tableSchema) + .equalityFieldColumns(equalityColumns) + .overwrite(overwrite) + .setAll(writeProps) + .flinkConf(readableConfig) + .append(); + } else { + return FlinkSink.forRowData(dataStream) + .tableLoader(tableLoader) + .tableSchema(tableSchema) + .equalityFieldColumns(equalityColumns) + .overwrite(overwrite) + .setAll(writeProps) + .flinkConf(readableConfig) + .append(); + } + }; } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java similarity index 99% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java index 5c3581aef3ec..688481e220e3 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java @@ -34,6 +34,7 @@ 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.variant.Variant; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.Schema; import org.apache.iceberg.parquet.ParquetSchemaUtil; @@ -802,6 +803,11 @@ public RawValueData getRawValue(int pos) { return (RawValueData) values[pos]; } + @Override + public Variant getVariant(int pos) { + return (Variant) values[pos]; + } + @Override public byte[] getBinary(int ordinal) { return (byte[]) values[ordinal]; diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkPlannedAvroReader.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkPlannedAvroReader.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkPlannedAvroReader.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkPlannedAvroReader.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java similarity index 51% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java index 6bb2693a0986..39aac237a8f6 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java @@ -30,8 +30,8 @@ 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.LogicalTypeAnnotation; import org.apache.parquet.schema.MessageType; -import org.apache.parquet.schema.OriginalType; import org.apache.parquet.schema.PrimitiveType; import org.apache.parquet.schema.Type; @@ -52,106 +52,100 @@ public static T visit( } 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 + LogicalTypeAnnotation annotation = group.getLogicalTypeAnnotation(); + if (annotation instanceof LogicalTypeAnnotation.ListLogicalTypeAnnotation) { + 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; + 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(); + } + } else if (annotation instanceof LogicalTypeAnnotation.MapLogicalTypeAnnotation) { + 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(); - } + return visitor.map(map, group, keyResult, valueResult); - default: + } finally { + visitor.fieldNames.pop(); } } Preconditions.checkArgument( diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java similarity index 98% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java index 9395b0e4810e..4144b04fe4eb 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java @@ -30,6 +30,7 @@ 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.types.variant.Variant; import org.apache.flink.util.StringUtils; import org.apache.iceberg.Schema; import org.apache.iceberg.flink.FlinkRowData; @@ -270,6 +271,11 @@ public RowData getRow(int pos, int numFields) { return (RowData) getValue(pos); } + @Override + public Variant getVariant(int pos) { + return (Variant) getValue(pos); + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java similarity index 98% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java index 34576a1e5c0b..b93e4346a47a 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java @@ -42,6 +42,7 @@ import org.apache.flink.table.data.StringData; import org.apache.flink.table.data.TimestampData; import org.apache.flink.types.RowKind; +import org.apache.flink.types.variant.Variant; 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; @@ -240,6 +241,11 @@ public RowData getRow(int pos, int numFields) { return isNullAt(pos) ? null : getStructRowData(pos); } + @Override + public Variant getVariant(int pos) { + return isNullAt(pos) ? null : struct.get(pos, Variant.class); + } + private StructRowData getStructRowData(int pos) { return new StructRowData( type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class)); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/DeleteOrphanFiles.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/DeleteOrphanFiles.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/DeleteOrphanFiles.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/DeleteOrphanFiles.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/FlinkMaintenanceConfig.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/FlinkMaintenanceConfig.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/FlinkMaintenanceConfig.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/FlinkMaintenanceConfig.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/JdbcLockFactory.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/JdbcLockFactory.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/JdbcLockFactory.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/JdbcLockFactory.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/LockConfig.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/LockConfig.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/LockConfig.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/LockConfig.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/RewriteDataFiles.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/RewriteDataFiles.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/RewriteDataFiles.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/RewriteDataFiles.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/RewriteDataFilesConfig.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/RewriteDataFilesConfig.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/RewriteDataFilesConfig.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/RewriteDataFilesConfig.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TaskResult.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TaskResult.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TaskResult.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TaskResult.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/Trigger.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/Trigger.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/Trigger.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/Trigger.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TriggerLockFactory.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TriggerLockFactory.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TriggerLockFactory.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TriggerLockFactory.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ZkLockFactory.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ZkLockFactory.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ZkLockFactory.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ZkLockFactory.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewriteCommitter.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewriteCommitter.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewriteCommitter.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewriteCommitter.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewritePlanner.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewritePlanner.java similarity index 98% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewritePlanner.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewritePlanner.java index 5403dfe19aae..81db62e8bf25 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewritePlanner.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewritePlanner.java @@ -22,7 +22,7 @@ import java.util.List; import java.util.Map; import org.apache.flink.annotation.Internal; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.metrics.Counter; import org.apache.flink.streaming.api.functions.ProcessFunction; import org.apache.flink.util.Collector; @@ -91,7 +91,7 @@ public DataFileRewritePlanner( } @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext context) throws Exception { tableLoader.open(); Table table = tableLoader.loadTable(); Preconditions.checkArgument( diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewriteRunner.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewriteRunner.java similarity index 98% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewriteRunner.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewriteRunner.java index c03b5cc1c8fd..ad3b0454008c 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewriteRunner.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewriteRunner.java @@ -24,7 +24,7 @@ import java.util.Set; import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.metrics.Counter; import org.apache.flink.streaming.api.functions.ProcessFunction; import org.apache.flink.table.data.RowData; @@ -75,7 +75,7 @@ public DataFileRewriteRunner(String tableName, String taskName, int taskIndex) { } @Override - public void open(Configuration parameters) { + public void open(OpenContext context) { this.errorCounter = TableMaintenanceMetrics.groupFor(getRuntimeContext(), tableName, taskName, taskIndex) .counter(TableMaintenanceMetrics.ERROR_COUNTER); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java similarity index 97% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java index 154512e27ba7..2db9585ebd8a 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java @@ -22,8 +22,8 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicLong; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.ProcessFunction; import org.apache.flink.util.Collector; import org.apache.flink.util.OutputTag; @@ -72,7 +72,7 @@ public ExpireSnapshotsProcessor( } @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext parameters) throws Exception { tableLoader.open(); this.table = tableLoader.loadTable(); this.plannerPool = diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/FileNameReader.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/FileNameReader.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/FileNameReader.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/FileNameReader.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/FileUriKeySelector.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/FileUriKeySelector.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/FileUriKeySelector.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/FileUriKeySelector.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ListFileSystemFiles.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ListFileSystemFiles.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ListFileSystemFiles.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ListFileSystemFiles.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ListMetadataFiles.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ListMetadataFiles.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ListMetadataFiles.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ListMetadataFiles.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockFactoryBuilder.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockFactoryBuilder.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockFactoryBuilder.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockFactoryBuilder.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LogUtil.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LogUtil.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LogUtil.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LogUtil.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MetadataTablePlanner.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MetadataTablePlanner.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MetadataTablePlanner.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MetadataTablePlanner.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/OrphanFilesDetector.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/OrphanFilesDetector.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/OrphanFilesDetector.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/OrphanFilesDetector.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/SingleThreadedIteratorSource.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/SingleThreadedIteratorSource.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/SingleThreadedIteratorSource.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/SingleThreadedIteratorSource.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/SkipOnError.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/SkipOnError.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/SkipOnError.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/SkipOnError.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableReader.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableReader.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableReader.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableReader.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TaskResultAggregator.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TaskResultAggregator.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TaskResultAggregator.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TaskResultAggregator.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java similarity index 99% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java index bd8424d726ec..f1f2b51c0943 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java @@ -21,13 +21,13 @@ import java.io.IOException; import java.util.List; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; 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.metrics.Counter; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.state.FunctionInitializationContext; @@ -121,7 +121,7 @@ public TriggerManager( } @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext parameters) throws Exception { MetricGroup mainGroup = TableMaintenanceMetrics.groupFor(getRuntimeContext(), tableName); this.rateLimiterTriggeredCounter = mainGroup.counter(TableMaintenanceMetrics.RATE_LIMITER_TRIGGERED); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/CommittableToTableChangeConverter.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/CommittableToTableChangeConverter.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/CommittableToTableChangeConverter.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/CommittableToTableChangeConverter.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java similarity index 98% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java index bca60745d479..d83a11d0f462 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java @@ -37,10 +37,11 @@ import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; -import org.apache.flink.table.api.TableSchema; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.util.DataFormatConverters; +import org.apache.flink.table.legacy.api.TableSchema; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.types.Row; @@ -154,7 +155,7 @@ public static Builder forRowData(DataStream input) { return new Builder().forRowData(input); } - public static class Builder { + public static class Builder implements IcebergSinkBuilder { private Function> inputCreator = null; private TableLoader tableLoader; private Table table; @@ -200,6 +201,7 @@ private Builder forMapperOutputType( * @param newTable the loaded iceberg table instance. * @return {@link Builder} to connect the iceberg table. */ + @Override public Builder table(Table newTable) { this.table = newTable; return this; @@ -213,6 +215,7 @@ public Builder table(Table newTable) { * @param newTableLoader to load iceberg table inside tasks. * @return {@link Builder} to connect the iceberg table. */ + @Override public Builder tableLoader(TableLoader newTableLoader) { this.tableLoader = newTableLoader; return this; @@ -231,31 +234,31 @@ public Builder set(String property, String value) { * Set the write properties for Flink sink. View the supported properties in {@link * FlinkWriteOptions} */ + @Override public Builder setAll(Map properties) { writeOptions.putAll(properties); return this; } - /** - * @deprecated since 1.10.0, will be removed in 2.0.0. Use {@link - * #resolvedSchema(ResolvedSchema)} instead. - */ - @Deprecated + @Override public Builder tableSchema(TableSchema newTableSchema) { this.tableSchema = newTableSchema; return this; } + @Override public Builder resolvedSchema(ResolvedSchema newResolvedSchema) { this.resolvedSchema = newResolvedSchema; return this; } + @Override public Builder overwrite(boolean newOverwrite) { writeOptions.put(FlinkWriteOptions.OVERWRITE_MODE.key(), Boolean.toString(newOverwrite)); return this; } + @Override public Builder flinkConf(ReadableConfig config) { this.readableConfig = config; return this; @@ -268,6 +271,7 @@ public Builder flinkConf(ReadableConfig config) { * @param mode to specify the write distribution mode. * @return {@link Builder} to connect the iceberg table. */ + @Override public Builder distributionMode(DistributionMode mode) { if (mode != null) { writeOptions.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), mode.modeName()); @@ -337,6 +341,7 @@ public Builder rangeDistributionSortKeyBaseWeight(double weight) { * @param newWriteParallelism the number of parallel iceberg stream writer. * @return {@link Builder} to connect the iceberg table. */ + @Override public Builder writeParallelism(int newWriteParallelism) { writeOptions.put( FlinkWriteOptions.WRITE_PARALLELISM.key(), Integer.toString(newWriteParallelism)); @@ -352,6 +357,7 @@ public Builder writeParallelism(int newWriteParallelism) { * @param enabled indicate whether it should transform all INSERT/UPDATE_AFTER events to UPSERT. * @return {@link Builder} to connect the iceberg table. */ + @Override public Builder upsert(boolean enabled) { writeOptions.put(FlinkWriteOptions.WRITE_UPSERT_ENABLED.key(), Boolean.toString(enabled)); return this; @@ -363,6 +369,7 @@ public Builder upsert(boolean enabled) { * @param columns defines the iceberg table's key. * @return {@link Builder} to connect the iceberg table. */ + @Override public Builder equalityFieldColumns(List columns) { this.equalityFieldColumns = columns; return this; @@ -407,6 +414,7 @@ public Builder setSnapshotProperty(String property, String value) { return this; } + @Override public Builder toBranch(String branch) { writeOptions.put(FlinkWriteOptions.BRANCH.key(), branch); return this; @@ -470,6 +478,7 @@ private DataStreamSink chainIcebergOperators() { * * @return {@link DataStreamSink} for sink. */ + @Override public DataStreamSink append() { return chainIcebergOperators(); } @@ -520,8 +529,8 @@ private DataStreamSink appendDummySink(SingleOutputStreamOperator co private SingleOutputStreamOperator appendCommitter( SingleOutputStreamOperator writerStream) { - IcebergFilesCommitter filesCommitter = - new IcebergFilesCommitter( + OneInputStreamOperatorFactory filesCommitterFactory = + new IcebergFilesCommitterFactory( tableLoader, flinkWriteConf.overwriteMode(), snapshotProperties, @@ -530,7 +539,8 @@ private SingleOutputStreamOperator appendCommitter( table.spec()); SingleOutputStreamOperator committerStream = writerStream - .transform(operatorName(ICEBERG_FILES_COMMITTER_NAME), Types.VOID, filesCommitter) + .transform( + operatorName(ICEBERG_FILES_COMMITTER_NAME), Types.VOID, filesCommitterFactory) .setParallelism(1) .setMaxParallelism(1); if (uidPrefix != null) { diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkWriteResult.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkWriteResult.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkWriteResult.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkWriteResult.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java similarity index 98% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java index b510dce28bac..89432cff2b29 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java @@ -37,6 +37,7 @@ 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.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.table.runtime.typeutils.SortedMapTypeInfo; import org.apache.iceberg.AppendFiles; @@ -124,12 +125,14 @@ class IcebergFilesCommitter extends AbstractStreamOperator private transient ExecutorService workerPool; IcebergFilesCommitter( + StreamOperatorParameters parameters, TableLoader tableLoader, boolean replacePartitions, Map snapshotProperties, Integer workerPoolSize, String branch, PartitionSpec spec) { + super(parameters); this.tableLoader = tableLoader; this.replacePartitions = replacePartitions; this.snapshotProperties = snapshotProperties; @@ -154,8 +157,8 @@ public void initializeState(StateInitializationContext context) throws Exception Preconditions.checkArgument( maxContinuousEmptyCommits > 0, MAX_CONTINUOUS_EMPTY_COMMITS + " must be positive"); - int subTaskId = getRuntimeContext().getIndexOfThisSubtask(); - int attemptId = getRuntimeContext().getAttemptNumber(); + int subTaskId = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(); + int attemptId = getRuntimeContext().getTaskInfo().getAttemptNumber(); this.manifestOutputFileFactory = FlinkManifestUtil.createOutputFileFactory( () -> table, table.properties(), flinkJobId, operatorUniqueId, subTaskId, attemptId); diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterFactory.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterFactory.java new file mode 100644 index 000000000000..88c304e5a0a6 --- /dev/null +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterFactory.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.sink; + +import java.util.Map; +import org.apache.flink.streaming.api.operators.ChainingStrategy; +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.iceberg.PartitionSpec; +import org.apache.iceberg.flink.TableLoader; + +class IcebergFilesCommitterFactory + implements OneInputStreamOperatorFactory { + + private final TableLoader tableLoader; + private final boolean overwriteMode; + private final Map snapshotProperties; + private final int workerPoolSize; + private final String branch; + private final PartitionSpec spec; + + IcebergFilesCommitterFactory( + TableLoader tableLoader, + boolean overwriteMode, + Map snapshotProperties, + int workerPoolSize, + String branch, + PartitionSpec spec) { + this.tableLoader = tableLoader; + this.overwriteMode = overwriteMode; + this.snapshotProperties = snapshotProperties; + this.workerPoolSize = workerPoolSize; + this.branch = branch; + this.spec = spec; + } + + @Override + public StreamOperator createStreamOperator(StreamOperatorParameters parameters) { + return new IcebergFilesCommitter( + parameters, tableLoader, overwriteMode, snapshotProperties, workerPoolSize, branch, spec); + } + + @Override + public void setChainingStrategy(ChainingStrategy strategy) {} + + @Override + public ChainingStrategy getChainingStrategy() { + return ChainingStrategy.ALWAYS; + } + + @Override + public Class> getStreamOperatorClass(ClassLoader classLoader) { + return IcebergFilesCommitter.class; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java similarity index 98% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java index 0ea0232278bd..752882a9d6c2 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java @@ -39,6 +39,7 @@ import org.apache.flink.api.connector.sink2.Sink; import org.apache.flink.api.connector.sink2.SinkWriter; import org.apache.flink.api.connector.sink2.SupportsCommitter; +import org.apache.flink.api.connector.sink2.WriterInitContext; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.io.SimpleVersionedSerializer; @@ -50,10 +51,10 @@ 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.table.api.TableSchema; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.util.DataFormatConverters; +import org.apache.flink.table.legacy.api.TableSchema; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.types.Row; @@ -200,7 +201,7 @@ private IcebergSink( } @Override - public SinkWriter createWriter(InitContext context) { + public SinkWriter createWriter(WriterInitContext context) { RowDataTaskWriterFactory taskWriterFactory = new RowDataTaskWriterFactory( tableSupplier, @@ -216,8 +217,8 @@ public SinkWriter createWriter(InitContext context) { tableSupplier.get().name(), taskWriterFactory, metrics, - context.getSubtaskId(), - context.getAttemptNumber()); + context.getTaskInfo().getIndexOfThisSubtask(), + context.getTaskInfo().getAttemptNumber()); } @Override @@ -318,7 +319,7 @@ public SimpleVersionedSerializer getWriteResultSerializer() { return new WriteResultSerializer(); } - public static class Builder { + public static class Builder implements IcebergSinkBuilder { private TableLoader tableLoader; private String uidSuffix = ""; private Function> inputCreator = null; @@ -393,6 +394,7 @@ private Builder forMapperOutputType( * @param newTable the loaded iceberg table instance. * @return {@link IcebergSink.Builder} to connect the iceberg table. */ + @Override public Builder table(Table newTable) { this.table = (SerializableTable) SerializableTable.copyOf(newTable); return this; @@ -407,6 +409,7 @@ public Builder table(Table newTable) { * @param newTableLoader to load iceberg table inside tasks. * @return {@link Builder} to connect the iceberg table. */ + @Override public Builder tableLoader(TableLoader newTableLoader) { this.tableLoader = newTableLoader; return this; @@ -429,31 +432,31 @@ public Builder set(String property, String value) { * Set the write properties for IcebergSink. View the supported properties in {@link * FlinkWriteOptions} */ + @Override public Builder setAll(Map properties) { writeOptions.putAll(properties); return this; } - /** - * @deprecated since 1.10.0, will be removed in 2.0.0. Use {@link - * #resolvedSchema(ResolvedSchema)} instead. - */ - @Deprecated + @Override public Builder tableSchema(TableSchema newTableSchema) { this.tableSchema = newTableSchema; return this; } + @Override public Builder resolvedSchema(ResolvedSchema newResolvedSchema) { this.resolvedSchema = newResolvedSchema; return this; } + @Override public Builder overwrite(boolean newOverwrite) { writeOptions.put(FlinkWriteOptions.OVERWRITE_MODE.key(), Boolean.toString(newOverwrite)); return this; } + @Override public Builder flinkConf(ReadableConfig config) { this.readableConfig = config; return this; @@ -467,6 +470,7 @@ public Builder flinkConf(ReadableConfig config) { * @param mode to specify the write distribution mode. * @return {@link IcebergSink.Builder} to connect the iceberg table. */ + @Override public Builder distributionMode(DistributionMode mode) { if (mode != null) { writeOptions.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), mode.modeName()); @@ -536,6 +540,7 @@ public Builder rangeDistributionSortKeyBaseWeight(double weight) { * @param newWriteParallelism the number of parallel iceberg stream writer. * @return {@link IcebergSink.Builder} to connect the iceberg table. */ + @Override public Builder writeParallelism(int newWriteParallelism) { writeOptions.put( FlinkWriteOptions.WRITE_PARALLELISM.key(), Integer.toString(newWriteParallelism)); @@ -551,6 +556,7 @@ public Builder writeParallelism(int newWriteParallelism) { * @param enabled indicate whether it should transform all INSERT/UPDATE_AFTER events to UPSERT. * @return {@link IcebergSink.Builder} to connect the iceberg table. */ + @Override public Builder upsert(boolean enabled) { writeOptions.put(FlinkWriteOptions.WRITE_UPSERT_ENABLED.key(), Boolean.toString(enabled)); return this; @@ -562,6 +568,7 @@ public Builder upsert(boolean enabled) { * @param columns defines the iceberg table's key. * @return {@link Builder} to connect the iceberg table. */ + @Override public Builder equalityFieldColumns(List columns) { this.equalityFieldColumns = columns; return this; @@ -604,6 +611,7 @@ public Builder setSnapshotProperty(String property, String value) { return this; } + @Override public Builder toBranch(String branch) { writeOptions.put(FlinkWriteOptions.BRANCH.key(), branch); return this; @@ -680,6 +688,7 @@ IcebergSink build() { * * @return {@link DataStreamSink} for sink. */ + @Override public DataStreamSink append() { IcebergSink sink = build(); String suffix = defaultSuffix(uidSuffix, table.name()); diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkBuilder.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkBuilder.java new file mode 100644 index 000000000000..577b2b9a4227 --- /dev/null +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkBuilder.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 java.util.List; +import java.util.Map; +import org.apache.flink.annotation.Internal; +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.catalog.ResolvedSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.legacy.api.TableSchema; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; + +/** + * This class is for internal purpose of transition between the previous implementation of Flink's + * sink ({@link FlinkSink}) and the new one implementation based on Flink SinkV2 API ({@link + * IcebergSink}). After we remove the previous implementation, all occurrences of this class would + * be replaced by direct {@link IcebergSink} usage. + */ +@Internal +interface IcebergSinkBuilder> { + + /** + * @deprecated Use {@link #resolvedSchema(ResolvedSchema)} instead. + */ + @Deprecated + T tableSchema(TableSchema newTableSchema); + + T resolvedSchema(ResolvedSchema newResolvedSchema); + + T tableLoader(TableLoader newTableLoader); + + T equalityFieldColumns(List columns); + + T overwrite(boolean newOverwrite); + + T setAll(Map properties); + + T flinkConf(ReadableConfig config); + + T table(Table newTable); + + T writeParallelism(int newWriteParallelism); + + T distributionMode(DistributionMode mode); + + T toBranch(String branch); + + T upsert(boolean enabled); + + DataStreamSink append(); + + /** + * @deprecated Use {@link #forRow(DataStream, ResolvedSchema, boolean)} instead. + */ + @Deprecated + static IcebergSinkBuilder forRow( + DataStream input, TableSchema tableSchema, boolean useV2Sink) { + if (useV2Sink) { + return IcebergSink.forRow(input, tableSchema); + } else { + return FlinkSink.forRow(input, tableSchema); + } + } + + static IcebergSinkBuilder forRow( + DataStream input, ResolvedSchema resolvedSchema, boolean useV2Sink) { + if (useV2Sink) { + return IcebergSink.forRow(input, resolvedSchema); + } else { + return FlinkSink.forRow(input, resolvedSchema); + } + } + + static IcebergSinkBuilder forRowData(DataStream input, boolean useV2Sink) { + if (useV2Sink) { + return IcebergSink.forRowData(input); + } else { + return FlinkSink.forRowData(input); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkWriter.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkWriter.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkWriter.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkWriter.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java similarity index 94% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java index 412d6c7081bf..adb53af27bd7 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java @@ -22,7 +22,6 @@ 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; @@ -46,13 +45,12 @@ class IcebergStreamWriter extends AbstractStreamOperator 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.subTaskId = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(); + this.attemptId = getRuntimeContext().getTaskInfo().getAttemptNumber(); this.writerMetrics = new IcebergStreamWriterMetrics(super.metrics, fullTableName); // Initialize the task writer factory. diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java similarity index 85% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java index 794ade577976..1dbb62363dce 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java @@ -21,6 +21,8 @@ import java.io.IOException; import java.util.Collection; import org.apache.flink.core.io.SimpleVersionedSerialization; +import org.apache.flink.runtime.checkpoint.CheckpointIDCounter; +import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; import org.apache.flink.streaming.api.connector.sink2.CommittableSummary; import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; @@ -43,18 +45,30 @@ class IcebergWriteAggregator extends AbstractStreamOperator> implements OneInputStreamOperator< CommittableMessage, CommittableMessage> { + private static final Logger LOG = LoggerFactory.getLogger(IcebergWriteAggregator.class); private static final byte[] EMPTY_MANIFEST_DATA = new byte[0]; + private final Collection results; + private final TableLoader tableLoader; + + private long lastCheckpointId = CheckpointIDCounter.INITIAL_CHECKPOINT_ID - 1; + private transient ManifestOutputFileFactory icebergManifestOutputFileFactory; private transient Table table; - private final TableLoader tableLoader; IcebergWriteAggregator(TableLoader tableLoader) { this.results = Sets.newHashSet(); this.tableLoader = tableLoader; } + @Override + public void initializeState(StateInitializationContext context) throws Exception { + context + .getRestoredCheckpointId() + .ifPresent(checkpointId -> this.lastCheckpointId = checkpointId); + } + @Override public void open() throws Exception { if (!tableLoader.isOpen()) { @@ -76,11 +90,21 @@ public void open() throws Exception { @Override public void finish() throws IOException { - prepareSnapshotPreBarrier(Long.MAX_VALUE); + prepareSnapshotPreBarrier(lastCheckpointId + 1); } @Override public void prepareSnapshotPreBarrier(long checkpointId) throws IOException { + if (checkpointId == lastCheckpointId) { + // Already flushed. This can happen when finish() above triggers flushing prior creating the + // final checkpoint. The calls are mutually exclusive, but we need to ensure we don't flush + // twice. + LOG.info("Aggregated writes for checkpoint id {} already flushed.", checkpointId); + return; + } + + this.lastCheckpointId = checkpointId; + IcebergCommittable committable = new IcebergCommittable( writeToManifest(results, checkpointId), diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/CompareSchemasVisitor.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/CompareSchemasVisitor.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/CompareSchemasVisitor.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/CompareSchemasVisitor.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DataConverter.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DataConverter.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DataConverter.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DataConverter.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitterMetrics.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitterMetrics.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitterMetrics.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitterMetrics.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicIcebergSink.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicIcebergSink.java similarity index 99% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicIcebergSink.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicIcebergSink.java index 9547de78d6ba..2715a01608d6 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicIcebergSink.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicIcebergSink.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg.flink.sink.dynamic; -import java.io.IOException; import java.util.Map; import java.util.Optional; import java.util.UUID; @@ -30,6 +29,7 @@ import org.apache.flink.api.connector.sink2.Sink; import org.apache.flink.api.connector.sink2.SinkWriter; import org.apache.flink.api.connector.sink2.SupportsCommitter; +import org.apache.flink.api.connector.sink2.WriterInitContext; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; @@ -109,7 +109,7 @@ public class DynamicIcebergSink } @Override - public SinkWriter createWriter(InitContext context) throws IOException { + public SinkWriter createWriter(WriterInitContext context) { return new DynamicWriter( catalogLoader.loadCatalog(), dataFileFormat, diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecord.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecord.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecord.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecord.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordGenerator.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordGenerator.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordGenerator.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordGenerator.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternal.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternal.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternal.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternal.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializer.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializer.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializer.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializer.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalType.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalType.java similarity index 89% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalType.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalType.java index 6be081aadf77..c18c8f670daf 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalType.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalType.java @@ -19,7 +19,6 @@ package org.apache.iceberg.flink.sink.dynamic; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.serialization.SerializerConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -75,12 +74,6 @@ public TypeSerializer createSerializer(SerializerConfig s new TableSerializerCache(catalogLoader, cacheSize), writeSchemaAndSpec); } - @Override - public TypeSerializer createSerializer(ExecutionConfig executionConfig) { - return new DynamicRecordInternalSerializer( - new TableSerializerCache(catalogLoader, cacheSize), writeSchemaAndSpec); - } - @Override public String toString() { return getClass().getName(); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicSinkUtil.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicSinkUtil.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicSinkUtil.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicSinkUtil.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResult.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResult.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResult.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResult.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java similarity index 89% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java index 58ba183dfcd4..c833527ecc55 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java @@ -26,6 +26,8 @@ import java.util.Map; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.core.io.SimpleVersionedSerialization; +import org.apache.flink.runtime.checkpoint.CheckpointIDCounter; +import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; import org.apache.flink.streaming.api.connector.sink2.CommittableSummary; import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; @@ -56,11 +58,15 @@ class DynamicWriteResultAggregator extends AbstractStreamOperator> implements OneInputStreamOperator< CommittableMessage, CommittableMessage> { + private static final Logger LOG = LoggerFactory.getLogger(DynamicWriteResultAggregator.class); private static final byte[] EMPTY_MANIFEST_DATA = new byte[0]; private static final Duration CACHE_EXPIRATION_DURATION = Duration.ofMinutes(1); private final CatalogLoader catalogLoader; + + private long lastCheckpointId = CheckpointIDCounter.INITIAL_CHECKPOINT_ID - 1; + private transient Map> results; private transient Cache> specs; private transient Cache outputFileFactories; @@ -74,6 +80,13 @@ class DynamicWriteResultAggregator this.catalogLoader = catalogLoader; } + @Override + public void initializeState(StateInitializationContext context) throws Exception { + context + .getRestoredCheckpointId() + .ifPresent(checkpointId -> this.lastCheckpointId = checkpointId); + } + @Override public void open() throws Exception { this.flinkJobId = getContainingTask().getEnvironment().getJobID().toString(); @@ -90,11 +103,21 @@ public void open() throws Exception { @Override public void finish() throws IOException { - prepareSnapshotPreBarrier(Long.MAX_VALUE); + prepareSnapshotPreBarrier(lastCheckpointId + 1); } @Override public void prepareSnapshotPreBarrier(long checkpointId) throws IOException { + if (checkpointId == lastCheckpointId) { + // Already flushed. This can happen when finish() above triggers flushing prior creating the + // final checkpoint. The calls are mutually exclusive, but we need to ensure we don't flush + // twice. + LOG.info("Aggregated writes for checkpoint id {} already flushed.", checkpointId); + return; + } + + this.lastCheckpointId = checkpointId; + Collection> committables = Sets.newHashSetWithExpectedSize(results.size()); int count = 0; diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultSerializer.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultSerializer.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultSerializer.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultSerializer.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/EvolveSchemaVisitor.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/EvolveSchemaVisitor.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/EvolveSchemaVisitor.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/EvolveSchemaVisitor.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/LRUCache.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/LRUCache.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/LRUCache.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/LRUCache.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecEvolution.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecEvolution.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecEvolution.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/PartitionSpecEvolution.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableSerializerCache.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableSerializerCache.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableSerializerCache.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableSerializerCache.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableUpdater.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableUpdater.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableUpdater.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableUpdater.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java similarity index 98% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java index 7995a8a5b181..a873136c9194 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java @@ -30,6 +30,7 @@ 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.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.table.data.RowData; import org.apache.iceberg.Schema; @@ -72,12 +73,14 @@ public class DataStatisticsOperator extends AbstractStreamOperator parameters, String operatorName, Schema schema, SortOrder sortOrder, OperatorEventGateway operatorEventGateway, int downstreamParallelism, StatisticsType statisticsType) { + super(parameters); this.operatorName = operatorName; this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); this.sortKey = new SortKey(schema, sortOrder); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java similarity index 94% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java index dc147bf36d13..7ece89fdcc69 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java @@ -79,10 +79,8 @@ public > T createStreamOperator( DataStatisticsOperator rangeStatisticsOperator = new DataStatisticsOperator( - operatorName, schema, sortOrder, gateway, downstreamParallelism, type); + parameters, operatorName, schema, sortOrder, gateway, downstreamParallelism, type); - rangeStatisticsOperator.setup( - parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); parameters .getOperatorEventDispatcher() .registerEventHandler(operatorId, rangeStatisticsOperator); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java similarity index 98% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java index acd078a61cd3..6f5bb6722771 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java @@ -68,12 +68,7 @@ class SortKeySerializer extends TypeSerializer { 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()); + Types.NestedField.from(sourceField).ofType(resultType).build(); transformedFields[i] = transformedField; } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java similarity index 92% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java index 1e5bdbbac3e4..891f2f02a241 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java @@ -45,12 +45,11 @@ static Schema sortKeySchema(Schema schema, SortOrder sortOrder) { // case. To resolve the collision, field id is set to transform index and field name is set to // sourceFieldName_transformIndex Types.NestedField transformedField = - Types.NestedField.of( - i, - sourceField.isOptional(), - sourceField.name() + '_' + i, - transformedType, - sourceField.doc()); + Types.NestedField.from(sourceField) + .withId(i) + .withName(sourceField.name() + '_' + i) + .ofType(transformedType) + .build(); transformedFields.add(transformedField); } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordTypeInformation.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordTypeInformation.java similarity index 94% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordTypeInformation.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordTypeInformation.java index 921ede9466e0..aa3af2bd2e38 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordTypeInformation.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordTypeInformation.java @@ -19,7 +19,6 @@ package org.apache.iceberg.flink.sink.shuffle; import java.util.Objects; -import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.serialization.SerializerConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -79,11 +78,6 @@ public TypeSerializer createSerializer(SerializerConfig conf return new StatisticsOrRecordSerializer(globalStatisticsSerializer, recordSerializer); } - @Override - public TypeSerializer createSerializer(ExecutionConfig config) { - return createSerializer(config.getSerializerConfig()); - } - @Override public String toString() { return "StatisticsOrRecord"; diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java similarity index 98% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java index b5a1ba85a6cb..e0c99107d549 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java @@ -28,9 +28,9 @@ 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.streaming.api.functions.source.SourceFunction; -import org.apache.flink.table.api.TableSchema; +import org.apache.flink.streaming.api.functions.source.legacy.SourceFunction; import org.apache.flink.table.data.RowData; +import org.apache.flink.table.legacy.api.TableSchema; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java similarity index 99% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java index 035682be8296..ec7cb010b6be 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -41,9 +41,9 @@ 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.catalog.ResolvedSchema; import org.apache.flink.table.data.RowData; +import org.apache.flink.table.legacy.api.TableSchema; import org.apache.iceberg.BaseMetadataTable; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java similarity index 97% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java index e2d131dc3b3d..aeecd43e7f14 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java @@ -27,7 +27,6 @@ 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.catalog.Column; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.connector.ChangelogMode; @@ -42,6 +41,8 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.table.expressions.ResolvedExpression; import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.legacy.api.TableSchema; +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; @@ -111,7 +112,7 @@ private IcebergTableSource( } @Override - public void applyProjection(int[][] projectFields) { + public void applyProjection(int[][] projectFields, DataType producedDataType) { this.projectedFields = new int[projectFields.length]; for (int i = 0; i < projectFields.length; i++) { Preconditions.checkArgument( diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java similarity index 95% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java index 391633924264..d27b2531eec0 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java @@ -24,8 +24,8 @@ import java.util.Collections; import java.util.List; import java.util.stream.Collectors; +import org.apache.flink.api.common.functions.OpenContext; 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; @@ -125,9 +125,9 @@ public RewriteMap( } @Override - public void open(Configuration parameters) { - this.subTaskId = getRuntimeContext().getIndexOfThisSubtask(); - this.attemptId = getRuntimeContext().getAttemptNumber(); + public void open(OpenContext parameters) { + this.subTaskId = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(); + this.attemptId = getRuntimeContext().getTaskInfo().getAttemptNumber(); // Initialize the task writer factory. this.taskWriterFactory.initialize(subTaskId, attemptId); } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java similarity index 98% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java index 39f615aeacc5..133859b657e5 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java @@ -22,15 +22,15 @@ import java.io.UncheckedIOException; import java.util.List; import java.util.concurrent.ExecutorService; +import org.apache.flink.api.common.functions.OpenContext; 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.functions.source.legacy.RichSourceFunction; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; @@ -97,7 +97,7 @@ public StreamingMonitorFunction(TableLoader tableLoader, ScanContext scanContext } @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext parameters) throws Exception { super.open(parameters); final RuntimeContext runtimeContext = getRuntimeContext(); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java similarity index 92% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java index ee6f7b63988d..6cc2ccd2c353 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java @@ -26,7 +26,7 @@ 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.functions.source.legacy.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; @@ -34,7 +34,7 @@ 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.operators.legacy.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; @@ -82,7 +82,11 @@ public class StreamingReaderOperator extends AbstractStreamOperator private transient SplitState currentSplitState; private StreamingReaderOperator( - FlinkInputFormat format, ProcessingTimeService timeService, MailboxExecutor mailboxExecutor) { + StreamOperatorParameters parameters, + FlinkInputFormat format, + ProcessingTimeService timeService, + MailboxExecutor mailboxExecutor) { + super(parameters); this.format = Preconditions.checkNotNull(format, "The InputFormat should not be null."); this.processingTimeService = timeService; this.executor = @@ -106,7 +110,7 @@ public void initializeState(StateInitializationContext context) throws Exception // Recover splits state from flink state backend if possible. splits = Lists.newLinkedList(); if (context.isRestored()) { - int subtaskIdx = getRuntimeContext().getIndexOfThisSubtask(); + int subtaskIdx = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(); LOG.info("Restoring state for the {} (taskIdx: {}).", getClass().getSimpleName(), subtaskIdx); for (FlinkInputSplit split : inputSplitsState.get()) { @@ -116,11 +120,10 @@ public void initializeState(StateInitializationContext context) throws Exception this.sourceContext = StreamSourceContexts.getSourceContext( - getOperatorConfig().getTimeCharacteristic(), getProcessingTimeService(), new Object(), // no actual locking needed output, - getRuntimeContext().getExecutionConfig().getAutoWatermarkInterval(), + getExecutionConfig().getAutoWatermarkInterval(), -1, true); @@ -232,9 +235,7 @@ public void setMailboxExecutor(MailboxExecutor mailboxExecutor) { public > O createStreamOperator( StreamOperatorParameters parameters) { StreamingReaderOperator operator = - new StreamingReaderOperator(format, processingTimeService, mailboxExecutor); - operator.setup( - parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); + new StreamingReaderOperator(parameters, format, processingTimeService, mailboxExecutor); return (O) operator; } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordConverter.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordConverter.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordConverter.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordConverter.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ConverterReaderFunction.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/ConverterReaderFunction.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ConverterReaderFunction.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/ConverterReaderFunction.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowConverter.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowConverter.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowConverter.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowConverter.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataConverter.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataConverter.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataConverter.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataConverter.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/ElapsedTimeGauge.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/util/ElapsedTimeGauge.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/ElapsedTimeGauge.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/util/ElapsedTimeGauge.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java similarity index 97% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java index 50fc83dba8e2..38bd73b87127 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java @@ -20,9 +20,9 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.table.api.Schema; -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.legacy.api.TableColumn; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.table.types.logical.RowType; diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java diff --git a/flink/v1.19/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink/v2.1/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory similarity index 94% rename from flink/v1.19/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory rename to flink/v2.1/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory index 29a9955a7e20..3034b2d3754a 100644 --- a/flink/v1.19/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ b/flink/v2.1/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -13,4 +13,5 @@ # See the License for the specific language governing permissions and # limitations under the License. +org.apache.iceberg.flink.FlinkCatalogFactory org.apache.iceberg.flink.FlinkDynamicTableFactory diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopTableExtension.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/HadoopTableExtension.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopTableExtension.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/HadoopTableExtension.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java similarity index 99% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java index ac15add88353..d9c9f7ad3f02 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java @@ -27,12 +27,12 @@ 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.catalog.Column; import org.apache.flink.table.catalog.ResolvedSchema; 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.legacy.api.TableSchema; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.types.RowKind; import org.apache.hadoop.conf.Configuration; diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SqlBase.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/SqlBase.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SqlBase.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/SqlBase.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestBase.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestBase.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestBase.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestBase.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java similarity index 96% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java index 8992cbd75187..fa8c09f66651 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java @@ -30,7 +30,7 @@ import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.util.Map; -import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializerConfigImpl; import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; import org.apache.flink.core.memory.DataInputDeserializer; import org.apache.flink.core.memory.DataOutputSerializer; @@ -155,7 +155,8 @@ public void testJavaSerialization() throws Exception { @Test public void testDataFileKryoSerialization() throws IOException { - KryoSerializer kryo = new KryoSerializer<>(DataFile.class, new ExecutionConfig()); + KryoSerializer kryo = + new KryoSerializer<>(DataFile.class, new SerializerConfigImpl()); DataOutputSerializer outputView = new DataOutputSerializer(1024); @@ -172,7 +173,8 @@ public void testDataFileKryoSerialization() throws IOException { @Test public void testDeleteFileKryoSerialization() throws IOException { - KryoSerializer kryo = new KryoSerializer<>(DeleteFile.class, new ExecutionConfig()); + KryoSerializer kryo = + new KryoSerializer<>(DeleteFile.class, new SerializerConfigImpl()); DataOutputSerializer outputView = new DataOutputSerializer(1024); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java similarity index 97% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java index 2cfee87e4631..f7848a5d22ef 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java @@ -34,6 +34,7 @@ import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.CommonCatalogOptions; import org.apache.flink.table.catalog.ObjectPath; import org.apache.flink.table.catalog.exceptions.TableNotExistException; import org.apache.iceberg.BaseTable; @@ -227,7 +228,13 @@ public void testCreateTableLikeInFlinkCatalog() throws TableNotExistException { .column("id", DataTypes.BIGINT()) .build()); - String srcCatalogProps = FlinkCreateTableOptions.toJson(catalogName, DATABASE, "tl", config); + // `type` option is filtered out by Flink + // https://github.com/apache/flink/blob/edc3d68736de73665440f4313ddcfd9142d8d42b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/FactoryUtil.java#L378 + Map filteredOptions = Maps.newHashMap(config); + filteredOptions.remove(CommonCatalogOptions.CATALOG_TYPE.key()); + + String srcCatalogProps = + FlinkCreateTableOptions.toJson(catalogName, DATABASE, "tl", filteredOptions); Map options = catalogTable.getOptions(); assertThat(options) .containsEntry( @@ -287,7 +294,7 @@ public void testCreateTableWithColumnComment() { } @TestTemplate - public void testCreateTableWithFormatV2ThroughTableProperty() { + public void testCreateTableWithFormatV2ThroughTableProperty() throws Exception { sql("CREATE TABLE tl(id BIGINT) WITH ('format-version'='2')"); Table table = table("tl"); @@ -295,7 +302,7 @@ public void testCreateTableWithFormatV2ThroughTableProperty() { } @TestTemplate - public void testUpgradeTableWithFormatV2ThroughTableProperty() { + public void testUpgradeTableWithFormatV2ThroughTableProperty() throws Exception { sql("CREATE TABLE tl(id BIGINT) WITH ('format-version'='1')"); Table table = table("tl"); @@ -310,7 +317,7 @@ public void testUpgradeTableWithFormatV2ThroughTableProperty() { } @TestTemplate - public void testDowngradeTableToFormatV1ThroughTablePropertyFails() { + public void testDowngradeTableToFormatV1ThroughTablePropertyFails() throws Exception { sql("CREATE TABLE tl(id BIGINT) WITH ('format-version'='2')"); Table table = table("tl"); @@ -342,7 +349,7 @@ public void testLoadTransformPartitionTable() throws TableNotExistException { } @TestTemplate - public void testAlterTableProperties() { + public void testAlterTableProperties() throws TableNotExistException { sql("CREATE TABLE tl(id BIGINT) WITH ('oldK'='oldV')"); Map properties = Maps.newHashMap(); properties.put("oldK", "oldV"); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java similarity index 98% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java index f19f0b447878..59b868ea1ef1 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java @@ -429,7 +429,11 @@ public Expression visit(UnresolvedCallExpression unresolvedCall) { .map(e -> (ResolvedExpression) e.accept(this)) .collect(Collectors.toList()); return new CallExpression( - unresolvedCall.getFunctionDefinition(), children, DataTypes.STRING()); + false, + unresolvedCall.getFunctionIdentifier().orElse(null), + unresolvedCall.getFunctionDefinition(), + children, + DataTypes.STRING()); } @Override diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java similarity index 99% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java index 6236dc6e2df2..ce18a1bb3d50 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java @@ -24,11 +24,11 @@ import java.util.Collections; import java.util.List; 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.catalog.Column; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.catalog.UniqueConstraint; +import org.apache.flink.table.legacy.api.TableSchema; import org.apache.flink.table.types.logical.BinaryType; import org.apache.flink.table.types.logical.CharType; import org.apache.flink.table.types.logical.LocalZonedTimestampType; diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkCompaction.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkCompaction.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkCompaction.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkCompaction.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java similarity index 99% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java index 26aa9d2b4c58..d8d3c5dc249b 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java @@ -34,7 +34,7 @@ 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.serialization.SerializerConfigImpl; 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; @@ -74,7 +74,7 @@ public class TestHelpers { private TestHelpers() {} public static T roundTripKryoSerialize(Class clazz, T table) throws IOException { - KryoSerializer kryo = new KryoSerializer<>(clazz, new ExecutionConfig()); + KryoSerializer kryo = new KryoSerializer<>(clazz, new SerializerConfigImpl()); DataOutputSerializer outputView = new DataOutputSerializer(1024); kryo.serialize(table, outputView); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java similarity index 99% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java index 47f5485df879..0a6d5e44caa2 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java @@ -252,7 +252,7 @@ public void testCatalogDatabaseConflictWithFlinkDatabase() { // Ensure that the table was created under the specific database. assertThatThrownBy( () -> sql("CREATE TABLE `default_catalog`.`%s`.`%s`", databaseName(), TABLE_NAME)) - .isInstanceOf(org.apache.flink.table.api.TableException.class) + .isInstanceOf(org.apache.flink.table.api.ValidationException.class) .hasMessageStartingWith("Could not execute CreateTable in path"); } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java similarity index 97% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java index 8f1f129e183b..5e7935be01c4 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java @@ -31,7 +31,7 @@ import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.file.Path; -import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializerConfigImpl; import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; import org.apache.flink.core.memory.DataInputDeserializer; import org.apache.flink.core.memory.DataOutputSerializer; @@ -108,7 +108,7 @@ public class TestManifestFileSerialization { @Test public void testKryoSerialization() throws IOException { KryoSerializer kryo = - new KryoSerializer<>(ManifestFile.class, new ExecutionConfig()); + new KryoSerializer<>(ManifestFile.class, new SerializerConfigImpl()); DataOutputSerializer outputView = new DataOutputSerializer(1024); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java similarity index 93% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java index 74b1da6007e6..5ffd31ad2010 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java @@ -18,8 +18,8 @@ */ package org.apache.iceberg.flink.data; +import org.apache.flink.api.common.functions.OpenContext; 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; @@ -38,7 +38,7 @@ public RowDataToRowMapper(RowType rowType) { } @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext parameters) throws Exception { this.converter = DataStructureConverters.getConverter(TypeConversions.fromLogicalToDataType(rowType)); } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java similarity index 99% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java index 004cc8234876..3b98939ea167 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java @@ -476,7 +476,6 @@ public void testListProjection() throws IOException { } @Test - @SuppressWarnings("unchecked") public void testListOfStructsProjection() throws IOException { Schema writeSchema = new Schema( diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskInfraExtension.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskInfraExtension.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskInfraExtension.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskInfraExtension.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskTestBase.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskTestBase.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskTestBase.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskTestBase.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestDeleteOrphanFiles.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestDeleteOrphanFiles.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestDeleteOrphanFiles.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestDeleteOrphanFiles.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestJdbcLockFactory.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestJdbcLockFactory.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestJdbcLockFactory.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestJdbcLockFactory.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestLockFactoryBase.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestLockFactoryBase.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestLockFactoryBase.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestLockFactoryBase.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestMaintenanceE2E.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestMaintenanceE2E.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestMaintenanceE2E.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestMaintenanceE2E.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestRewriteDataFiles.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestRewriteDataFiles.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestRewriteDataFiles.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestRewriteDataFiles.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestRewriteDataFilesConfig.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestRewriteDataFilesConfig.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestRewriteDataFilesConfig.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestRewriteDataFilesConfig.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestTableMaintenance.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestTableMaintenance.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestTableMaintenance.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestTableMaintenance.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestZkLockFactory.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestZkLockFactory.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestZkLockFactory.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestZkLockFactory.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/CollectingSink.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/CollectingSink.java similarity index 96% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/CollectingSink.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/CollectingSink.java index e7e818ba6887..e09e312be1dd 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/CollectingSink.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/CollectingSink.java @@ -28,6 +28,7 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.flink.api.connector.sink2.Sink; import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.api.connector.sink2.WriterInitContext; import org.apache.iceberg.relocated.com.google.common.collect.Lists; /** Sink for collecting output during testing. */ @@ -86,7 +87,7 @@ public T poll(Duration timeout) throws TimeoutException { } @Override - public SinkWriter createWriter(InitContext context) { + public SinkWriter createWriter(WriterInitContext context) { return new CollectingWriter<>(index); } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ManualSource.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ManualSource.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ManualSource.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ManualSource.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/MetricsReporterFactoryForTests.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/MetricsReporterFactoryForTests.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/MetricsReporterFactoryForTests.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/MetricsReporterFactoryForTests.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java similarity index 97% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java index 8460b392e278..f9cbc9715cce 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java @@ -24,11 +24,11 @@ import java.io.File; import java.io.IOException; import java.nio.file.Path; +import javax.annotation.Nullable; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.MetricOptions; import org.apache.flink.core.execution.JobClient; import org.apache.flink.core.execution.SavepointFormatType; -import org.apache.flink.runtime.jobgraph.SavepointConfigOptions; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.graph.StreamGraphGenerator; @@ -254,24 +254,21 @@ protected TableLoader tableLoader() { * stop without a savepoint. * @return configuration for restarting the job from the savepoint */ - protected static Configuration closeJobClient(JobClient jobClient, File savepointDir) { - Configuration conf = new Configuration(); + @Nullable + protected static String closeJobClient(JobClient jobClient, File savepointDir) { if (jobClient != null) { if (savepointDir != null) { // Stop with savepoint jobClient.stopWithSavepoint(false, savepointDir.getPath(), SavepointFormatType.CANONICAL); // Wait until the savepoint is created and the job has been stopped Awaitility.await().until(() -> savepointDir.listFiles(File::isDirectory).length == 1); - conf.set( - SavepointConfigOptions.SAVEPOINT_PATH, - savepointDir.listFiles(File::isDirectory)[0].getAbsolutePath()); + return savepointDir.listFiles(File::isDirectory)[0].getAbsolutePath(); } else { jobClient.cancel(); } // Wait until the job has been stopped Awaitility.await().until(() -> jobClient.getJobStatus().get().isTerminalState()); - return conf; } return null; diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/RewriteUtil.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/RewriteUtil.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/RewriteUtil.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/RewriteUtil.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewriteCommitter.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewriteCommitter.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewriteCommitter.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewriteCommitter.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewritePlanner.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewritePlanner.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewritePlanner.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewritePlanner.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewriteRunner.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewriteRunner.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewriteRunner.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewriteRunner.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDeleteFilesProcessor.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDeleteFilesProcessor.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDeleteFilesProcessor.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDeleteFilesProcessor.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestExpireSnapshotsProcessor.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestExpireSnapshotsProcessor.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestExpireSnapshotsProcessor.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestExpireSnapshotsProcessor.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestListFileSystemFiles.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestListFileSystemFiles.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestListFileSystemFiles.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestListFileSystemFiles.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestListMetadataFiles.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestListMetadataFiles.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestListMetadataFiles.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestListMetadataFiles.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockConfig.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockConfig.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockConfig.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockConfig.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockFactoryBuilder.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockFactoryBuilder.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockFactoryBuilder.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockFactoryBuilder.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java similarity index 99% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java index a163dcaa71f7..7b88f20e376a 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java @@ -34,6 +34,7 @@ import org.apache.flink.api.connector.sink2.Sink; import org.apache.flink.api.connector.sink2.SinkWriter; import org.apache.flink.api.connector.sink2.SupportsCommitter; +import org.apache.flink.api.connector.sink2.WriterInitContext; import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.execution.JobClient; @@ -354,7 +355,7 @@ private static class SinkTest SupportsCommitter, SupportsPostCommitTopology { @Override - public SinkWriter createWriter(InitContext initContext) { + public SinkWriter createWriter(WriterInitContext initContext) { return new CommittingSinkWriter() { private final Collection received = Lists.newArrayList(); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java similarity index 93% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java index c561c7054eae..9c1ea2f2295a 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java @@ -25,15 +25,21 @@ import java.io.IOException; import java.time.Duration; import java.util.List; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicReference; +import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; +import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.execution.JobClient; import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.graph.StreamGraph; +import org.apache.flink.test.junit5.InjectClusterClient; import org.apache.hadoop.fs.Path; import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; @@ -157,7 +163,9 @@ void testSource() throws Exception { /** Check that the {@link MonitorSource} operator state is restored correctly. */ @Test - void testStateRestore(@TempDir File savepointDir) throws Exception { + void testStateRestore( + @TempDir File savepointDir, @InjectClusterClient ClusterClient clusterClient) + throws Exception { Table table = createTable(); insert(table, 1, "a"); TableLoader tableLoader = tableLoader(); @@ -180,7 +188,7 @@ void testStateRestore(@TempDir File savepointDir) throws Exception { events.sinkTo(result); // Start the job - Configuration conf; + String savepointPath; JobClient jobClient = null; AtomicReference firstNonEmptyEvent = new AtomicReference<>(); try { @@ -201,11 +209,11 @@ void testStateRestore(@TempDir File savepointDir) throws Exception { }); } finally { // Stop with savepoint - conf = closeJobClient(jobClient, savepointDir); + savepointPath = closeJobClient(jobClient, savepointDir); } // Restore from savepoint, create the same topology with a different env - env = StreamExecutionEnvironment.getExecutionEnvironment(conf); + env = StreamExecutionEnvironment.getExecutionEnvironment(); events = env.fromSource( new MonitorSource(tableLoader, LOW_RATE, Long.MAX_VALUE), @@ -216,13 +224,13 @@ void testStateRestore(@TempDir File savepointDir) throws Exception { events.sinkTo(resultWithSavepoint); // Make sure that the job with restored source does not read new records from the table - JobClient clientWithSavepoint = null; + StreamGraph streamGraph = env.getStreamGraph(); + streamGraph.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(savepointPath)); + CompletableFuture jobIDCompletableFuture = clusterClient.submitJob(streamGraph); try { - clientWithSavepoint = env.executeAsync("Table Change Source test with savepoint"); - assertThat(resultWithSavepoint.poll(Duration.ofSeconds(5L))).isEqualTo(EMPTY_EVENT); } finally { - closeJobClient(clientWithSavepoint, null); + clusterClient.cancel(jobIDCompletableFuture.get()); } // Restore without savepoint diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestOrphanFilesDetector.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestOrphanFilesDetector.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestOrphanFilesDetector.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestOrphanFilesDetector.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestSkipOnError.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestSkipOnError.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestSkipOnError.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestSkipOnError.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTablePlanerAndReader.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTablePlanerAndReader.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTablePlanerAndReader.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTablePlanerAndReader.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTaskResultAggregator.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTaskResultAggregator.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTaskResultAggregator.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTaskResultAggregator.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/SinkTestUtil.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/SinkTestUtil.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/SinkTestUtil.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/SinkTestUtil.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCommittableToTableChangeConverter.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestCommittableToTableChangeConverter.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCommittableToTableChangeConverter.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestCommittableToTableChangeConverter.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java similarity index 98% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java index fe37aa9e56eb..55e00d39b316 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java @@ -28,10 +28,10 @@ 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.table.api.TableSchema; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.util.DataFormatConverters; +import org.apache.flink.table.legacy.api.TableSchema; import org.apache.flink.table.runtime.typeutils.ExternalTypeInfo; import org.apache.flink.table.types.DataType; import org.apache.flink.test.junit5.MiniClusterExtension; diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java similarity index 99% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java index fefd88cf57fe..a77ddead3003 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java @@ -25,8 +25,8 @@ 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.table.catalog.ResolvedSchema; +import org.apache.flink.table.legacy.api.TableSchema; import org.apache.flink.types.Row; import org.apache.iceberg.DistributionMode; import org.apache.iceberg.FileFormat; diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2DistributionMode.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2DistributionMode.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2DistributionMode.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2DistributionMode.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java similarity index 98% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java index 584b9962eaee..abac7e16d9f0 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java @@ -367,8 +367,7 @@ public void testSingleCommit() throws Exception { SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) .hasFailedCommittables(committableSummary.getNumberOfFailedCommittables()) - .hasOverallCommittables(committableSummary.getNumberOfCommittables()) - .hasPendingCommittables(0); + .hasOverallCommittables(committableSummary.getNumberOfCommittables()); SinkV2Assertions.assertThat(extractAndAssertCommittableWithLineage(output.get(1))) .hasSubtaskId(0) @@ -427,8 +426,7 @@ public void testStateRestoreFromPreJobWithUncommitted() throws Exception { SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) .hasFailedCommittables(committableSummary.getNumberOfFailedCommittables()) - .hasOverallCommittables(committableSummary.getNumberOfCommittables()) - .hasPendingCommittables(0); + .hasOverallCommittables(committableSummary.getNumberOfCommittables()); SinkV2Assertions.assertThat(extractAndAssertCommittableWithLineage(output.get(1))) .hasCheckpointId(0L) @@ -453,8 +451,7 @@ public void testStateRestoreFromPreJobWithUncommitted() throws Exception { List output2 = transformsToStreamElement(restored.getOutput()); SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output2.get(0))) .hasFailedCommittables(committableSummary2.getNumberOfFailedCommittables()) - .hasOverallCommittables(committableSummary2.getNumberOfCommittables()) - .hasPendingCommittables(0); + .hasOverallCommittables(committableSummary2.getNumberOfCommittables()); SinkV2Assertions.assertThat(extractAndAssertCommittableWithLineage(output2.get(1))) .hasCheckpointId(0L) @@ -503,8 +500,7 @@ public void testStateRestoreFromPreJobWithCommitted() throws Exception { SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) .hasFailedCommittables(committableSummary.getNumberOfFailedCommittables()) - .hasOverallCommittables(committableSummary.getNumberOfCommittables()) - .hasPendingCommittables(0); + .hasOverallCommittables(committableSummary.getNumberOfCommittables()); assertSnapshotSize(1); assertMaxCommittedCheckpointId(jobId1, checkpointId); @@ -549,8 +545,7 @@ public void testStateRestoreFromPreJobWithCommitted() throws Exception { assertThat(output3).hasSize(4); SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output3.get(0))) .hasFailedCommittables(committableSummary2.getNumberOfFailedCommittables()) - .hasOverallCommittables(committableSummary2.getNumberOfCommittables()) - .hasPendingCommittables(0); + .hasOverallCommittables(committableSummary2.getNumberOfCommittables()); } assertSnapshotSize(2); @@ -609,8 +604,7 @@ public void testStateRestoreFromCurrJob() throws Exception { SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) .hasFailedCommittables(committableSummary.getNumberOfFailedCommittables()) - .hasOverallCommittables(committableSummary.getNumberOfCommittables()) - .hasPendingCommittables(0); + .hasOverallCommittables(committableSummary.getNumberOfCommittables()); table.refresh(); Snapshot currentSnapshot = table.snapshot(branch); @@ -631,8 +625,7 @@ public void testStateRestoreFromCurrJob() throws Exception { List output2 = transformsToStreamElement(restored.getOutput()); SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output2.get(0))) .hasFailedCommittables(committableSummary2.getNumberOfFailedCommittables()) - .hasOverallCommittables(committableSummary2.getNumberOfCommittables()) - .hasPendingCommittables(0); + .hasOverallCommittables(committableSummary2.getNumberOfCommittables()); restored.close(); assertSnapshotSize(2); @@ -715,8 +708,8 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except processElement(jobId, checkpointId, harness, 1, operatorId.toString(), dataFile); snapshot = harness.snapshot(++checkpointId, ++timestamp); - assertFlinkManifests(1); + harness.notifyOfCompletedCheckpoint(checkpointId); } // Redeploying flink job from external checkpoint. @@ -1051,19 +1044,18 @@ public void testHandleEndInput() throws Exception { testHarness.open(); - long checkpointId = Long.MAX_VALUE; + long checkpointId = 1; processElement(jobId, checkpointId, testHarness, 1, OPERATOR_ID, dataFileTest1); testHarness.endInput(); - assertMaxCommittedCheckpointId(jobId, OPERATOR_ID, Long.MAX_VALUE); + assertMaxCommittedCheckpointId(jobId, OPERATOR_ID, checkpointId); List output = transformsToStreamElement(testHarness.getOutput()); assertThat(output).hasSize(2); SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) .hasCheckpointId(checkpointId) - .hasPendingCommittables(0) .hasOverallCommittables(1) .hasFailedCommittables(0); @@ -1102,7 +1094,6 @@ public void testDeleteFiles() throws Exception { assertThat(output).hasSize(2); SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) .hasCheckpointId(checkpointId) - .hasPendingCommittables(0) .hasOverallCommittables(1) .hasFailedCommittables(0); SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); @@ -1127,7 +1118,6 @@ public void testDeleteFiles() throws Exception { assertThat(output2).hasSize(2 + 2); SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output2.get(2))) .hasCheckpointId(checkpointId) - .hasPendingCommittables(0) .hasOverallCommittables(1) .hasFailedCommittables(0); @@ -1159,7 +1149,6 @@ public void testDeleteFiles() throws Exception { assertThat(output3).hasSize(2 + 2 + 2); SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output3.get(4))) .hasCheckpointId(checkpointId) - .hasPendingCommittables(0) .hasOverallCommittables(1) .hasFailedCommittables(0); } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java similarity index 99% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java index bc6e00b03d9b..65fb9b8f69b4 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java @@ -1217,16 +1217,16 @@ private static TestOperatorFactory of(String tablePath, String branch, Partition @Override @SuppressWarnings("unchecked") public > T createStreamOperator( - StreamOperatorParameters param) { + StreamOperatorParameters params) { IcebergFilesCommitter committer = new IcebergFilesCommitter( + params, 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; } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSink.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSink.java similarity index 99% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSink.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSink.java index bf7f7b5e9815..4f2b09ee55ff 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSink.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSink.java @@ -31,9 +31,9 @@ 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.catalog.ResolvedSchema; import org.apache.flink.table.data.RowData; +import org.apache.flink.table.legacy.api.TableSchema; import org.apache.flink.types.Row; import org.apache.iceberg.DistributionMode; import org.apache.iceberg.FileFormat; diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java similarity index 98% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java index ee5560712657..ddcb57f6ca33 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java @@ -24,8 +24,8 @@ 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.table.catalog.ResolvedSchema; +import org.apache.flink.table.legacy.api.TableSchema; import org.apache.flink.types.Row; import org.apache.iceberg.DistributionMode; import org.apache.iceberg.FileFormat; diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkCompact.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkCompact.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkCompact.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkCompact.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2Branch.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2Branch.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2Branch.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2Branch.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerTestBase.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerTestBase.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerTestBase.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerTestBase.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestCompareSchemasVisitor.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestCompareSchemasVisitor.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestCompareSchemasVisitor.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestCompareSchemasVisitor.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommittableSerializer.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommittableSerializer.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommittableSerializer.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommittableSerializer.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchema.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchema.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchema.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchema.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaId.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaId.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaId.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaId.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicTableUpdateOperator.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicTableUpdateOperator.java similarity index 96% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicTableUpdateOperator.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicTableUpdateOperator.java index 01e2c440df67..618074f412f9 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicTableUpdateOperator.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicTableUpdateOperator.java @@ -23,7 +23,6 @@ import static org.assertj.core.api.Assertions.assertThat; import java.util.Collections; -import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.table.data.GenericRowData; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; @@ -63,7 +62,7 @@ void testDynamicTableUpdateOperatorNewTable() throws Exception { cacheMaximumSize, cacheRefreshMs, inputSchemaCacheMaximumSize); - operator.open((OpenContext) null); + operator.open(null); DynamicRecordInternal input = new DynamicRecordInternal( @@ -95,7 +94,7 @@ void testDynamicTableUpdateOperatorSchemaChange() throws Exception { cacheMaximumSize, cacheRefreshMs, inputSchemaCacheMaximumSize); - operator.open((OpenContext) null); + operator.open(null); catalog.createTable(table, SCHEMA1); DynamicRecordInternal input = diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultAggregator.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultAggregator.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultAggregator.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultAggregator.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultSerializer.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultSerializer.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultSerializer.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultSerializer.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestEvolveSchemaVisitor.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestEvolveSchemaVisitor.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestEvolveSchemaVisitor.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestEvolveSchemaVisitor.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestLRUCache.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestLRUCache.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestLRUCache.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestLRUCache.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestPartitionSpecEvolution.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestPartitionSpecEvolution.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestPartitionSpecEvolution.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestPartitionSpecEvolution.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestRowDataConverter.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestRowDataConverter.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestRowDataConverter.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestRowDataConverter.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableSerializerCache.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableSerializerCache.java similarity index 99% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableSerializerCache.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableSerializerCache.java index ec610a3357ba..1cf2c8bae001 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableSerializerCache.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableSerializerCache.java @@ -38,7 +38,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; -public class TestTableSerializerCache { +class TestTableSerializerCache { @RegisterExtension static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension("db", "table"); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/DataDistributionUtil.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/DataDistributionUtil.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/DataDistributionUtil.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/DataDistributionUtil.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataDistributionUtil.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataDistributionUtil.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataDistributionUtil.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataDistributionUtil.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java similarity index 97% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java index f7a7a147e73a..09b2b6371e8b 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java @@ -47,10 +47,7 @@ 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; @@ -58,7 +55,6 @@ import org.apache.iceberg.SortKey; 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.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; @@ -91,20 +87,16 @@ private DataStatisticsOperator createOperator(StatisticsType type, int downstrea } private DataStatisticsOperator createOperator( - StatisticsType type, int downstreamParallelism, MockOperatorEventGateway mockGateway) - throws Exception { + StatisticsType type, int downstreamParallelism, MockOperatorEventGateway mockGateway) { DataStatisticsOperator operator = new DataStatisticsOperator( + null, "testOperator", Fixtures.SCHEMA, Fixtures.SORT_ORDER, mockGateway, downstreamParallelism, type); - operator.setup( - new OneInputStreamTask(env), - new MockStreamConfig(new Configuration(), 1), - new MockOutput<>(Lists.newArrayList())); return operator; } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitionerSkew.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitionerSkew.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitionerSkew.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitionerSkew.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestStatisticsOrRecordTypeInformation.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestStatisticsOrRecordTypeInformation.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestStatisticsOrRecordTypeInformation.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestStatisticsOrRecordTypeInformation.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java similarity index 98% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java index f97937dfef9c..134858f5055e 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java @@ -32,7 +32,7 @@ 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.streaming.api.functions.source.legacy.SourceFunction; import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.connector.ChangelogMode; diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java similarity index 98% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java index 7b435d059845..e412006176b9 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java @@ -23,7 +23,7 @@ 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.flink.streaming.api.functions.source.legacy.SourceFunction; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; /** diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java similarity index 97% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java index f89d63ac73e3..c83a9a1baa15 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java @@ -61,7 +61,7 @@ protected TableEnvironment getTableEnv() { super.getTableEnv() .getConfig() .getConfiguration() - .setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), useFlip27Source); + .set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, useFlip27Source); return super.getTableEnv(); } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java similarity index 99% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java index c8b65e131c33..d11bb8640412 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java @@ -29,7 +29,7 @@ import java.util.Map; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.legacy.api.TableSchema; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.types.Row; import org.apache.iceberg.FileFormat; diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java similarity index 97% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java index dd50170f0fd7..0e5b0f335418 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java @@ -22,8 +22,8 @@ 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.table.legacy.api.TableColumn; +import org.apache.flink.table.legacy.api.TableSchema; import org.apache.flink.types.Row; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.flink.FlinkSchemaUtil; diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java similarity index 98% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java index e3dccd76393e..8c1e53e15f15 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java @@ -116,7 +116,7 @@ protected List run( StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(1); Configuration config = new Configuration(); - config.setInteger(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 128); + config.set(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 128); Table table; try (TableLoader tableLoader = tableLoader()) { tableLoader.open(); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedConverterBase.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedConverterBase.java similarity index 96% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedConverterBase.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedConverterBase.java index eb4ad4e0402d..19804dec8088 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedConverterBase.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedConverterBase.java @@ -145,7 +145,7 @@ private List run( env.getConfig().enableObjectReuse(); Configuration config = new Configuration(); - config.setInteger(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 128); + config.set(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 128); Table table; try (TableLoader tableLoader = tableLoader()) { tableLoader.open(); @@ -209,10 +209,11 @@ private IcebergSource.Builder createSourceBuilderWithReaderFunction( } protected abstract org.apache.iceberg.flink.source.reader.RowDataConverter getConverter( - Schema icebergSchema, Table table) throws Exception; + org.apache.iceberg.Schema icebergSchema, Table table) throws Exception; protected ReaderFunction getReaderFunction( - Schema icebergSchema, Table table, List filters) throws Exception { + org.apache.iceberg.Schema icebergSchema, Table table, List filters) + throws Exception { throw new UnsupportedOperationException("No default implementation for getReaderFunction"); } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedRow.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedRow.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedRow.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedRow.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java similarity index 95% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java index 0aea5b0b0a9b..310c8be2eff6 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java @@ -29,14 +29,14 @@ 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.client.program.ClusterClient; +import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.RestartStrategyOptions; import org.apache.flink.core.execution.JobClient; import org.apache.flink.core.execution.SavepointFormatType; import org.apache.flink.runtime.highavailability.nonha.embedded.HaLeadershipControl; -import org.apache.flink.runtime.jobgraph.SavepointConfigOptions; import org.apache.flink.runtime.minicluster.MiniCluster; import org.apache.flink.runtime.minicluster.RpcServiceSharing; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; @@ -184,7 +184,7 @@ public void testBoundedWithSavepoint(@InjectClusterClient ClusterClient clust // New env from the savepoint Configuration conf = new Configuration(); - conf.set(SavepointConfigOptions.SAVEPOINT_PATH, savepointPath); + conf.set(CheckpointingOptions.SAVEPOINT_DIRECTORY, savepointPath); env = StreamExecutionEnvironment.getExecutionEnvironment(conf); createBoundedStreams(env, DO_NOT_FAIL); @@ -217,8 +217,13 @@ private void testBoundedIcebergSource(FailoverType failoverType, MiniCluster min dataAppender.appendToTable(records); } - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); + Configuration config = new Configuration(); + config.set( + RestartStrategyOptions.RESTART_STRATEGY, + RestartStrategyOptions.RestartStrategyType.FIXED_DELAY.getMainValue()); + config.set(RestartStrategyOptions.RESTART_STRATEGY_FIXED_DELAY_ATTEMPTS, 1); + config.set(RestartStrategyOptions.RESTART_STRATEGY_FIXED_DELAY_DELAY, Duration.ofSeconds(0)); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(config); createBoundedStreams(env, 2); JobClient jobClient = env.executeAsync("Bounded Iceberg Source Failover Test"); @@ -256,7 +261,7 @@ private void testContinuousIcebergSource(FailoverType failoverType, MiniCluster env.setParallelism(PARALLELISM); env.enableCheckpointing(10L); Configuration config = new Configuration(); - config.setInteger(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 128); + config.set(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 128); DataStream stream = env.fromSource( diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java similarity index 98% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index 70889f4f76aa..f84cf7fb1aae 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -26,6 +26,7 @@ import java.time.Instant; import java.time.LocalDateTime; import java.time.ZoneId; +import java.time.temporal.ChronoUnit; import java.util.List; import java.util.Optional; import java.util.concurrent.ConcurrentMap; @@ -47,7 +48,6 @@ 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; @@ -177,7 +177,7 @@ public void testWindowing() throws Exception { TypeInformation.of(RowData.class)); stream - .windowAll(TumblingEventTimeWindows.of(Time.minutes(5))) + .windowAll(TumblingEventTimeWindows.of(Duration.ofMinutes(5))) .apply( new AllWindowFunction() { @Override @@ -368,7 +368,7 @@ protected Record generateRecord(int minutes, String str) { Record record = GenericRecord.create(TestFixtures.TS_SCHEMA); LocalDateTime ts = LocalDateTime.ofInstant( - Instant.ofEpochMilli(Time.of(minutes, TimeUnit.MINUTES).toMilliseconds()), + Instant.ofEpochMilli(Duration.of(minutes, ChronoUnit.MINUTES).toMillis()), ZoneId.of("Z")); record.setField("ts", ts); record.setField("str", str); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java similarity index 98% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java index 7bd98c69ff36..61a587e7786a 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java @@ -186,7 +186,7 @@ private static Configuration configure() { Configuration configuration = new Configuration(); configuration.set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); configuration.set(RestOptions.BIND_PORT, "0"); - configuration.set(JobManagerOptions.SLOT_REQUEST_TIMEOUT, 5000L); + configuration.set(JobManagerOptions.SLOT_REQUEST_TIMEOUT, Duration.ofSeconds(5)); // Use FLIP-27 source configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java similarity index 92% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java index b701419a7499..db85f108ab1b 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java @@ -44,7 +44,7 @@ public void testInferedParallelism() throws IOException { assertThat(parallelism).isEqualTo(1); // 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); + configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX, 1); parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 2); assertThat(parallelism).isEqualTo(1); @@ -54,7 +54,7 @@ public void testInferedParallelism() throws IOException { assertThat(parallelism).isEqualTo(1); // 2 splits, infer parallelism is disabled, the parallelism is flink default parallelism 1 - configuration.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); + configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); parallelism = SourceUtil.inferParallelism(configuration, 3, () -> 2); assertThat(parallelism).isEqualTo(1); } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java similarity index 96% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java index dd63154fe03b..fe4c32731055 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java @@ -143,16 +143,14 @@ public void testExposeLocality() throws Exception { // test sql api Configuration tableConf = getTableEnv().getConfig().getConfiguration(); - tableConf.setBoolean( - FlinkConfigOptions.TABLE_EXEC_ICEBERG_EXPOSE_SPLIT_LOCALITY_INFO.key(), false); + tableConf.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_EXPOSE_SPLIT_LOCALITY_INFO, 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); + tableConf.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_EXPOSE_SPLIT_LOCALITY_INFO, 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 diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java similarity index 99% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java index 1080362af278..3c747a05c16a 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java @@ -28,7 +28,7 @@ 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.functions.source.legacy.SourceFunction; import org.apache.flink.streaming.api.operators.StreamSource; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness; diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java similarity index 98% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java index 59c618f7a888..56965417a54a 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java @@ -26,7 +26,6 @@ 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; @@ -275,7 +274,6 @@ private OneInputStreamOperatorTestHarness createReader StreamingReaderOperator.factory(inputFormat); OneInputStreamOperatorTestHarness harness = new OneInputStreamOperatorTestHarness<>(factory, 1, 1, 0); - harness.getStreamConfig().setTimeCharacteristic(TimeCharacteristic.ProcessingTime); return harness; } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java similarity index 97% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java rename to flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java index 11a563709f1a..8f765ff21fa8 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java @@ -29,7 +29,7 @@ public class TestFlinkPackage { /** This unit test would need to be adjusted as new Flink version is supported. */ @Test public void testVersion() { - assertThat(FlinkPackage.version()).isEqualTo("1.19.2"); + assertThat(FlinkPackage.version()).isEqualTo("2.1.0"); } @Test diff --git a/flink/v1.19/flink/src/test/resources/META-INF/services/org.apache.flink.metrics.reporter.MetricReporterFactory b/flink/v2.1/flink/src/test/resources/META-INF/services/org.apache.flink.metrics.reporter.MetricReporterFactory similarity index 100% rename from flink/v1.19/flink/src/test/resources/META-INF/services/org.apache.flink.metrics.reporter.MetricReporterFactory rename to flink/v2.1/flink/src/test/resources/META-INF/services/org.apache.flink.metrics.reporter.MetricReporterFactory diff --git a/flink/v1.19/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink/v2.1/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory similarity index 100% rename from flink/v1.19/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory rename to flink/v2.1/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory diff --git a/gradle.properties b/gradle.properties index 5da56c59de41..0f70b49eb722 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=2.0 -systemProp.knownFlinkVersions=1.19,1.20,2.0 +systemProp.defaultFlinkVersions=2.1 +systemProp.knownFlinkVersions=1.20,2.0,2.1 systemProp.defaultSparkVersions=4.0 systemProp.knownSparkVersions=3.4,3.5,4.0 systemProp.defaultKafkaVersions=3 diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index c18d33476f61..47e90612b7b4 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -46,9 +46,9 @@ esotericsoftware-kryo = "4.0.3" errorprone-annotations = "2.41.0" failsafe = "3.3.2" findbugs-jsr305 = "3.0.2" -flink119 = { strictly = "1.19.2"} flink120 = { strictly = "1.20.1"} flink20 = { strictly = "2.0.0"} +flink21 = { strictly = "2.1.0"} google-libraries-bom = "26.67.0" guava = "33.4.8-jre" hadoop3 = "3.4.1" @@ -113,12 +113,6 @@ delta-standalone = { module = "io.delta:delta-standalone_2.12", version.ref = "d errorprone-annotations = { module = "com.google.errorprone:error_prone_annotations", version.ref = "errorprone-annotations" } failsafe = { module = "dev.failsafe:failsafe", version.ref = "failsafe"} findbugs-jsr305 = { module = "com.google.code.findbugs:jsr305", version.ref = "findbugs-jsr305" } -flink119-avro = { module = "org.apache.flink:flink-avro", version.ref = "flink119" } -flink119-connector-base = { module = "org.apache.flink:flink-connector-base", version.ref = "flink119" } -flink119-connector-files = { module = "org.apache.flink:flink-connector-files", version.ref = "flink119" } -flink119-metrics-dropwizard = { module = "org.apache.flink:flink-metrics-dropwizard", version.ref = "flink119" } -flink119-streaming-java = { module = "org.apache.flink:flink-streaming-java", version.ref = "flink119" } -flink119-table-api-java-bridge = { module = "org.apache.flink:flink-table-api-java-bridge", version.ref = "flink119" } flink120-avro = { module = "org.apache.flink:flink-avro", version.ref = "flink120" } flink120-connector-base = { module = "org.apache.flink:flink-connector-base", version.ref = "flink120" } flink120-connector-files = { module = "org.apache.flink:flink-connector-files", version.ref = "flink120" } @@ -131,6 +125,12 @@ flink20-connector-files = { module = "org.apache.flink:flink-connector-files", v flink20-metrics-dropwizard = { module = "org.apache.flink:flink-metrics-dropwizard", version.ref = "flink20" } flink20-streaming-java = { module = "org.apache.flink:flink-streaming-java", version.ref = "flink20" } flink20-table-api-java-bridge = { module = "org.apache.flink:flink-table-api-java-bridge", version.ref = "flink20" } +flink21-avro = { module = "org.apache.flink:flink-avro", version.ref = "flink21" } +flink21-connector-base = { module = "org.apache.flink:flink-connector-base", version.ref = "flink21" } +flink21-connector-files = { module = "org.apache.flink:flink-connector-files", version.ref = "flink21" } +flink21-metrics-dropwizard = { module = "org.apache.flink:flink-metrics-dropwizard", version.ref = "flink21" } +flink21-streaming-java = { module = "org.apache.flink:flink-streaming-java", version.ref = "flink21" } +flink21-table-api-java-bridge = { module = "org.apache.flink:flink-table-api-java-bridge", version.ref = "flink21" } google-libraries-bom = { module = "com.google.cloud:libraries-bom", version.ref = "google-libraries-bom" } guava-guava = { module = "com.google.guava:guava", version.ref = "guava" } hadoop3-client = { module = "org.apache.hadoop:hadoop-client", version.ref = "hadoop3" } @@ -180,11 +180,6 @@ delta-spark = { module = "io.delta:delta-spark_2.12", version.ref = "delta-spark derby-core = { module = "org.apache.derby:derby", version.ref = "derby"} derby-tools = { module = "org.apache.derby:derbytools", version.ref = "derby"} esotericsoftware-kryo = { module = "com.esotericsoftware:kryo", version.ref = "esotericsoftware-kryo" } -flink119-connector-test-utils = { module = "org.apache.flink:flink-connector-test-utils", version.ref = "flink119" } -flink119-core = { module = "org.apache.flink:flink-core", version.ref = "flink119" } -flink119-runtime = { module = "org.apache.flink:flink-runtime", version.ref = "flink119" } -flink119-test-utils = { module = "org.apache.flink:flink-test-utils", version.ref = "flink119" } -flink119-test-utilsjunit = { module = "org.apache.flink:flink-test-utils-junit", version.ref = "flink119" } flink120-connector-test-utils = { module = "org.apache.flink:flink-connector-test-utils", version.ref = "flink120" } flink120-core = { module = "org.apache.flink:flink-core", version.ref = "flink120" } flink120-runtime = { module = "org.apache.flink:flink-runtime", version.ref = "flink120" } @@ -195,6 +190,11 @@ flink20-core = { module = "org.apache.flink:flink-core", version.ref = "flink20" flink20-runtime = { module = "org.apache.flink:flink-runtime", version.ref = "flink20" } flink20-test-utils = { module = "org.apache.flink:flink-test-utils", version.ref = "flink20" } flink20-test-utilsjunit = { module = "org.apache.flink:flink-test-utils-junit", version.ref = "flink20" } +flink21-connector-test-utils = { module = "org.apache.flink:flink-connector-test-utils", version.ref = "flink21" } +flink21-core = { module = "org.apache.flink:flink-core", version.ref = "flink21" } +flink21-runtime = { module = "org.apache.flink:flink-runtime", version.ref = "flink21" } +flink21-test-utils = { module = "org.apache.flink:flink-test-utils", version.ref = "flink21" } +flink21-test-utilsjunit = { module = "org.apache.flink:flink-test-utils-junit", version.ref = "flink21" } guava-testlib = { module = "com.google.guava:guava-testlib", version.ref = "guava" } jakarta-el-api = { module = "jakarta.el:jakarta.el-api", version.ref = "jakarta-el-api" } jakarta-servlet = {module = "jakarta.servlet:jakarta.servlet-api", version.ref = "jakarta-servlet-api"} diff --git a/settings.gradle b/settings.gradle index 57b8f7b58c70..de342dda1476 100644 --- a/settings.gradle +++ b/settings.gradle @@ -112,15 +112,6 @@ if (!flinkVersions.isEmpty()) { project(':flink').name = 'iceberg-flink' } -if (flinkVersions.contains("1.19")) { - include ":iceberg-flink:flink-1.19" - include ":iceberg-flink:flink-runtime-1.19" - project(":iceberg-flink:flink-1.19").projectDir = file('flink/v1.19/flink') - project(":iceberg-flink:flink-1.19").name = "iceberg-flink-1.19" - project(":iceberg-flink:flink-runtime-1.19").projectDir = file('flink/v1.19/flink-runtime') - project(":iceberg-flink:flink-runtime-1.19").name = "iceberg-flink-runtime-1.19" -} - if (flinkVersions.contains("1.20")) { include ":iceberg-flink:flink-1.20" include ":iceberg-flink:flink-runtime-1.20" @@ -139,6 +130,15 @@ if (flinkVersions.contains("2.0")) { project(":iceberg-flink:flink-runtime-2.0").name = "iceberg-flink-runtime-2.0" } +if (flinkVersions.contains("2.1")) { + include ":iceberg-flink:flink-2.1" + include ":iceberg-flink:flink-runtime-2.1" + project(":iceberg-flink:flink-2.1").projectDir = file('flink/v2.1/flink') + project(":iceberg-flink:flink-2.1").name = "iceberg-flink-2.1" + project(":iceberg-flink:flink-runtime-2.1").projectDir = file('flink/v2.1/flink-runtime') + project(":iceberg-flink:flink-runtime-2.1").name = "iceberg-flink-runtime-2.1" +} + if (sparkVersions.contains("3.4")) { include ":iceberg-spark:spark-3.4_${scalaVersion}" include ":iceberg-spark:spark-extensions-3.4_${scalaVersion}"