-
Notifications
You must be signed in to change notification settings - Fork 3k
support create table like in flink catalog #12199
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
Changes from 6 commits
c193d17
d4a464d
44d45af
b4d7ffe
8f90cd5
bf55619
6745fef
6beeb2b
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 |
|---|---|---|
| @@ -0,0 +1,115 @@ | ||
| /* | ||
| * 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<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"; | ||
stevenzwu marked this conversation as resolved.
Show resolved
Hide resolved
Contributor
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. also does Flink create table ... like ... work for Hive tables as source? I assume no. otherwise, we can refer to how that is implemented. It feels tacky to use special property key to carry over catalog props and source table identifier. Does it make sense to modify Flink API to support this more elegantly?
Contributor
Author
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. Right this is not supported in hive, Supported in Kafka, but yeah there is no concept of catalog , mostly flat properties of cluster for kafka connector.
Contributor
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. yeah, Kafka doesn't have catalog concept. we can take this discussion separately. it won't be a blocker for this PR |
||
| public static final String CONNECTOR_PROPS_KEY = "connector"; | ||
|
|
||
| static String toJson( | ||
| String catalogName, String catalogDb, String catalogTable, Map<String, String> catalogProps) { | ||
| 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(), catalogProps, 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 catalogName() { | ||
| return catalogName; | ||
| } | ||
|
|
||
| String catalogDb() { | ||
| return catalogDb; | ||
| } | ||
|
|
||
| String catalogTable() { | ||
| return catalogTable; | ||
| } | ||
|
|
||
| Map<String, String> catalogProps() { | ||
| 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); | ||
|
Contributor
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,42 @@ private static TableLoader createTableLoader( | |
| flinkCatalog.getCatalogLoader(), TableIdentifier.of(catalogDatabase, catalogTable)); | ||
| } | ||
|
|
||
| /** | ||
| * Merges source catalog properties with connector properties. Iceberg Catalog properties are | ||
| * serialized as json in FlinkCatalog#getTable to be able to isolate catalog props from iceberg | ||
| * table props, Here, we flatten and merge them back to use to create catalog. | ||
| * | ||
| * @param tableProps the existing table properties | ||
| * @return a map of merged properties, with source catalog properties taking precedence when keys | ||
| * conflict | ||
| */ | ||
| 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.catalogName()); | ||
| mergedProps.put( | ||
| FlinkCreateTableOptions.CATALOG_DATABASE.key(), createTableOptions.catalogDb()); | ||
| mergedProps.put( | ||
| FlinkCreateTableOptions.CATALOG_TABLE.key(), createTableOptions.catalogTable()); | ||
| mergedProps.putAll(createTableOptions.catalogProps()); | ||
|
|
||
| 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)); | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.