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



##########
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());
+    }
+
+    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);
+    return options;
   }
 
   @Override
   public Set<ConfigOption<?>> optionalOptions() {
-    throw new UnsupportedOperationException("Iceberg Table Factory can not be 
loaded from Java SPI");
+    Set<ConfigOption<?>> options = Sets.newHashSet();
+    options.add(CATALOG_DATABASE);
+    return options;
   }
 
   @Override
   public String factoryIdentifier() {
-    throw new UnsupportedOperationException("Iceberg Table Factory can not be 
loaded from Java SPI");
+    return FACTORY_IDENTIFIER;
+  }
+
+  private static TableLoader createTableLoader(CatalogBaseTable catalogTable,
+                                               Map<String, String> tableProps,
+                                               String tableName) {
+    Configuration flinkConf = new Configuration();
+    tableProps.forEach(flinkConf::setString);
+
+    String catalogName = flinkConf.getString(CATALOG_NAME);
+    Preconditions.checkNotNull(catalogName, "Table property '%s' cannot be 
null", CATALOG_NAME.key());
+
+    String catalogDatabase = flinkConf.getString(CATALOG_DATABASE);
+    Preconditions.checkNotNull(catalogDatabase, "Table property '%s' cannot be 
null", CATALOG_DATABASE.key());
+
+    org.apache.hadoop.conf.Configuration hadoopConf = 
FlinkCatalogFactory.clusterHadoopConf();
+    FlinkCatalogFactory factory = new FlinkCatalogFactory();
+    FlinkCatalog flinkCatalog = (FlinkCatalog) 
factory.createCatalog(catalogName, tableProps, hadoopConf);
+    ObjectPath objectPath = new ObjectPath(catalogDatabase, tableName);
+
+    // Create database if not exists in the external catalog.
+    if (!flinkCatalog.databaseExists(catalogDatabase)) {
+      try {
+        flinkCatalog.createDatabase(catalogDatabase, new 
CatalogDatabaseImpl(Maps.newHashMap(), null), true);
+      } catch (DatabaseAlreadyExistException | CatalogException e) {
+        throw new RuntimeException(String.format("Failed to create database 
%s.%s", catalogName, catalogDatabase), e);
+      }
+    }
+
+    // Create table if not exists in the external catalog.
+    if (!flinkCatalog.tableExists(objectPath)) {
+      try {
+        flinkCatalog.createTable(objectPath, catalogTable, true);
+      } catch (TableAlreadyExistException | CatalogException e) {
+        throw new RuntimeException(String.format("Failed to create table 
%s.%s", catalogName, objectPath), e);

Review comment:
       Same here. I think it is better to throw Iceberg exceptions if they 
exist.




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

Reply via email to