Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
The table of contents is too big for display.
Diff view
Diff view
  •  
  •  
  •  
2 changes: 1 addition & 1 deletion .github/workflows/flink-ci.yml
Original file line number Diff line number Diff line change
Expand Up @@ -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:
Expand Down
8 changes: 4 additions & 4 deletions flink/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -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")
}

This file was deleted.

36 changes: 18 additions & 18 deletions flink/v1.19/build.gradle → flink/v2.1/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -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}") {
Expand All @@ -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
Expand Down Expand Up @@ -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'
}
Expand Down Expand Up @@ -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
Expand All @@ -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
Expand Down
File renamed without changes.
File renamed without changes.
Original file line number Diff line number Diff line change
@@ -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 {}
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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;
Expand Down Expand Up @@ -662,11 +660,16 @@ static CatalogTable toCatalogTableWithProps(Table table, Map<String, String> pro
List<String> 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) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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;

/**
Expand All @@ -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";
Expand All @@ -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
Expand Down Expand Up @@ -122,21 +122,23 @@ static CatalogLoader createCatalogLoader(
}

@Override
public Map<String, String> requiredContext() {
Map<String, String> context = Maps.newHashMap();
context.put(TYPE, "iceberg");
context.put(PROPERTY_VERSION, "1");
return context;
public String factoryIdentifier() {
return FACTORY_IDENTIFIER;
}

@Override
public Set<ConfigOption<?>> requiredOptions() {
return ImmutableSet.<ConfigOption<?>>builder().build();
}

@Override
public List<String> supportedProperties() {
return ImmutableList.of("*");
public Set<ConfigOption<?>> optionalOptions() {
return ImmutableSet.<ConfigOption<?>>builder().build();
}

@Override
public Catalog createCatalog(String name, Map<String, String> properties) {
return createCatalog(name, properties, clusterHadoopConf());
public Catalog createCatalog(Context context) {
return createCatalog(context.getName(), context.getOptions(), clusterHadoopConf());
}

protected Catalog createCatalog(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -88,7 +88,7 @@ private FlinkConfigOptions() {}
public static final ConfigOption<Boolean> 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<Boolean> TABLE_EXEC_ICEBERG_USE_V2_SINK =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down
Loading