openinx commented on a change in pull request #2666:
URL: https://github.com/apache/iceberg/pull/2666#discussion_r659467383
##########
File path:
flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java
##########
@@ -19,56 +19,155 @@
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.CatalogException;
+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.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
public class FlinkDynamicTableFactory implements DynamicTableSinkFactory,
DynamicTableSourceFactory {
+ private 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 that managed in the iceberg
catalog.");
+
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();
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.getObjectName());
Review comment:
I think this comment is similar to the @stevenzwu 's
[question](https://github.com/apache/iceberg/pull/2666#discussion_r647120788) .
Saying if the flink table identifier is
`flink_catalog.flink_database.table_name` (when creating table using the
`connector=iceberg`, not the iceberg catalog approach), then it's mapping the
flink table with name `table_name` to the underlying iceberg table `table_name`
with the configured database in table property. The flink's `flink_catalog`
and `flink_database` does not has any relationship to the iceberg's catalog &
database.
Maybe it's better to forbidden people to specify a `connector=iceberg`
property when creating table under the iceberg catalog ( adding a
Precondition.checkArgument in `createDynamicTableSource` and
`createDynamicTableSink`), then we don't mixed the two approaches to create
flink+iceberg table or map flink table to iceberg table.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]