-
Notifications
You must be signed in to change notification settings - Fork 2.4k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
support create table like in flink catalog #12199
base: main
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -97,17 +97,20 @@ public class FlinkCatalog extends AbstractCatalog { | |
private final Namespace baseNamespace; | ||
private final SupportsNamespaces asNamespaceCatalog; | ||
private final Closeable closeable; | ||
private final Map<String, String> catalogProps; | ||
private final boolean cacheEnabled; | ||
|
||
public FlinkCatalog( | ||
String catalogName, | ||
String defaultDatabase, | ||
Namespace baseNamespace, | ||
CatalogLoader catalogLoader, | ||
Map<String, String> catalogProps, | ||
boolean cacheEnabled, | ||
long cacheExpirationIntervalMs) { | ||
super(catalogName, defaultDatabase); | ||
this.catalogLoader = catalogLoader; | ||
this.catalogProps = catalogProps; | ||
this.baseNamespace = baseNamespace; | ||
this.cacheEnabled = cacheEnabled; | ||
|
||
|
@@ -332,7 +335,17 @@ public List<String> listTables(String databaseName) | |
public CatalogTable getTable(ObjectPath tablePath) | ||
throws TableNotExistException, CatalogException { | ||
Table table = loadIcebergTable(tablePath); | ||
return toCatalogTable(table); | ||
|
||
String srcCatalogProps = | ||
FlinkCreateTableOptions.toJson( | ||
getName(), tablePath.getDatabaseName(), tablePath.getObjectName(), catalogProps); | ||
|
||
ImmutableMap.Builder<String, String> mergedProps = ImmutableMap.builder(); | ||
mergedProps.put("connector", FlinkDynamicTableFactory.FACTORY_IDENTIFIER); | ||
mergedProps.put(FlinkCreateTableOptions.SRC_CATALOG_PROPS_KEY, srcCatalogProps); | ||
mergedProps.putAll(table.properties()); | ||
|
||
return toCatalogTableWithCustomProps(table, mergedProps.build()); | ||
} | ||
|
||
private Table loadIcebergTable(ObjectPath tablePath) throws TableNotExistException { | ||
|
@@ -384,13 +397,16 @@ 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( | ||
pvary marked this conversation as resolved.
Show resolved
Hide resolved
|
||
table.getOptions().get("connector"), FlinkDynamicTableFactory.FACTORY_IDENTIFIER)) { | ||
table.getOptions().get("connector"), 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); | ||
} | ||
|
@@ -406,7 +422,8 @@ void createIcebergTable(ObjectPath tablePath, ResolvedCatalogTable table, boolea | |
for (Map.Entry<String, String> entry : table.getOptions().entrySet()) { | ||
if ("location".equalsIgnoreCase(entry.getKey())) { | ||
location = entry.getValue(); | ||
} else { | ||
} else if (!("connector".equalsIgnoreCase(entry.getKey()) | ||
|| FlinkCreateTableOptions.SRC_CATALOG_PROPS_KEY.equalsIgnoreCase(entry.getKey()))) { | ||
Comment on lines
+425
to
+426
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It took me some time to decipher this 😄
Or it is just me? |
||
properties.put(entry.getKey(), entry.getValue()); | ||
} | ||
} | ||
|
@@ -625,7 +642,7 @@ private static List<String> toPartitionKeys(PartitionSpec spec, Schema icebergSc | |
return partitionKeysBuilder.build(); | ||
} | ||
|
||
static CatalogTable toCatalogTable(Table table) { | ||
static CatalogTable toCatalogTableWithCustomProps(Table table, Map<String, String> customProps) { | ||
TableSchema schema = FlinkSchemaUtil.toSchema(table.schema()); | ||
List<String> partitionKeys = toPartitionKeys(table.spec(), table.schema()); | ||
|
||
|
@@ -634,7 +651,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, customProps, null); | ||
} | ||
|
||
static CatalogTable toCatalogTable(Table table) { | ||
return toCatalogTableWithCustomProps(table, table.properties()); | ||
} | ||
|
||
@Override | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,114 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, | ||
* software distributed under the License is distributed on an | ||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
* KIND, either express or implied. See the License for the | ||
* specific language governing permissions and limitations | ||
* under the License. | ||
*/ | ||
package org.apache.iceberg.flink; | ||
|
||
import java.util.Map; | ||
import org.apache.flink.configuration.ConfigOption; | ||
import org.apache.flink.configuration.ConfigOptions; | ||
import org.apache.iceberg.util.JsonUtil; | ||
|
||
public class FlinkCreateTableOptions { | ||
private final String catalogName; | ||
private final String catalogDb; | ||
private final String catalogTable; | ||
private final Map<String, String> catalogProps; | ||
|
||
private FlinkCreateTableOptions( | ||
String catalogName, String catalogDb, String catalogTable, Map<String, String> props) { | ||
this.catalogName = catalogName; | ||
this.catalogDb = catalogDb; | ||
this.catalogTable = catalogTable; | ||
this.catalogProps = props; | ||
} | ||
|
||
public static final ConfigOption<String> CATALOG_NAME = | ||
ConfigOptions.key("catalog-name") | ||
.stringType() | ||
.noDefaultValue() | ||
.withDescription("Catalog name"); | ||
|
||
public static final ConfigOption<String> CATALOG_TYPE = | ||
ConfigOptions.key(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE) | ||
.stringType() | ||
.noDefaultValue() | ||
.withDescription("Catalog type, the optional types are: custom, hadoop, hive."); | ||
|
||
public static final ConfigOption<String> CATALOG_DATABASE = | ||
ConfigOptions.key("catalog-database") | ||
.stringType() | ||
.defaultValue(FlinkCatalogFactory.DEFAULT_DATABASE_NAME) | ||
.withDescription("Database name managed in the iceberg catalog."); | ||
|
||
public static final ConfigOption<String> CATALOG_TABLE = | ||
ConfigOptions.key("catalog-table") | ||
.stringType() | ||
.noDefaultValue() | ||
.withDescription("Table name managed in the underlying iceberg catalog and database."); | ||
|
||
public static final ConfigOption<Map<String, String>> CATALOG_PROPS = | ||
ConfigOptions.key("catalog-props") | ||
.mapType() | ||
.noDefaultValue() | ||
.withDescription("Properties for the underlying catalog for iceberg table."); | ||
|
||
public static final String SRC_CATALOG_PROPS_KEY = "src-catalog"; | ||
|
||
static String toJson( | ||
String catalogName, String catalogDb, String catalogTable, Map<String, String> props) { | ||
return JsonUtil.generate( | ||
gen -> { | ||
gen.writeStartObject(); | ||
gen.writeStringField(CATALOG_NAME.key(), catalogName); | ||
gen.writeStringField(CATALOG_DATABASE.key(), catalogDb); | ||
gen.writeStringField(CATALOG_TABLE.key(), catalogTable); | ||
JsonUtil.writeStringMap(CATALOG_PROPS.key(), props, gen); | ||
gen.writeEndObject(); | ||
}, | ||
false); | ||
} | ||
|
||
static FlinkCreateTableOptions fromJson(String createTableOptions) { | ||
return JsonUtil.parse( | ||
createTableOptions, | ||
node -> { | ||
String catalogName = JsonUtil.getString(CATALOG_NAME.key(), node); | ||
String catalogDb = JsonUtil.getString(CATALOG_DATABASE.key(), node); | ||
String catalogTable = JsonUtil.getString(CATALOG_TABLE.key(), node); | ||
Map<String, String> catalogProps = JsonUtil.getStringMap(CATALOG_PROPS.key(), node); | ||
|
||
return new FlinkCreateTableOptions(catalogName, catalogDb, catalogTable, catalogProps); | ||
}); | ||
} | ||
|
||
String getCatalogName() { | ||
return catalogName; | ||
} | ||
|
||
String getCatalogDb() { | ||
return catalogDb; | ||
} | ||
|
||
String getCatalogTable() { | ||
return catalogTable; | ||
} | ||
|
||
Map<String, String> getCatalogProps() { | ||
return catalogProps; | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -18,10 +18,10 @@ | |
*/ | ||
package org.apache.iceberg.flink; | ||
|
||
import java.util.Collections; | ||
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.CatalogDatabaseImpl; | ||
|
@@ -45,31 +45,6 @@ | |
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() { | ||
|
@@ -127,16 +102,16 @@ public DynamicTableSink createDynamicTableSink(Context context) { | |
@Override | ||
public Set<ConfigOption<?>> requiredOptions() { | ||
Set<ConfigOption<?>> options = Sets.newHashSet(); | ||
options.add(CATALOG_TYPE); | ||
options.add(CATALOG_NAME); | ||
options.add(FlinkCreateTableOptions.CATALOG_TYPE); | ||
options.add(FlinkCreateTableOptions.CATALOG_NAME); | ||
return options; | ||
} | ||
|
||
@Override | ||
public Set<ConfigOption<?>> optionalOptions() { | ||
Set<ConfigOption<?>> options = Sets.newHashSet(); | ||
options.add(CATALOG_DATABASE); | ||
options.add(CATALOG_TABLE); | ||
options.add(FlinkCreateTableOptions.CATALOG_DATABASE); | ||
options.add(FlinkCreateTableOptions.CATALOG_TABLE); | ||
return options; | ||
} | ||
|
||
|
@@ -151,22 +126,28 @@ private static TableLoader createTableLoader( | |
String databaseName, | ||
String tableName) { | ||
Configuration flinkConf = new Configuration(); | ||
tableProps.forEach(flinkConf::setString); | ||
|
||
String catalogName = flinkConf.getString(CATALOG_NAME); | ||
Map<String, String> mergedProps = mergeSrcCatalogProps(tableProps); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Either here, or in the javadoc for the |
||
|
||
mergedProps.forEach(flinkConf::setString); | ||
|
||
String catalogName = flinkConf.getString(FlinkCreateTableOptions.CATALOG_NAME); | ||
Preconditions.checkNotNull( | ||
catalogName, "Table property '%s' cannot be null", CATALOG_NAME.key()); | ||
catalogName, | ||
"Table property '%s' cannot be null", | ||
FlinkCreateTableOptions.CATALOG_NAME.key()); | ||
|
||
String catalogDatabase = flinkConf.getString(CATALOG_DATABASE, databaseName); | ||
String catalogDatabase = | ||
flinkConf.getString(FlinkCreateTableOptions.CATALOG_DATABASE, databaseName); | ||
Preconditions.checkNotNull(catalogDatabase, "The iceberg database name cannot be null"); | ||
|
||
String catalogTable = flinkConf.getString(CATALOG_TABLE, tableName); | ||
String catalogTable = flinkConf.getString(FlinkCreateTableOptions.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); | ||
(FlinkCatalog) factory.createCatalog(catalogName, mergedProps, hadoopConf); | ||
ObjectPath objectPath = new ObjectPath(catalogDatabase, catalogTable); | ||
|
||
// Create database if not exists in the external catalog. | ||
|
@@ -201,6 +182,34 @@ private static TableLoader createTableLoader( | |
flinkCatalog.getCatalogLoader(), TableIdentifier.of(catalogDatabase, catalogTable)); | ||
} | ||
|
||
private static Map<String, String> mergeSrcCatalogProps(Map<String, String> tableProps) { | ||
String srcCatalogProps = tableProps.get(FlinkCreateTableOptions.SRC_CATALOG_PROPS_KEY); | ||
if (srcCatalogProps != null) { | ||
Map<String, String> mergedProps = Maps.newHashMap(); | ||
FlinkCreateTableOptions createTableOptions = | ||
FlinkCreateTableOptions.fromJson(srcCatalogProps); | ||
|
||
mergedProps.put( | ||
FlinkCreateTableOptions.CATALOG_NAME.key(), createTableOptions.getCatalogName()); | ||
mergedProps.put( | ||
FlinkCreateTableOptions.CATALOG_DATABASE.key(), createTableOptions.getCatalogDb()); | ||
mergedProps.put( | ||
FlinkCreateTableOptions.CATALOG_TABLE.key(), createTableOptions.getCatalogTable()); | ||
mergedProps.putAll(createTableOptions.getCatalogProps()); | ||
|
||
tableProps.forEach( | ||
(k, v) -> { | ||
if (!FlinkCreateTableOptions.SRC_CATALOG_PROPS_KEY.equals(k)) { | ||
mergedProps.put(k, v); | ||
} | ||
}); | ||
|
||
return Collections.unmodifiableMap(mergedProps); | ||
} | ||
|
||
return tableProps; | ||
} | ||
|
||
private static TableLoader createTableLoader(FlinkCatalog catalog, ObjectPath objectPath) { | ||
Preconditions.checkNotNull(catalog, "Flink catalog cannot be null"); | ||
return TableLoader.fromCatalog(catalog.getCatalogLoader(), catalog.toIdentifier(objectPath)); | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Could we please add a comment here, that we store the catalog properties in the merged property list to work around the Flink API limitations?