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



##########
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:
       Okay, I thought that this was a way to run DDL even if you don't have an 
Iceberg catalog defined. It sort of does that, but it also creates a reference 
in the in-memory catalog. That's fine, but it does bring up a couple other 
questions:
   * How do you create an in-memory catalog table pointing to an Iceberg table 
if the Iceberg table already exists? What if the DDL, like the schema, doesn't 
match?
   * Why share the table name between the in-memory and external catalog but 
not the database? I think it makes sense to default the external database and 
table name using the ones from the DDL command but allow both to be overridden.

##########
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();

Review comment:
       Sorry, I meant is there a case when `context.getCatalogTable()` is 
non-null but passed to a source where `catalog` is set in the constructor?

##########
File path: 
flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java
##########
@@ -0,0 +1,315 @@
+/*
+ * 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.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Map;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.catalog.Catalog;
+import org.apache.flink.table.catalog.ObjectPath;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.thrift.TException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestIcebergConnector extends FlinkTestBase {
+
+  private static final String TABLE_NAME = "test_table";
+
+  @ClassRule
+  public static final TemporaryFolder WAREHOUSE = new TemporaryFolder();
+
+  private final String catalogName;
+  private final Map<String, String> properties;
+  private final boolean isStreaming;
+  private volatile TableEnvironment tEnv;
+
+  @Parameterized.Parameters(name = "catalogName = {0}, properties = {1}, 
isStreaming={2}")
+  public static Iterable<Object[]> parameters() {
+    return Lists.newArrayList(
+        // Create iceberg table in the hadoop catalog and default database.
+        new Object[] {
+            "testhadoop",
+            ImmutableMap.of(
+                "connector", "iceberg",
+                "catalog-type", "hadoop"
+            ),
+            true
+        },
+        new Object[] {
+            "testhadoop",
+            ImmutableMap.of(
+                "connector", "iceberg",
+                "catalog-type", "hadoop"
+            ),
+            false
+        },
+        // Create iceberg table in the hadoop catalog and not_existing_db.
+        new Object[] {
+            "testhadoop",
+            ImmutableMap.of(
+                "connector", "iceberg",
+                "catalog-type", "hadoop",
+                "catalog-database", "not_existing_db"
+            ),
+            true
+        },
+        new Object[] {
+            "testhadoop",
+            ImmutableMap.of(
+                "connector", "iceberg",
+                "catalog-type", "hadoop",
+                "catalog-database", "not_existing_db"
+            ),
+            false
+        },
+        // Create iceberg table in the hive catalog and default database.
+        new Object[] {
+            "testhive",
+            ImmutableMap.of(
+                "connector", "iceberg",
+                "catalog-type", "hive"
+            ),
+            true
+        },
+        new Object[] {
+            "testhive",
+            ImmutableMap.of(
+                "connector", "iceberg",
+                "catalog-type", "hive"
+            ),
+            false
+        },
+        // Create iceberg table in the hive catalog and not_existing_db.
+        new Object[] {
+            "testhive",
+            ImmutableMap.of(
+                "connector", "iceberg",
+                "catalog-type", "hive",
+                "catalog-database", "not_existing_db"
+            ),
+            true
+        },
+        new Object[] {
+            "testhive",
+            ImmutableMap.of(
+                "connector", "iceberg",
+                "catalog-type", "hive",
+                "catalog-database", "not_existing_db"
+            ),
+            false
+        }
+    );
+  }
+
+  public TestIcebergConnector(String catalogName, Map<String, String> 
properties, boolean isStreaming) {
+    this.catalogName = catalogName;
+    this.properties = properties;
+    this.isStreaming = isStreaming;
+  }
+
+  @Override
+  protected TableEnvironment getTableEnv() {
+    if (tEnv == null) {
+      synchronized (this) {
+        if (tEnv == null) {
+          EnvironmentSettings.Builder settingsBuilder = EnvironmentSettings
+              .newInstance()
+              .useBlinkPlanner();
+          if (isStreaming) {
+            settingsBuilder.inStreamingMode();
+            StreamExecutionEnvironment env = StreamExecutionEnvironment
+                
.getExecutionEnvironment(MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG);
+            env.enableCheckpointing(400);
+            env.setMaxParallelism(2);
+            env.setParallelism(2);
+            tEnv = StreamTableEnvironment.create(env, settingsBuilder.build());
+          } else {
+            settingsBuilder.inBatchMode();
+            tEnv = TableEnvironment.create(settingsBuilder.build());
+          }
+          // Set only one parallelism.
+          tEnv.getConfig().getConfiguration()
+              .set(CoreOptions.DEFAULT_PARALLELISM, 1)
+              
.set(FlinkTableOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false);
+        }
+      }
+    }
+    return tEnv;
+  }
+
+  @After
+  public void after() throws TException {
+    sql("DROP TABLE IF EXISTS %s", TABLE_NAME);
+
+    // Clean the created orphan databases and tables from hive-metastore.
+    if (isHiveCatalog()) {
+      HiveMetaStoreClient metaStoreClient = new HiveMetaStoreClient(hiveConf);
+      try {
+        metaStoreClient.dropTable(databaseName(), TABLE_NAME);
+        if (!isDefaultDatabaseName()) {
+          try {
+            metaStoreClient.dropDatabase(databaseName());
+          } catch (Exception ignored) {
+            // Ignore
+          }
+        }
+      } finally {
+        metaStoreClient.close();
+      }
+    }
+  }
+
+  private void testCreateConnectorTable() {
+    Map<String, String> tableProps = createTableProps();
+
+    // Create table under the flink's current database.
+    sql("CREATE TABLE %s (id BIGINT, data STRING) WITH %s", TABLE_NAME, 
toWithClause(tableProps));
+    sql("INSERT INTO %s VALUES (1, 'AAA'), (2, 'BBB'), (3, 'CCC')", 
TABLE_NAME);
+    Assert.assertEquals("Should have expected rows",
+        Lists.newArrayList(Row.of(1L, "AAA"), Row.of(2L, "BBB"), Row.of(3L, 
"CCC")),
+        sql("SELECT * FROM %s", TABLE_NAME));
+
+    FlinkCatalogFactory factory = new FlinkCatalogFactory();
+    Catalog flinkCatalog = factory.createCatalog(catalogName, tableProps, new 
Configuration());
+    Assert.assertTrue("Should have created the expected database", 
flinkCatalog.databaseExists(databaseName()));
+    Assert.assertTrue("Should have created the expected table",
+        flinkCatalog.tableExists(new ObjectPath(databaseName(), TABLE_NAME)));
+
+    // Drop and create it again.
+    sql("DROP TABLE %s", TABLE_NAME);
+    sql("CREATE TABLE %s (id BIGINT, data STRING) WITH %s", TABLE_NAME, 
toWithClause(tableProps));

Review comment:
       Is it possible to omit the schema for cases where the underlying table 
already exists?




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