Skip to content
Merged
11 changes: 11 additions & 0 deletions flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java
Original file line number Diff line number Diff line change
Expand Up @@ -360,6 +360,17 @@ public void renameTable(ObjectPath tablePath, String newTableName, boolean ignor
@Override
public void createTable(ObjectPath tablePath, CatalogBaseTable table, boolean ignoreIfExists)
throws CatalogException, TableAlreadyExistException {
if (Objects.equals(table.getOptions().get("connector"), FlinkDynamicTableFactory.FACTORY_IDENTIFIER)) {
throw new IllegalArgumentException("Cannot create the table with 'connector'='iceberg' table property in " +
"an iceberg catalog, Please create table with 'connector'='iceberg' property in a non-iceberg catalog or " +
"create table without 'connector'='iceberg' related properties in an iceberg table.");
}

createIcebergTable(tablePath, table, ignoreIfExists);
}

void createIcebergTable(ObjectPath tablePath, CatalogBaseTable table, boolean ignoreIfExists)
throws CatalogException, TableAlreadyExistException {
validateFlinkTable(table);

Schema icebergSchema = FlinkSchemaUtil.convert(table.getSchema());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,7 @@ public class FlinkCatalogFactory implements CatalogFactory {

public static final String HIVE_CONF_DIR = "hive-conf-dir";
public static final String DEFAULT_DATABASE = "default-database";
public static final String DEFAULT_DATABASE_NAME = "default";
public static final String BASE_NAMESPACE = "base-namespace";
public static final String CACHE_ENABLED = "cache-enabled";

Expand All @@ -77,7 +78,7 @@ public class FlinkCatalogFactory implements CatalogFactory {
* @param hadoopConf Hadoop configuration for catalog
* @return an Iceberg catalog loader
*/
protected CatalogLoader createCatalogLoader(String name, Map<String, String> properties, Configuration hadoopConf) {
static CatalogLoader createCatalogLoader(String name, Map<String, String> properties, Configuration hadoopConf) {
String catalogImpl = properties.get(CatalogProperties.CATALOG_IMPL);
if (catalogImpl != null) {
return CatalogLoader.custom(name, properties, hadoopConf, catalogImpl);
Expand Down Expand Up @@ -120,7 +121,7 @@ public Catalog createCatalog(String name, Map<String, String> properties) {

protected Catalog createCatalog(String name, Map<String, String> properties, Configuration hadoopConf) {
CatalogLoader catalogLoader = createCatalogLoader(name, properties, hadoopConf);
String defaultDatabase = properties.getOrDefault(DEFAULT_DATABASE, "default");
String defaultDatabase = properties.getOrDefault(DEFAULT_DATABASE, DEFAULT_DATABASE_NAME);

Namespace baseNamespace = Namespace.empty();
if (properties.containsKey(BASE_NAMESPACE)) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,56 +19,170 @@

package org.apache.iceberg.flink;

import java.util.Map;
import java.util.Set;
import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.configuration.ConfigOptions;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.table.api.TableSchema;
import org.apache.flink.table.catalog.CatalogBaseTable;
import org.apache.flink.table.catalog.CatalogDatabaseImpl;
import org.apache.flink.table.catalog.CatalogTable;
import org.apache.flink.table.catalog.ObjectIdentifier;
import org.apache.flink.table.catalog.ObjectPath;
import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException;
import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
import org.apache.flink.table.connector.sink.DynamicTableSink;
import org.apache.flink.table.connector.source.DynamicTableSource;
import org.apache.flink.table.factories.DynamicTableSinkFactory;
import org.apache.flink.table.factories.DynamicTableSourceFactory;
import org.apache.flink.table.utils.TableSchemaUtils;
import org.apache.flink.util.Preconditions;
import org.apache.iceberg.catalog.TableIdentifier;
import org.apache.iceberg.exceptions.AlreadyExistsException;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.iceberg.relocated.com.google.common.collect.Sets;

public class FlinkDynamicTableFactory implements DynamicTableSinkFactory, DynamicTableSourceFactory {
static final String FACTORY_IDENTIFIER = "iceberg";

private static final ConfigOption<String> CATALOG_NAME =
ConfigOptions.key("catalog-name")
.stringType()
.noDefaultValue()
.withDescription("Catalog name");

private static final ConfigOption<String> CATALOG_TYPE =
ConfigOptions.key(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE)
.stringType()
.noDefaultValue()
.withDescription("Catalog type, the optional types are: custom, hadoop, hive.");

private static final ConfigOption<String> CATALOG_DATABASE =
ConfigOptions.key("catalog-database")
.stringType()
.defaultValue(FlinkCatalogFactory.DEFAULT_DATABASE_NAME)
.withDescription("Database name managed in the iceberg catalog.");

private static final ConfigOption<String> CATALOG_TABLE =
ConfigOptions.key("catalog-table")
.stringType()
.noDefaultValue()
.withDescription("Table name managed in the underlying iceberg catalog and database.");

private final FlinkCatalog catalog;

public FlinkDynamicTableFactory() {
this.catalog = null;
}

public FlinkDynamicTableFactory(FlinkCatalog catalog) {
this.catalog = catalog;
}

@Override
public DynamicTableSource createDynamicTableSource(Context context) {
ObjectPath objectPath = context.getObjectIdentifier().toObjectPath();
TableLoader tableLoader = createTableLoader(objectPath);
ObjectIdentifier objectIdentifier = context.getObjectIdentifier();
Map<String, String> tableProps = context.getCatalogTable().getOptions();
CatalogTable catalogTable = context.getCatalogTable();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

When is catalogTable set in the context? Is it possible that it is set and the factory's context is non-null? In that case, should an error be thrown?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think you are talking about the issue : Is it possible that we access the catalogTable by context.getCatalogTable() before setting it inside the context, if so then the catalogTable will be a null object.

I think we don't have to concern this issue because the catalogTable was set in the context contructor here. And in the flink code path, the set catalogTable must not be null. So I think it's OK here :-)

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry, I meant is there a case when context.getCatalogTable() is non-null but passed to a source where catalog is set in the constructor?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I got your question, yes, it's possible in your case. For example, when we create a table under the iceberg catalog in flink sql, then it will create the FlinkDynamicTableFactory with the specified iceberg catalog.

There should be no problem in that case. the catalogTable is the correct flink table which is parsed from SQL such as INSERT INTO iceberg_catalog.iceberg_db.iceberg_table , although we don't use it in the current code path.

TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema());
return new IcebergTableSource(tableLoader, tableSchema, context.getCatalogTable().getOptions(),
context.getConfiguration());

TableLoader tableLoader;
if (catalog != null) {
tableLoader = createTableLoader(catalog, objectIdentifier.toObjectPath());
} else {
tableLoader = createTableLoader(catalogTable, tableProps, objectIdentifier.getDatabaseName(),
objectIdentifier.getObjectName());
}

return new IcebergTableSource(tableLoader, tableSchema, tableProps, context.getConfiguration());
}

@Override
public DynamicTableSink createDynamicTableSink(Context context) {
ObjectPath objectPath = context.getObjectIdentifier().toObjectPath();
TableLoader tableLoader = createTableLoader(objectPath);
Map<String, String> tableProps = context.getCatalogTable().getOptions();
CatalogTable catalogTable = context.getCatalogTable();
TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema());

TableLoader tableLoader;
if (catalog != null) {
tableLoader = createTableLoader(catalog, objectPath);
} else {
tableLoader = createTableLoader(catalogTable, tableProps, objectPath.getDatabaseName(),
objectPath.getObjectName());
}

return new IcebergTableSink(tableLoader, tableSchema);
}

@Override
public Set<ConfigOption<?>> requiredOptions() {
throw new UnsupportedOperationException("Iceberg Table Factory can not be loaded from Java SPI");
Set<ConfigOption<?>> options = Sets.newHashSet();
options.add(CATALOG_TYPE);
options.add(CATALOG_NAME);
return options;
}

@Override
public Set<ConfigOption<?>> optionalOptions() {
throw new UnsupportedOperationException("Iceberg Table Factory can not be loaded from Java SPI");
Set<ConfigOption<?>> options = Sets.newHashSet();
options.add(CATALOG_DATABASE);
options.add(CATALOG_TABLE);
return options;
}

@Override
public String factoryIdentifier() {
throw new UnsupportedOperationException("Iceberg Table Factory can not be loaded from Java SPI");
return FACTORY_IDENTIFIER;
}

private static TableLoader createTableLoader(CatalogBaseTable catalogBaseTable,
Map<String, String> tableProps,
String databaseName,
String tableName) {
Configuration flinkConf = new Configuration();
tableProps.forEach(flinkConf::setString);

String catalogName = flinkConf.getString(CATALOG_NAME);
Preconditions.checkNotNull(catalogName, "Table property '%s' cannot be null", CATALOG_NAME.key());

String catalogDatabase = flinkConf.getString(CATALOG_DATABASE, databaseName);
Preconditions.checkNotNull(catalogDatabase, "The iceberg database name cannot be null");

String catalogTable = flinkConf.getString(CATALOG_TABLE, tableName);
Preconditions.checkNotNull(catalogTable, "The iceberg table name cannot be null");

org.apache.hadoop.conf.Configuration hadoopConf = FlinkCatalogFactory.clusterHadoopConf();
FlinkCatalogFactory factory = new FlinkCatalogFactory();
FlinkCatalog flinkCatalog = (FlinkCatalog) factory.createCatalog(catalogName, tableProps, hadoopConf);
ObjectPath objectPath = new ObjectPath(catalogDatabase, catalogTable);

// Create database if not exists in the external catalog.
if (!flinkCatalog.databaseExists(catalogDatabase)) {
try {
flinkCatalog.createDatabase(catalogDatabase, new CatalogDatabaseImpl(Maps.newHashMap(), null), true);
} catch (DatabaseAlreadyExistException e) {
throw new AlreadyExistsException(e, "Database %s already exists in the iceberg catalog %s.", catalogName,
catalogDatabase);
}
}

// Create table if not exists in the external catalog.
if (!flinkCatalog.tableExists(objectPath)) {
try {
flinkCatalog.createIcebergTable(objectPath, catalogBaseTable, true);
} catch (TableAlreadyExistException e) {
throw new AlreadyExistsException(e, "Table %s already exists in the database %s and catalog %s",
catalogTable, catalogDatabase, catalogName);
}
}

return TableLoader.fromCatalog(flinkCatalog.getCatalogLoader(), TableIdentifier.of(catalogDatabase, catalogTable));
}

private TableLoader createTableLoader(ObjectPath objectPath) {
private static TableLoader createTableLoader(FlinkCatalog catalog, ObjectPath objectPath) {
Preconditions.checkNotNull(catalog, "Flink catalog cannot be null");
return TableLoader.fromCatalog(catalog.getCatalogLoader(), catalog.toIdentifier(objectPath));
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,16 @@
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.

org.apache.iceberg.flink.FlinkDynamicTableFactory
Loading