kbendick commented on a change in pull request #2666:
URL: https://github.com/apache/iceberg/pull/2666#discussion_r645737621



##########
File path: 
flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java
##########
@@ -19,56 +19,150 @@
 
 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.Catalog;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+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.DatabaseNotExistException;
+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.Sets;
 
 public class FlinkDynamicTableFactory implements DynamicTableSinkFactory, 
DynamicTableSourceFactory {
+  private static final String FACTORY_IDENTIFIER = "iceberg";
+
+  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.");
+
+  public static final ConfigOption<String> CATALOG_DATABASE =
+      ConfigOptions.key("catalog-database")
+          .stringType()
+          .defaultValue(FlinkCatalogFactory.DEFAULT_DATABASE_VALUE)
+          .withDeprecatedKeys("Catalog 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();
     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());
+    }
+
+    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.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);
+    options.add(CATALOG_DATABASE);
+    return Sets.newHashSet();

Review comment:
       Are you intending to return `options` here? Possibly I am not 
understanding the interface fully, but it seems like `options` should be 
returned.




-- 
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.

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to