diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java
index 86295d78cc13..76c6dc758265 100644
--- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java
+++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java
@@ -27,6 +27,7 @@
import java.util.Optional;
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;
@@ -91,12 +92,14 @@
*
The Iceberg table manages its partitions by itself. The partition of the Iceberg table is
* independent of the partition of Flink.
*/
+@Internal
public class FlinkCatalog extends AbstractCatalog {
private final CatalogLoader catalogLoader;
private final Catalog icebergCatalog;
private final Namespace baseNamespace;
private final SupportsNamespaces asNamespaceCatalog;
private final Closeable closeable;
+ private final Map catalogProps;
private final boolean cacheEnabled;
public FlinkCatalog(
@@ -104,10 +107,12 @@ public FlinkCatalog(
String defaultDatabase,
Namespace baseNamespace,
CatalogLoader catalogLoader,
+ Map catalogProps,
boolean cacheEnabled,
long cacheExpirationIntervalMs) {
super(catalogName, defaultDatabase);
this.catalogLoader = catalogLoader;
+ this.catalogProps = catalogProps;
this.baseNamespace = baseNamespace;
this.cacheEnabled = cacheEnabled;
@@ -332,7 +337,34 @@ public List listTables(String databaseName)
public CatalogTable getTable(ObjectPath tablePath)
throws TableNotExistException, CatalogException {
Table table = loadIcebergTable(tablePath);
- return toCatalogTable(table);
+
+ // Flink's CREATE TABLE LIKE clause relies on properties sent back here to create new table.
+ // Inorder to create such table in non iceberg catalog, we need to send across catalog
+ // properties also.
+ // As Flink API accepts only Map for props, here we are serializing catalog
+ // props as json string to distinguish between catalog and table properties in createTable.
+ String srcCatalogProps =
+ FlinkCreateTableOptions.toJson(
+ getName(), tablePath.getDatabaseName(), tablePath.getObjectName(), catalogProps);
+
+ Map tableProps = table.properties();
+ if (tableProps.containsKey(FlinkCreateTableOptions.CONNECTOR_PROPS_KEY)
+ || tableProps.containsKey(FlinkCreateTableOptions.SRC_CATALOG_PROPS_KEY)) {
+ throw new IllegalArgumentException(
+ String.format(
+ "Source table %s contains one/all of the reserved property keys: %s, %s.",
+ tablePath,
+ FlinkCreateTableOptions.CONNECTOR_PROPS_KEY,
+ FlinkCreateTableOptions.SRC_CATALOG_PROPS_KEY));
+ }
+
+ ImmutableMap.Builder mergedProps = ImmutableMap.builder();
+ mergedProps.put(
+ FlinkCreateTableOptions.CONNECTOR_PROPS_KEY, FlinkDynamicTableFactory.FACTORY_IDENTIFIER);
+ mergedProps.put(FlinkCreateTableOptions.SRC_CATALOG_PROPS_KEY, srcCatalogProps);
+ mergedProps.putAll(tableProps);
+
+ return toCatalogTableWithProps(table, mergedProps.build());
}
private Table loadIcebergTable(ObjectPath tablePath) throws TableNotExistException {
@@ -384,13 +416,17 @@ public void renameTable(ObjectPath tablePath, String newTableName, boolean ignor
@Override
public void createTable(ObjectPath tablePath, CatalogBaseTable table, boolean ignoreIfExists)
throws CatalogException, TableAlreadyExistException {
+ // Creating Iceberg table using connector is allowed only when table is created using LIKE
if (Objects.equals(
- table.getOptions().get("connector"), FlinkDynamicTableFactory.FACTORY_IDENTIFIER)) {
+ table.getOptions().get(FlinkCreateTableOptions.CONNECTOR_PROPS_KEY),
+ FlinkDynamicTableFactory.FACTORY_IDENTIFIER)
+ && table.getOptions().get(FlinkCreateTableOptions.SRC_CATALOG_PROPS_KEY) == null) {
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.");
}
+
Preconditions.checkArgument(table instanceof ResolvedCatalogTable, "table should be resolved");
createIcebergTable(tablePath, (ResolvedCatalogTable) table, ignoreIfExists);
}
@@ -404,10 +440,14 @@ void createIcebergTable(ObjectPath tablePath, ResolvedCatalogTable table, boolea
ImmutableMap.Builder properties = ImmutableMap.builder();
String location = null;
for (Map.Entry entry : table.getOptions().entrySet()) {
- if ("location".equalsIgnoreCase(entry.getKey())) {
- location = entry.getValue();
- } else {
+ if (!isReservedProperty(entry.getKey())) {
properties.put(entry.getKey(), entry.getValue());
+ } else {
+ // Filtering reserved properties like catalog properties(added to support CREATE TABLE LIKE
+ // in getTable()), location and not persisting on table properties.
+ if (FlinkCreateTableOptions.LOCATION_KEY.equalsIgnoreCase(entry.getKey())) {
+ location = entry.getValue();
+ }
}
}
@@ -421,6 +461,12 @@ void createIcebergTable(ObjectPath tablePath, ResolvedCatalogTable table, boolea
}
}
+ private boolean isReservedProperty(String prop) {
+ return FlinkCreateTableOptions.LOCATION_KEY.equalsIgnoreCase(prop)
+ || FlinkCreateTableOptions.CONNECTOR_PROPS_KEY.equalsIgnoreCase(prop)
+ || FlinkCreateTableOptions.SRC_CATALOG_PROPS_KEY.equalsIgnoreCase(prop);
+ }
+
private static void validateTableSchemaAndPartition(CatalogTable ct1, CatalogTable ct2) {
TableSchema ts1 = ct1.getSchema();
TableSchema ts2 = ct2.getSchema();
@@ -501,7 +547,7 @@ public void alterTable(ObjectPath tablePath, CatalogBaseTable newTable, boolean
continue;
}
- if ("location".equalsIgnoreCase(key)) {
+ if (FlinkCreateTableOptions.LOCATION_KEY.equalsIgnoreCase(key)) {
setLocation = value;
} else if ("current-snapshot-id".equalsIgnoreCase(key)) {
setSnapshotId = value;
@@ -558,7 +604,7 @@ public void alterTable(
if (change instanceof TableChange.SetOption) {
TableChange.SetOption set = (TableChange.SetOption) change;
- if ("location".equalsIgnoreCase(set.getKey())) {
+ if (FlinkCreateTableOptions.LOCATION_KEY.equalsIgnoreCase(set.getKey())) {
setLocation = set.getValue();
} else if ("current-snapshot-id".equalsIgnoreCase(set.getKey())) {
setSnapshotId = set.getValue();
@@ -625,7 +671,7 @@ private static List toPartitionKeys(PartitionSpec spec, Schema icebergSc
return partitionKeysBuilder.build();
}
- static CatalogTable toCatalogTable(Table table) {
+ static CatalogTable toCatalogTableWithProps(Table table, Map props) {
TableSchema schema = FlinkSchemaUtil.toSchema(table.schema());
List partitionKeys = toPartitionKeys(table.spec(), table.schema());
@@ -634,7 +680,11 @@ static CatalogTable toCatalogTable(Table table) {
// CatalogTableImpl to copy a new catalog table.
// Let's re-loading table from Iceberg catalog when creating source/sink operators.
// Iceberg does not have Table comment, so pass a null (Default comment value in Flink).
- return new CatalogTableImpl(schema, partitionKeys, table.properties(), null);
+ return new CatalogTableImpl(schema, partitionKeys, props, null);
+ }
+
+ static CatalogTable toCatalogTable(Table table) {
+ return toCatalogTableWithProps(table, table.properties());
}
@Override
diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java
index 9b0c7a938920..dd065617bd88 100644
--- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java
+++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java
@@ -72,7 +72,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";
@@ -169,6 +168,7 @@ protected Catalog createCatalog(
defaultDatabase,
baseNamespace,
catalogLoader,
+ properties,
cacheEnabled,
cacheExpirationIntervalMs);
}
diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCreateTableOptions.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCreateTableOptions.java
new file mode 100644
index 000000000000..ab69ec5adc7f
--- /dev/null
+++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCreateTableOptions.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.flink;
+
+import java.util.Map;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.iceberg.util.JsonUtil;
+
+class FlinkCreateTableOptions {
+ private final String catalogName;
+ private final String catalogDb;
+ private final String catalogTable;
+ private final Map catalogProps;
+
+ private FlinkCreateTableOptions(
+ String catalogName, String catalogDb, String catalogTable, Map props) {
+ this.catalogName = catalogName;
+ this.catalogDb = catalogDb;
+ this.catalogTable = catalogTable;
+ this.catalogProps = props;
+ }
+
+ public static final ConfigOption CATALOG_NAME =
+ ConfigOptions.key("catalog-name")
+ .stringType()
+ .noDefaultValue()
+ .withDescription("Catalog name");
+
+ public static final ConfigOption CATALOG_TYPE =
+ ConfigOptions.key(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE)
+ .stringType()
+ .noDefaultValue()
+ .withDescription("Catalog type, the optional types are: custom, hadoop, hive.");
+
+ public static final ConfigOption CATALOG_DATABASE =
+ ConfigOptions.key("catalog-database")
+ .stringType()
+ .defaultValue(FlinkCatalogFactory.DEFAULT_DATABASE_NAME)
+ .withDescription("Database name managed in the iceberg catalog.");
+
+ public static final ConfigOption CATALOG_TABLE =
+ ConfigOptions.key("catalog-table")
+ .stringType()
+ .noDefaultValue()
+ .withDescription("Table name managed in the underlying iceberg catalog and database.");
+
+ public static final ConfigOption