You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2020/07/08 06:12:50 UTC

[GitHub] [iceberg] JingsongLi opened a new pull request #1182: Integrate Iceberg catalog to Flink catalog

JingsongLi opened a new pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182


   Like Spark 3, Flink also has Catalog interface, we can integrate Iceberg catalog to Flink catalog, iceberg as a Flink catalog, users can use Flink DDLs to manipulate iceberg metadata. And query iceberg tables directly.
   
   The mapping between Flink database and Iceberg namespace:
   - Supplying a base namespace for a given catalog, so if you have a catalog that supports a 2-level namespace, you would supply the first level in the catalog configuration and the second level would be exposed as Flink databases.
   - The Iceberg table manages its partitions by itself. The partition of the Iceberg table is independent of the partition of Flink.
   
   This PR depends on:
   - https://github.com/apache/iceberg/pull/1174
   - https://github.com/apache/iceberg/issues/1176


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on pull request #1182: Flink: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#issuecomment-661557042


   Thanks @rdblue for your patient review~


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453423717



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java
##########
@@ -0,0 +1,508 @@
+/*
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.AbstractCatalog;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogDatabase;
+import org.apache.flink.table.catalog.CatalogDatabaseImpl;
+import org.apache.flink.table.catalog.CatalogFunction;
+import org.apache.flink.table.catalog.CatalogPartition;
+import org.apache.flink.table.catalog.CatalogPartitionSpec;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+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.DatabaseNotEmptyException;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
+import org.apache.flink.table.catalog.exceptions.FunctionNotExistException;
+import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.catalog.stats.CatalogColumnStatistics;
+import org.apache.flink.table.catalog.stats.CatalogTableStatistics;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.util.StringUtils;
+import org.apache.iceberg.CachingCatalog;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+
+/**
+ * A Flink Catalog implementation that wraps an Iceberg {@link Catalog}.
+ * <p>
+ * The mapping between Flink database and Iceberg namespace:
+ * Supplying a base namespace for a given catalog, so if you have a catalog that supports a 2-level namespace, you
+ * would supply the first level in the catalog configuration and the second level would be exposed as Flink databases.
+ * <p>
+ * The Iceberg table manages its partitions by itself. The partition of the Iceberg table is independent of the
+ * partition of Flink.
+ */
+public class FlinkCatalog extends AbstractCatalog {
+
+  private final Catalog originalCatalog;
+  private final Catalog icebergCatalog;
+  private final String[] baseNamespace;
+  private final SupportsNamespaces asNamespaceCatalog;
+
+  public FlinkCatalog(
+      String catalogName,
+      String defaultDatabase,
+      String[] baseNamespace,
+      Catalog icebergCatalog,
+      boolean cacheEnabled) {
+    super(catalogName, defaultDatabase);
+    this.originalCatalog = icebergCatalog;
+    this.icebergCatalog = cacheEnabled ? CachingCatalog.wrap(icebergCatalog) : icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    if (icebergCatalog instanceof SupportsNamespaces) {
+      asNamespaceCatalog = (SupportsNamespaces) icebergCatalog;
+    } else {
+      asNamespaceCatalog = null;
+    }
+  }
+
+  @Override
+  public void open() throws CatalogException {
+  }
+
+  @Override
+  public void close() throws CatalogException {
+    if (originalCatalog instanceof Closeable) {
+      try {
+        ((Closeable) originalCatalog).close();
+      } catch (IOException e) {
+        throw new CatalogException(e);
+      }
+    }
+  }
+
+  private Namespace toNamespace(String database) {
+    String[] namespace = new String[baseNamespace.length + 1];
+    System.arraycopy(baseNamespace, 0, namespace, 0, baseNamespace.length);
+    namespace[baseNamespace.length] = database;
+    return Namespace.of(namespace);
+  }
+
+  private TableIdentifier toIdentifier(ObjectPath path) {
+    return TableIdentifier.of(toNamespace(path.getDatabaseName()), path.getObjectName());
+  }
+
+  @Override
+  public List<String> listDatabases() throws CatalogException {
+    if (asNamespaceCatalog == null) {
+      return Collections.singletonList(getDefaultDatabase());
+    }
+
+    return listAllNamespaces(Namespace.empty()).stream()
+        .map(n -> n.level(n.levels().length - 1))
+        .collect(Collectors.toList());
+  }
+
+  private List<Namespace> listAllNamespaces(Namespace namespace) {
+    if (asNamespaceCatalog == null) {
+      throw new RuntimeException("The asNamespaceCatalog should not be null.");
+    }
+
+    String[] levels = namespace.levels();
+    if (levels.length == baseNamespace.length + 1) {
+      return Collections.singletonList(namespace);
+    }
+    if (levels.length < baseNamespace.length + 1) {
+      for (int i = 0; i < levels.length; i++) {
+        if (!baseNamespace[i].equals(levels[i])) {
+          return Collections.emptyList();
+        }
+      }
+      List<Namespace> ret = new ArrayList<>();
+      asNamespaceCatalog.listNamespaces(namespace).forEach(n -> ret.addAll(listAllNamespaces(n)));
+      return ret;
+    }
+    return Collections.emptyList();
+  }
+
+  @Override
+  public CatalogDatabase getDatabase(String databaseName) throws DatabaseNotExistException, CatalogException {
+    if (asNamespaceCatalog == null) {
+      if (!getDefaultDatabase().equals(databaseName)) {
+        throw new DatabaseNotExistException(getName(), databaseName);
+      } else {
+        return new CatalogDatabaseImpl(new HashMap<>(), "");
+      }
+    } else {
+      try {
+        Map<String, String> metadata =
+            new HashMap<>(asNamespaceCatalog.loadNamespaceMetadata(toNamespace(databaseName)));
+        String comment = metadata.remove("comment");
+        return new CatalogDatabaseImpl(metadata, comment);
+      } catch (NoSuchNamespaceException e) {
+        throw new DatabaseNotExistException(getName(), databaseName, e);
+      }
+    }
+  }
+
+  @Override
+  public boolean databaseExists(String databaseName) throws CatalogException {
+    try {
+      getDatabase(databaseName);
+      return true;
+    } catch (DatabaseNotExistException ignore) {
+      return false;
+    }
+  }
+
+  @Override
+  public void createDatabase(String name, CatalogDatabase database, boolean ignoreIfExists)
+      throws DatabaseAlreadyExistException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      try {
+        asNamespaceCatalog.createNamespace(
+            toNamespace(name),
+            mergeComment(database.getProperties(), database.getComment()));
+      } catch (AlreadyExistsException e) {
+        if (!ignoreIfExists) {
+          throw new DatabaseAlreadyExistException(getName(), name, e);
+        }
+      }
+    } else {
+      throw new UnsupportedOperationException("Namespaces are not supported by catalog: " + getName());
+    }
+  }
+
+  private Map<String, String> mergeComment(Map<String, String> metadata, String comment) {
+    Map<String, String> ret = new HashMap<>(metadata);
+    if (metadata.containsKey("comment")) {
+      throw new CatalogException("Database properties should not contain key: 'comment'.");
+    }
+    if (!StringUtils.isNullOrWhitespaceOnly(comment)) {
+      ret.put("comment", comment);
+    }
+    return ret;
+  }
+
+  @Override
+  public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade)
+      throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      try {
+        boolean success = asNamespaceCatalog.dropNamespace(toNamespace(name));
+        if (!success && !ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name);
+        }
+      } catch (NoSuchNamespaceException e) {
+        if (!ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name, e);
+        }
+      } catch (NamespaceNotEmptyException e) {
+        throw new DatabaseNotEmptyException(getName(), name, e);
+      }
+    } else {
+      if (!ignoreIfNotExists) {
+        throw new DatabaseNotExistException(getName(), name);
+      }
+    }
+  }
+
+  @Override
+  public void alterDatabase(String name, CatalogDatabase newDatabase, boolean ignoreIfNotExists)
+      throws DatabaseNotExistException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      Namespace namespace = toNamespace(name);
+      Map<String, String> updates = Maps.newHashMap();
+      Set<String> removals = Sets.newHashSet();
+
+      try {
+        Map<String, String> oldOptions = asNamespaceCatalog.loadNamespaceMetadata(namespace);
+        Map<String, String> newOptions = mergeComment(newDatabase.getProperties(), newDatabase.getComment());
+
+        for (String key : oldOptions.keySet()) {
+          if (!newOptions.containsKey(key)) {
+            removals.add(key);
+          }
+        }
+
+        for (Map.Entry<String, String> entry : newOptions.entrySet()) {
+          if (!entry.getValue().equals(oldOptions.get(entry.getKey()))) {
+            updates.put(entry.getKey(), entry.getValue());
+          }
+        }
+
+        if (!updates.isEmpty()) {
+          asNamespaceCatalog.setProperties(namespace, updates);
+        }
+
+        if (!removals.isEmpty()) {
+          asNamespaceCatalog.removeProperties(namespace, removals);
+        }
+
+      } catch (org.apache.iceberg.exceptions.NoSuchNamespaceException e) {
+        if (!ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name, e);
+        }
+      }
+    } else {
+      if (!ignoreIfNotExists) {
+        throw new DatabaseNotExistException(getName(), name);
+      }
+    }
+  }
+
+  @Override
+  public List<String> listTables(String databaseName) throws DatabaseNotExistException, CatalogException {
+    try {
+      return icebergCatalog.listTables(toNamespace(databaseName)).stream()
+          .map(TableIdentifier::name)
+          .collect(Collectors.toList());
+    } catch (NoSuchNamespaceException e) {
+      throw new DatabaseNotExistException(getName(), databaseName, e);
+    }
+  }
+
+  @Override
+  public CatalogBaseTable getTable(ObjectPath tablePath) throws TableNotExistException, CatalogException {
+    try {
+      Table table = icebergCatalog.loadTable(toIdentifier(tablePath));
+      TableSchema tableSchema = FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(table.schema()));
+
+      // NOTE: We can not create a IcebergCatalogTable, because Flink optimizer may use CatalogTableImpl to copy a new
+      // catalog table.
+      // Let's re-loading table from Iceberg catalog when creating source/sink operators.
+      return new CatalogTableImpl(tableSchema, table.properties(), "");
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      throw new TableNotExistException(getName(), tablePath, e);
+    }
+  }
+
+  @Override
+  public boolean tableExists(ObjectPath tablePath) throws CatalogException {
+    return icebergCatalog.tableExists(toIdentifier(tablePath));
+  }
+
+  @Override
+  public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists)
+      throws TableNotExistException, CatalogException {
+    try {
+      icebergCatalog.dropTable(toIdentifier(tablePath));
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      throw new TableNotExistException(getName(), tablePath, e);
+    }
+  }
+
+  @Override
+  public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists)
+      throws TableNotExistException, TableAlreadyExistException, CatalogException {
+    try {
+      icebergCatalog.renameTable(
+          toIdentifier(tablePath),
+          toIdentifier(new ObjectPath(tablePath.getDatabaseName(), newTableName)));
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      throw new TableNotExistException(getName(), tablePath, e);
+    } catch (AlreadyExistsException e) {
+      throw new TableAlreadyExistException(getName(), tablePath, e);
+    }
+  }
+
+  /**
+   * TODO Implement DDL-string parser for PartitionSpec.

Review comment:
       After some discussions with Flink developers, we can map Iceberg Partition Transform to Flink Computed Column and Partition. We can support it in future.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453400242



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java
##########
@@ -0,0 +1,508 @@
+/*
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.AbstractCatalog;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogDatabase;
+import org.apache.flink.table.catalog.CatalogDatabaseImpl;
+import org.apache.flink.table.catalog.CatalogFunction;
+import org.apache.flink.table.catalog.CatalogPartition;
+import org.apache.flink.table.catalog.CatalogPartitionSpec;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+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.DatabaseNotEmptyException;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
+import org.apache.flink.table.catalog.exceptions.FunctionNotExistException;
+import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.catalog.stats.CatalogColumnStatistics;
+import org.apache.flink.table.catalog.stats.CatalogTableStatistics;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.util.StringUtils;
+import org.apache.iceberg.CachingCatalog;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+
+/**
+ * A Flink Catalog implementation that wraps an Iceberg {@link Catalog}.
+ * <p>
+ * The mapping between Flink database and Iceberg namespace:
+ * Supplying a base namespace for a given catalog, so if you have a catalog that supports a 2-level namespace, you
+ * would supply the first level in the catalog configuration and the second level would be exposed as Flink databases.
+ * <p>
+ * The Iceberg table manages its partitions by itself. The partition of the Iceberg table is independent of the
+ * partition of Flink.
+ */
+public class FlinkCatalog extends AbstractCatalog {
+
+  private final Catalog originalCatalog;
+  private final Catalog icebergCatalog;
+  private final String[] baseNamespace;
+  private final SupportsNamespaces asNamespaceCatalog;
+
+  public FlinkCatalog(
+      String catalogName,
+      String defaultDatabase,
+      String[] baseNamespace,
+      Catalog icebergCatalog,
+      boolean cacheEnabled) {
+    super(catalogName, defaultDatabase);
+    this.originalCatalog = icebergCatalog;
+    this.icebergCatalog = cacheEnabled ? CachingCatalog.wrap(icebergCatalog) : icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    if (icebergCatalog instanceof SupportsNamespaces) {
+      asNamespaceCatalog = (SupportsNamespaces) icebergCatalog;
+    } else {
+      asNamespaceCatalog = null;
+    }
+  }
+
+  @Override
+  public void open() throws CatalogException {
+  }
+
+  @Override
+  public void close() throws CatalogException {
+    if (originalCatalog instanceof Closeable) {
+      try {
+        ((Closeable) originalCatalog).close();
+      } catch (IOException e) {
+        throw new CatalogException(e);
+      }
+    }
+  }
+
+  private Namespace toNamespace(String database) {
+    String[] namespace = new String[baseNamespace.length + 1];
+    System.arraycopy(baseNamespace, 0, namespace, 0, baseNamespace.length);
+    namespace[baseNamespace.length] = database;
+    return Namespace.of(namespace);
+  }
+
+  private TableIdentifier toIdentifier(ObjectPath path) {
+    return TableIdentifier.of(toNamespace(path.getDatabaseName()), path.getObjectName());
+  }
+
+  @Override
+  public List<String> listDatabases() throws CatalogException {
+    if (asNamespaceCatalog == null) {
+      return Collections.singletonList(getDefaultDatabase());
+    }
+
+    return listAllNamespaces(Namespace.empty()).stream()
+        .map(n -> n.level(n.levels().length - 1))
+        .collect(Collectors.toList());
+  }
+
+  private List<Namespace> listAllNamespaces(Namespace namespace) {
+    if (asNamespaceCatalog == null) {
+      throw new RuntimeException("The asNamespaceCatalog should not be null.");
+    }
+
+    String[] levels = namespace.levels();
+    if (levels.length == baseNamespace.length + 1) {
+      return Collections.singletonList(namespace);
+    }
+    if (levels.length < baseNamespace.length + 1) {
+      for (int i = 0; i < levels.length; i++) {
+        if (!baseNamespace[i].equals(levels[i])) {
+          return Collections.emptyList();
+        }
+      }
+      List<Namespace> ret = new ArrayList<>();
+      asNamespaceCatalog.listNamespaces(namespace).forEach(n -> ret.addAll(listAllNamespaces(n)));
+      return ret;
+    }
+    return Collections.emptyList();
+  }
+
+  @Override
+  public CatalogDatabase getDatabase(String databaseName) throws DatabaseNotExistException, CatalogException {
+    if (asNamespaceCatalog == null) {
+      if (!getDefaultDatabase().equals(databaseName)) {
+        throw new DatabaseNotExistException(getName(), databaseName);
+      } else {
+        return new CatalogDatabaseImpl(new HashMap<>(), "");
+      }
+    } else {
+      try {
+        Map<String, String> metadata =
+            new HashMap<>(asNamespaceCatalog.loadNamespaceMetadata(toNamespace(databaseName)));
+        String comment = metadata.remove("comment");
+        return new CatalogDatabaseImpl(metadata, comment);
+      } catch (NoSuchNamespaceException e) {
+        throw new DatabaseNotExistException(getName(), databaseName, e);
+      }
+    }
+  }
+
+  @Override
+  public boolean databaseExists(String databaseName) throws CatalogException {
+    try {
+      getDatabase(databaseName);
+      return true;
+    } catch (DatabaseNotExistException ignore) {
+      return false;
+    }
+  }
+
+  @Override
+  public void createDatabase(String name, CatalogDatabase database, boolean ignoreIfExists)
+      throws DatabaseAlreadyExistException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      try {
+        asNamespaceCatalog.createNamespace(
+            toNamespace(name),
+            mergeComment(database.getProperties(), database.getComment()));
+      } catch (AlreadyExistsException e) {
+        if (!ignoreIfExists) {
+          throw new DatabaseAlreadyExistException(getName(), name, e);
+        }
+      }
+    } else {
+      throw new UnsupportedOperationException("Namespaces are not supported by catalog: " + getName());
+    }
+  }
+
+  private Map<String, String> mergeComment(Map<String, String> metadata, String comment) {
+    Map<String, String> ret = new HashMap<>(metadata);
+    if (metadata.containsKey("comment")) {
+      throw new CatalogException("Database properties should not contain key: 'comment'.");
+    }
+    if (!StringUtils.isNullOrWhitespaceOnly(comment)) {
+      ret.put("comment", comment);
+    }
+    return ret;
+  }
+
+  @Override
+  public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade)
+      throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      try {
+        boolean success = asNamespaceCatalog.dropNamespace(toNamespace(name));
+        if (!success && !ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name);
+        }
+      } catch (NoSuchNamespaceException e) {
+        if (!ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name, e);
+        }
+      } catch (NamespaceNotEmptyException e) {
+        throw new DatabaseNotEmptyException(getName(), name, e);
+      }
+    } else {
+      if (!ignoreIfNotExists) {
+        throw new DatabaseNotExistException(getName(), name);
+      }
+    }
+  }
+
+  @Override
+  public void alterDatabase(String name, CatalogDatabase newDatabase, boolean ignoreIfNotExists)
+      throws DatabaseNotExistException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      Namespace namespace = toNamespace(name);
+      Map<String, String> updates = Maps.newHashMap();
+      Set<String> removals = Sets.newHashSet();
+
+      try {
+        Map<String, String> oldOptions = asNamespaceCatalog.loadNamespaceMetadata(namespace);
+        Map<String, String> newOptions = mergeComment(newDatabase.getProperties(), newDatabase.getComment());
+
+        for (String key : oldOptions.keySet()) {
+          if (!newOptions.containsKey(key)) {
+            removals.add(key);
+          }
+        }
+
+        for (Map.Entry<String, String> entry : newOptions.entrySet()) {
+          if (!entry.getValue().equals(oldOptions.get(entry.getKey()))) {
+            updates.put(entry.getKey(), entry.getValue());
+          }
+        }
+
+        if (!updates.isEmpty()) {
+          asNamespaceCatalog.setProperties(namespace, updates);
+        }
+
+        if (!removals.isEmpty()) {
+          asNamespaceCatalog.removeProperties(namespace, removals);
+        }
+
+      } catch (org.apache.iceberg.exceptions.NoSuchNamespaceException e) {
+        if (!ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name, e);
+        }
+      }
+    } else {
+      if (!ignoreIfNotExists) {
+        throw new DatabaseNotExistException(getName(), name);
+      }
+    }
+  }
+
+  @Override
+  public List<String> listTables(String databaseName) throws DatabaseNotExistException, CatalogException {
+    try {
+      return icebergCatalog.listTables(toNamespace(databaseName)).stream()
+          .map(TableIdentifier::name)
+          .collect(Collectors.toList());
+    } catch (NoSuchNamespaceException e) {
+      throw new DatabaseNotExistException(getName(), databaseName, e);
+    }
+  }
+
+  @Override
+  public CatalogBaseTable getTable(ObjectPath tablePath) throws TableNotExistException, CatalogException {
+    try {
+      Table table = icebergCatalog.loadTable(toIdentifier(tablePath));
+      TableSchema tableSchema = FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(table.schema()));
+
+      // NOTE: We can not create a IcebergCatalogTable, because Flink optimizer may use CatalogTableImpl to copy a new
+      // catalog table.
+      // Let's re-loading table from Iceberg catalog when creating source/sink operators.
+      return new CatalogTableImpl(tableSchema, table.properties(), "");
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      throw new TableNotExistException(getName(), tablePath, e);
+    }
+  }
+
+  @Override
+  public boolean tableExists(ObjectPath tablePath) throws CatalogException {
+    return icebergCatalog.tableExists(toIdentifier(tablePath));
+  }
+
+  @Override
+  public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists)
+      throws TableNotExistException, CatalogException {
+    try {
+      icebergCatalog.dropTable(toIdentifier(tablePath));
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      throw new TableNotExistException(getName(), tablePath, e);
+    }
+  }
+
+  @Override
+  public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists)
+      throws TableNotExistException, TableAlreadyExistException, CatalogException {
+    try {
+      icebergCatalog.renameTable(
+          toIdentifier(tablePath),
+          toIdentifier(new ObjectPath(tablePath.getDatabaseName(), newTableName)));
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      throw new TableNotExistException(getName(), tablePath, e);
+    } catch (AlreadyExistsException e) {
+      throw new TableAlreadyExistException(getName(), tablePath, e);
+    }
+  }
+
+  /**
+   * TODO Implement DDL-string parser for PartitionSpec.

Review comment:
       I prefer that adding partitioning to the Flink DDL parser. I'll modify the comments.
   Using `PartitionSpecParser.fromJson` looks very difficult to use.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453113164



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java
##########
@@ -0,0 +1,241 @@
+/*
+ * 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.File;
+import java.util.Map;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.types.Types;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Test;
+
+public class TestFlinkCatalogDatabase extends FlinkCatalogTestBase {
+
+  public TestFlinkCatalogDatabase(String catalogName, String[] baseNamepace) {
+    super(catalogName, baseNamepace);
+  }
+
+  @After
+  public void clean() {
+    sql("DROP TABLE IF EXISTS %s.tl", flinkIdentifier);
+    sql("DROP DATABASE IF EXISTS %s", flinkIdentifier);
+  }
+
+  @Test
+  public void testCreateNamespace() {
+    Assert.assertFalse(
+        "Database should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+
+    Assert.assertTrue("Database should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+  }
+
+  @Test
+  public void testDefaultDatabase() {
+    sql("USE CATALOG %s", catalogName);
+
+    Assert.assertEquals("Should use the current catalog", tEnv.getCurrentCatalog(), catalogName);
+    Assert.assertEquals("Should use the configured default namespace", tEnv.getCurrentDatabase(), "default");
+  }
+
+  @Test
+  public void testDropEmptyDatabase() {
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("DROP DATABASE %s", flinkIdentifier);
+
+    Assert.assertFalse(
+        "Namespace should have been dropped",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+  }
+
+  @Test
+  public void testDropNonEmptyNamespace() {
+    Assume.assumeFalse("Hadoop catalog throws IOException: Directory is not empty.", isHadoopCatalog);
+
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+
+    validationCatalog.createTable(
+        TableIdentifier.of(icebergNamespace, "tl"),
+        new Schema(Types.NestedField.optional(0, "id", Types.LongType.get())));
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+    Assert.assertTrue("Table should exist", validationCatalog.tableExists(TableIdentifier.of(icebergNamespace, "tl")));
+
+    AssertHelpers.assertThrowsCause(

Review comment:
       Why does this require `assertThrowsCause`? Is it wrapped in a generic SQL failure exception?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453107873



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java
##########
@@ -0,0 +1,508 @@
+/*
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.AbstractCatalog;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogDatabase;
+import org.apache.flink.table.catalog.CatalogDatabaseImpl;
+import org.apache.flink.table.catalog.CatalogFunction;
+import org.apache.flink.table.catalog.CatalogPartition;
+import org.apache.flink.table.catalog.CatalogPartitionSpec;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+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.DatabaseNotEmptyException;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
+import org.apache.flink.table.catalog.exceptions.FunctionNotExistException;
+import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.catalog.stats.CatalogColumnStatistics;
+import org.apache.flink.table.catalog.stats.CatalogTableStatistics;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.util.StringUtils;
+import org.apache.iceberg.CachingCatalog;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+
+/**
+ * A Flink Catalog implementation that wraps an Iceberg {@link Catalog}.
+ * <p>
+ * The mapping between Flink database and Iceberg namespace:
+ * Supplying a base namespace for a given catalog, so if you have a catalog that supports a 2-level namespace, you
+ * would supply the first level in the catalog configuration and the second level would be exposed as Flink databases.
+ * <p>
+ * The Iceberg table manages its partitions by itself. The partition of the Iceberg table is independent of the
+ * partition of Flink.
+ */
+public class FlinkCatalog extends AbstractCatalog {
+
+  private final Catalog originalCatalog;
+  private final Catalog icebergCatalog;
+  private final String[] baseNamespace;
+  private final SupportsNamespaces asNamespaceCatalog;
+
+  public FlinkCatalog(
+      String catalogName,
+      String defaultDatabase,
+      String[] baseNamespace,
+      Catalog icebergCatalog,
+      boolean cacheEnabled) {
+    super(catalogName, defaultDatabase);
+    this.originalCatalog = icebergCatalog;
+    this.icebergCatalog = cacheEnabled ? CachingCatalog.wrap(icebergCatalog) : icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    if (icebergCatalog instanceof SupportsNamespaces) {
+      asNamespaceCatalog = (SupportsNamespaces) icebergCatalog;
+    } else {
+      asNamespaceCatalog = null;
+    }
+  }
+
+  @Override
+  public void open() throws CatalogException {
+  }
+
+  @Override
+  public void close() throws CatalogException {
+    if (originalCatalog instanceof Closeable) {
+      try {
+        ((Closeable) originalCatalog).close();
+      } catch (IOException e) {
+        throw new CatalogException(e);
+      }
+    }
+  }
+
+  private Namespace toNamespace(String database) {
+    String[] namespace = new String[baseNamespace.length + 1];
+    System.arraycopy(baseNamespace, 0, namespace, 0, baseNamespace.length);
+    namespace[baseNamespace.length] = database;
+    return Namespace.of(namespace);
+  }
+
+  private TableIdentifier toIdentifier(ObjectPath path) {
+    return TableIdentifier.of(toNamespace(path.getDatabaseName()), path.getObjectName());
+  }
+
+  @Override
+  public List<String> listDatabases() throws CatalogException {
+    if (asNamespaceCatalog == null) {
+      return Collections.singletonList(getDefaultDatabase());
+    }
+
+    return listAllNamespaces(Namespace.empty()).stream()
+        .map(n -> n.level(n.levels().length - 1))
+        .collect(Collectors.toList());
+  }
+
+  private List<Namespace> listAllNamespaces(Namespace namespace) {
+    if (asNamespaceCatalog == null) {
+      throw new RuntimeException("The asNamespaceCatalog should not be null.");
+    }
+
+    String[] levels = namespace.levels();
+    if (levels.length == baseNamespace.length + 1) {
+      return Collections.singletonList(namespace);
+    }
+    if (levels.length < baseNamespace.length + 1) {
+      for (int i = 0; i < levels.length; i++) {
+        if (!baseNamespace[i].equals(levels[i])) {
+          return Collections.emptyList();
+        }
+      }
+      List<Namespace> ret = new ArrayList<>();
+      asNamespaceCatalog.listNamespaces(namespace).forEach(n -> ret.addAll(listAllNamespaces(n)));
+      return ret;
+    }
+    return Collections.emptyList();
+  }
+
+  @Override
+  public CatalogDatabase getDatabase(String databaseName) throws DatabaseNotExistException, CatalogException {
+    if (asNamespaceCatalog == null) {
+      if (!getDefaultDatabase().equals(databaseName)) {
+        throw new DatabaseNotExistException(getName(), databaseName);
+      } else {
+        return new CatalogDatabaseImpl(new HashMap<>(), "");
+      }
+    } else {
+      try {
+        Map<String, String> metadata =
+            new HashMap<>(asNamespaceCatalog.loadNamespaceMetadata(toNamespace(databaseName)));
+        String comment = metadata.remove("comment");
+        return new CatalogDatabaseImpl(metadata, comment);
+      } catch (NoSuchNamespaceException e) {
+        throw new DatabaseNotExistException(getName(), databaseName, e);
+      }
+    }
+  }
+
+  @Override
+  public boolean databaseExists(String databaseName) throws CatalogException {
+    try {
+      getDatabase(databaseName);
+      return true;
+    } catch (DatabaseNotExistException ignore) {
+      return false;
+    }
+  }
+
+  @Override
+  public void createDatabase(String name, CatalogDatabase database, boolean ignoreIfExists)
+      throws DatabaseAlreadyExistException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      try {
+        asNamespaceCatalog.createNamespace(
+            toNamespace(name),
+            mergeComment(database.getProperties(), database.getComment()));
+      } catch (AlreadyExistsException e) {
+        if (!ignoreIfExists) {
+          throw new DatabaseAlreadyExistException(getName(), name, e);
+        }
+      }
+    } else {
+      throw new UnsupportedOperationException("Namespaces are not supported by catalog: " + getName());
+    }
+  }
+
+  private Map<String, String> mergeComment(Map<String, String> metadata, String comment) {
+    Map<String, String> ret = new HashMap<>(metadata);
+    if (metadata.containsKey("comment")) {
+      throw new CatalogException("Database properties should not contain key: 'comment'.");
+    }
+    if (!StringUtils.isNullOrWhitespaceOnly(comment)) {
+      ret.put("comment", comment);
+    }
+    return ret;
+  }
+
+  @Override
+  public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade)
+      throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      try {
+        boolean success = asNamespaceCatalog.dropNamespace(toNamespace(name));
+        if (!success && !ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name);
+        }
+      } catch (NoSuchNamespaceException e) {
+        if (!ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name, e);
+        }
+      } catch (NamespaceNotEmptyException e) {
+        throw new DatabaseNotEmptyException(getName(), name, e);
+      }
+    } else {
+      if (!ignoreIfNotExists) {
+        throw new DatabaseNotExistException(getName(), name);
+      }
+    }
+  }
+
+  @Override
+  public void alterDatabase(String name, CatalogDatabase newDatabase, boolean ignoreIfNotExists)
+      throws DatabaseNotExistException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      Namespace namespace = toNamespace(name);
+      Map<String, String> updates = Maps.newHashMap();
+      Set<String> removals = Sets.newHashSet();
+
+      try {
+        Map<String, String> oldOptions = asNamespaceCatalog.loadNamespaceMetadata(namespace);
+        Map<String, String> newOptions = mergeComment(newDatabase.getProperties(), newDatabase.getComment());
+
+        for (String key : oldOptions.keySet()) {
+          if (!newOptions.containsKey(key)) {
+            removals.add(key);
+          }
+        }
+
+        for (Map.Entry<String, String> entry : newOptions.entrySet()) {
+          if (!entry.getValue().equals(oldOptions.get(entry.getKey()))) {
+            updates.put(entry.getKey(), entry.getValue());
+          }
+        }
+
+        if (!updates.isEmpty()) {
+          asNamespaceCatalog.setProperties(namespace, updates);
+        }
+
+        if (!removals.isEmpty()) {
+          asNamespaceCatalog.removeProperties(namespace, removals);
+        }
+
+      } catch (org.apache.iceberg.exceptions.NoSuchNamespaceException e) {
+        if (!ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name, e);
+        }
+      }
+    } else {
+      if (!ignoreIfNotExists) {

Review comment:
       This doesn't seem to handle the default database. What is the correct behavior when the catalog doesn't support namespaces, but this is called for the default database?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453112078



##########
File path: flink/src/test/java/org/apache/iceberg/flink/FlinkCatalogTestBase.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.util.ArrayUtils;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public abstract class FlinkCatalogTestBase extends FlinkTestBase {
+
+  protected static final String DATABASE = "db";
+  private static File warehouse = null;
+
+  @BeforeClass
+  public static void createWarehouse() throws IOException {
+    FlinkCatalogTestBase.warehouse = File.createTempFile("warehouse", null);
+    Assert.assertTrue(warehouse.delete());
+  }
+
+  @AfterClass
+  public static void dropWarehouse() {
+    if (warehouse != null && warehouse.exists()) {
+      warehouse.delete();
+    }
+  }
+
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { "testhive", new String[0] },
+        new Object[] { "testhadoop", new String[0] },
+        new Object[] { "testhadoop", new String[] { "l0", "l1" }},
+    };
+  }
+
+  protected final TableEnvironment tEnv =
+      TableEnvironment.create(EnvironmentSettings.newInstance().useBlinkPlanner().inBatchMode().build());
+
+  protected final String catalogName;
+  protected final String[] baseNamespace;
+  protected final Catalog validationCatalog;
+  protected final SupportsNamespaces validationNamespaceCatalog;
+  protected final org.apache.flink.table.catalog.Catalog flinkCatalog;
+
+  protected final String flinkIdentifier;
+  protected final Namespace icebergNamespace;
+  protected final boolean isHadoopCatalog;
+
+  public FlinkCatalogTestBase(String catalogName, String[] baseNamespace) {
+    this.catalogName = catalogName;
+    this.baseNamespace = baseNamespace;
+    this.isHadoopCatalog = catalogName.equals("testhadoop");
+    this.validationCatalog = isHadoopCatalog ?
+        new HadoopCatalog(hiveConf, "file:" + warehouse) :
+        catalog;
+    this.validationNamespaceCatalog = (SupportsNamespaces) validationCatalog;
+
+    Map<String, String> config = new HashMap<>();
+    config.put("type", "iceberg");
+    config.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, isHadoopCatalog ? "hadoop" : "hive");
+    config.put(FlinkCatalogFactory.HADOOP_WAREHOUSE_LOCATION, "file:" + warehouse);
+    if (baseNamespace.length > 0) {
+      config.put(FlinkCatalogFactory.BASE_NAMESPACE, Joiner.on(".").join(baseNamespace));
+    }
+
+    FlinkCatalogFactory factory = new FlinkCatalogFactory() {
+      @Override
+      protected Catalog buildIcebergCatalog(String name, Map<String, String> options) {
+        // Flink hadoop configuration depends on system env, it is quiet hard to set from testing. So directly pass
+        // correct hadoop configuration.
+        return super.buildIcebergCatalog(name, options, hiveConf);
+      }
+    };
+    flinkCatalog = factory.createCatalog(catalogName, config);
+    tEnv.registerCatalog(catalogName, flinkCatalog);
+
+    this.flinkIdentifier = catalogName + "." + DATABASE;
+    this.icebergNamespace = Namespace.of(ArrayUtils.concat(baseNamespace, new String[] { DATABASE }));
+  }
+
+  @After

Review comment:
       Won't this close the catalog after every test method?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1182: Flink: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r457813260



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java
##########
@@ -0,0 +1,508 @@
+/*
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.AbstractCatalog;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogDatabase;
+import org.apache.flink.table.catalog.CatalogDatabaseImpl;
+import org.apache.flink.table.catalog.CatalogFunction;
+import org.apache.flink.table.catalog.CatalogPartition;
+import org.apache.flink.table.catalog.CatalogPartitionSpec;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+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.DatabaseNotEmptyException;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
+import org.apache.flink.table.catalog.exceptions.FunctionNotExistException;
+import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.catalog.stats.CatalogColumnStatistics;
+import org.apache.flink.table.catalog.stats.CatalogTableStatistics;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.util.StringUtils;
+import org.apache.iceberg.CachingCatalog;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+
+/**
+ * A Flink Catalog implementation that wraps an Iceberg {@link Catalog}.
+ * <p>
+ * The mapping between Flink database and Iceberg namespace:
+ * Supplying a base namespace for a given catalog, so if you have a catalog that supports a 2-level namespace, you
+ * would supply the first level in the catalog configuration and the second level would be exposed as Flink databases.
+ * <p>
+ * The Iceberg table manages its partitions by itself. The partition of the Iceberg table is independent of the
+ * partition of Flink.
+ */
+public class FlinkCatalog extends AbstractCatalog {
+
+  private final Catalog originalCatalog;
+  private final Catalog icebergCatalog;
+  private final String[] baseNamespace;
+  private final SupportsNamespaces asNamespaceCatalog;
+
+  public FlinkCatalog(
+      String catalogName,
+      String defaultDatabase,
+      String[] baseNamespace,
+      Catalog icebergCatalog,
+      boolean cacheEnabled) {
+    super(catalogName, defaultDatabase);
+    this.originalCatalog = icebergCatalog;
+    this.icebergCatalog = cacheEnabled ? CachingCatalog.wrap(icebergCatalog) : icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    if (icebergCatalog instanceof SupportsNamespaces) {
+      asNamespaceCatalog = (SupportsNamespaces) icebergCatalog;
+    } else {
+      asNamespaceCatalog = null;
+    }
+  }
+
+  @Override
+  public void open() throws CatalogException {
+  }
+
+  @Override
+  public void close() throws CatalogException {
+    if (originalCatalog instanceof Closeable) {
+      try {
+        ((Closeable) originalCatalog).close();
+      } catch (IOException e) {
+        throw new CatalogException(e);
+      }
+    }
+  }
+
+  private Namespace toNamespace(String database) {
+    String[] namespace = new String[baseNamespace.length + 1];
+    System.arraycopy(baseNamespace, 0, namespace, 0, baseNamespace.length);
+    namespace[baseNamespace.length] = database;
+    return Namespace.of(namespace);
+  }
+
+  private TableIdentifier toIdentifier(ObjectPath path) {
+    return TableIdentifier.of(toNamespace(path.getDatabaseName()), path.getObjectName());
+  }
+
+  @Override
+  public List<String> listDatabases() throws CatalogException {
+    if (asNamespaceCatalog == null) {
+      return Collections.singletonList(getDefaultDatabase());
+    }
+
+    return listAllNamespaces(Namespace.empty()).stream()
+        .map(n -> n.level(n.levels().length - 1))
+        .collect(Collectors.toList());
+  }
+
+  private List<Namespace> listAllNamespaces(Namespace namespace) {
+    if (asNamespaceCatalog == null) {
+      throw new RuntimeException("The asNamespaceCatalog should not be null.");
+    }
+
+    String[] levels = namespace.levels();
+    if (levels.length == baseNamespace.length + 1) {
+      return Collections.singletonList(namespace);
+    }
+    if (levels.length < baseNamespace.length + 1) {
+      for (int i = 0; i < levels.length; i++) {
+        if (!baseNamespace[i].equals(levels[i])) {
+          return Collections.emptyList();
+        }
+      }
+      List<Namespace> ret = new ArrayList<>();
+      asNamespaceCatalog.listNamespaces(namespace).forEach(n -> ret.addAll(listAllNamespaces(n)));
+      return ret;
+    }
+    return Collections.emptyList();
+  }
+
+  @Override
+  public CatalogDatabase getDatabase(String databaseName) throws DatabaseNotExistException, CatalogException {
+    if (asNamespaceCatalog == null) {
+      if (!getDefaultDatabase().equals(databaseName)) {
+        throw new DatabaseNotExistException(getName(), databaseName);
+      } else {
+        return new CatalogDatabaseImpl(new HashMap<>(), "");
+      }
+    } else {
+      try {
+        Map<String, String> metadata =
+            new HashMap<>(asNamespaceCatalog.loadNamespaceMetadata(toNamespace(databaseName)));
+        String comment = metadata.remove("comment");
+        return new CatalogDatabaseImpl(metadata, comment);
+      } catch (NoSuchNamespaceException e) {
+        throw new DatabaseNotExistException(getName(), databaseName, e);
+      }
+    }
+  }
+
+  @Override
+  public boolean databaseExists(String databaseName) throws CatalogException {
+    try {
+      getDatabase(databaseName);
+      return true;
+    } catch (DatabaseNotExistException ignore) {
+      return false;
+    }
+  }
+
+  @Override
+  public void createDatabase(String name, CatalogDatabase database, boolean ignoreIfExists)
+      throws DatabaseAlreadyExistException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      try {
+        asNamespaceCatalog.createNamespace(
+            toNamespace(name),
+            mergeComment(database.getProperties(), database.getComment()));
+      } catch (AlreadyExistsException e) {
+        if (!ignoreIfExists) {
+          throw new DatabaseAlreadyExistException(getName(), name, e);
+        }
+      }
+    } else {
+      throw new UnsupportedOperationException("Namespaces are not supported by catalog: " + getName());
+    }
+  }
+
+  private Map<String, String> mergeComment(Map<String, String> metadata, String comment) {
+    Map<String, String> ret = new HashMap<>(metadata);
+    if (metadata.containsKey("comment")) {
+      throw new CatalogException("Database properties should not contain key: 'comment'.");
+    }
+    if (!StringUtils.isNullOrWhitespaceOnly(comment)) {
+      ret.put("comment", comment);
+    }
+    return ret;
+  }
+
+  @Override
+  public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade)
+      throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      try {
+        boolean success = asNamespaceCatalog.dropNamespace(toNamespace(name));
+        if (!success && !ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name);
+        }
+      } catch (NoSuchNamespaceException e) {
+        if (!ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name, e);
+        }
+      } catch (NamespaceNotEmptyException e) {
+        throw new DatabaseNotEmptyException(getName(), name, e);
+      }
+    } else {
+      if (!ignoreIfNotExists) {
+        throw new DatabaseNotExistException(getName(), name);
+      }
+    }
+  }
+
+  @Override
+  public void alterDatabase(String name, CatalogDatabase newDatabase, boolean ignoreIfNotExists)
+      throws DatabaseNotExistException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      Namespace namespace = toNamespace(name);
+      Map<String, String> updates = Maps.newHashMap();
+      Set<String> removals = Sets.newHashSet();
+
+      try {
+        Map<String, String> oldOptions = asNamespaceCatalog.loadNamespaceMetadata(namespace);
+        Map<String, String> newOptions = mergeComment(newDatabase.getProperties(), newDatabase.getComment());
+
+        for (String key : oldOptions.keySet()) {
+          if (!newOptions.containsKey(key)) {
+            removals.add(key);
+          }
+        }
+
+        for (Map.Entry<String, String> entry : newOptions.entrySet()) {
+          if (!entry.getValue().equals(oldOptions.get(entry.getKey()))) {
+            updates.put(entry.getKey(), entry.getValue());
+          }
+        }
+
+        if (!updates.isEmpty()) {
+          asNamespaceCatalog.setProperties(namespace, updates);
+        }
+
+        if (!removals.isEmpty()) {
+          asNamespaceCatalog.removeProperties(namespace, removals);
+        }
+
+      } catch (org.apache.iceberg.exceptions.NoSuchNamespaceException e) {
+        if (!ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name, e);
+        }
+      }
+    } else {
+      if (!ignoreIfNotExists) {
+        throw new DatabaseNotExistException(getName(), name);
+      }
+    }
+  }
+
+  @Override
+  public List<String> listTables(String databaseName) throws DatabaseNotExistException, CatalogException {
+    try {
+      return icebergCatalog.listTables(toNamespace(databaseName)).stream()
+          .map(TableIdentifier::name)
+          .collect(Collectors.toList());
+    } catch (NoSuchNamespaceException e) {
+      throw new DatabaseNotExistException(getName(), databaseName, e);
+    }
+  }
+
+  @Override
+  public CatalogBaseTable getTable(ObjectPath tablePath) throws TableNotExistException, CatalogException {
+    try {
+      Table table = icebergCatalog.loadTable(toIdentifier(tablePath));
+      TableSchema tableSchema = FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(table.schema()));
+
+      // NOTE: We can not create a IcebergCatalogTable, because Flink optimizer may use CatalogTableImpl to copy a new
+      // catalog table.
+      // Let's re-loading table from Iceberg catalog when creating source/sink operators.
+      return new CatalogTableImpl(tableSchema, table.properties(), "");
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      throw new TableNotExistException(getName(), tablePath, e);
+    }
+  }
+
+  @Override
+  public boolean tableExists(ObjectPath tablePath) throws CatalogException {
+    return icebergCatalog.tableExists(toIdentifier(tablePath));
+  }
+
+  @Override
+  public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists)
+      throws TableNotExistException, CatalogException {
+    try {
+      icebergCatalog.dropTable(toIdentifier(tablePath));
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      throw new TableNotExistException(getName(), tablePath, e);
+    }
+  }
+
+  @Override
+  public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists)
+      throws TableNotExistException, TableAlreadyExistException, CatalogException {
+    try {
+      icebergCatalog.renameTable(
+          toIdentifier(tablePath),
+          toIdentifier(new ObjectPath(tablePath.getDatabaseName(), newTableName)));
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      throw new TableNotExistException(getName(), tablePath, e);
+    } catch (AlreadyExistsException e) {
+      throw new TableAlreadyExistException(getName(), tablePath, e);
+    }
+  }
+
+  /**
+   * TODO Implement DDL-string parser for PartitionSpec.

Review comment:
       Good points.
   
   For Flink SQL, computed columns are virtual columns, the source and sink can just ignore them, the source just produces columns without computed columns, the Flink core will generate computed columns for input records. For sink, Flink core just give the records without computed columns to connector sink.
   
   - I see, you mean https://iceberg.apache.org/evolution/#partition-evolution , the computed columns should be calculated by Flink core, iceberg should just deal with its physical logical.
   - There are three types of function: 1.`hour,day,month,year` are the same as Flink's functions. 2. For `truncate`, Flink also supports this function, but not support `truncate` with input type string and bytes, iceberg can provides catalog function (`Catalog.getFunction`), users can use `iceberg_catalog.truncate` to create computed column. 3. For bucket, Flink not support this function, so iceberg can provides catalog functions, users can directly use it.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java
##########
@@ -0,0 +1,508 @@
+/*
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.AbstractCatalog;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogDatabase;
+import org.apache.flink.table.catalog.CatalogDatabaseImpl;
+import org.apache.flink.table.catalog.CatalogFunction;
+import org.apache.flink.table.catalog.CatalogPartition;
+import org.apache.flink.table.catalog.CatalogPartitionSpec;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+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.DatabaseNotEmptyException;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
+import org.apache.flink.table.catalog.exceptions.FunctionNotExistException;
+import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.catalog.stats.CatalogColumnStatistics;
+import org.apache.flink.table.catalog.stats.CatalogTableStatistics;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.util.StringUtils;
+import org.apache.iceberg.CachingCatalog;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+
+/**
+ * A Flink Catalog implementation that wraps an Iceberg {@link Catalog}.
+ * <p>
+ * The mapping between Flink database and Iceberg namespace:
+ * Supplying a base namespace for a given catalog, so if you have a catalog that supports a 2-level namespace, you
+ * would supply the first level in the catalog configuration and the second level would be exposed as Flink databases.
+ * <p>
+ * The Iceberg table manages its partitions by itself. The partition of the Iceberg table is independent of the
+ * partition of Flink.
+ */
+public class FlinkCatalog extends AbstractCatalog {
+
+  private final Catalog originalCatalog;
+  private final Catalog icebergCatalog;
+  private final String[] baseNamespace;
+  private final SupportsNamespaces asNamespaceCatalog;
+
+  public FlinkCatalog(
+      String catalogName,
+      String defaultDatabase,
+      String[] baseNamespace,
+      Catalog icebergCatalog,
+      boolean cacheEnabled) {
+    super(catalogName, defaultDatabase);
+    this.originalCatalog = icebergCatalog;
+    this.icebergCatalog = cacheEnabled ? CachingCatalog.wrap(icebergCatalog) : icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    if (icebergCatalog instanceof SupportsNamespaces) {
+      asNamespaceCatalog = (SupportsNamespaces) icebergCatalog;
+    } else {
+      asNamespaceCatalog = null;
+    }
+  }
+
+  @Override
+  public void open() throws CatalogException {
+  }
+
+  @Override
+  public void close() throws CatalogException {
+    if (originalCatalog instanceof Closeable) {
+      try {
+        ((Closeable) originalCatalog).close();
+      } catch (IOException e) {
+        throw new CatalogException(e);
+      }
+    }
+  }
+
+  private Namespace toNamespace(String database) {
+    String[] namespace = new String[baseNamespace.length + 1];
+    System.arraycopy(baseNamespace, 0, namespace, 0, baseNamespace.length);
+    namespace[baseNamespace.length] = database;
+    return Namespace.of(namespace);
+  }
+
+  private TableIdentifier toIdentifier(ObjectPath path) {
+    return TableIdentifier.of(toNamespace(path.getDatabaseName()), path.getObjectName());
+  }
+
+  @Override
+  public List<String> listDatabases() throws CatalogException {
+    if (asNamespaceCatalog == null) {
+      return Collections.singletonList(getDefaultDatabase());
+    }
+
+    return listAllNamespaces(Namespace.empty()).stream()
+        .map(n -> n.level(n.levels().length - 1))
+        .collect(Collectors.toList());
+  }
+
+  private List<Namespace> listAllNamespaces(Namespace namespace) {
+    if (asNamespaceCatalog == null) {
+      throw new RuntimeException("The asNamespaceCatalog should not be null.");
+    }
+
+    String[] levels = namespace.levels();
+    if (levels.length == baseNamespace.length + 1) {
+      return Collections.singletonList(namespace);
+    }
+    if (levels.length < baseNamespace.length + 1) {
+      for (int i = 0; i < levels.length; i++) {
+        if (!baseNamespace[i].equals(levels[i])) {
+          return Collections.emptyList();
+        }
+      }
+      List<Namespace> ret = new ArrayList<>();
+      asNamespaceCatalog.listNamespaces(namespace).forEach(n -> ret.addAll(listAllNamespaces(n)));
+      return ret;
+    }
+    return Collections.emptyList();
+  }
+
+  @Override
+  public CatalogDatabase getDatabase(String databaseName) throws DatabaseNotExistException, CatalogException {
+    if (asNamespaceCatalog == null) {
+      if (!getDefaultDatabase().equals(databaseName)) {
+        throw new DatabaseNotExistException(getName(), databaseName);
+      } else {
+        return new CatalogDatabaseImpl(new HashMap<>(), "");
+      }
+    } else {
+      try {
+        Map<String, String> metadata =
+            new HashMap<>(asNamespaceCatalog.loadNamespaceMetadata(toNamespace(databaseName)));
+        String comment = metadata.remove("comment");
+        return new CatalogDatabaseImpl(metadata, comment);
+      } catch (NoSuchNamespaceException e) {
+        throw new DatabaseNotExistException(getName(), databaseName, e);
+      }
+    }
+  }
+
+  @Override
+  public boolean databaseExists(String databaseName) throws CatalogException {
+    try {
+      getDatabase(databaseName);
+      return true;
+    } catch (DatabaseNotExistException ignore) {
+      return false;
+    }
+  }
+
+  @Override
+  public void createDatabase(String name, CatalogDatabase database, boolean ignoreIfExists)
+      throws DatabaseAlreadyExistException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      try {
+        asNamespaceCatalog.createNamespace(
+            toNamespace(name),
+            mergeComment(database.getProperties(), database.getComment()));
+      } catch (AlreadyExistsException e) {
+        if (!ignoreIfExists) {
+          throw new DatabaseAlreadyExistException(getName(), name, e);
+        }
+      }
+    } else {
+      throw new UnsupportedOperationException("Namespaces are not supported by catalog: " + getName());
+    }
+  }
+
+  private Map<String, String> mergeComment(Map<String, String> metadata, String comment) {
+    Map<String, String> ret = new HashMap<>(metadata);
+    if (metadata.containsKey("comment")) {
+      throw new CatalogException("Database properties should not contain key: 'comment'.");
+    }
+    if (!StringUtils.isNullOrWhitespaceOnly(comment)) {
+      ret.put("comment", comment);
+    }
+    return ret;
+  }
+
+  @Override
+  public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade)
+      throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      try {
+        boolean success = asNamespaceCatalog.dropNamespace(toNamespace(name));
+        if (!success && !ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name);
+        }
+      } catch (NoSuchNamespaceException e) {
+        if (!ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name, e);
+        }
+      } catch (NamespaceNotEmptyException e) {
+        throw new DatabaseNotEmptyException(getName(), name, e);
+      }
+    } else {
+      if (!ignoreIfNotExists) {
+        throw new DatabaseNotExistException(getName(), name);
+      }
+    }
+  }
+
+  @Override
+  public void alterDatabase(String name, CatalogDatabase newDatabase, boolean ignoreIfNotExists)
+      throws DatabaseNotExistException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      Namespace namespace = toNamespace(name);
+      Map<String, String> updates = Maps.newHashMap();
+      Set<String> removals = Sets.newHashSet();
+
+      try {
+        Map<String, String> oldOptions = asNamespaceCatalog.loadNamespaceMetadata(namespace);
+        Map<String, String> newOptions = mergeComment(newDatabase.getProperties(), newDatabase.getComment());
+
+        for (String key : oldOptions.keySet()) {
+          if (!newOptions.containsKey(key)) {
+            removals.add(key);
+          }
+        }
+
+        for (Map.Entry<String, String> entry : newOptions.entrySet()) {
+          if (!entry.getValue().equals(oldOptions.get(entry.getKey()))) {
+            updates.put(entry.getKey(), entry.getValue());
+          }
+        }
+
+        if (!updates.isEmpty()) {
+          asNamespaceCatalog.setProperties(namespace, updates);
+        }
+
+        if (!removals.isEmpty()) {
+          asNamespaceCatalog.removeProperties(namespace, removals);
+        }
+
+      } catch (org.apache.iceberg.exceptions.NoSuchNamespaceException e) {
+        if (!ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name, e);
+        }
+      }
+    } else {
+      if (!ignoreIfNotExists) {
+        throw new DatabaseNotExistException(getName(), name);
+      }
+    }
+  }
+
+  @Override
+  public List<String> listTables(String databaseName) throws DatabaseNotExistException, CatalogException {
+    try {
+      return icebergCatalog.listTables(toNamespace(databaseName)).stream()
+          .map(TableIdentifier::name)
+          .collect(Collectors.toList());
+    } catch (NoSuchNamespaceException e) {
+      throw new DatabaseNotExistException(getName(), databaseName, e);
+    }
+  }
+
+  @Override
+  public CatalogBaseTable getTable(ObjectPath tablePath) throws TableNotExistException, CatalogException {
+    try {
+      Table table = icebergCatalog.loadTable(toIdentifier(tablePath));
+      TableSchema tableSchema = FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(table.schema()));
+
+      // NOTE: We can not create a IcebergCatalogTable, because Flink optimizer may use CatalogTableImpl to copy a new
+      // catalog table.
+      // Let's re-loading table from Iceberg catalog when creating source/sink operators.
+      return new CatalogTableImpl(tableSchema, table.properties(), "");
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      throw new TableNotExistException(getName(), tablePath, e);
+    }
+  }
+
+  @Override
+  public boolean tableExists(ObjectPath tablePath) throws CatalogException {
+    return icebergCatalog.tableExists(toIdentifier(tablePath));
+  }
+
+  @Override
+  public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists)
+      throws TableNotExistException, CatalogException {
+    try {
+      icebergCatalog.dropTable(toIdentifier(tablePath));
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      throw new TableNotExistException(getName(), tablePath, e);
+    }
+  }
+
+  @Override
+  public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists)
+      throws TableNotExistException, TableAlreadyExistException, CatalogException {
+    try {
+      icebergCatalog.renameTable(
+          toIdentifier(tablePath),
+          toIdentifier(new ObjectPath(tablePath.getDatabaseName(), newTableName)));
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      throw new TableNotExistException(getName(), tablePath, e);
+    } catch (AlreadyExistsException e) {
+      throw new TableAlreadyExistException(getName(), tablePath, e);
+    }
+  }
+
+  /**
+   * TODO Implement DDL-string parser for PartitionSpec.

Review comment:
       Good points.
   
   For Flink SQL, computed columns are virtual columns, the source and sink can just ignore them, the source just produces columns without computed columns, the Flink core will generate computed columns for input records. For sink, Flink core just give the records without computed columns to connector sink.
   
   - I see, you mean https://iceberg.apache.org/evolution/#partition-evolution , the computed columns should be calculated by Flink core, iceberg should just deal with its physical logical.
   - There are three types of function: 1.`hour,day,month,year` are the same as Flink's functions. 2. For `truncate`, Flink also supports this function, but not support `truncate` with input type string and bytes, iceberg can provides catalog function (`Catalog.getFunction`), users can use `iceberg_catalog.truncate` to create computed column. 3. For bucket, Flink not support this function, so iceberg can provides catalog functions, users can directly use it.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi edited a comment on pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
JingsongLi edited a comment on pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#issuecomment-657931071


   > @JingsongLi, I think this needs to be rebased now that #1180 is in.
   > 
   > Also, should we get #1174 updated for the comments here so we can merge them separately? If we can, I'd prefer to make the commits smaller.
   
   Yes, I think we can, I'll update #1174 , create PR for HadoopCatalog bug, create PR for Flink 1.11..


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453793475



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java
##########
@@ -21,10 +21,19 @@
 
 import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.types.FieldsDataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.utils.TypeConversions;
 import org.apache.iceberg.Schema;
 import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
 import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.TypeUtil;
 
+/**
+ * Converter between Flink types and Iceberg type.
+ * The conversion is not a 1:1 mapping that not allows back-and-forth conversion. So some information might get lost
+ * during the back-and-forth conversion.

Review comment:
       If I understand correctly, this is lossy because Iceberg doesn't represent some types that Flink supports, like `CHAR(N)`. Is that right?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r454054081



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java
##########
@@ -0,0 +1,508 @@
+/*
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.AbstractCatalog;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogDatabase;
+import org.apache.flink.table.catalog.CatalogDatabaseImpl;
+import org.apache.flink.table.catalog.CatalogFunction;
+import org.apache.flink.table.catalog.CatalogPartition;
+import org.apache.flink.table.catalog.CatalogPartitionSpec;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+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.DatabaseNotEmptyException;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
+import org.apache.flink.table.catalog.exceptions.FunctionNotExistException;
+import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.catalog.stats.CatalogColumnStatistics;
+import org.apache.flink.table.catalog.stats.CatalogTableStatistics;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.util.StringUtils;
+import org.apache.iceberg.CachingCatalog;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+
+/**
+ * A Flink Catalog implementation that wraps an Iceberg {@link Catalog}.
+ * <p>
+ * The mapping between Flink database and Iceberg namespace:
+ * Supplying a base namespace for a given catalog, so if you have a catalog that supports a 2-level namespace, you
+ * would supply the first level in the catalog configuration and the second level would be exposed as Flink databases.
+ * <p>
+ * The Iceberg table manages its partitions by itself. The partition of the Iceberg table is independent of the
+ * partition of Flink.
+ */
+public class FlinkCatalog extends AbstractCatalog {
+
+  private final Catalog originalCatalog;
+  private final Catalog icebergCatalog;
+  private final String[] baseNamespace;
+  private final SupportsNamespaces asNamespaceCatalog;
+
+  public FlinkCatalog(
+      String catalogName,
+      String defaultDatabase,
+      String[] baseNamespace,
+      Catalog icebergCatalog,
+      boolean cacheEnabled) {
+    super(catalogName, defaultDatabase);
+    this.originalCatalog = icebergCatalog;
+    this.icebergCatalog = cacheEnabled ? CachingCatalog.wrap(icebergCatalog) : icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    if (icebergCatalog instanceof SupportsNamespaces) {
+      asNamespaceCatalog = (SupportsNamespaces) icebergCatalog;
+    } else {
+      asNamespaceCatalog = null;
+    }
+  }
+
+  @Override
+  public void open() throws CatalogException {
+  }
+
+  @Override
+  public void close() throws CatalogException {
+    if (originalCatalog instanceof Closeable) {
+      try {
+        ((Closeable) originalCatalog).close();
+      } catch (IOException e) {
+        throw new CatalogException(e);
+      }
+    }
+  }
+
+  private Namespace toNamespace(String database) {
+    String[] namespace = new String[baseNamespace.length + 1];
+    System.arraycopy(baseNamespace, 0, namespace, 0, baseNamespace.length);
+    namespace[baseNamespace.length] = database;
+    return Namespace.of(namespace);
+  }
+
+  private TableIdentifier toIdentifier(ObjectPath path) {
+    return TableIdentifier.of(toNamespace(path.getDatabaseName()), path.getObjectName());
+  }
+
+  @Override
+  public List<String> listDatabases() throws CatalogException {
+    if (asNamespaceCatalog == null) {
+      return Collections.singletonList(getDefaultDatabase());
+    }
+
+    return listAllNamespaces(Namespace.empty()).stream()
+        .map(n -> n.level(n.levels().length - 1))
+        .collect(Collectors.toList());
+  }
+
+  private List<Namespace> listAllNamespaces(Namespace namespace) {
+    if (asNamespaceCatalog == null) {
+      throw new RuntimeException("The asNamespaceCatalog should not be null.");
+    }
+
+    String[] levels = namespace.levels();
+    if (levels.length == baseNamespace.length + 1) {
+      return Collections.singletonList(namespace);
+    }
+    if (levels.length < baseNamespace.length + 1) {
+      for (int i = 0; i < levels.length; i++) {
+        if (!baseNamespace[i].equals(levels[i])) {
+          return Collections.emptyList();
+        }
+      }
+      List<Namespace> ret = new ArrayList<>();
+      asNamespaceCatalog.listNamespaces(namespace).forEach(n -> ret.addAll(listAllNamespaces(n)));
+      return ret;
+    }
+    return Collections.emptyList();
+  }
+
+  @Override
+  public CatalogDatabase getDatabase(String databaseName) throws DatabaseNotExistException, CatalogException {
+    if (asNamespaceCatalog == null) {
+      if (!getDefaultDatabase().equals(databaseName)) {
+        throw new DatabaseNotExistException(getName(), databaseName);
+      } else {
+        return new CatalogDatabaseImpl(new HashMap<>(), "");
+      }
+    } else {
+      try {
+        Map<String, String> metadata =
+            new HashMap<>(asNamespaceCatalog.loadNamespaceMetadata(toNamespace(databaseName)));
+        String comment = metadata.remove("comment");
+        return new CatalogDatabaseImpl(metadata, comment);
+      } catch (NoSuchNamespaceException e) {
+        throw new DatabaseNotExistException(getName(), databaseName, e);
+      }
+    }
+  }
+
+  @Override
+  public boolean databaseExists(String databaseName) throws CatalogException {
+    try {
+      getDatabase(databaseName);
+      return true;
+    } catch (DatabaseNotExistException ignore) {
+      return false;
+    }
+  }
+
+  @Override
+  public void createDatabase(String name, CatalogDatabase database, boolean ignoreIfExists)
+      throws DatabaseAlreadyExistException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      try {
+        asNamespaceCatalog.createNamespace(
+            toNamespace(name),
+            mergeComment(database.getProperties(), database.getComment()));
+      } catch (AlreadyExistsException e) {
+        if (!ignoreIfExists) {
+          throw new DatabaseAlreadyExistException(getName(), name, e);
+        }
+      }
+    } else {
+      throw new UnsupportedOperationException("Namespaces are not supported by catalog: " + getName());
+    }
+  }
+
+  private Map<String, String> mergeComment(Map<String, String> metadata, String comment) {
+    Map<String, String> ret = new HashMap<>(metadata);
+    if (metadata.containsKey("comment")) {
+      throw new CatalogException("Database properties should not contain key: 'comment'.");
+    }
+    if (!StringUtils.isNullOrWhitespaceOnly(comment)) {
+      ret.put("comment", comment);
+    }
+    return ret;
+  }
+
+  @Override
+  public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade)
+      throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      try {
+        boolean success = asNamespaceCatalog.dropNamespace(toNamespace(name));
+        if (!success && !ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name);
+        }
+      } catch (NoSuchNamespaceException e) {
+        if (!ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name, e);
+        }
+      } catch (NamespaceNotEmptyException e) {
+        throw new DatabaseNotEmptyException(getName(), name, e);
+      }
+    } else {
+      if (!ignoreIfNotExists) {
+        throw new DatabaseNotExistException(getName(), name);
+      }
+    }
+  }
+
+  @Override
+  public void alterDatabase(String name, CatalogDatabase newDatabase, boolean ignoreIfNotExists)
+      throws DatabaseNotExistException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      Namespace namespace = toNamespace(name);
+      Map<String, String> updates = Maps.newHashMap();
+      Set<String> removals = Sets.newHashSet();
+
+      try {
+        Map<String, String> oldOptions = asNamespaceCatalog.loadNamespaceMetadata(namespace);
+        Map<String, String> newOptions = mergeComment(newDatabase.getProperties(), newDatabase.getComment());
+
+        for (String key : oldOptions.keySet()) {
+          if (!newOptions.containsKey(key)) {
+            removals.add(key);
+          }
+        }
+
+        for (Map.Entry<String, String> entry : newOptions.entrySet()) {
+          if (!entry.getValue().equals(oldOptions.get(entry.getKey()))) {
+            updates.put(entry.getKey(), entry.getValue());
+          }
+        }
+
+        if (!updates.isEmpty()) {
+          asNamespaceCatalog.setProperties(namespace, updates);
+        }
+
+        if (!removals.isEmpty()) {
+          asNamespaceCatalog.removeProperties(namespace, removals);
+        }
+
+      } catch (org.apache.iceberg.exceptions.NoSuchNamespaceException e) {
+        if (!ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name, e);
+        }
+      }
+    } else {
+      if (!ignoreIfNotExists) {
+        throw new DatabaseNotExistException(getName(), name);
+      }
+    }
+  }
+
+  @Override
+  public List<String> listTables(String databaseName) throws DatabaseNotExistException, CatalogException {
+    try {
+      return icebergCatalog.listTables(toNamespace(databaseName)).stream()
+          .map(TableIdentifier::name)
+          .collect(Collectors.toList());
+    } catch (NoSuchNamespaceException e) {
+      throw new DatabaseNotExistException(getName(), databaseName, e);
+    }
+  }
+
+  @Override
+  public CatalogBaseTable getTable(ObjectPath tablePath) throws TableNotExistException, CatalogException {
+    try {
+      Table table = icebergCatalog.loadTable(toIdentifier(tablePath));
+      TableSchema tableSchema = FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(table.schema()));
+
+      // NOTE: We can not create a IcebergCatalogTable, because Flink optimizer may use CatalogTableImpl to copy a new
+      // catalog table.
+      // Let's re-loading table from Iceberg catalog when creating source/sink operators.
+      return new CatalogTableImpl(tableSchema, table.properties(), "");
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      throw new TableNotExistException(getName(), tablePath, e);
+    }
+  }
+
+  @Override
+  public boolean tableExists(ObjectPath tablePath) throws CatalogException {
+    return icebergCatalog.tableExists(toIdentifier(tablePath));
+  }
+
+  @Override
+  public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists)
+      throws TableNotExistException, CatalogException {
+    try {
+      icebergCatalog.dropTable(toIdentifier(tablePath));
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      throw new TableNotExistException(getName(), tablePath, e);
+    }
+  }
+
+  @Override
+  public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists)
+      throws TableNotExistException, TableAlreadyExistException, CatalogException {
+    try {
+      icebergCatalog.renameTable(
+          toIdentifier(tablePath),
+          toIdentifier(new ObjectPath(tablePath.getDatabaseName(), newTableName)));
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      throw new TableNotExistException(getName(), tablePath, e);
+    } catch (AlreadyExistsException e) {
+      throw new TableAlreadyExistException(getName(), tablePath, e);
+    }
+  }
+
+  /**
+   * TODO Implement DDL-string parser for PartitionSpec.

Review comment:
       A rough idea, Flink support computed column: https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/create.html#create-table
   
   Flink DDL `CREATE TABLE T (pk INT, ... dt STRING, year AS YEAR(dt), month AS MONTH(dt), d AS DAY(dt)) PARTITIONED BY(year, month, d)` should be same to Spark DDL `CREATE TABLE T (pk INT, ... dt STRING) PARTITIONED BY(YEAR(dt), MONTH(dt), DAY(dt))`.
   
   The computed columns are not stored in the real data, they are just virtual columns, which means we can map they to iceberg partition transforms of iceberg table in iceberg Flink Catalog.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453110983



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java
##########
@@ -21,10 +21,19 @@
 
 import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.types.FieldsDataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.utils.TypeConversions;
 import org.apache.iceberg.Schema;
 import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
 import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.TypeUtil;
 
+/**
+ * Converter between Flink types and Iceberg type.
+ * The conversion is not a 1:1 mapping that not allows back-and-forth conversion. So some information might get lost
+ * during the back-and-forth conversion.

Review comment:
       Can you be more specific about this? What is a case where information is lost?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453428163



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java
##########
@@ -0,0 +1,241 @@
+/*
+ * 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.File;
+import java.util.Map;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.types.Types;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Test;
+
+public class TestFlinkCatalogDatabase extends FlinkCatalogTestBase {
+
+  public TestFlinkCatalogDatabase(String catalogName, String[] baseNamepace) {
+    super(catalogName, baseNamepace);
+  }
+
+  @After
+  public void clean() {
+    sql("DROP TABLE IF EXISTS %s.tl", flinkIdentifier);
+    sql("DROP DATABASE IF EXISTS %s", flinkIdentifier);
+  }
+
+  @Test
+  public void testCreateNamespace() {
+    Assert.assertFalse(
+        "Database should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+
+    Assert.assertTrue("Database should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+  }
+
+  @Test
+  public void testDefaultDatabase() {
+    sql("USE CATALOG %s", catalogName);
+
+    Assert.assertEquals("Should use the current catalog", tEnv.getCurrentCatalog(), catalogName);
+    Assert.assertEquals("Should use the configured default namespace", tEnv.getCurrentDatabase(), "default");
+  }
+
+  @Test
+  public void testDropEmptyDatabase() {
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("DROP DATABASE %s", flinkIdentifier);
+
+    Assert.assertFalse(
+        "Namespace should have been dropped",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+  }
+
+  @Test
+  public void testDropNonEmptyNamespace() {
+    Assume.assumeFalse("Hadoop catalog throws IOException: Directory is not empty.", isHadoopCatalog);

Review comment:
       I'll modify in this PR. Tell me if I need create a new PR for fixing hadoop catalog.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453794419



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java
##########
@@ -0,0 +1,241 @@
+/*
+ * 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.File;
+import java.util.Map;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.types.Types;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Test;
+
+public class TestFlinkCatalogDatabase extends FlinkCatalogTestBase {
+
+  public TestFlinkCatalogDatabase(String catalogName, String[] baseNamepace) {
+    super(catalogName, baseNamepace);
+  }
+
+  @After
+  public void clean() {
+    sql("DROP TABLE IF EXISTS %s.tl", flinkIdentifier);
+    sql("DROP DATABASE IF EXISTS %s", flinkIdentifier);
+  }
+
+  @Test
+  public void testCreateNamespace() {
+    Assert.assertFalse(
+        "Database should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+
+    Assert.assertTrue("Database should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+  }
+
+  @Test
+  public void testDefaultDatabase() {
+    sql("USE CATALOG %s", catalogName);
+
+    Assert.assertEquals("Should use the current catalog", tEnv.getCurrentCatalog(), catalogName);
+    Assert.assertEquals("Should use the configured default namespace", tEnv.getCurrentDatabase(), "default");
+  }
+
+  @Test
+  public void testDropEmptyDatabase() {
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("DROP DATABASE %s", flinkIdentifier);
+
+    Assert.assertFalse(
+        "Namespace should have been dropped",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+  }
+
+  @Test
+  public void testDropNonEmptyNamespace() {
+    Assume.assumeFalse("Hadoop catalog throws IOException: Directory is not empty.", isHadoopCatalog);

Review comment:
       I think it would be better to fix the Hadoop catalog in a separate PR and leave this one with the Assume until it is merged.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453429022



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java
##########
@@ -0,0 +1,241 @@
+/*
+ * 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.File;
+import java.util.Map;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.types.Types;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Test;
+
+public class TestFlinkCatalogDatabase extends FlinkCatalogTestBase {
+
+  public TestFlinkCatalogDatabase(String catalogName, String[] baseNamepace) {
+    super(catalogName, baseNamepace);
+  }
+
+  @After
+  public void clean() {
+    sql("DROP TABLE IF EXISTS %s.tl", flinkIdentifier);
+    sql("DROP DATABASE IF EXISTS %s", flinkIdentifier);
+  }
+
+  @Test
+  public void testCreateNamespace() {
+    Assert.assertFalse(
+        "Database should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+
+    Assert.assertTrue("Database should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+  }
+
+  @Test
+  public void testDefaultDatabase() {
+    sql("USE CATALOG %s", catalogName);
+
+    Assert.assertEquals("Should use the current catalog", tEnv.getCurrentCatalog(), catalogName);
+    Assert.assertEquals("Should use the configured default namespace", tEnv.getCurrentDatabase(), "default");
+  }
+
+  @Test
+  public void testDropEmptyDatabase() {
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("DROP DATABASE %s", flinkIdentifier);
+
+    Assert.assertFalse(
+        "Namespace should have been dropped",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+  }
+
+  @Test
+  public void testDropNonEmptyNamespace() {
+    Assume.assumeFalse("Hadoop catalog throws IOException: Directory is not empty.", isHadoopCatalog);
+
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+
+    validationCatalog.createTable(
+        TableIdentifier.of(icebergNamespace, "tl"),
+        new Schema(Types.NestedField.optional(0, "id", Types.LongType.get())));
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+    Assert.assertTrue("Table should exist", validationCatalog.tableExists(TableIdentifier.of(icebergNamespace, "tl")));
+
+    AssertHelpers.assertThrowsCause(

Review comment:
       Yes, in Flink, will wrap an exception like: `throw new ValidationException("Could not execute DROP DATABASE", e);`, in the top exception, there is no the message that we want to check.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453395113



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java
##########
@@ -0,0 +1,508 @@
+/*
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.AbstractCatalog;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogDatabase;
+import org.apache.flink.table.catalog.CatalogDatabaseImpl;
+import org.apache.flink.table.catalog.CatalogFunction;
+import org.apache.flink.table.catalog.CatalogPartition;
+import org.apache.flink.table.catalog.CatalogPartitionSpec;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+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.DatabaseNotEmptyException;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
+import org.apache.flink.table.catalog.exceptions.FunctionNotExistException;
+import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.catalog.stats.CatalogColumnStatistics;
+import org.apache.flink.table.catalog.stats.CatalogTableStatistics;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.util.StringUtils;
+import org.apache.iceberg.CachingCatalog;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+
+/**
+ * A Flink Catalog implementation that wraps an Iceberg {@link Catalog}.
+ * <p>
+ * The mapping between Flink database and Iceberg namespace:
+ * Supplying a base namespace for a given catalog, so if you have a catalog that supports a 2-level namespace, you
+ * would supply the first level in the catalog configuration and the second level would be exposed as Flink databases.
+ * <p>
+ * The Iceberg table manages its partitions by itself. The partition of the Iceberg table is independent of the
+ * partition of Flink.
+ */
+public class FlinkCatalog extends AbstractCatalog {
+
+  private final Catalog originalCatalog;
+  private final Catalog icebergCatalog;
+  private final String[] baseNamespace;
+  private final SupportsNamespaces asNamespaceCatalog;
+
+  public FlinkCatalog(
+      String catalogName,
+      String defaultDatabase,
+      String[] baseNamespace,
+      Catalog icebergCatalog,
+      boolean cacheEnabled) {
+    super(catalogName, defaultDatabase);
+    this.originalCatalog = icebergCatalog;
+    this.icebergCatalog = cacheEnabled ? CachingCatalog.wrap(icebergCatalog) : icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    if (icebergCatalog instanceof SupportsNamespaces) {
+      asNamespaceCatalog = (SupportsNamespaces) icebergCatalog;
+    } else {
+      asNamespaceCatalog = null;
+    }
+  }
+
+  @Override
+  public void open() throws CatalogException {
+  }
+
+  @Override
+  public void close() throws CatalogException {
+    if (originalCatalog instanceof Closeable) {
+      try {
+        ((Closeable) originalCatalog).close();
+      } catch (IOException e) {
+        throw new CatalogException(e);
+      }
+    }
+  }
+
+  private Namespace toNamespace(String database) {
+    String[] namespace = new String[baseNamespace.length + 1];
+    System.arraycopy(baseNamespace, 0, namespace, 0, baseNamespace.length);
+    namespace[baseNamespace.length] = database;
+    return Namespace.of(namespace);
+  }
+
+  private TableIdentifier toIdentifier(ObjectPath path) {
+    return TableIdentifier.of(toNamespace(path.getDatabaseName()), path.getObjectName());
+  }
+
+  @Override
+  public List<String> listDatabases() throws CatalogException {
+    if (asNamespaceCatalog == null) {
+      return Collections.singletonList(getDefaultDatabase());
+    }
+
+    return listAllNamespaces(Namespace.empty()).stream()
+        .map(n -> n.level(n.levels().length - 1))
+        .collect(Collectors.toList());
+  }
+
+  private List<Namespace> listAllNamespaces(Namespace namespace) {
+    if (asNamespaceCatalog == null) {
+      throw new RuntimeException("The asNamespaceCatalog should not be null.");
+    }
+
+    String[] levels = namespace.levels();
+    if (levels.length == baseNamespace.length + 1) {
+      return Collections.singletonList(namespace);
+    }
+    if (levels.length < baseNamespace.length + 1) {
+      for (int i = 0; i < levels.length; i++) {
+        if (!baseNamespace[i].equals(levels[i])) {
+          return Collections.emptyList();
+        }
+      }
+      List<Namespace> ret = new ArrayList<>();
+      asNamespaceCatalog.listNamespaces(namespace).forEach(n -> ret.addAll(listAllNamespaces(n)));

Review comment:
       You are right, `asNamespaceCatalog.listNamespaces(Namespace.of(baseNamespace))` is enough.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453793940



##########
File path: flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.List;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.BinaryType;
+import org.apache.flink.table.types.logical.BooleanType;
+import org.apache.flink.table.types.logical.CharType;
+import org.apache.flink.table.types.logical.DateType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.DoubleType;
+import org.apache.flink.table.types.logical.FloatType;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.LocalZonedTimestampType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.TimeType;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.VarBinaryType;
+import org.apache.flink.table.types.logical.VarCharType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types;
+
+class TypeToFlinkType extends TypeUtil.SchemaVisitor<LogicalType> {
+  TypeToFlinkType() {
+  }
+
+  @Override
+  public LogicalType schema(Schema schema, LogicalType structType) {
+    return structType;
+  }
+
+  @Override
+  public LogicalType struct(Types.StructType struct, List<LogicalType> fieldResults) {
+    List<Types.NestedField> fields = struct.fields();
+
+    List<RowType.RowField> flinkFields = Lists.newArrayListWithExpectedSize(fieldResults.size());
+    for (int i = 0; i < fields.size(); i += 1) {
+      Types.NestedField field = fields.get(i);
+      LogicalType type = fieldResults.get(i);
+      RowType.RowField flinkField = new RowType.RowField(
+          field.name(), type.copy(field.isOptional()), field.doc());
+      flinkFields.add(flinkField);
+    }
+
+    return new RowType(flinkFields);
+  }
+
+  @Override
+  public LogicalType field(Types.NestedField field, LogicalType fieldResult) {
+    return fieldResult;
+  }
+
+  @Override
+  public LogicalType list(Types.ListType list, LogicalType elementResult) {
+    return new ArrayType(elementResult.copy(list.isElementOptional()));
+  }
+
+  @Override
+  public LogicalType map(Types.MapType map, LogicalType keyResult, LogicalType valueResult) {
+    // keys in map are not allowed to be null.
+    return new MapType(keyResult.copy(false), valueResult.copy(map.isValueOptional()));
+  }
+
+  @Override
+  public LogicalType primitive(Type.PrimitiveType primitive) {
+    switch (primitive.typeId()) {
+      case BOOLEAN:
+        return new BooleanType();
+      case INTEGER:
+        return new IntType();
+      case LONG:
+        return new BigIntType();
+      case FLOAT:
+        return new FloatType();
+      case DOUBLE:
+        return new DoubleType();
+      case DATE:
+        return new DateType();
+      case TIME:
+        // MICROS
+        return new TimeType(6);
+      case TIMESTAMP:
+        Types.TimestampType timestamp = (Types.TimestampType) primitive;
+        if (timestamp.shouldAdjustToUTC()) {
+          // MICROS
+          return new LocalZonedTimestampType(6);
+        } else {
+          // MICROS
+          return new TimestampType(6);
+        }
+      case STRING:
+        return new VarCharType(VarCharType.MAX_LENGTH);
+      case UUID:
+        // UUID length is 16
+        return new CharType(16);

Review comment:
       I think either `CHAR(36)` or `VARBINARY(16)` would work, but not `CHAR(16)`.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453113502



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java
##########
@@ -0,0 +1,241 @@
+/*
+ * 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.File;
+import java.util.Map;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.types.Types;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Test;
+
+public class TestFlinkCatalogDatabase extends FlinkCatalogTestBase {
+
+  public TestFlinkCatalogDatabase(String catalogName, String[] baseNamepace) {
+    super(catalogName, baseNamepace);
+  }
+
+  @After
+  public void clean() {
+    sql("DROP TABLE IF EXISTS %s.tl", flinkIdentifier);
+    sql("DROP DATABASE IF EXISTS %s", flinkIdentifier);
+  }
+
+  @Test
+  public void testCreateNamespace() {
+    Assert.assertFalse(
+        "Database should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+
+    Assert.assertTrue("Database should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+  }
+
+  @Test
+  public void testDefaultDatabase() {
+    sql("USE CATALOG %s", catalogName);
+
+    Assert.assertEquals("Should use the current catalog", tEnv.getCurrentCatalog(), catalogName);
+    Assert.assertEquals("Should use the configured default namespace", tEnv.getCurrentDatabase(), "default");
+  }
+
+  @Test
+  public void testDropEmptyDatabase() {
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("DROP DATABASE %s", flinkIdentifier);
+
+    Assert.assertFalse(
+        "Namespace should have been dropped",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+  }
+
+  @Test
+  public void testDropNonEmptyNamespace() {
+    Assume.assumeFalse("Hadoop catalog throws IOException: Directory is not empty.", isHadoopCatalog);
+
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+
+    validationCatalog.createTable(
+        TableIdentifier.of(icebergNamespace, "tl"),
+        new Schema(Types.NestedField.optional(0, "id", Types.LongType.get())));
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+    Assert.assertTrue("Table should exist", validationCatalog.tableExists(TableIdentifier.of(icebergNamespace, "tl")));
+
+    AssertHelpers.assertThrowsCause(
+        "Should fail if trying to delete a non-empty database",
+        DatabaseNotEmptyException.class,
+        String.format("Database %s in catalog %s is not empty.", DATABASE, catalogName),
+        () -> sql("DROP DATABASE %s", flinkIdentifier));
+
+    sql("DROP TABLE %s.tl", flinkIdentifier);
+  }
+
+  @Test
+  public void testListTables() {
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+    sql("USE CATALOG %s", catalogName);
+    sql("USE %s", DATABASE);
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    Assert.assertEquals("Should not list any tables", 0, tEnv.listTables().length);

Review comment:
       Should this call `SHOW TABLES`?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453417946



##########
File path: flink/src/test/java/org/apache/iceberg/flink/FlinkCatalogTestBase.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.util.ArrayUtils;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public abstract class FlinkCatalogTestBase extends FlinkTestBase {
+
+  protected static final String DATABASE = "db";
+  private static File warehouse = null;
+
+  @BeforeClass
+  public static void createWarehouse() throws IOException {
+    FlinkCatalogTestBase.warehouse = File.createTempFile("warehouse", null);
+    Assert.assertTrue(warehouse.delete());
+  }
+
+  @AfterClass
+  public static void dropWarehouse() {
+    if (warehouse != null && warehouse.exists()) {
+      warehouse.delete();
+    }
+  }
+
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { "testhive", new String[0] },
+        new Object[] { "testhadoop", new String[0] },
+        new Object[] { "testhadoop", new String[] { "l0", "l1" }},
+    };
+  }
+
+  protected final TableEnvironment tEnv =
+      TableEnvironment.create(EnvironmentSettings.newInstance().useBlinkPlanner().inBatchMode().build());
+
+  protected final String catalogName;
+  protected final String[] baseNamespace;
+  protected final Catalog validationCatalog;
+  protected final SupportsNamespaces validationNamespaceCatalog;
+  protected final org.apache.flink.table.catalog.Catalog flinkCatalog;
+
+  protected final String flinkIdentifier;
+  protected final Namespace icebergNamespace;
+  protected final boolean isHadoopCatalog;
+
+  public FlinkCatalogTestBase(String catalogName, String[] baseNamespace) {
+    this.catalogName = catalogName;
+    this.baseNamespace = baseNamespace;
+    this.isHadoopCatalog = catalogName.equals("testhadoop");
+    this.validationCatalog = isHadoopCatalog ?
+        new HadoopCatalog(hiveConf, "file:" + warehouse) :
+        catalog;
+    this.validationNamespaceCatalog = (SupportsNamespaces) validationCatalog;
+
+    Map<String, String> config = new HashMap<>();
+    config.put("type", "iceberg");
+    config.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, isHadoopCatalog ? "hadoop" : "hive");
+    config.put(FlinkCatalogFactory.HADOOP_WAREHOUSE_LOCATION, "file:" + warehouse);
+    if (baseNamespace.length > 0) {
+      config.put(FlinkCatalogFactory.BASE_NAMESPACE, Joiner.on(".").join(baseNamespace));
+    }
+
+    FlinkCatalogFactory factory = new FlinkCatalogFactory() {
+      @Override
+      protected Catalog buildIcebergCatalog(String name, Map<String, String> options) {
+        // Flink hadoop configuration depends on system env, it is quiet hard to set from testing. So directly pass
+        // correct hadoop configuration.
+        return super.buildIcebergCatalog(name, options, hiveConf);
+      }
+    };
+    flinkCatalog = factory.createCatalog(catalogName, config);
+    tEnv.registerCatalog(catalogName, flinkCatalog);
+
+    this.flinkIdentifier = catalogName + "." + DATABASE;
+    this.icebergNamespace = Namespace.of(ArrayUtils.concat(baseNamespace, new String[] { DATABASE }));
+  }
+
+  @After

Review comment:
       Yes, every test method will create a new catalog too.
   But it seems we can reuse them by catalog name.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r454051515



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java
##########
@@ -0,0 +1,241 @@
+/*
+ * 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.File;
+import java.util.Map;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.types.Types;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Test;
+
+public class TestFlinkCatalogDatabase extends FlinkCatalogTestBase {
+
+  public TestFlinkCatalogDatabase(String catalogName, String[] baseNamepace) {
+    super(catalogName, baseNamepace);
+  }
+
+  @After
+  public void clean() {
+    sql("DROP TABLE IF EXISTS %s.tl", flinkIdentifier);
+    sql("DROP DATABASE IF EXISTS %s", flinkIdentifier);
+  }
+
+  @Test
+  public void testCreateNamespace() {
+    Assert.assertFalse(
+        "Database should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+
+    Assert.assertTrue("Database should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+  }
+
+  @Test
+  public void testDefaultDatabase() {
+    sql("USE CATALOG %s", catalogName);
+
+    Assert.assertEquals("Should use the current catalog", tEnv.getCurrentCatalog(), catalogName);
+    Assert.assertEquals("Should use the configured default namespace", tEnv.getCurrentDatabase(), "default");
+  }
+
+  @Test
+  public void testDropEmptyDatabase() {
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("DROP DATABASE %s", flinkIdentifier);
+
+    Assert.assertFalse(
+        "Namespace should have been dropped",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+  }
+
+  @Test
+  public void testDropNonEmptyNamespace() {
+    Assume.assumeFalse("Hadoop catalog throws IOException: Directory is not empty.", isHadoopCatalog);

Review comment:
       I'll create it.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue merged pull request #1182: Flink: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
rdblue merged pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182


   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453114529



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java
##########
@@ -0,0 +1,241 @@
+/*
+ * 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.File;
+import java.util.Map;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.types.Types;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Test;
+
+public class TestFlinkCatalogDatabase extends FlinkCatalogTestBase {
+
+  public TestFlinkCatalogDatabase(String catalogName, String[] baseNamepace) {
+    super(catalogName, baseNamepace);
+  }
+
+  @After
+  public void clean() {
+    sql("DROP TABLE IF EXISTS %s.tl", flinkIdentifier);
+    sql("DROP DATABASE IF EXISTS %s", flinkIdentifier);
+  }
+
+  @Test
+  public void testCreateNamespace() {
+    Assert.assertFalse(
+        "Database should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+
+    Assert.assertTrue("Database should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+  }
+
+  @Test
+  public void testDefaultDatabase() {
+    sql("USE CATALOG %s", catalogName);
+
+    Assert.assertEquals("Should use the current catalog", tEnv.getCurrentCatalog(), catalogName);
+    Assert.assertEquals("Should use the configured default namespace", tEnv.getCurrentDatabase(), "default");
+  }
+
+  @Test
+  public void testDropEmptyDatabase() {
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("DROP DATABASE %s", flinkIdentifier);
+
+    Assert.assertFalse(
+        "Namespace should have been dropped",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+  }
+
+  @Test
+  public void testDropNonEmptyNamespace() {
+    Assume.assumeFalse("Hadoop catalog throws IOException: Directory is not empty.", isHadoopCatalog);
+
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+
+    validationCatalog.createTable(
+        TableIdentifier.of(icebergNamespace, "tl"),
+        new Schema(Types.NestedField.optional(0, "id", Types.LongType.get())));
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+    Assert.assertTrue("Table should exist", validationCatalog.tableExists(TableIdentifier.of(icebergNamespace, "tl")));
+
+    AssertHelpers.assertThrowsCause(
+        "Should fail if trying to delete a non-empty database",
+        DatabaseNotEmptyException.class,
+        String.format("Database %s in catalog %s is not empty.", DATABASE, catalogName),
+        () -> sql("DROP DATABASE %s", flinkIdentifier));
+
+    sql("DROP TABLE %s.tl", flinkIdentifier);
+  }
+
+  @Test
+  public void testListTables() {
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+    sql("USE CATALOG %s", catalogName);
+    sql("USE %s", DATABASE);
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    Assert.assertEquals("Should not list any tables", 0, tEnv.listTables().length);
+
+    validationCatalog.createTable(
+        TableIdentifier.of(icebergNamespace, "tl"),
+        new Schema(Types.NestedField.optional(0, "id", Types.LongType.get())));
+
+    Assert.assertEquals("Only 1 table", 1, tEnv.listTables().length);
+    Assert.assertEquals("Table name should match", "tl", tEnv.listTables()[0]);
+  }
+
+  @Test
+  public void testListNamespace() {
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+    sql("USE CATALOG %s", catalogName);
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    String[] databases = tEnv.listDatabases();
+
+    if (isHadoopCatalog) {
+      Assert.assertEquals("Should have 1 database", 1, databases.length);
+      Assert.assertEquals("Should have only db database", "db", databases[0]);
+
+      if (baseNamespace.length > 0) {
+        // test namespace not belongs to this catalog
+        validationNamespaceCatalog.createNamespace(Namespace.of(baseNamespace[0], "UNKNOWN_NAMESPACE"));
+        databases = tEnv.listDatabases();
+        Assert.assertEquals("Should have 1 database", 1, databases.length);
+        Assert.assertEquals("Should have only db database", "db", databases[0]);
+      }
+    } else {
+      Assert.assertEquals("Should have 2 databases", 2, databases.length);
+      Assert.assertEquals(
+          "Should have default and db databases",
+          ImmutableSet.of("default", "db"),
+          ImmutableSet.copyOf(databases));
+    }
+  }
+
+  @Test
+  public void testCreateNamespaceWithMetadata() {
+    Assume.assumeFalse("HadoopCatalog does not support namespace metadata", isHadoopCatalog);
+
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s WITH ('prop'='value')", flinkIdentifier);
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    Map<String, String> nsMetadata = validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace);
+
+    Assert.assertEquals("Namespace should have expected prop value", "value", nsMetadata.get("prop"));
+  }
+
+  @Test
+  public void testCreateNamespaceWithComment() {
+    Assume.assumeFalse("HadoopCatalog does not support namespace metadata", isHadoopCatalog);
+
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s COMMENT 'namespace doc'", flinkIdentifier);
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    Map<String, String> nsMetadata = validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace);
+
+    Assert.assertEquals("Namespace should have expected comment", "namespace doc", nsMetadata.get("comment"));
+  }
+
+  @Test
+  public void testCreateNamespaceWithLocation() throws Exception {
+    Assume.assumeFalse("HadoopCatalog does not support namespace locations", isHadoopCatalog);
+
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    File location = TEMPORARY_FOLDER.newFile();

Review comment:
       What is `TEMPORARY_FOLDER`? I don't see it elsewhere in this PR.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#issuecomment-657931071


   > @JingsongLi, I think this needs to be rebased now that #1180 is in.
   > 
   > Also, should we get #1174 updated for the comments here so we can merge them separately? If we can, I'd prefer to make the commits smaller.
   
   Yes, I think we can, I'll update #1174 , create PR for HadoopCatalog bug, create PR for Flink 1.11. maybe create PR for `TestHiveMetastore` bug.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#issuecomment-656933167


   Thanks @JingsongLi, this looks close. I just had a few questions.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1182: Flink: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r457712322



##########
File path: flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.concurrent.ConcurrentMap;
+import org.apache.flink.table.catalog.Catalog;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.iceberg.hive.HiveCatalog;
+import org.apache.iceberg.hive.TestHiveMetastore;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+public abstract class FlinkTestBase extends AbstractTestBase {
+
+  private static TestHiveMetastore metastore = null;
+  protected static HiveConf hiveConf = null;
+  protected static HiveCatalog catalog = null;
+  protected static ConcurrentMap<String, Catalog> flinkCatalogs;
+
+  @BeforeClass
+  public static void startMetastoreAndSpark() {

Review comment:
       Nit: the method names weren't updated.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453114831



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java
##########
@@ -0,0 +1,241 @@
+/*
+ * 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.File;
+import java.util.Map;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.types.Types;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Test;
+
+public class TestFlinkCatalogDatabase extends FlinkCatalogTestBase {
+
+  public TestFlinkCatalogDatabase(String catalogName, String[] baseNamepace) {
+    super(catalogName, baseNamepace);
+  }
+
+  @After
+  public void clean() {
+    sql("DROP TABLE IF EXISTS %s.tl", flinkIdentifier);
+    sql("DROP DATABASE IF EXISTS %s", flinkIdentifier);
+  }
+
+  @Test
+  public void testCreateNamespace() {
+    Assert.assertFalse(
+        "Database should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+
+    Assert.assertTrue("Database should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+  }
+
+  @Test
+  public void testDefaultDatabase() {
+    sql("USE CATALOG %s", catalogName);
+
+    Assert.assertEquals("Should use the current catalog", tEnv.getCurrentCatalog(), catalogName);
+    Assert.assertEquals("Should use the configured default namespace", tEnv.getCurrentDatabase(), "default");
+  }
+
+  @Test
+  public void testDropEmptyDatabase() {
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("DROP DATABASE %s", flinkIdentifier);
+
+    Assert.assertFalse(
+        "Namespace should have been dropped",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+  }
+
+  @Test
+  public void testDropNonEmptyNamespace() {
+    Assume.assumeFalse("Hadoop catalog throws IOException: Directory is not empty.", isHadoopCatalog);
+
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+
+    validationCatalog.createTable(
+        TableIdentifier.of(icebergNamespace, "tl"),
+        new Schema(Types.NestedField.optional(0, "id", Types.LongType.get())));
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+    Assert.assertTrue("Table should exist", validationCatalog.tableExists(TableIdentifier.of(icebergNamespace, "tl")));
+
+    AssertHelpers.assertThrowsCause(
+        "Should fail if trying to delete a non-empty database",
+        DatabaseNotEmptyException.class,
+        String.format("Database %s in catalog %s is not empty.", DATABASE, catalogName),
+        () -> sql("DROP DATABASE %s", flinkIdentifier));
+
+    sql("DROP TABLE %s.tl", flinkIdentifier);
+  }
+
+  @Test
+  public void testListTables() {
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+    sql("USE CATALOG %s", catalogName);
+    sql("USE %s", DATABASE);
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    Assert.assertEquals("Should not list any tables", 0, tEnv.listTables().length);
+
+    validationCatalog.createTable(
+        TableIdentifier.of(icebergNamespace, "tl"),
+        new Schema(Types.NestedField.optional(0, "id", Types.LongType.get())));
+
+    Assert.assertEquals("Only 1 table", 1, tEnv.listTables().length);
+    Assert.assertEquals("Table name should match", "tl", tEnv.listTables()[0]);
+  }
+
+  @Test
+  public void testListNamespace() {
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+    sql("USE CATALOG %s", catalogName);
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    String[] databases = tEnv.listDatabases();
+
+    if (isHadoopCatalog) {
+      Assert.assertEquals("Should have 1 database", 1, databases.length);
+      Assert.assertEquals("Should have only db database", "db", databases[0]);
+
+      if (baseNamespace.length > 0) {
+        // test namespace not belongs to this catalog
+        validationNamespaceCatalog.createNamespace(Namespace.of(baseNamespace[0], "UNKNOWN_NAMESPACE"));
+        databases = tEnv.listDatabases();
+        Assert.assertEquals("Should have 1 database", 1, databases.length);
+        Assert.assertEquals("Should have only db database", "db", databases[0]);
+      }
+    } else {
+      Assert.assertEquals("Should have 2 databases", 2, databases.length);
+      Assert.assertEquals(
+          "Should have default and db databases",
+          ImmutableSet.of("default", "db"),
+          ImmutableSet.copyOf(databases));
+    }
+  }
+
+  @Test
+  public void testCreateNamespaceWithMetadata() {
+    Assume.assumeFalse("HadoopCatalog does not support namespace metadata", isHadoopCatalog);
+
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s WITH ('prop'='value')", flinkIdentifier);
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    Map<String, String> nsMetadata = validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace);
+
+    Assert.assertEquals("Namespace should have expected prop value", "value", nsMetadata.get("prop"));
+  }
+
+  @Test
+  public void testCreateNamespaceWithComment() {
+    Assume.assumeFalse("HadoopCatalog does not support namespace metadata", isHadoopCatalog);
+
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s COMMENT 'namespace doc'", flinkIdentifier);
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    Map<String, String> nsMetadata = validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace);
+
+    Assert.assertEquals("Namespace should have expected comment", "namespace doc", nsMetadata.get("comment"));
+  }
+
+  @Test
+  public void testCreateNamespaceWithLocation() throws Exception {
+    Assume.assumeFalse("HadoopCatalog does not support namespace locations", isHadoopCatalog);

Review comment:
       Do we need a test to validate that the `CREATE DATABASE` statement fails for Hadoop?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#issuecomment-657827883


   @JingsongLi, I think this needs to be rebased now that #1180 is in.
   
   Also, should we get #1174 updated for the comments here so we can merge them separately? If we can, I'd prefer to make the commits smaller.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#issuecomment-657371042


   Thanks @rdblue for your review, I have addressed your comments.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453399671



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java
##########
@@ -0,0 +1,508 @@
+/*
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.AbstractCatalog;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogDatabase;
+import org.apache.flink.table.catalog.CatalogDatabaseImpl;
+import org.apache.flink.table.catalog.CatalogFunction;
+import org.apache.flink.table.catalog.CatalogPartition;
+import org.apache.flink.table.catalog.CatalogPartitionSpec;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+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.DatabaseNotEmptyException;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
+import org.apache.flink.table.catalog.exceptions.FunctionNotExistException;
+import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.catalog.stats.CatalogColumnStatistics;
+import org.apache.flink.table.catalog.stats.CatalogTableStatistics;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.util.StringUtils;
+import org.apache.iceberg.CachingCatalog;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+
+/**
+ * A Flink Catalog implementation that wraps an Iceberg {@link Catalog}.
+ * <p>
+ * The mapping between Flink database and Iceberg namespace:
+ * Supplying a base namespace for a given catalog, so if you have a catalog that supports a 2-level namespace, you
+ * would supply the first level in the catalog configuration and the second level would be exposed as Flink databases.
+ * <p>
+ * The Iceberg table manages its partitions by itself. The partition of the Iceberg table is independent of the
+ * partition of Flink.
+ */
+public class FlinkCatalog extends AbstractCatalog {
+
+  private final Catalog originalCatalog;
+  private final Catalog icebergCatalog;
+  private final String[] baseNamespace;
+  private final SupportsNamespaces asNamespaceCatalog;
+
+  public FlinkCatalog(
+      String catalogName,
+      String defaultDatabase,
+      String[] baseNamespace,
+      Catalog icebergCatalog,
+      boolean cacheEnabled) {
+    super(catalogName, defaultDatabase);
+    this.originalCatalog = icebergCatalog;
+    this.icebergCatalog = cacheEnabled ? CachingCatalog.wrap(icebergCatalog) : icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    if (icebergCatalog instanceof SupportsNamespaces) {
+      asNamespaceCatalog = (SupportsNamespaces) icebergCatalog;
+    } else {
+      asNamespaceCatalog = null;
+    }
+  }
+
+  @Override
+  public void open() throws CatalogException {
+  }
+
+  @Override
+  public void close() throws CatalogException {
+    if (originalCatalog instanceof Closeable) {
+      try {
+        ((Closeable) originalCatalog).close();
+      } catch (IOException e) {
+        throw new CatalogException(e);
+      }
+    }
+  }
+
+  private Namespace toNamespace(String database) {
+    String[] namespace = new String[baseNamespace.length + 1];
+    System.arraycopy(baseNamespace, 0, namespace, 0, baseNamespace.length);
+    namespace[baseNamespace.length] = database;
+    return Namespace.of(namespace);
+  }
+
+  private TableIdentifier toIdentifier(ObjectPath path) {
+    return TableIdentifier.of(toNamespace(path.getDatabaseName()), path.getObjectName());
+  }
+
+  @Override
+  public List<String> listDatabases() throws CatalogException {
+    if (asNamespaceCatalog == null) {
+      return Collections.singletonList(getDefaultDatabase());
+    }
+
+    return listAllNamespaces(Namespace.empty()).stream()
+        .map(n -> n.level(n.levels().length - 1))
+        .collect(Collectors.toList());
+  }
+
+  private List<Namespace> listAllNamespaces(Namespace namespace) {
+    if (asNamespaceCatalog == null) {
+      throw new RuntimeException("The asNamespaceCatalog should not be null.");
+    }
+
+    String[] levels = namespace.levels();
+    if (levels.length == baseNamespace.length + 1) {
+      return Collections.singletonList(namespace);
+    }
+    if (levels.length < baseNamespace.length + 1) {
+      for (int i = 0; i < levels.length; i++) {
+        if (!baseNamespace[i].equals(levels[i])) {
+          return Collections.emptyList();
+        }
+      }
+      List<Namespace> ret = new ArrayList<>();
+      asNamespaceCatalog.listNamespaces(namespace).forEach(n -> ret.addAll(listAllNamespaces(n)));
+      return ret;
+    }
+    return Collections.emptyList();
+  }
+
+  @Override
+  public CatalogDatabase getDatabase(String databaseName) throws DatabaseNotExistException, CatalogException {
+    if (asNamespaceCatalog == null) {
+      if (!getDefaultDatabase().equals(databaseName)) {
+        throw new DatabaseNotExistException(getName(), databaseName);
+      } else {
+        return new CatalogDatabaseImpl(new HashMap<>(), "");
+      }
+    } else {
+      try {
+        Map<String, String> metadata =
+            new HashMap<>(asNamespaceCatalog.loadNamespaceMetadata(toNamespace(databaseName)));
+        String comment = metadata.remove("comment");
+        return new CatalogDatabaseImpl(metadata, comment);
+      } catch (NoSuchNamespaceException e) {
+        throw new DatabaseNotExistException(getName(), databaseName, e);
+      }
+    }
+  }
+
+  @Override
+  public boolean databaseExists(String databaseName) throws CatalogException {
+    try {
+      getDatabase(databaseName);
+      return true;
+    } catch (DatabaseNotExistException ignore) {
+      return false;
+    }
+  }
+
+  @Override
+  public void createDatabase(String name, CatalogDatabase database, boolean ignoreIfExists)
+      throws DatabaseAlreadyExistException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      try {
+        asNamespaceCatalog.createNamespace(
+            toNamespace(name),
+            mergeComment(database.getProperties(), database.getComment()));
+      } catch (AlreadyExistsException e) {
+        if (!ignoreIfExists) {
+          throw new DatabaseAlreadyExistException(getName(), name, e);
+        }
+      }
+    } else {
+      throw new UnsupportedOperationException("Namespaces are not supported by catalog: " + getName());
+    }
+  }
+
+  private Map<String, String> mergeComment(Map<String, String> metadata, String comment) {
+    Map<String, String> ret = new HashMap<>(metadata);
+    if (metadata.containsKey("comment")) {
+      throw new CatalogException("Database properties should not contain key: 'comment'.");
+    }
+    if (!StringUtils.isNullOrWhitespaceOnly(comment)) {
+      ret.put("comment", comment);
+    }
+    return ret;
+  }
+
+  @Override
+  public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade)
+      throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      try {
+        boolean success = asNamespaceCatalog.dropNamespace(toNamespace(name));
+        if (!success && !ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name);
+        }
+      } catch (NoSuchNamespaceException e) {
+        if (!ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name, e);
+        }
+      } catch (NamespaceNotEmptyException e) {
+        throw new DatabaseNotEmptyException(getName(), name, e);
+      }
+    } else {
+      if (!ignoreIfNotExists) {
+        throw new DatabaseNotExistException(getName(), name);
+      }
+    }
+  }
+
+  @Override
+  public void alterDatabase(String name, CatalogDatabase newDatabase, boolean ignoreIfNotExists)
+      throws DatabaseNotExistException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      Namespace namespace = toNamespace(name);
+      Map<String, String> updates = Maps.newHashMap();
+      Set<String> removals = Sets.newHashSet();
+
+      try {
+        Map<String, String> oldOptions = asNamespaceCatalog.loadNamespaceMetadata(namespace);
+        Map<String, String> newOptions = mergeComment(newDatabase.getProperties(), newDatabase.getComment());
+
+        for (String key : oldOptions.keySet()) {
+          if (!newOptions.containsKey(key)) {
+            removals.add(key);
+          }
+        }
+
+        for (Map.Entry<String, String> entry : newOptions.entrySet()) {
+          if (!entry.getValue().equals(oldOptions.get(entry.getKey()))) {
+            updates.put(entry.getKey(), entry.getValue());
+          }
+        }
+
+        if (!updates.isEmpty()) {
+          asNamespaceCatalog.setProperties(namespace, updates);
+        }
+
+        if (!removals.isEmpty()) {
+          asNamespaceCatalog.removeProperties(namespace, removals);
+        }
+
+      } catch (org.apache.iceberg.exceptions.NoSuchNamespaceException e) {
+        if (!ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name, e);
+        }
+      }
+    } else {
+      if (!ignoreIfNotExists) {
+        throw new DatabaseNotExistException(getName(), name);
+      }
+    }
+  }
+
+  @Override
+  public List<String> listTables(String databaseName) throws DatabaseNotExistException, CatalogException {
+    try {
+      return icebergCatalog.listTables(toNamespace(databaseName)).stream()
+          .map(TableIdentifier::name)
+          .collect(Collectors.toList());
+    } catch (NoSuchNamespaceException e) {
+      throw new DatabaseNotExistException(getName(), databaseName, e);
+    }
+  }
+
+  @Override
+  public CatalogBaseTable getTable(ObjectPath tablePath) throws TableNotExistException, CatalogException {
+    try {
+      Table table = icebergCatalog.loadTable(toIdentifier(tablePath));
+      TableSchema tableSchema = FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(table.schema()));
+
+      // NOTE: We can not create a IcebergCatalogTable, because Flink optimizer may use CatalogTableImpl to copy a new
+      // catalog table.
+      // Let's re-loading table from Iceberg catalog when creating source/sink operators.
+      return new CatalogTableImpl(tableSchema, table.properties(), "");

Review comment:
       I'll modify it to null.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453404061



##########
File path: flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.List;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.BinaryType;
+import org.apache.flink.table.types.logical.BooleanType;
+import org.apache.flink.table.types.logical.CharType;
+import org.apache.flink.table.types.logical.DateType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.DoubleType;
+import org.apache.flink.table.types.logical.FloatType;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.LocalZonedTimestampType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.TimeType;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.VarBinaryType;
+import org.apache.flink.table.types.logical.VarCharType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types;
+
+class TypeToFlinkType extends TypeUtil.SchemaVisitor<LogicalType> {
+  TypeToFlinkType() {
+  }
+
+  @Override
+  public LogicalType schema(Schema schema, LogicalType structType) {
+    return structType;
+  }
+
+  @Override
+  public LogicalType struct(Types.StructType struct, List<LogicalType> fieldResults) {
+    List<Types.NestedField> fields = struct.fields();
+
+    List<RowType.RowField> flinkFields = Lists.newArrayListWithExpectedSize(fieldResults.size());
+    for (int i = 0; i < fields.size(); i += 1) {
+      Types.NestedField field = fields.get(i);
+      LogicalType type = fieldResults.get(i);
+      RowType.RowField flinkField = new RowType.RowField(
+          field.name(), type.copy(field.isOptional()), field.doc());
+      flinkFields.add(flinkField);
+    }
+
+    return new RowType(flinkFields);
+  }
+
+  @Override
+  public LogicalType field(Types.NestedField field, LogicalType fieldResult) {
+    return fieldResult;
+  }
+
+  @Override
+  public LogicalType list(Types.ListType list, LogicalType elementResult) {
+    return new ArrayType(elementResult.copy(list.isElementOptional()));
+  }
+
+  @Override
+  public LogicalType map(Types.MapType map, LogicalType keyResult, LogicalType valueResult) {
+    // keys in map are not allowed to be null.
+    return new MapType(keyResult.copy(false), valueResult.copy(map.isValueOptional()));
+  }
+
+  @Override
+  public LogicalType primitive(Type.PrimitiveType primitive) {
+    switch (primitive.typeId()) {
+      case BOOLEAN:
+        return new BooleanType();
+      case INTEGER:
+        return new IntType();
+      case LONG:
+        return new BigIntType();
+      case FLOAT:
+        return new FloatType();
+      case DOUBLE:
+        return new DoubleType();
+      case DATE:
+        return new DateType();
+      case TIME:
+        // MICROS
+        return new TimeType(6);
+      case TIMESTAMP:
+        Types.TimestampType timestamp = (Types.TimestampType) primitive;
+        if (timestamp.shouldAdjustToUTC()) {
+          // MICROS
+          return new LocalZonedTimestampType(6);
+        } else {
+          // MICROS
+          return new TimestampType(6);
+        }
+      case STRING:
+        return new VarCharType(VarCharType.MAX_LENGTH);
+      case UUID:
+        // UUID length is 16
+        return new CharType(16);

Review comment:
       I thought UUID should be a `Char` with 36 precision because:
   - In Spark, UUID function returns `StringType`.
   - In Flink, UUID function returns `CharType` with 36 precision.
   
   But you are right, in Orc and Parquet, UUID just be treated as a fixed-length binary.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453108323



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java
##########
@@ -0,0 +1,508 @@
+/*
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.AbstractCatalog;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogDatabase;
+import org.apache.flink.table.catalog.CatalogDatabaseImpl;
+import org.apache.flink.table.catalog.CatalogFunction;
+import org.apache.flink.table.catalog.CatalogPartition;
+import org.apache.flink.table.catalog.CatalogPartitionSpec;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+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.DatabaseNotEmptyException;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
+import org.apache.flink.table.catalog.exceptions.FunctionNotExistException;
+import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.catalog.stats.CatalogColumnStatistics;
+import org.apache.flink.table.catalog.stats.CatalogTableStatistics;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.util.StringUtils;
+import org.apache.iceberg.CachingCatalog;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+
+/**
+ * A Flink Catalog implementation that wraps an Iceberg {@link Catalog}.
+ * <p>
+ * The mapping between Flink database and Iceberg namespace:
+ * Supplying a base namespace for a given catalog, so if you have a catalog that supports a 2-level namespace, you
+ * would supply the first level in the catalog configuration and the second level would be exposed as Flink databases.
+ * <p>
+ * The Iceberg table manages its partitions by itself. The partition of the Iceberg table is independent of the
+ * partition of Flink.
+ */
+public class FlinkCatalog extends AbstractCatalog {
+
+  private final Catalog originalCatalog;
+  private final Catalog icebergCatalog;
+  private final String[] baseNamespace;
+  private final SupportsNamespaces asNamespaceCatalog;
+
+  public FlinkCatalog(
+      String catalogName,
+      String defaultDatabase,
+      String[] baseNamespace,
+      Catalog icebergCatalog,
+      boolean cacheEnabled) {
+    super(catalogName, defaultDatabase);
+    this.originalCatalog = icebergCatalog;
+    this.icebergCatalog = cacheEnabled ? CachingCatalog.wrap(icebergCatalog) : icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    if (icebergCatalog instanceof SupportsNamespaces) {
+      asNamespaceCatalog = (SupportsNamespaces) icebergCatalog;
+    } else {
+      asNamespaceCatalog = null;
+    }
+  }
+
+  @Override
+  public void open() throws CatalogException {
+  }
+
+  @Override
+  public void close() throws CatalogException {
+    if (originalCatalog instanceof Closeable) {
+      try {
+        ((Closeable) originalCatalog).close();
+      } catch (IOException e) {
+        throw new CatalogException(e);
+      }
+    }
+  }
+
+  private Namespace toNamespace(String database) {
+    String[] namespace = new String[baseNamespace.length + 1];
+    System.arraycopy(baseNamespace, 0, namespace, 0, baseNamespace.length);
+    namespace[baseNamespace.length] = database;
+    return Namespace.of(namespace);
+  }
+
+  private TableIdentifier toIdentifier(ObjectPath path) {
+    return TableIdentifier.of(toNamespace(path.getDatabaseName()), path.getObjectName());
+  }
+
+  @Override
+  public List<String> listDatabases() throws CatalogException {
+    if (asNamespaceCatalog == null) {
+      return Collections.singletonList(getDefaultDatabase());
+    }
+
+    return listAllNamespaces(Namespace.empty()).stream()
+        .map(n -> n.level(n.levels().length - 1))
+        .collect(Collectors.toList());
+  }
+
+  private List<Namespace> listAllNamespaces(Namespace namespace) {
+    if (asNamespaceCatalog == null) {
+      throw new RuntimeException("The asNamespaceCatalog should not be null.");
+    }
+
+    String[] levels = namespace.levels();
+    if (levels.length == baseNamespace.length + 1) {
+      return Collections.singletonList(namespace);
+    }
+    if (levels.length < baseNamespace.length + 1) {
+      for (int i = 0; i < levels.length; i++) {
+        if (!baseNamespace[i].equals(levels[i])) {
+          return Collections.emptyList();
+        }
+      }
+      List<Namespace> ret = new ArrayList<>();
+      asNamespaceCatalog.listNamespaces(namespace).forEach(n -> ret.addAll(listAllNamespaces(n)));
+      return ret;
+    }
+    return Collections.emptyList();
+  }
+
+  @Override
+  public CatalogDatabase getDatabase(String databaseName) throws DatabaseNotExistException, CatalogException {
+    if (asNamespaceCatalog == null) {
+      if (!getDefaultDatabase().equals(databaseName)) {
+        throw new DatabaseNotExistException(getName(), databaseName);
+      } else {
+        return new CatalogDatabaseImpl(new HashMap<>(), "");
+      }
+    } else {
+      try {
+        Map<String, String> metadata =
+            new HashMap<>(asNamespaceCatalog.loadNamespaceMetadata(toNamespace(databaseName)));
+        String comment = metadata.remove("comment");
+        return new CatalogDatabaseImpl(metadata, comment);
+      } catch (NoSuchNamespaceException e) {
+        throw new DatabaseNotExistException(getName(), databaseName, e);
+      }
+    }
+  }
+
+  @Override
+  public boolean databaseExists(String databaseName) throws CatalogException {
+    try {
+      getDatabase(databaseName);
+      return true;
+    } catch (DatabaseNotExistException ignore) {
+      return false;
+    }
+  }
+
+  @Override
+  public void createDatabase(String name, CatalogDatabase database, boolean ignoreIfExists)
+      throws DatabaseAlreadyExistException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      try {
+        asNamespaceCatalog.createNamespace(
+            toNamespace(name),
+            mergeComment(database.getProperties(), database.getComment()));
+      } catch (AlreadyExistsException e) {
+        if (!ignoreIfExists) {
+          throw new DatabaseAlreadyExistException(getName(), name, e);
+        }
+      }
+    } else {
+      throw new UnsupportedOperationException("Namespaces are not supported by catalog: " + getName());
+    }
+  }
+
+  private Map<String, String> mergeComment(Map<String, String> metadata, String comment) {
+    Map<String, String> ret = new HashMap<>(metadata);
+    if (metadata.containsKey("comment")) {
+      throw new CatalogException("Database properties should not contain key: 'comment'.");
+    }
+    if (!StringUtils.isNullOrWhitespaceOnly(comment)) {
+      ret.put("comment", comment);
+    }
+    return ret;
+  }
+
+  @Override
+  public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade)
+      throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      try {
+        boolean success = asNamespaceCatalog.dropNamespace(toNamespace(name));
+        if (!success && !ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name);
+        }
+      } catch (NoSuchNamespaceException e) {
+        if (!ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name, e);
+        }
+      } catch (NamespaceNotEmptyException e) {
+        throw new DatabaseNotEmptyException(getName(), name, e);
+      }
+    } else {
+      if (!ignoreIfNotExists) {
+        throw new DatabaseNotExistException(getName(), name);
+      }
+    }
+  }
+
+  @Override
+  public void alterDatabase(String name, CatalogDatabase newDatabase, boolean ignoreIfNotExists)
+      throws DatabaseNotExistException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      Namespace namespace = toNamespace(name);
+      Map<String, String> updates = Maps.newHashMap();
+      Set<String> removals = Sets.newHashSet();
+
+      try {
+        Map<String, String> oldOptions = asNamespaceCatalog.loadNamespaceMetadata(namespace);
+        Map<String, String> newOptions = mergeComment(newDatabase.getProperties(), newDatabase.getComment());
+
+        for (String key : oldOptions.keySet()) {
+          if (!newOptions.containsKey(key)) {
+            removals.add(key);
+          }
+        }
+
+        for (Map.Entry<String, String> entry : newOptions.entrySet()) {
+          if (!entry.getValue().equals(oldOptions.get(entry.getKey()))) {
+            updates.put(entry.getKey(), entry.getValue());
+          }
+        }
+
+        if (!updates.isEmpty()) {
+          asNamespaceCatalog.setProperties(namespace, updates);
+        }
+
+        if (!removals.isEmpty()) {
+          asNamespaceCatalog.removeProperties(namespace, removals);
+        }
+
+      } catch (org.apache.iceberg.exceptions.NoSuchNamespaceException e) {
+        if (!ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name, e);
+        }
+      }
+    } else {
+      if (!ignoreIfNotExists) {
+        throw new DatabaseNotExistException(getName(), name);
+      }
+    }
+  }
+
+  @Override
+  public List<String> listTables(String databaseName) throws DatabaseNotExistException, CatalogException {
+    try {
+      return icebergCatalog.listTables(toNamespace(databaseName)).stream()
+          .map(TableIdentifier::name)
+          .collect(Collectors.toList());
+    } catch (NoSuchNamespaceException e) {
+      throw new DatabaseNotExistException(getName(), databaseName, e);
+    }
+  }
+
+  @Override
+  public CatalogBaseTable getTable(ObjectPath tablePath) throws TableNotExistException, CatalogException {
+    try {
+      Table table = icebergCatalog.loadTable(toIdentifier(tablePath));
+      TableSchema tableSchema = FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(table.schema()));
+
+      // NOTE: We can not create a IcebergCatalogTable, because Flink optimizer may use CatalogTableImpl to copy a new
+      // catalog table.
+      // Let's re-loading table from Iceberg catalog when creating source/sink operators.
+      return new CatalogTableImpl(tableSchema, table.properties(), "");

Review comment:
       Passing an empty string is suspicious. Should that be null or omitted to let the impl default?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#issuecomment-657390245


   I updated the branch, but the changes were not synchronized to this PR. It seems something wrong in github...


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453791968



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java
##########
@@ -0,0 +1,508 @@
+/*
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.AbstractCatalog;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogDatabase;
+import org.apache.flink.table.catalog.CatalogDatabaseImpl;
+import org.apache.flink.table.catalog.CatalogFunction;
+import org.apache.flink.table.catalog.CatalogPartition;
+import org.apache.flink.table.catalog.CatalogPartitionSpec;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+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.DatabaseNotEmptyException;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
+import org.apache.flink.table.catalog.exceptions.FunctionNotExistException;
+import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.catalog.stats.CatalogColumnStatistics;
+import org.apache.flink.table.catalog.stats.CatalogTableStatistics;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.util.StringUtils;
+import org.apache.iceberg.CachingCatalog;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+
+/**
+ * A Flink Catalog implementation that wraps an Iceberg {@link Catalog}.
+ * <p>
+ * The mapping between Flink database and Iceberg namespace:
+ * Supplying a base namespace for a given catalog, so if you have a catalog that supports a 2-level namespace, you
+ * would supply the first level in the catalog configuration and the second level would be exposed as Flink databases.
+ * <p>
+ * The Iceberg table manages its partitions by itself. The partition of the Iceberg table is independent of the
+ * partition of Flink.
+ */
+public class FlinkCatalog extends AbstractCatalog {
+
+  private final Catalog originalCatalog;
+  private final Catalog icebergCatalog;
+  private final String[] baseNamespace;
+  private final SupportsNamespaces asNamespaceCatalog;
+
+  public FlinkCatalog(
+      String catalogName,
+      String defaultDatabase,
+      String[] baseNamespace,
+      Catalog icebergCatalog,
+      boolean cacheEnabled) {
+    super(catalogName, defaultDatabase);
+    this.originalCatalog = icebergCatalog;
+    this.icebergCatalog = cacheEnabled ? CachingCatalog.wrap(icebergCatalog) : icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    if (icebergCatalog instanceof SupportsNamespaces) {
+      asNamespaceCatalog = (SupportsNamespaces) icebergCatalog;
+    } else {
+      asNamespaceCatalog = null;
+    }
+  }
+
+  @Override
+  public void open() throws CatalogException {
+  }
+
+  @Override
+  public void close() throws CatalogException {
+    if (originalCatalog instanceof Closeable) {
+      try {
+        ((Closeable) originalCatalog).close();
+      } catch (IOException e) {
+        throw new CatalogException(e);
+      }
+    }
+  }
+
+  private Namespace toNamespace(String database) {
+    String[] namespace = new String[baseNamespace.length + 1];
+    System.arraycopy(baseNamespace, 0, namespace, 0, baseNamespace.length);
+    namespace[baseNamespace.length] = database;
+    return Namespace.of(namespace);
+  }
+
+  private TableIdentifier toIdentifier(ObjectPath path) {
+    return TableIdentifier.of(toNamespace(path.getDatabaseName()), path.getObjectName());
+  }
+
+  @Override
+  public List<String> listDatabases() throws CatalogException {
+    if (asNamespaceCatalog == null) {
+      return Collections.singletonList(getDefaultDatabase());
+    }
+
+    return listAllNamespaces(Namespace.empty()).stream()
+        .map(n -> n.level(n.levels().length - 1))
+        .collect(Collectors.toList());
+  }
+
+  private List<Namespace> listAllNamespaces(Namespace namespace) {
+    if (asNamespaceCatalog == null) {
+      throw new RuntimeException("The asNamespaceCatalog should not be null.");
+    }
+
+    String[] levels = namespace.levels();
+    if (levels.length == baseNamespace.length + 1) {
+      return Collections.singletonList(namespace);
+    }
+    if (levels.length < baseNamespace.length + 1) {
+      for (int i = 0; i < levels.length; i++) {
+        if (!baseNamespace[i].equals(levels[i])) {
+          return Collections.emptyList();
+        }
+      }
+      List<Namespace> ret = new ArrayList<>();
+      asNamespaceCatalog.listNamespaces(namespace).forEach(n -> ret.addAll(listAllNamespaces(n)));
+      return ret;
+    }
+    return Collections.emptyList();
+  }
+
+  @Override
+  public CatalogDatabase getDatabase(String databaseName) throws DatabaseNotExistException, CatalogException {
+    if (asNamespaceCatalog == null) {
+      if (!getDefaultDatabase().equals(databaseName)) {
+        throw new DatabaseNotExistException(getName(), databaseName);
+      } else {
+        return new CatalogDatabaseImpl(new HashMap<>(), "");
+      }
+    } else {
+      try {
+        Map<String, String> metadata =
+            new HashMap<>(asNamespaceCatalog.loadNamespaceMetadata(toNamespace(databaseName)));
+        String comment = metadata.remove("comment");
+        return new CatalogDatabaseImpl(metadata, comment);
+      } catch (NoSuchNamespaceException e) {
+        throw new DatabaseNotExistException(getName(), databaseName, e);
+      }
+    }
+  }
+
+  @Override
+  public boolean databaseExists(String databaseName) throws CatalogException {
+    try {
+      getDatabase(databaseName);
+      return true;
+    } catch (DatabaseNotExistException ignore) {
+      return false;
+    }
+  }
+
+  @Override
+  public void createDatabase(String name, CatalogDatabase database, boolean ignoreIfExists)
+      throws DatabaseAlreadyExistException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      try {
+        asNamespaceCatalog.createNamespace(
+            toNamespace(name),
+            mergeComment(database.getProperties(), database.getComment()));
+      } catch (AlreadyExistsException e) {
+        if (!ignoreIfExists) {
+          throw new DatabaseAlreadyExistException(getName(), name, e);
+        }
+      }
+    } else {
+      throw new UnsupportedOperationException("Namespaces are not supported by catalog: " + getName());
+    }
+  }
+
+  private Map<String, String> mergeComment(Map<String, String> metadata, String comment) {
+    Map<String, String> ret = new HashMap<>(metadata);
+    if (metadata.containsKey("comment")) {
+      throw new CatalogException("Database properties should not contain key: 'comment'.");
+    }
+    if (!StringUtils.isNullOrWhitespaceOnly(comment)) {
+      ret.put("comment", comment);
+    }
+    return ret;
+  }
+
+  @Override
+  public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade)
+      throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      try {
+        boolean success = asNamespaceCatalog.dropNamespace(toNamespace(name));
+        if (!success && !ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name);
+        }
+      } catch (NoSuchNamespaceException e) {
+        if (!ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name, e);
+        }
+      } catch (NamespaceNotEmptyException e) {
+        throw new DatabaseNotEmptyException(getName(), name, e);
+      }
+    } else {
+      if (!ignoreIfNotExists) {
+        throw new DatabaseNotExistException(getName(), name);
+      }
+    }
+  }
+
+  @Override
+  public void alterDatabase(String name, CatalogDatabase newDatabase, boolean ignoreIfNotExists)
+      throws DatabaseNotExistException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      Namespace namespace = toNamespace(name);
+      Map<String, String> updates = Maps.newHashMap();
+      Set<String> removals = Sets.newHashSet();
+
+      try {
+        Map<String, String> oldOptions = asNamespaceCatalog.loadNamespaceMetadata(namespace);
+        Map<String, String> newOptions = mergeComment(newDatabase.getProperties(), newDatabase.getComment());
+
+        for (String key : oldOptions.keySet()) {
+          if (!newOptions.containsKey(key)) {
+            removals.add(key);
+          }
+        }
+
+        for (Map.Entry<String, String> entry : newOptions.entrySet()) {
+          if (!entry.getValue().equals(oldOptions.get(entry.getKey()))) {
+            updates.put(entry.getKey(), entry.getValue());
+          }
+        }
+
+        if (!updates.isEmpty()) {
+          asNamespaceCatalog.setProperties(namespace, updates);
+        }
+
+        if (!removals.isEmpty()) {
+          asNamespaceCatalog.removeProperties(namespace, removals);
+        }
+
+      } catch (org.apache.iceberg.exceptions.NoSuchNamespaceException e) {
+        if (!ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name, e);
+        }
+      }
+    } else {
+      if (!ignoreIfNotExists) {
+        throw new DatabaseNotExistException(getName(), name);
+      }
+    }
+  }
+
+  @Override
+  public List<String> listTables(String databaseName) throws DatabaseNotExistException, CatalogException {
+    try {
+      return icebergCatalog.listTables(toNamespace(databaseName)).stream()
+          .map(TableIdentifier::name)
+          .collect(Collectors.toList());
+    } catch (NoSuchNamespaceException e) {
+      throw new DatabaseNotExistException(getName(), databaseName, e);
+    }
+  }
+
+  @Override
+  public CatalogBaseTable getTable(ObjectPath tablePath) throws TableNotExistException, CatalogException {
+    try {
+      Table table = icebergCatalog.loadTable(toIdentifier(tablePath));
+      TableSchema tableSchema = FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(table.schema()));
+
+      // NOTE: We can not create a IcebergCatalogTable, because Flink optimizer may use CatalogTableImpl to copy a new
+      // catalog table.
+      // Let's re-loading table from Iceberg catalog when creating source/sink operators.
+      return new CatalogTableImpl(tableSchema, table.properties(), "");
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      throw new TableNotExistException(getName(), tablePath, e);
+    }
+  }
+
+  @Override
+  public boolean tableExists(ObjectPath tablePath) throws CatalogException {
+    return icebergCatalog.tableExists(toIdentifier(tablePath));
+  }
+
+  @Override
+  public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists)
+      throws TableNotExistException, CatalogException {
+    try {
+      icebergCatalog.dropTable(toIdentifier(tablePath));
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      throw new TableNotExistException(getName(), tablePath, e);
+    }
+  }
+
+  @Override
+  public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists)
+      throws TableNotExistException, TableAlreadyExistException, CatalogException {
+    try {
+      icebergCatalog.renameTable(
+          toIdentifier(tablePath),
+          toIdentifier(new ObjectPath(tablePath.getDatabaseName(), newTableName)));
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      throw new TableNotExistException(getName(), tablePath, e);
+    } catch (AlreadyExistsException e) {
+      throw new TableAlreadyExistException(getName(), tablePath, e);
+    }
+  }
+
+  /**
+   * TODO Implement DDL-string parser for PartitionSpec.

Review comment:
       `PartitionSpecParser.fromJson` is how we serialize partition specs internally. It isn't great to expose it directly to users, but would at least make it possible to configure partitioning. If you have a different approach, that is much better!
   
   How would the computed column and partition approach work?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453105170



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java
##########
@@ -0,0 +1,508 @@
+/*
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.AbstractCatalog;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogDatabase;
+import org.apache.flink.table.catalog.CatalogDatabaseImpl;
+import org.apache.flink.table.catalog.CatalogFunction;
+import org.apache.flink.table.catalog.CatalogPartition;
+import org.apache.flink.table.catalog.CatalogPartitionSpec;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+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.DatabaseNotEmptyException;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
+import org.apache.flink.table.catalog.exceptions.FunctionNotExistException;
+import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.catalog.stats.CatalogColumnStatistics;
+import org.apache.flink.table.catalog.stats.CatalogTableStatistics;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.util.StringUtils;
+import org.apache.iceberg.CachingCatalog;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+
+/**
+ * A Flink Catalog implementation that wraps an Iceberg {@link Catalog}.
+ * <p>
+ * The mapping between Flink database and Iceberg namespace:
+ * Supplying a base namespace for a given catalog, so if you have a catalog that supports a 2-level namespace, you
+ * would supply the first level in the catalog configuration and the second level would be exposed as Flink databases.
+ * <p>
+ * The Iceberg table manages its partitions by itself. The partition of the Iceberg table is independent of the
+ * partition of Flink.
+ */
+public class FlinkCatalog extends AbstractCatalog {
+
+  private final Catalog originalCatalog;
+  private final Catalog icebergCatalog;
+  private final String[] baseNamespace;
+  private final SupportsNamespaces asNamespaceCatalog;
+
+  public FlinkCatalog(
+      String catalogName,
+      String defaultDatabase,
+      String[] baseNamespace,
+      Catalog icebergCatalog,
+      boolean cacheEnabled) {
+    super(catalogName, defaultDatabase);
+    this.originalCatalog = icebergCatalog;
+    this.icebergCatalog = cacheEnabled ? CachingCatalog.wrap(icebergCatalog) : icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    if (icebergCatalog instanceof SupportsNamespaces) {
+      asNamespaceCatalog = (SupportsNamespaces) icebergCatalog;
+    } else {
+      asNamespaceCatalog = null;
+    }
+  }
+
+  @Override
+  public void open() throws CatalogException {
+  }
+
+  @Override
+  public void close() throws CatalogException {
+    if (originalCatalog instanceof Closeable) {
+      try {
+        ((Closeable) originalCatalog).close();
+      } catch (IOException e) {
+        throw new CatalogException(e);
+      }
+    }
+  }
+
+  private Namespace toNamespace(String database) {
+    String[] namespace = new String[baseNamespace.length + 1];
+    System.arraycopy(baseNamespace, 0, namespace, 0, baseNamespace.length);
+    namespace[baseNamespace.length] = database;
+    return Namespace.of(namespace);
+  }
+
+  private TableIdentifier toIdentifier(ObjectPath path) {
+    return TableIdentifier.of(toNamespace(path.getDatabaseName()), path.getObjectName());
+  }
+
+  @Override
+  public List<String> listDatabases() throws CatalogException {
+    if (asNamespaceCatalog == null) {
+      return Collections.singletonList(getDefaultDatabase());
+    }
+
+    return listAllNamespaces(Namespace.empty()).stream()
+        .map(n -> n.level(n.levels().length - 1))
+        .collect(Collectors.toList());
+  }
+
+  private List<Namespace> listAllNamespaces(Namespace namespace) {
+    if (asNamespaceCatalog == null) {
+      throw new RuntimeException("The asNamespaceCatalog should not be null.");
+    }
+
+    String[] levels = namespace.levels();
+    if (levels.length == baseNamespace.length + 1) {
+      return Collections.singletonList(namespace);
+    }
+    if (levels.length < baseNamespace.length + 1) {

Review comment:
       Nit: we like to add empty lines after control flow blocks.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453429303



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java
##########
@@ -0,0 +1,241 @@
+/*
+ * 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.File;
+import java.util.Map;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.types.Types;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Test;
+
+public class TestFlinkCatalogDatabase extends FlinkCatalogTestBase {
+
+  public TestFlinkCatalogDatabase(String catalogName, String[] baseNamepace) {
+    super(catalogName, baseNamepace);
+  }
+
+  @After
+  public void clean() {
+    sql("DROP TABLE IF EXISTS %s.tl", flinkIdentifier);
+    sql("DROP DATABASE IF EXISTS %s", flinkIdentifier);
+  }
+
+  @Test
+  public void testCreateNamespace() {
+    Assert.assertFalse(
+        "Database should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+
+    Assert.assertTrue("Database should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+  }
+
+  @Test
+  public void testDefaultDatabase() {
+    sql("USE CATALOG %s", catalogName);
+
+    Assert.assertEquals("Should use the current catalog", tEnv.getCurrentCatalog(), catalogName);
+    Assert.assertEquals("Should use the configured default namespace", tEnv.getCurrentDatabase(), "default");
+  }
+
+  @Test
+  public void testDropEmptyDatabase() {
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("DROP DATABASE %s", flinkIdentifier);
+
+    Assert.assertFalse(
+        "Namespace should have been dropped",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+  }
+
+  @Test
+  public void testDropNonEmptyNamespace() {
+    Assume.assumeFalse("Hadoop catalog throws IOException: Directory is not empty.", isHadoopCatalog);
+
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+
+    validationCatalog.createTable(
+        TableIdentifier.of(icebergNamespace, "tl"),
+        new Schema(Types.NestedField.optional(0, "id", Types.LongType.get())));
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+    Assert.assertTrue("Table should exist", validationCatalog.tableExists(TableIdentifier.of(icebergNamespace, "tl")));
+
+    AssertHelpers.assertThrowsCause(
+        "Should fail if trying to delete a non-empty database",
+        DatabaseNotEmptyException.class,
+        String.format("Database %s in catalog %s is not empty.", DATABASE, catalogName),
+        () -> sql("DROP DATABASE %s", flinkIdentifier));
+
+    sql("DROP TABLE %s.tl", flinkIdentifier);
+  }
+
+  @Test
+  public void testListTables() {
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+    sql("USE CATALOG %s", catalogName);
+    sql("USE %s", DATABASE);
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    Assert.assertEquals("Should not list any tables", 0, tEnv.listTables().length);

Review comment:
       In Flink 1.10, not support `SHOW TABLES`. It is supported in 1.11.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453397368



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java
##########
@@ -0,0 +1,508 @@
+/*
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.AbstractCatalog;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogDatabase;
+import org.apache.flink.table.catalog.CatalogDatabaseImpl;
+import org.apache.flink.table.catalog.CatalogFunction;
+import org.apache.flink.table.catalog.CatalogPartition;
+import org.apache.flink.table.catalog.CatalogPartitionSpec;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+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.DatabaseNotEmptyException;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
+import org.apache.flink.table.catalog.exceptions.FunctionNotExistException;
+import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.catalog.stats.CatalogColumnStatistics;
+import org.apache.flink.table.catalog.stats.CatalogTableStatistics;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.util.StringUtils;
+import org.apache.iceberg.CachingCatalog;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+
+/**
+ * A Flink Catalog implementation that wraps an Iceberg {@link Catalog}.
+ * <p>
+ * The mapping between Flink database and Iceberg namespace:
+ * Supplying a base namespace for a given catalog, so if you have a catalog that supports a 2-level namespace, you
+ * would supply the first level in the catalog configuration and the second level would be exposed as Flink databases.
+ * <p>
+ * The Iceberg table manages its partitions by itself. The partition of the Iceberg table is independent of the
+ * partition of Flink.
+ */
+public class FlinkCatalog extends AbstractCatalog {
+
+  private final Catalog originalCatalog;
+  private final Catalog icebergCatalog;
+  private final String[] baseNamespace;
+  private final SupportsNamespaces asNamespaceCatalog;
+
+  public FlinkCatalog(
+      String catalogName,
+      String defaultDatabase,
+      String[] baseNamespace,
+      Catalog icebergCatalog,
+      boolean cacheEnabled) {
+    super(catalogName, defaultDatabase);
+    this.originalCatalog = icebergCatalog;
+    this.icebergCatalog = cacheEnabled ? CachingCatalog.wrap(icebergCatalog) : icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    if (icebergCatalog instanceof SupportsNamespaces) {
+      asNamespaceCatalog = (SupportsNamespaces) icebergCatalog;
+    } else {
+      asNamespaceCatalog = null;
+    }
+  }
+
+  @Override
+  public void open() throws CatalogException {
+  }
+
+  @Override
+  public void close() throws CatalogException {
+    if (originalCatalog instanceof Closeable) {
+      try {
+        ((Closeable) originalCatalog).close();
+      } catch (IOException e) {
+        throw new CatalogException(e);
+      }
+    }
+  }
+
+  private Namespace toNamespace(String database) {
+    String[] namespace = new String[baseNamespace.length + 1];
+    System.arraycopy(baseNamespace, 0, namespace, 0, baseNamespace.length);
+    namespace[baseNamespace.length] = database;
+    return Namespace.of(namespace);
+  }
+
+  private TableIdentifier toIdentifier(ObjectPath path) {
+    return TableIdentifier.of(toNamespace(path.getDatabaseName()), path.getObjectName());
+  }
+
+  @Override
+  public List<String> listDatabases() throws CatalogException {
+    if (asNamespaceCatalog == null) {
+      return Collections.singletonList(getDefaultDatabase());
+    }
+
+    return listAllNamespaces(Namespace.empty()).stream()
+        .map(n -> n.level(n.levels().length - 1))
+        .collect(Collectors.toList());
+  }
+
+  private List<Namespace> listAllNamespaces(Namespace namespace) {
+    if (asNamespaceCatalog == null) {
+      throw new RuntimeException("The asNamespaceCatalog should not be null.");
+    }
+
+    String[] levels = namespace.levels();
+    if (levels.length == baseNamespace.length + 1) {
+      return Collections.singletonList(namespace);
+    }
+    if (levels.length < baseNamespace.length + 1) {
+      for (int i = 0; i < levels.length; i++) {
+        if (!baseNamespace[i].equals(levels[i])) {
+          return Collections.emptyList();
+        }
+      }
+      List<Namespace> ret = new ArrayList<>();
+      asNamespaceCatalog.listNamespaces(namespace).forEach(n -> ret.addAll(listAllNamespaces(n)));
+      return ret;
+    }
+    return Collections.emptyList();
+  }
+
+  @Override
+  public CatalogDatabase getDatabase(String databaseName) throws DatabaseNotExistException, CatalogException {
+    if (asNamespaceCatalog == null) {
+      if (!getDefaultDatabase().equals(databaseName)) {
+        throw new DatabaseNotExistException(getName(), databaseName);
+      } else {
+        return new CatalogDatabaseImpl(new HashMap<>(), "");
+      }
+    } else {
+      try {
+        Map<String, String> metadata =
+            new HashMap<>(asNamespaceCatalog.loadNamespaceMetadata(toNamespace(databaseName)));
+        String comment = metadata.remove("comment");
+        return new CatalogDatabaseImpl(metadata, comment);
+      } catch (NoSuchNamespaceException e) {
+        throw new DatabaseNotExistException(getName(), databaseName, e);
+      }
+    }
+  }
+
+  @Override
+  public boolean databaseExists(String databaseName) throws CatalogException {
+    try {
+      getDatabase(databaseName);
+      return true;
+    } catch (DatabaseNotExistException ignore) {
+      return false;
+    }
+  }
+
+  @Override
+  public void createDatabase(String name, CatalogDatabase database, boolean ignoreIfExists)
+      throws DatabaseAlreadyExistException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      try {
+        asNamespaceCatalog.createNamespace(
+            toNamespace(name),
+            mergeComment(database.getProperties(), database.getComment()));
+      } catch (AlreadyExistsException e) {
+        if (!ignoreIfExists) {
+          throw new DatabaseAlreadyExistException(getName(), name, e);
+        }
+      }
+    } else {
+      throw new UnsupportedOperationException("Namespaces are not supported by catalog: " + getName());
+    }
+  }
+
+  private Map<String, String> mergeComment(Map<String, String> metadata, String comment) {
+    Map<String, String> ret = new HashMap<>(metadata);
+    if (metadata.containsKey("comment")) {
+      throw new CatalogException("Database properties should not contain key: 'comment'.");
+    }
+    if (!StringUtils.isNullOrWhitespaceOnly(comment)) {
+      ret.put("comment", comment);
+    }
+    return ret;
+  }
+
+  @Override
+  public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade)
+      throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      try {
+        boolean success = asNamespaceCatalog.dropNamespace(toNamespace(name));
+        if (!success && !ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name);
+        }
+      } catch (NoSuchNamespaceException e) {
+        if (!ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name, e);
+        }
+      } catch (NamespaceNotEmptyException e) {
+        throw new DatabaseNotEmptyException(getName(), name, e);
+      }
+    } else {
+      if (!ignoreIfNotExists) {
+        throw new DatabaseNotExistException(getName(), name);
+      }
+    }
+  }
+
+  @Override
+  public void alterDatabase(String name, CatalogDatabase newDatabase, boolean ignoreIfNotExists)
+      throws DatabaseNotExistException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      Namespace namespace = toNamespace(name);
+      Map<String, String> updates = Maps.newHashMap();
+      Set<String> removals = Sets.newHashSet();
+
+      try {
+        Map<String, String> oldOptions = asNamespaceCatalog.loadNamespaceMetadata(namespace);
+        Map<String, String> newOptions = mergeComment(newDatabase.getProperties(), newDatabase.getComment());
+
+        for (String key : oldOptions.keySet()) {
+          if (!newOptions.containsKey(key)) {
+            removals.add(key);
+          }
+        }
+
+        for (Map.Entry<String, String> entry : newOptions.entrySet()) {
+          if (!entry.getValue().equals(oldOptions.get(entry.getKey()))) {
+            updates.put(entry.getKey(), entry.getValue());
+          }
+        }
+
+        if (!updates.isEmpty()) {
+          asNamespaceCatalog.setProperties(namespace, updates);
+        }
+
+        if (!removals.isEmpty()) {
+          asNamespaceCatalog.removeProperties(namespace, removals);
+        }
+
+      } catch (org.apache.iceberg.exceptions.NoSuchNamespaceException e) {
+        if (!ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name, e);
+        }
+      }
+    } else {
+      if (!ignoreIfNotExists) {

Review comment:
       Yes, we need handle default database, the correct behavior when the catalog doesn't support namespaces should throw a exception to tell users that the default database can not be altered.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java
##########
@@ -0,0 +1,508 @@
+/*
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.AbstractCatalog;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogDatabase;
+import org.apache.flink.table.catalog.CatalogDatabaseImpl;
+import org.apache.flink.table.catalog.CatalogFunction;
+import org.apache.flink.table.catalog.CatalogPartition;
+import org.apache.flink.table.catalog.CatalogPartitionSpec;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+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.DatabaseNotEmptyException;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
+import org.apache.flink.table.catalog.exceptions.FunctionNotExistException;
+import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.catalog.stats.CatalogColumnStatistics;
+import org.apache.flink.table.catalog.stats.CatalogTableStatistics;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.util.StringUtils;
+import org.apache.iceberg.CachingCatalog;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+
+/**
+ * A Flink Catalog implementation that wraps an Iceberg {@link Catalog}.
+ * <p>
+ * The mapping between Flink database and Iceberg namespace:
+ * Supplying a base namespace for a given catalog, so if you have a catalog that supports a 2-level namespace, you
+ * would supply the first level in the catalog configuration and the second level would be exposed as Flink databases.
+ * <p>
+ * The Iceberg table manages its partitions by itself. The partition of the Iceberg table is independent of the
+ * partition of Flink.
+ */
+public class FlinkCatalog extends AbstractCatalog {
+
+  private final Catalog originalCatalog;
+  private final Catalog icebergCatalog;
+  private final String[] baseNamespace;
+  private final SupportsNamespaces asNamespaceCatalog;
+
+  public FlinkCatalog(
+      String catalogName,
+      String defaultDatabase,
+      String[] baseNamespace,
+      Catalog icebergCatalog,
+      boolean cacheEnabled) {
+    super(catalogName, defaultDatabase);
+    this.originalCatalog = icebergCatalog;
+    this.icebergCatalog = cacheEnabled ? CachingCatalog.wrap(icebergCatalog) : icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    if (icebergCatalog instanceof SupportsNamespaces) {
+      asNamespaceCatalog = (SupportsNamespaces) icebergCatalog;
+    } else {
+      asNamespaceCatalog = null;
+    }
+  }
+
+  @Override
+  public void open() throws CatalogException {
+  }
+
+  @Override
+  public void close() throws CatalogException {
+    if (originalCatalog instanceof Closeable) {
+      try {
+        ((Closeable) originalCatalog).close();
+      } catch (IOException e) {
+        throw new CatalogException(e);
+      }
+    }
+  }
+
+  private Namespace toNamespace(String database) {
+    String[] namespace = new String[baseNamespace.length + 1];
+    System.arraycopy(baseNamespace, 0, namespace, 0, baseNamespace.length);
+    namespace[baseNamespace.length] = database;
+    return Namespace.of(namespace);
+  }
+
+  private TableIdentifier toIdentifier(ObjectPath path) {
+    return TableIdentifier.of(toNamespace(path.getDatabaseName()), path.getObjectName());
+  }
+
+  @Override
+  public List<String> listDatabases() throws CatalogException {
+    if (asNamespaceCatalog == null) {
+      return Collections.singletonList(getDefaultDatabase());
+    }
+
+    return listAllNamespaces(Namespace.empty()).stream()
+        .map(n -> n.level(n.levels().length - 1))
+        .collect(Collectors.toList());
+  }
+
+  private List<Namespace> listAllNamespaces(Namespace namespace) {
+    if (asNamespaceCatalog == null) {
+      throw new RuntimeException("The asNamespaceCatalog should not be null.");
+    }
+
+    String[] levels = namespace.levels();
+    if (levels.length == baseNamespace.length + 1) {
+      return Collections.singletonList(namespace);
+    }
+    if (levels.length < baseNamespace.length + 1) {
+      for (int i = 0; i < levels.length; i++) {
+        if (!baseNamespace[i].equals(levels[i])) {
+          return Collections.emptyList();
+        }
+      }
+      List<Namespace> ret = new ArrayList<>();
+      asNamespaceCatalog.listNamespaces(namespace).forEach(n -> ret.addAll(listAllNamespaces(n)));
+      return ret;
+    }
+    return Collections.emptyList();
+  }
+
+  @Override
+  public CatalogDatabase getDatabase(String databaseName) throws DatabaseNotExistException, CatalogException {
+    if (asNamespaceCatalog == null) {
+      if (!getDefaultDatabase().equals(databaseName)) {
+        throw new DatabaseNotExistException(getName(), databaseName);
+      } else {
+        return new CatalogDatabaseImpl(new HashMap<>(), "");
+      }
+    } else {
+      try {
+        Map<String, String> metadata =
+            new HashMap<>(asNamespaceCatalog.loadNamespaceMetadata(toNamespace(databaseName)));
+        String comment = metadata.remove("comment");
+        return new CatalogDatabaseImpl(metadata, comment);
+      } catch (NoSuchNamespaceException e) {
+        throw new DatabaseNotExistException(getName(), databaseName, e);
+      }
+    }
+  }
+
+  @Override
+  public boolean databaseExists(String databaseName) throws CatalogException {
+    try {
+      getDatabase(databaseName);
+      return true;
+    } catch (DatabaseNotExistException ignore) {
+      return false;
+    }
+  }
+
+  @Override
+  public void createDatabase(String name, CatalogDatabase database, boolean ignoreIfExists)
+      throws DatabaseAlreadyExistException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      try {
+        asNamespaceCatalog.createNamespace(
+            toNamespace(name),
+            mergeComment(database.getProperties(), database.getComment()));
+      } catch (AlreadyExistsException e) {
+        if (!ignoreIfExists) {
+          throw new DatabaseAlreadyExistException(getName(), name, e);
+        }
+      }
+    } else {
+      throw new UnsupportedOperationException("Namespaces are not supported by catalog: " + getName());
+    }
+  }
+
+  private Map<String, String> mergeComment(Map<String, String> metadata, String comment) {
+    Map<String, String> ret = new HashMap<>(metadata);
+    if (metadata.containsKey("comment")) {
+      throw new CatalogException("Database properties should not contain key: 'comment'.");
+    }
+    if (!StringUtils.isNullOrWhitespaceOnly(comment)) {
+      ret.put("comment", comment);
+    }
+    return ret;
+  }
+
+  @Override
+  public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade)
+      throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      try {
+        boolean success = asNamespaceCatalog.dropNamespace(toNamespace(name));
+        if (!success && !ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name);
+        }
+      } catch (NoSuchNamespaceException e) {
+        if (!ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name, e);
+        }
+      } catch (NamespaceNotEmptyException e) {
+        throw new DatabaseNotEmptyException(getName(), name, e);
+      }
+    } else {
+      if (!ignoreIfNotExists) {
+        throw new DatabaseNotExistException(getName(), name);
+      }
+    }
+  }
+
+  @Override
+  public void alterDatabase(String name, CatalogDatabase newDatabase, boolean ignoreIfNotExists)
+      throws DatabaseNotExistException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      Namespace namespace = toNamespace(name);
+      Map<String, String> updates = Maps.newHashMap();
+      Set<String> removals = Sets.newHashSet();
+
+      try {
+        Map<String, String> oldOptions = asNamespaceCatalog.loadNamespaceMetadata(namespace);
+        Map<String, String> newOptions = mergeComment(newDatabase.getProperties(), newDatabase.getComment());
+
+        for (String key : oldOptions.keySet()) {
+          if (!newOptions.containsKey(key)) {
+            removals.add(key);
+          }
+        }
+
+        for (Map.Entry<String, String> entry : newOptions.entrySet()) {
+          if (!entry.getValue().equals(oldOptions.get(entry.getKey()))) {
+            updates.put(entry.getKey(), entry.getValue());
+          }
+        }
+
+        if (!updates.isEmpty()) {
+          asNamespaceCatalog.setProperties(namespace, updates);
+        }
+
+        if (!removals.isEmpty()) {
+          asNamespaceCatalog.removeProperties(namespace, removals);
+        }
+
+      } catch (org.apache.iceberg.exceptions.NoSuchNamespaceException e) {
+        if (!ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name, e);
+        }
+      }
+    } else {
+      if (!ignoreIfNotExists) {

Review comment:
       Yes, we need handle default database, the correct behavior when the catalog doesn't support namespaces should throw an exception to tell users that the default database can not be altered.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453111418



##########
File path: flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.List;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.BinaryType;
+import org.apache.flink.table.types.logical.BooleanType;
+import org.apache.flink.table.types.logical.CharType;
+import org.apache.flink.table.types.logical.DateType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.DoubleType;
+import org.apache.flink.table.types.logical.FloatType;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.LocalZonedTimestampType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.TimeType;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.VarBinaryType;
+import org.apache.flink.table.types.logical.VarCharType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types;
+
+class TypeToFlinkType extends TypeUtil.SchemaVisitor<LogicalType> {
+  TypeToFlinkType() {
+  }
+
+  @Override
+  public LogicalType schema(Schema schema, LogicalType structType) {
+    return structType;
+  }
+
+  @Override
+  public LogicalType struct(Types.StructType struct, List<LogicalType> fieldResults) {
+    List<Types.NestedField> fields = struct.fields();
+
+    List<RowType.RowField> flinkFields = Lists.newArrayListWithExpectedSize(fieldResults.size());
+    for (int i = 0; i < fields.size(); i += 1) {
+      Types.NestedField field = fields.get(i);
+      LogicalType type = fieldResults.get(i);
+      RowType.RowField flinkField = new RowType.RowField(
+          field.name(), type.copy(field.isOptional()), field.doc());
+      flinkFields.add(flinkField);
+    }
+
+    return new RowType(flinkFields);
+  }
+
+  @Override
+  public LogicalType field(Types.NestedField field, LogicalType fieldResult) {
+    return fieldResult;
+  }
+
+  @Override
+  public LogicalType list(Types.ListType list, LogicalType elementResult) {
+    return new ArrayType(elementResult.copy(list.isElementOptional()));
+  }
+
+  @Override
+  public LogicalType map(Types.MapType map, LogicalType keyResult, LogicalType valueResult) {
+    // keys in map are not allowed to be null.
+    return new MapType(keyResult.copy(false), valueResult.copy(map.isValueOptional()));
+  }
+
+  @Override
+  public LogicalType primitive(Type.PrimitiveType primitive) {
+    switch (primitive.typeId()) {
+      case BOOLEAN:
+        return new BooleanType();
+      case INTEGER:
+        return new IntType();
+      case LONG:
+        return new BigIntType();
+      case FLOAT:
+        return new FloatType();
+      case DOUBLE:
+        return new DoubleType();
+      case DATE:
+        return new DateType();
+      case TIME:
+        // MICROS
+        return new TimeType(6);
+      case TIMESTAMP:
+        Types.TimestampType timestamp = (Types.TimestampType) primitive;
+        if (timestamp.shouldAdjustToUTC()) {
+          // MICROS
+          return new LocalZonedTimestampType(6);
+        } else {
+          // MICROS
+          return new TimestampType(6);
+        }
+      case STRING:
+        return new VarCharType(VarCharType.MAX_LENGTH);
+      case UUID:
+        // UUID length is 16
+        return new CharType(16);

Review comment:
       `Char`? Wouldn't this be fixed-length binary?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453404061



##########
File path: flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.List;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.BinaryType;
+import org.apache.flink.table.types.logical.BooleanType;
+import org.apache.flink.table.types.logical.CharType;
+import org.apache.flink.table.types.logical.DateType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.DoubleType;
+import org.apache.flink.table.types.logical.FloatType;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.LocalZonedTimestampType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.TimeType;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.VarBinaryType;
+import org.apache.flink.table.types.logical.VarCharType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types;
+
+class TypeToFlinkType extends TypeUtil.SchemaVisitor<LogicalType> {
+  TypeToFlinkType() {
+  }
+
+  @Override
+  public LogicalType schema(Schema schema, LogicalType structType) {
+    return structType;
+  }
+
+  @Override
+  public LogicalType struct(Types.StructType struct, List<LogicalType> fieldResults) {
+    List<Types.NestedField> fields = struct.fields();
+
+    List<RowType.RowField> flinkFields = Lists.newArrayListWithExpectedSize(fieldResults.size());
+    for (int i = 0; i < fields.size(); i += 1) {
+      Types.NestedField field = fields.get(i);
+      LogicalType type = fieldResults.get(i);
+      RowType.RowField flinkField = new RowType.RowField(
+          field.name(), type.copy(field.isOptional()), field.doc());
+      flinkFields.add(flinkField);
+    }
+
+    return new RowType(flinkFields);
+  }
+
+  @Override
+  public LogicalType field(Types.NestedField field, LogicalType fieldResult) {
+    return fieldResult;
+  }
+
+  @Override
+  public LogicalType list(Types.ListType list, LogicalType elementResult) {
+    return new ArrayType(elementResult.copy(list.isElementOptional()));
+  }
+
+  @Override
+  public LogicalType map(Types.MapType map, LogicalType keyResult, LogicalType valueResult) {
+    // keys in map are not allowed to be null.
+    return new MapType(keyResult.copy(false), valueResult.copy(map.isValueOptional()));
+  }
+
+  @Override
+  public LogicalType primitive(Type.PrimitiveType primitive) {
+    switch (primitive.typeId()) {
+      case BOOLEAN:
+        return new BooleanType();
+      case INTEGER:
+        return new IntType();
+      case LONG:
+        return new BigIntType();
+      case FLOAT:
+        return new FloatType();
+      case DOUBLE:
+        return new DoubleType();
+      case DATE:
+        return new DateType();
+      case TIME:
+        // MICROS
+        return new TimeType(6);
+      case TIMESTAMP:
+        Types.TimestampType timestamp = (Types.TimestampType) primitive;
+        if (timestamp.shouldAdjustToUTC()) {
+          // MICROS
+          return new LocalZonedTimestampType(6);
+        } else {
+          // MICROS
+          return new TimestampType(6);
+        }
+      case STRING:
+        return new VarCharType(VarCharType.MAX_LENGTH);
+      case UUID:
+        // UUID length is 16
+        return new CharType(16);

Review comment:
       I think UUID should be a `Char` with 36 precision. Like:
   - In Spark, UUID function returns `StringType`.
   - In Flink, UUID function returns `CharType` with 36 precision.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1182: Flink: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r457716399



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java
##########
@@ -0,0 +1,508 @@
+/*
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.AbstractCatalog;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogDatabase;
+import org.apache.flink.table.catalog.CatalogDatabaseImpl;
+import org.apache.flink.table.catalog.CatalogFunction;
+import org.apache.flink.table.catalog.CatalogPartition;
+import org.apache.flink.table.catalog.CatalogPartitionSpec;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+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.DatabaseNotEmptyException;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
+import org.apache.flink.table.catalog.exceptions.FunctionNotExistException;
+import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.catalog.stats.CatalogColumnStatistics;
+import org.apache.flink.table.catalog.stats.CatalogTableStatistics;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.util.StringUtils;
+import org.apache.iceberg.CachingCatalog;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+
+/**
+ * A Flink Catalog implementation that wraps an Iceberg {@link Catalog}.
+ * <p>
+ * The mapping between Flink database and Iceberg namespace:
+ * Supplying a base namespace for a given catalog, so if you have a catalog that supports a 2-level namespace, you
+ * would supply the first level in the catalog configuration and the second level would be exposed as Flink databases.
+ * <p>
+ * The Iceberg table manages its partitions by itself. The partition of the Iceberg table is independent of the
+ * partition of Flink.
+ */
+public class FlinkCatalog extends AbstractCatalog {
+
+  private final Catalog originalCatalog;
+  private final Catalog icebergCatalog;
+  private final String[] baseNamespace;
+  private final SupportsNamespaces asNamespaceCatalog;
+
+  public FlinkCatalog(
+      String catalogName,
+      String defaultDatabase,
+      String[] baseNamespace,
+      Catalog icebergCatalog,
+      boolean cacheEnabled) {
+    super(catalogName, defaultDatabase);
+    this.originalCatalog = icebergCatalog;
+    this.icebergCatalog = cacheEnabled ? CachingCatalog.wrap(icebergCatalog) : icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    if (icebergCatalog instanceof SupportsNamespaces) {
+      asNamespaceCatalog = (SupportsNamespaces) icebergCatalog;
+    } else {
+      asNamespaceCatalog = null;
+    }
+  }
+
+  @Override
+  public void open() throws CatalogException {
+  }
+
+  @Override
+  public void close() throws CatalogException {
+    if (originalCatalog instanceof Closeable) {
+      try {
+        ((Closeable) originalCatalog).close();
+      } catch (IOException e) {
+        throw new CatalogException(e);
+      }
+    }
+  }
+
+  private Namespace toNamespace(String database) {
+    String[] namespace = new String[baseNamespace.length + 1];
+    System.arraycopy(baseNamespace, 0, namespace, 0, baseNamespace.length);
+    namespace[baseNamespace.length] = database;
+    return Namespace.of(namespace);
+  }
+
+  private TableIdentifier toIdentifier(ObjectPath path) {
+    return TableIdentifier.of(toNamespace(path.getDatabaseName()), path.getObjectName());
+  }
+
+  @Override
+  public List<String> listDatabases() throws CatalogException {
+    if (asNamespaceCatalog == null) {
+      return Collections.singletonList(getDefaultDatabase());
+    }
+
+    return listAllNamespaces(Namespace.empty()).stream()
+        .map(n -> n.level(n.levels().length - 1))
+        .collect(Collectors.toList());
+  }
+
+  private List<Namespace> listAllNamespaces(Namespace namespace) {
+    if (asNamespaceCatalog == null) {
+      throw new RuntimeException("The asNamespaceCatalog should not be null.");
+    }
+
+    String[] levels = namespace.levels();
+    if (levels.length == baseNamespace.length + 1) {
+      return Collections.singletonList(namespace);
+    }
+    if (levels.length < baseNamespace.length + 1) {
+      for (int i = 0; i < levels.length; i++) {
+        if (!baseNamespace[i].equals(levels[i])) {
+          return Collections.emptyList();
+        }
+      }
+      List<Namespace> ret = new ArrayList<>();
+      asNamespaceCatalog.listNamespaces(namespace).forEach(n -> ret.addAll(listAllNamespaces(n)));
+      return ret;
+    }
+    return Collections.emptyList();
+  }
+
+  @Override
+  public CatalogDatabase getDatabase(String databaseName) throws DatabaseNotExistException, CatalogException {
+    if (asNamespaceCatalog == null) {
+      if (!getDefaultDatabase().equals(databaseName)) {
+        throw new DatabaseNotExistException(getName(), databaseName);
+      } else {
+        return new CatalogDatabaseImpl(new HashMap<>(), "");
+      }
+    } else {
+      try {
+        Map<String, String> metadata =
+            new HashMap<>(asNamespaceCatalog.loadNamespaceMetadata(toNamespace(databaseName)));
+        String comment = metadata.remove("comment");
+        return new CatalogDatabaseImpl(metadata, comment);
+      } catch (NoSuchNamespaceException e) {
+        throw new DatabaseNotExistException(getName(), databaseName, e);
+      }
+    }
+  }
+
+  @Override
+  public boolean databaseExists(String databaseName) throws CatalogException {
+    try {
+      getDatabase(databaseName);
+      return true;
+    } catch (DatabaseNotExistException ignore) {
+      return false;
+    }
+  }
+
+  @Override
+  public void createDatabase(String name, CatalogDatabase database, boolean ignoreIfExists)
+      throws DatabaseAlreadyExistException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      try {
+        asNamespaceCatalog.createNamespace(
+            toNamespace(name),
+            mergeComment(database.getProperties(), database.getComment()));
+      } catch (AlreadyExistsException e) {
+        if (!ignoreIfExists) {
+          throw new DatabaseAlreadyExistException(getName(), name, e);
+        }
+      }
+    } else {
+      throw new UnsupportedOperationException("Namespaces are not supported by catalog: " + getName());
+    }
+  }
+
+  private Map<String, String> mergeComment(Map<String, String> metadata, String comment) {
+    Map<String, String> ret = new HashMap<>(metadata);
+    if (metadata.containsKey("comment")) {
+      throw new CatalogException("Database properties should not contain key: 'comment'.");
+    }
+    if (!StringUtils.isNullOrWhitespaceOnly(comment)) {
+      ret.put("comment", comment);
+    }
+    return ret;
+  }
+
+  @Override
+  public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade)
+      throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      try {
+        boolean success = asNamespaceCatalog.dropNamespace(toNamespace(name));
+        if (!success && !ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name);
+        }
+      } catch (NoSuchNamespaceException e) {
+        if (!ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name, e);
+        }
+      } catch (NamespaceNotEmptyException e) {
+        throw new DatabaseNotEmptyException(getName(), name, e);
+      }
+    } else {
+      if (!ignoreIfNotExists) {
+        throw new DatabaseNotExistException(getName(), name);
+      }
+    }
+  }
+
+  @Override
+  public void alterDatabase(String name, CatalogDatabase newDatabase, boolean ignoreIfNotExists)
+      throws DatabaseNotExistException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      Namespace namespace = toNamespace(name);
+      Map<String, String> updates = Maps.newHashMap();
+      Set<String> removals = Sets.newHashSet();
+
+      try {
+        Map<String, String> oldOptions = asNamespaceCatalog.loadNamespaceMetadata(namespace);
+        Map<String, String> newOptions = mergeComment(newDatabase.getProperties(), newDatabase.getComment());
+
+        for (String key : oldOptions.keySet()) {
+          if (!newOptions.containsKey(key)) {
+            removals.add(key);
+          }
+        }
+
+        for (Map.Entry<String, String> entry : newOptions.entrySet()) {
+          if (!entry.getValue().equals(oldOptions.get(entry.getKey()))) {
+            updates.put(entry.getKey(), entry.getValue());
+          }
+        }
+
+        if (!updates.isEmpty()) {
+          asNamespaceCatalog.setProperties(namespace, updates);
+        }
+
+        if (!removals.isEmpty()) {
+          asNamespaceCatalog.removeProperties(namespace, removals);
+        }
+
+      } catch (org.apache.iceberg.exceptions.NoSuchNamespaceException e) {
+        if (!ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name, e);
+        }
+      }
+    } else {
+      if (!ignoreIfNotExists) {
+        throw new DatabaseNotExistException(getName(), name);
+      }
+    }
+  }
+
+  @Override
+  public List<String> listTables(String databaseName) throws DatabaseNotExistException, CatalogException {
+    try {
+      return icebergCatalog.listTables(toNamespace(databaseName)).stream()
+          .map(TableIdentifier::name)
+          .collect(Collectors.toList());
+    } catch (NoSuchNamespaceException e) {
+      throw new DatabaseNotExistException(getName(), databaseName, e);
+    }
+  }
+
+  @Override
+  public CatalogBaseTable getTable(ObjectPath tablePath) throws TableNotExistException, CatalogException {
+    try {
+      Table table = icebergCatalog.loadTable(toIdentifier(tablePath));
+      TableSchema tableSchema = FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(table.schema()));
+
+      // NOTE: We can not create a IcebergCatalogTable, because Flink optimizer may use CatalogTableImpl to copy a new
+      // catalog table.
+      // Let's re-loading table from Iceberg catalog when creating source/sink operators.
+      return new CatalogTableImpl(tableSchema, table.properties(), "");
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      throw new TableNotExistException(getName(), tablePath, e);
+    }
+  }
+
+  @Override
+  public boolean tableExists(ObjectPath tablePath) throws CatalogException {
+    return icebergCatalog.tableExists(toIdentifier(tablePath));
+  }
+
+  @Override
+  public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists)
+      throws TableNotExistException, CatalogException {
+    try {
+      icebergCatalog.dropTable(toIdentifier(tablePath));
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      throw new TableNotExistException(getName(), tablePath, e);
+    }
+  }
+
+  @Override
+  public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists)
+      throws TableNotExistException, TableAlreadyExistException, CatalogException {
+    try {
+      icebergCatalog.renameTable(
+          toIdentifier(tablePath),
+          toIdentifier(new ObjectPath(tablePath.getDatabaseName(), newTableName)));
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      throw new TableNotExistException(getName(), tablePath, e);
+    } catch (AlreadyExistsException e) {
+      throw new TableAlreadyExistException(getName(), tablePath, e);
+    }
+  }
+
+  /**
+   * TODO Implement DDL-string parser for PartitionSpec.

Review comment:
       Good idea, but there are a couple of things to watch out for:
   
   * Where possible, we avoid exposing the actual partition values, in order to maintain a separation between logical queries and physical layout. That way, the physical layout can change, but the logical queries will continue to work. In this case, we would need to make sure that the computed columns are tracked separately so that we don't drop the `day` column when the table gets converted to partitioning by `hour`.
   * Year, month, and day are functions with concrete behavior for Flink SQL, and Iceberg's partitioning may not align with that behavior. So we probably would not want to supply the data for those columns using Iceberg partition values. Instead, I think we should derive them from the `dt` field.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453429540



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java
##########
@@ -0,0 +1,241 @@
+/*
+ * 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.File;
+import java.util.Map;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.types.Types;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Test;
+
+public class TestFlinkCatalogDatabase extends FlinkCatalogTestBase {
+
+  public TestFlinkCatalogDatabase(String catalogName, String[] baseNamepace) {
+    super(catalogName, baseNamepace);
+  }
+
+  @After
+  public void clean() {
+    sql("DROP TABLE IF EXISTS %s.tl", flinkIdentifier);
+    sql("DROP DATABASE IF EXISTS %s", flinkIdentifier);
+  }
+
+  @Test
+  public void testCreateNamespace() {
+    Assert.assertFalse(
+        "Database should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+
+    Assert.assertTrue("Database should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+  }
+
+  @Test
+  public void testDefaultDatabase() {
+    sql("USE CATALOG %s", catalogName);
+
+    Assert.assertEquals("Should use the current catalog", tEnv.getCurrentCatalog(), catalogName);
+    Assert.assertEquals("Should use the configured default namespace", tEnv.getCurrentDatabase(), "default");
+  }
+
+  @Test
+  public void testDropEmptyDatabase() {
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("DROP DATABASE %s", flinkIdentifier);
+
+    Assert.assertFalse(
+        "Namespace should have been dropped",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+  }
+
+  @Test
+  public void testDropNonEmptyNamespace() {
+    Assume.assumeFalse("Hadoop catalog throws IOException: Directory is not empty.", isHadoopCatalog);
+
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+
+    validationCatalog.createTable(
+        TableIdentifier.of(icebergNamespace, "tl"),
+        new Schema(Types.NestedField.optional(0, "id", Types.LongType.get())));
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+    Assert.assertTrue("Table should exist", validationCatalog.tableExists(TableIdentifier.of(icebergNamespace, "tl")));
+
+    AssertHelpers.assertThrowsCause(
+        "Should fail if trying to delete a non-empty database",
+        DatabaseNotEmptyException.class,
+        String.format("Database %s in catalog %s is not empty.", DATABASE, catalogName),
+        () -> sql("DROP DATABASE %s", flinkIdentifier));
+
+    sql("DROP TABLE %s.tl", flinkIdentifier);
+  }
+
+  @Test
+  public void testListTables() {
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+    sql("USE CATALOG %s", catalogName);
+    sql("USE %s", DATABASE);
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    Assert.assertEquals("Should not list any tables", 0, tEnv.listTables().length);
+
+    validationCatalog.createTable(
+        TableIdentifier.of(icebergNamespace, "tl"),
+        new Schema(Types.NestedField.optional(0, "id", Types.LongType.get())));
+
+    Assert.assertEquals("Only 1 table", 1, tEnv.listTables().length);
+    Assert.assertEquals("Table name should match", "tl", tEnv.listTables()[0]);
+  }
+
+  @Test
+  public void testListNamespace() {
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+    sql("USE CATALOG %s", catalogName);
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    String[] databases = tEnv.listDatabases();
+
+    if (isHadoopCatalog) {
+      Assert.assertEquals("Should have 1 database", 1, databases.length);
+      Assert.assertEquals("Should have only db database", "db", databases[0]);
+
+      if (baseNamespace.length > 0) {
+        // test namespace not belongs to this catalog
+        validationNamespaceCatalog.createNamespace(Namespace.of(baseNamespace[0], "UNKNOWN_NAMESPACE"));
+        databases = tEnv.listDatabases();
+        Assert.assertEquals("Should have 1 database", 1, databases.length);
+        Assert.assertEquals("Should have only db database", "db", databases[0]);
+      }
+    } else {
+      Assert.assertEquals("Should have 2 databases", 2, databases.length);
+      Assert.assertEquals(
+          "Should have default and db databases",
+          ImmutableSet.of("default", "db"),
+          ImmutableSet.copyOf(databases));
+    }
+  }
+
+  @Test
+  public void testCreateNamespaceWithMetadata() {
+    Assume.assumeFalse("HadoopCatalog does not support namespace metadata", isHadoopCatalog);
+
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s WITH ('prop'='value')", flinkIdentifier);
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    Map<String, String> nsMetadata = validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace);
+
+    Assert.assertEquals("Namespace should have expected prop value", "value", nsMetadata.get("prop"));
+  }
+
+  @Test
+  public void testCreateNamespaceWithComment() {
+    Assume.assumeFalse("HadoopCatalog does not support namespace metadata", isHadoopCatalog);
+
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s COMMENT 'namespace doc'", flinkIdentifier);
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    Map<String, String> nsMetadata = validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace);
+
+    Assert.assertEquals("Namespace should have expected comment", "namespace doc", nsMetadata.get("comment"));
+  }
+
+  @Test
+  public void testCreateNamespaceWithLocation() throws Exception {
+    Assume.assumeFalse("HadoopCatalog does not support namespace locations", isHadoopCatalog);
+
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    File location = TEMPORARY_FOLDER.newFile();

Review comment:
       `TEMPORARY_FOLDER` is from Flink test base class `AbstractTestBase`.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453110670



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.runtime.util.HadoopUtils;
+import org.apache.flink.table.catalog.Catalog;
+import org.apache.flink.table.descriptors.CatalogDescriptorValidator;
+import org.apache.flink.table.factories.CatalogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.hive.HiveCatalog;
+import org.apache.iceberg.relocated.com.google.common.base.Splitter;
+
+/**
+ * A Flink Catalog factory implementation that creates {@link FlinkCatalog}.
+ * <p>
+ * This supports the following catalog configuration options:
+ * <ul>
+ *   <li><tt>type</tt> - Flink catalog factory key, should be "iceberg"</li>
+ *   <li><tt>catalog-type</tt> - iceberg catalog type, "hive" or "hadoop"</li>
+ *   <li><tt>uri</tt> - the Hive Metastore URI (Hive catalog only)</li>
+ *   <li><tt>clients</tt> - the Hive Client Pool Size (Hive catalog only)</li>
+ *   <li><tt>warehouse</tt> - the warehouse path (Hadoop catalog only)</li>
+ *   <li><tt>default-database</tt> - a database name to use as the default</li>
+ *   <li><tt>base-namespace</tt> - a base namespace as the prefix for all databases</li>

Review comment:
       This should also be labelled `(Hadoop catalog only)` because the Hive catalog supports only database.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r454054919



##########
File path: flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.List;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.BinaryType;
+import org.apache.flink.table.types.logical.BooleanType;
+import org.apache.flink.table.types.logical.CharType;
+import org.apache.flink.table.types.logical.DateType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.DoubleType;
+import org.apache.flink.table.types.logical.FloatType;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.LocalZonedTimestampType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.TimeType;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.VarBinaryType;
+import org.apache.flink.table.types.logical.VarCharType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types;
+
+class TypeToFlinkType extends TypeUtil.SchemaVisitor<LogicalType> {
+  TypeToFlinkType() {
+  }
+
+  @Override
+  public LogicalType schema(Schema schema, LogicalType structType) {
+    return structType;
+  }
+
+  @Override
+  public LogicalType struct(Types.StructType struct, List<LogicalType> fieldResults) {
+    List<Types.NestedField> fields = struct.fields();
+
+    List<RowType.RowField> flinkFields = Lists.newArrayListWithExpectedSize(fieldResults.size());
+    for (int i = 0; i < fields.size(); i += 1) {
+      Types.NestedField field = fields.get(i);
+      LogicalType type = fieldResults.get(i);
+      RowType.RowField flinkField = new RowType.RowField(
+          field.name(), type.copy(field.isOptional()), field.doc());
+      flinkFields.add(flinkField);
+    }
+
+    return new RowType(flinkFields);
+  }
+
+  @Override
+  public LogicalType field(Types.NestedField field, LogicalType fieldResult) {
+    return fieldResult;
+  }
+
+  @Override
+  public LogicalType list(Types.ListType list, LogicalType elementResult) {
+    return new ArrayType(elementResult.copy(list.isElementOptional()));
+  }
+
+  @Override
+  public LogicalType map(Types.MapType map, LogicalType keyResult, LogicalType valueResult) {
+    // keys in map are not allowed to be null.
+    return new MapType(keyResult.copy(false), valueResult.copy(map.isValueOptional()));
+  }
+
+  @Override
+  public LogicalType primitive(Type.PrimitiveType primitive) {
+    switch (primitive.typeId()) {
+      case BOOLEAN:
+        return new BooleanType();
+      case INTEGER:
+        return new IntType();
+      case LONG:
+        return new BigIntType();
+      case FLOAT:
+        return new FloatType();
+      case DOUBLE:
+        return new DoubleType();
+      case DATE:
+        return new DateType();
+      case TIME:
+        // MICROS
+        return new TimeType(6);
+      case TIMESTAMP:
+        Types.TimestampType timestamp = (Types.TimestampType) primitive;
+        if (timestamp.shouldAdjustToUTC()) {
+          // MICROS
+          return new LocalZonedTimestampType(6);
+        } else {
+          // MICROS
+          return new TimestampType(6);
+        }
+      case STRING:
+        return new VarCharType(VarCharType.MAX_LENGTH);
+      case UUID:
+        // UUID length is 16
+        return new CharType(16);

Review comment:
       I choose fixed-length binary(16).




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453112859



##########
File path: flink/src/test/java/org/apache/iceberg/flink/FlinkCatalogTestBase.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.util.ArrayUtils;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public abstract class FlinkCatalogTestBase extends FlinkTestBase {
+
+  protected static final String DATABASE = "db";
+  private static File warehouse = null;
+
+  @BeforeClass
+  public static void createWarehouse() throws IOException {
+    FlinkCatalogTestBase.warehouse = File.createTempFile("warehouse", null);
+    Assert.assertTrue(warehouse.delete());
+  }
+
+  @AfterClass
+  public static void dropWarehouse() {
+    if (warehouse != null && warehouse.exists()) {
+      warehouse.delete();
+    }
+  }
+
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { "testhive", new String[0] },
+        new Object[] { "testhadoop", new String[0] },
+        new Object[] { "testhadoop", new String[] { "l0", "l1" }},
+    };
+  }
+
+  protected final TableEnvironment tEnv =
+      TableEnvironment.create(EnvironmentSettings.newInstance().useBlinkPlanner().inBatchMode().build());
+
+  protected final String catalogName;
+  protected final String[] baseNamespace;
+  protected final Catalog validationCatalog;
+  protected final SupportsNamespaces validationNamespaceCatalog;
+  protected final org.apache.flink.table.catalog.Catalog flinkCatalog;
+
+  protected final String flinkIdentifier;

Review comment:
       It isn't clear from this name that this is for a database. How about renaming it to `flinkDatabase`?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453105019



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java
##########
@@ -0,0 +1,508 @@
+/*
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.AbstractCatalog;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogDatabase;
+import org.apache.flink.table.catalog.CatalogDatabaseImpl;
+import org.apache.flink.table.catalog.CatalogFunction;
+import org.apache.flink.table.catalog.CatalogPartition;
+import org.apache.flink.table.catalog.CatalogPartitionSpec;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+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.DatabaseNotEmptyException;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
+import org.apache.flink.table.catalog.exceptions.FunctionNotExistException;
+import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.catalog.stats.CatalogColumnStatistics;
+import org.apache.flink.table.catalog.stats.CatalogTableStatistics;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.util.StringUtils;
+import org.apache.iceberg.CachingCatalog;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+
+/**
+ * A Flink Catalog implementation that wraps an Iceberg {@link Catalog}.
+ * <p>
+ * The mapping between Flink database and Iceberg namespace:
+ * Supplying a base namespace for a given catalog, so if you have a catalog that supports a 2-level namespace, you
+ * would supply the first level in the catalog configuration and the second level would be exposed as Flink databases.
+ * <p>
+ * The Iceberg table manages its partitions by itself. The partition of the Iceberg table is independent of the
+ * partition of Flink.
+ */
+public class FlinkCatalog extends AbstractCatalog {
+
+  private final Catalog originalCatalog;
+  private final Catalog icebergCatalog;
+  private final String[] baseNamespace;
+  private final SupportsNamespaces asNamespaceCatalog;
+
+  public FlinkCatalog(
+      String catalogName,
+      String defaultDatabase,
+      String[] baseNamespace,
+      Catalog icebergCatalog,
+      boolean cacheEnabled) {
+    super(catalogName, defaultDatabase);
+    this.originalCatalog = icebergCatalog;
+    this.icebergCatalog = cacheEnabled ? CachingCatalog.wrap(icebergCatalog) : icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    if (icebergCatalog instanceof SupportsNamespaces) {
+      asNamespaceCatalog = (SupportsNamespaces) icebergCatalog;
+    } else {
+      asNamespaceCatalog = null;
+    }
+  }
+
+  @Override
+  public void open() throws CatalogException {
+  }
+
+  @Override
+  public void close() throws CatalogException {
+    if (originalCatalog instanceof Closeable) {
+      try {
+        ((Closeable) originalCatalog).close();
+      } catch (IOException e) {
+        throw new CatalogException(e);
+      }
+    }
+  }
+
+  private Namespace toNamespace(String database) {
+    String[] namespace = new String[baseNamespace.length + 1];
+    System.arraycopy(baseNamespace, 0, namespace, 0, baseNamespace.length);
+    namespace[baseNamespace.length] = database;
+    return Namespace.of(namespace);
+  }
+
+  private TableIdentifier toIdentifier(ObjectPath path) {
+    return TableIdentifier.of(toNamespace(path.getDatabaseName()), path.getObjectName());
+  }
+
+  @Override
+  public List<String> listDatabases() throws CatalogException {
+    if (asNamespaceCatalog == null) {
+      return Collections.singletonList(getDefaultDatabase());
+    }
+
+    return listAllNamespaces(Namespace.empty()).stream()
+        .map(n -> n.level(n.levels().length - 1))
+        .collect(Collectors.toList());
+  }
+
+  private List<Namespace> listAllNamespaces(Namespace namespace) {
+    if (asNamespaceCatalog == null) {
+      throw new RuntimeException("The asNamespaceCatalog should not be null.");

Review comment:
       We would normally use `Preconditions.checkArgument` in this case.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453108883



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java
##########
@@ -0,0 +1,508 @@
+/*
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.AbstractCatalog;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogDatabase;
+import org.apache.flink.table.catalog.CatalogDatabaseImpl;
+import org.apache.flink.table.catalog.CatalogFunction;
+import org.apache.flink.table.catalog.CatalogPartition;
+import org.apache.flink.table.catalog.CatalogPartitionSpec;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+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.DatabaseNotEmptyException;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
+import org.apache.flink.table.catalog.exceptions.FunctionNotExistException;
+import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.catalog.stats.CatalogColumnStatistics;
+import org.apache.flink.table.catalog.stats.CatalogTableStatistics;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.util.StringUtils;
+import org.apache.iceberg.CachingCatalog;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+
+/**
+ * A Flink Catalog implementation that wraps an Iceberg {@link Catalog}.
+ * <p>
+ * The mapping between Flink database and Iceberg namespace:
+ * Supplying a base namespace for a given catalog, so if you have a catalog that supports a 2-level namespace, you
+ * would supply the first level in the catalog configuration and the second level would be exposed as Flink databases.
+ * <p>
+ * The Iceberg table manages its partitions by itself. The partition of the Iceberg table is independent of the
+ * partition of Flink.
+ */
+public class FlinkCatalog extends AbstractCatalog {
+
+  private final Catalog originalCatalog;
+  private final Catalog icebergCatalog;
+  private final String[] baseNamespace;
+  private final SupportsNamespaces asNamespaceCatalog;
+
+  public FlinkCatalog(
+      String catalogName,
+      String defaultDatabase,
+      String[] baseNamespace,
+      Catalog icebergCatalog,
+      boolean cacheEnabled) {
+    super(catalogName, defaultDatabase);
+    this.originalCatalog = icebergCatalog;
+    this.icebergCatalog = cacheEnabled ? CachingCatalog.wrap(icebergCatalog) : icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    if (icebergCatalog instanceof SupportsNamespaces) {
+      asNamespaceCatalog = (SupportsNamespaces) icebergCatalog;
+    } else {
+      asNamespaceCatalog = null;
+    }
+  }
+
+  @Override
+  public void open() throws CatalogException {
+  }
+
+  @Override
+  public void close() throws CatalogException {
+    if (originalCatalog instanceof Closeable) {
+      try {
+        ((Closeable) originalCatalog).close();
+      } catch (IOException e) {
+        throw new CatalogException(e);
+      }
+    }
+  }
+
+  private Namespace toNamespace(String database) {
+    String[] namespace = new String[baseNamespace.length + 1];
+    System.arraycopy(baseNamespace, 0, namespace, 0, baseNamespace.length);
+    namespace[baseNamespace.length] = database;
+    return Namespace.of(namespace);
+  }
+
+  private TableIdentifier toIdentifier(ObjectPath path) {
+    return TableIdentifier.of(toNamespace(path.getDatabaseName()), path.getObjectName());
+  }
+
+  @Override
+  public List<String> listDatabases() throws CatalogException {
+    if (asNamespaceCatalog == null) {
+      return Collections.singletonList(getDefaultDatabase());
+    }
+
+    return listAllNamespaces(Namespace.empty()).stream()
+        .map(n -> n.level(n.levels().length - 1))
+        .collect(Collectors.toList());
+  }
+
+  private List<Namespace> listAllNamespaces(Namespace namespace) {
+    if (asNamespaceCatalog == null) {
+      throw new RuntimeException("The asNamespaceCatalog should not be null.");
+    }
+
+    String[] levels = namespace.levels();
+    if (levels.length == baseNamespace.length + 1) {
+      return Collections.singletonList(namespace);
+    }
+    if (levels.length < baseNamespace.length + 1) {
+      for (int i = 0; i < levels.length; i++) {
+        if (!baseNamespace[i].equals(levels[i])) {
+          return Collections.emptyList();
+        }
+      }
+      List<Namespace> ret = new ArrayList<>();
+      asNamespaceCatalog.listNamespaces(namespace).forEach(n -> ret.addAll(listAllNamespaces(n)));
+      return ret;
+    }
+    return Collections.emptyList();
+  }
+
+  @Override
+  public CatalogDatabase getDatabase(String databaseName) throws DatabaseNotExistException, CatalogException {
+    if (asNamespaceCatalog == null) {
+      if (!getDefaultDatabase().equals(databaseName)) {
+        throw new DatabaseNotExistException(getName(), databaseName);
+      } else {
+        return new CatalogDatabaseImpl(new HashMap<>(), "");
+      }
+    } else {
+      try {
+        Map<String, String> metadata =
+            new HashMap<>(asNamespaceCatalog.loadNamespaceMetadata(toNamespace(databaseName)));
+        String comment = metadata.remove("comment");
+        return new CatalogDatabaseImpl(metadata, comment);
+      } catch (NoSuchNamespaceException e) {
+        throw new DatabaseNotExistException(getName(), databaseName, e);
+      }
+    }
+  }
+
+  @Override
+  public boolean databaseExists(String databaseName) throws CatalogException {
+    try {
+      getDatabase(databaseName);
+      return true;
+    } catch (DatabaseNotExistException ignore) {
+      return false;
+    }
+  }
+
+  @Override
+  public void createDatabase(String name, CatalogDatabase database, boolean ignoreIfExists)
+      throws DatabaseAlreadyExistException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      try {
+        asNamespaceCatalog.createNamespace(
+            toNamespace(name),
+            mergeComment(database.getProperties(), database.getComment()));
+      } catch (AlreadyExistsException e) {
+        if (!ignoreIfExists) {
+          throw new DatabaseAlreadyExistException(getName(), name, e);
+        }
+      }
+    } else {
+      throw new UnsupportedOperationException("Namespaces are not supported by catalog: " + getName());
+    }
+  }
+
+  private Map<String, String> mergeComment(Map<String, String> metadata, String comment) {
+    Map<String, String> ret = new HashMap<>(metadata);
+    if (metadata.containsKey("comment")) {
+      throw new CatalogException("Database properties should not contain key: 'comment'.");
+    }
+    if (!StringUtils.isNullOrWhitespaceOnly(comment)) {
+      ret.put("comment", comment);
+    }
+    return ret;
+  }
+
+  @Override
+  public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade)
+      throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      try {
+        boolean success = asNamespaceCatalog.dropNamespace(toNamespace(name));
+        if (!success && !ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name);
+        }
+      } catch (NoSuchNamespaceException e) {
+        if (!ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name, e);
+        }
+      } catch (NamespaceNotEmptyException e) {
+        throw new DatabaseNotEmptyException(getName(), name, e);
+      }
+    } else {
+      if (!ignoreIfNotExists) {
+        throw new DatabaseNotExistException(getName(), name);
+      }
+    }
+  }
+
+  @Override
+  public void alterDatabase(String name, CatalogDatabase newDatabase, boolean ignoreIfNotExists)
+      throws DatabaseNotExistException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      Namespace namespace = toNamespace(name);
+      Map<String, String> updates = Maps.newHashMap();
+      Set<String> removals = Sets.newHashSet();
+
+      try {
+        Map<String, String> oldOptions = asNamespaceCatalog.loadNamespaceMetadata(namespace);
+        Map<String, String> newOptions = mergeComment(newDatabase.getProperties(), newDatabase.getComment());
+
+        for (String key : oldOptions.keySet()) {
+          if (!newOptions.containsKey(key)) {
+            removals.add(key);
+          }
+        }
+
+        for (Map.Entry<String, String> entry : newOptions.entrySet()) {
+          if (!entry.getValue().equals(oldOptions.get(entry.getKey()))) {
+            updates.put(entry.getKey(), entry.getValue());
+          }
+        }
+
+        if (!updates.isEmpty()) {
+          asNamespaceCatalog.setProperties(namespace, updates);
+        }
+
+        if (!removals.isEmpty()) {
+          asNamespaceCatalog.removeProperties(namespace, removals);
+        }
+
+      } catch (org.apache.iceberg.exceptions.NoSuchNamespaceException e) {
+        if (!ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name, e);
+        }
+      }
+    } else {
+      if (!ignoreIfNotExists) {
+        throw new DatabaseNotExistException(getName(), name);
+      }
+    }
+  }
+
+  @Override
+  public List<String> listTables(String databaseName) throws DatabaseNotExistException, CatalogException {
+    try {
+      return icebergCatalog.listTables(toNamespace(databaseName)).stream()
+          .map(TableIdentifier::name)
+          .collect(Collectors.toList());
+    } catch (NoSuchNamespaceException e) {
+      throw new DatabaseNotExistException(getName(), databaseName, e);
+    }
+  }
+
+  @Override
+  public CatalogBaseTable getTable(ObjectPath tablePath) throws TableNotExistException, CatalogException {
+    try {
+      Table table = icebergCatalog.loadTable(toIdentifier(tablePath));
+      TableSchema tableSchema = FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(table.schema()));
+
+      // NOTE: We can not create a IcebergCatalogTable, because Flink optimizer may use CatalogTableImpl to copy a new
+      // catalog table.
+      // Let's re-loading table from Iceberg catalog when creating source/sink operators.
+      return new CatalogTableImpl(tableSchema, table.properties(), "");
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      throw new TableNotExistException(getName(), tablePath, e);
+    }
+  }
+
+  @Override
+  public boolean tableExists(ObjectPath tablePath) throws CatalogException {
+    return icebergCatalog.tableExists(toIdentifier(tablePath));
+  }
+
+  @Override
+  public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists)
+      throws TableNotExistException, CatalogException {
+    try {
+      icebergCatalog.dropTable(toIdentifier(tablePath));
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      throw new TableNotExistException(getName(), tablePath, e);
+    }
+  }
+
+  @Override
+  public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists)
+      throws TableNotExistException, TableAlreadyExistException, CatalogException {
+    try {
+      icebergCatalog.renameTable(
+          toIdentifier(tablePath),
+          toIdentifier(new ObjectPath(tablePath.getDatabaseName(), newTableName)));
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      throw new TableNotExistException(getName(), tablePath, e);
+    } catch (AlreadyExistsException e) {
+      throw new TableAlreadyExistException(getName(), tablePath, e);
+    }
+  }
+
+  /**
+   * TODO Implement DDL-string parser for PartitionSpec.

Review comment:
       Can we add partitioning to the Flink DDL parser instead? That seems like a more appropriate place for it.
   
   Otherwise, I'd recommend just using the `PartitionSpecParser.fromJson` method.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453106175



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java
##########
@@ -0,0 +1,508 @@
+/*
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.AbstractCatalog;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogDatabase;
+import org.apache.flink.table.catalog.CatalogDatabaseImpl;
+import org.apache.flink.table.catalog.CatalogFunction;
+import org.apache.flink.table.catalog.CatalogPartition;
+import org.apache.flink.table.catalog.CatalogPartitionSpec;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+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.DatabaseNotEmptyException;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
+import org.apache.flink.table.catalog.exceptions.FunctionNotExistException;
+import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.catalog.stats.CatalogColumnStatistics;
+import org.apache.flink.table.catalog.stats.CatalogTableStatistics;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.util.StringUtils;
+import org.apache.iceberg.CachingCatalog;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+
+/**
+ * A Flink Catalog implementation that wraps an Iceberg {@link Catalog}.
+ * <p>
+ * The mapping between Flink database and Iceberg namespace:
+ * Supplying a base namespace for a given catalog, so if you have a catalog that supports a 2-level namespace, you
+ * would supply the first level in the catalog configuration and the second level would be exposed as Flink databases.
+ * <p>
+ * The Iceberg table manages its partitions by itself. The partition of the Iceberg table is independent of the
+ * partition of Flink.
+ */
+public class FlinkCatalog extends AbstractCatalog {
+
+  private final Catalog originalCatalog;
+  private final Catalog icebergCatalog;
+  private final String[] baseNamespace;
+  private final SupportsNamespaces asNamespaceCatalog;
+
+  public FlinkCatalog(
+      String catalogName,
+      String defaultDatabase,
+      String[] baseNamespace,
+      Catalog icebergCatalog,
+      boolean cacheEnabled) {
+    super(catalogName, defaultDatabase);
+    this.originalCatalog = icebergCatalog;
+    this.icebergCatalog = cacheEnabled ? CachingCatalog.wrap(icebergCatalog) : icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    if (icebergCatalog instanceof SupportsNamespaces) {
+      asNamespaceCatalog = (SupportsNamespaces) icebergCatalog;
+    } else {
+      asNamespaceCatalog = null;
+    }
+  }
+
+  @Override
+  public void open() throws CatalogException {
+  }
+
+  @Override
+  public void close() throws CatalogException {
+    if (originalCatalog instanceof Closeable) {
+      try {
+        ((Closeable) originalCatalog).close();
+      } catch (IOException e) {
+        throw new CatalogException(e);
+      }
+    }
+  }
+
+  private Namespace toNamespace(String database) {
+    String[] namespace = new String[baseNamespace.length + 1];
+    System.arraycopy(baseNamespace, 0, namespace, 0, baseNamespace.length);
+    namespace[baseNamespace.length] = database;
+    return Namespace.of(namespace);
+  }
+
+  private TableIdentifier toIdentifier(ObjectPath path) {
+    return TableIdentifier.of(toNamespace(path.getDatabaseName()), path.getObjectName());
+  }
+
+  @Override
+  public List<String> listDatabases() throws CatalogException {
+    if (asNamespaceCatalog == null) {
+      return Collections.singletonList(getDefaultDatabase());
+    }
+
+    return listAllNamespaces(Namespace.empty()).stream()
+        .map(n -> n.level(n.levels().length - 1))
+        .collect(Collectors.toList());
+  }
+
+  private List<Namespace> listAllNamespaces(Namespace namespace) {
+    if (asNamespaceCatalog == null) {
+      throw new RuntimeException("The asNamespaceCatalog should not be null.");
+    }
+
+    String[] levels = namespace.levels();
+    if (levels.length == baseNamespace.length + 1) {
+      return Collections.singletonList(namespace);
+    }
+    if (levels.length < baseNamespace.length + 1) {
+      for (int i = 0; i < levels.length; i++) {
+        if (!baseNamespace[i].equals(levels[i])) {
+          return Collections.emptyList();
+        }
+      }
+      List<Namespace> ret = new ArrayList<>();
+      asNamespaceCatalog.listNamespaces(namespace).forEach(n -> ret.addAll(listAllNamespaces(n)));

Review comment:
       Why is this recursive? It seems unnecessary.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453109755



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java
##########
@@ -0,0 +1,508 @@
+/*
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.AbstractCatalog;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogDatabase;
+import org.apache.flink.table.catalog.CatalogDatabaseImpl;
+import org.apache.flink.table.catalog.CatalogFunction;
+import org.apache.flink.table.catalog.CatalogPartition;
+import org.apache.flink.table.catalog.CatalogPartitionSpec;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+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.DatabaseNotEmptyException;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
+import org.apache.flink.table.catalog.exceptions.FunctionNotExistException;
+import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.catalog.stats.CatalogColumnStatistics;
+import org.apache.flink.table.catalog.stats.CatalogTableStatistics;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.util.StringUtils;
+import org.apache.iceberg.CachingCatalog;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+
+/**
+ * A Flink Catalog implementation that wraps an Iceberg {@link Catalog}.
+ * <p>
+ * The mapping between Flink database and Iceberg namespace:
+ * Supplying a base namespace for a given catalog, so if you have a catalog that supports a 2-level namespace, you
+ * would supply the first level in the catalog configuration and the second level would be exposed as Flink databases.
+ * <p>
+ * The Iceberg table manages its partitions by itself. The partition of the Iceberg table is independent of the
+ * partition of Flink.
+ */
+public class FlinkCatalog extends AbstractCatalog {
+
+  private final Catalog originalCatalog;
+  private final Catalog icebergCatalog;
+  private final String[] baseNamespace;
+  private final SupportsNamespaces asNamespaceCatalog;
+
+  public FlinkCatalog(
+      String catalogName,
+      String defaultDatabase,
+      String[] baseNamespace,
+      Catalog icebergCatalog,
+      boolean cacheEnabled) {
+    super(catalogName, defaultDatabase);
+    this.originalCatalog = icebergCatalog;
+    this.icebergCatalog = cacheEnabled ? CachingCatalog.wrap(icebergCatalog) : icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    if (icebergCatalog instanceof SupportsNamespaces) {
+      asNamespaceCatalog = (SupportsNamespaces) icebergCatalog;
+    } else {
+      asNamespaceCatalog = null;
+    }
+  }
+
+  @Override
+  public void open() throws CatalogException {
+  }
+
+  @Override
+  public void close() throws CatalogException {
+    if (originalCatalog instanceof Closeable) {
+      try {
+        ((Closeable) originalCatalog).close();
+      } catch (IOException e) {
+        throw new CatalogException(e);
+      }
+    }
+  }
+
+  private Namespace toNamespace(String database) {
+    String[] namespace = new String[baseNamespace.length + 1];
+    System.arraycopy(baseNamespace, 0, namespace, 0, baseNamespace.length);
+    namespace[baseNamespace.length] = database;
+    return Namespace.of(namespace);
+  }
+
+  private TableIdentifier toIdentifier(ObjectPath path) {
+    return TableIdentifier.of(toNamespace(path.getDatabaseName()), path.getObjectName());
+  }
+
+  @Override
+  public List<String> listDatabases() throws CatalogException {
+    if (asNamespaceCatalog == null) {
+      return Collections.singletonList(getDefaultDatabase());
+    }
+
+    return listAllNamespaces(Namespace.empty()).stream()
+        .map(n -> n.level(n.levels().length - 1))
+        .collect(Collectors.toList());
+  }
+
+  private List<Namespace> listAllNamespaces(Namespace namespace) {
+    if (asNamespaceCatalog == null) {
+      throw new RuntimeException("The asNamespaceCatalog should not be null.");
+    }
+
+    String[] levels = namespace.levels();
+    if (levels.length == baseNamespace.length + 1) {
+      return Collections.singletonList(namespace);
+    }
+    if (levels.length < baseNamespace.length + 1) {
+      for (int i = 0; i < levels.length; i++) {
+        if (!baseNamespace[i].equals(levels[i])) {
+          return Collections.emptyList();
+        }
+      }
+      List<Namespace> ret = new ArrayList<>();
+      asNamespaceCatalog.listNamespaces(namespace).forEach(n -> ret.addAll(listAllNamespaces(n)));
+      return ret;
+    }
+    return Collections.emptyList();
+  }
+
+  @Override
+  public CatalogDatabase getDatabase(String databaseName) throws DatabaseNotExistException, CatalogException {
+    if (asNamespaceCatalog == null) {
+      if (!getDefaultDatabase().equals(databaseName)) {
+        throw new DatabaseNotExistException(getName(), databaseName);
+      } else {
+        return new CatalogDatabaseImpl(new HashMap<>(), "");
+      }
+    } else {
+      try {
+        Map<String, String> metadata =
+            new HashMap<>(asNamespaceCatalog.loadNamespaceMetadata(toNamespace(databaseName)));
+        String comment = metadata.remove("comment");
+        return new CatalogDatabaseImpl(metadata, comment);
+      } catch (NoSuchNamespaceException e) {
+        throw new DatabaseNotExistException(getName(), databaseName, e);
+      }
+    }
+  }
+
+  @Override
+  public boolean databaseExists(String databaseName) throws CatalogException {
+    try {
+      getDatabase(databaseName);
+      return true;
+    } catch (DatabaseNotExistException ignore) {
+      return false;
+    }
+  }
+
+  @Override
+  public void createDatabase(String name, CatalogDatabase database, boolean ignoreIfExists)
+      throws DatabaseAlreadyExistException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      try {
+        asNamespaceCatalog.createNamespace(
+            toNamespace(name),
+            mergeComment(database.getProperties(), database.getComment()));
+      } catch (AlreadyExistsException e) {
+        if (!ignoreIfExists) {
+          throw new DatabaseAlreadyExistException(getName(), name, e);
+        }
+      }
+    } else {
+      throw new UnsupportedOperationException("Namespaces are not supported by catalog: " + getName());
+    }
+  }
+
+  private Map<String, String> mergeComment(Map<String, String> metadata, String comment) {
+    Map<String, String> ret = new HashMap<>(metadata);
+    if (metadata.containsKey("comment")) {
+      throw new CatalogException("Database properties should not contain key: 'comment'.");
+    }
+    if (!StringUtils.isNullOrWhitespaceOnly(comment)) {
+      ret.put("comment", comment);
+    }
+    return ret;
+  }
+
+  @Override
+  public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade)
+      throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      try {
+        boolean success = asNamespaceCatalog.dropNamespace(toNamespace(name));
+        if (!success && !ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name);
+        }
+      } catch (NoSuchNamespaceException e) {
+        if (!ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name, e);
+        }
+      } catch (NamespaceNotEmptyException e) {
+        throw new DatabaseNotEmptyException(getName(), name, e);
+      }
+    } else {
+      if (!ignoreIfNotExists) {
+        throw new DatabaseNotExistException(getName(), name);
+      }
+    }
+  }
+
+  @Override
+  public void alterDatabase(String name, CatalogDatabase newDatabase, boolean ignoreIfNotExists)
+      throws DatabaseNotExistException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      Namespace namespace = toNamespace(name);
+      Map<String, String> updates = Maps.newHashMap();
+      Set<String> removals = Sets.newHashSet();
+
+      try {
+        Map<String, String> oldOptions = asNamespaceCatalog.loadNamespaceMetadata(namespace);
+        Map<String, String> newOptions = mergeComment(newDatabase.getProperties(), newDatabase.getComment());
+
+        for (String key : oldOptions.keySet()) {
+          if (!newOptions.containsKey(key)) {
+            removals.add(key);
+          }
+        }
+
+        for (Map.Entry<String, String> entry : newOptions.entrySet()) {
+          if (!entry.getValue().equals(oldOptions.get(entry.getKey()))) {
+            updates.put(entry.getKey(), entry.getValue());
+          }
+        }
+
+        if (!updates.isEmpty()) {
+          asNamespaceCatalog.setProperties(namespace, updates);
+        }
+
+        if (!removals.isEmpty()) {
+          asNamespaceCatalog.removeProperties(namespace, removals);
+        }
+
+      } catch (org.apache.iceberg.exceptions.NoSuchNamespaceException e) {
+        if (!ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name, e);
+        }
+      }
+    } else {
+      if (!ignoreIfNotExists) {
+        throw new DatabaseNotExistException(getName(), name);
+      }
+    }
+  }
+
+  @Override
+  public List<String> listTables(String databaseName) throws DatabaseNotExistException, CatalogException {
+    try {
+      return icebergCatalog.listTables(toNamespace(databaseName)).stream()
+          .map(TableIdentifier::name)
+          .collect(Collectors.toList());
+    } catch (NoSuchNamespaceException e) {
+      throw new DatabaseNotExistException(getName(), databaseName, e);
+    }
+  }
+
+  @Override
+  public CatalogBaseTable getTable(ObjectPath tablePath) throws TableNotExistException, CatalogException {
+    try {
+      Table table = icebergCatalog.loadTable(toIdentifier(tablePath));
+      TableSchema tableSchema = FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(table.schema()));
+
+      // NOTE: We can not create a IcebergCatalogTable, because Flink optimizer may use CatalogTableImpl to copy a new
+      // catalog table.
+      // Let's re-loading table from Iceberg catalog when creating source/sink operators.
+      return new CatalogTableImpl(tableSchema, table.properties(), "");
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      throw new TableNotExistException(getName(), tablePath, e);
+    }
+  }
+
+  @Override
+  public boolean tableExists(ObjectPath tablePath) throws CatalogException {
+    return icebergCatalog.tableExists(toIdentifier(tablePath));
+  }
+
+  @Override
+  public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists)
+      throws TableNotExistException, CatalogException {
+    try {
+      icebergCatalog.dropTable(toIdentifier(tablePath));
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      throw new TableNotExistException(getName(), tablePath, e);
+    }
+  }
+
+  @Override
+  public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists)
+      throws TableNotExistException, TableAlreadyExistException, CatalogException {
+    try {
+      icebergCatalog.renameTable(
+          toIdentifier(tablePath),
+          toIdentifier(new ObjectPath(tablePath.getDatabaseName(), newTableName)));
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      throw new TableNotExistException(getName(), tablePath, e);
+    } catch (AlreadyExistsException e) {
+      throw new TableAlreadyExistException(getName(), tablePath, e);
+    }
+  }
+
+  /**
+   * TODO Implement DDL-string parser for PartitionSpec.
+   */
+  @Override
+  public void createTable(ObjectPath tablePath, CatalogBaseTable table, boolean ignoreIfExists)
+      throws CatalogException {
+    throw new UnsupportedOperationException("Not support createTable now.");
+  }
+
+  @Override
+  public void alterTable(ObjectPath tablePath, CatalogBaseTable newTable, boolean ignoreIfNotExists)
+      throws CatalogException {
+    throw new UnsupportedOperationException("Not support alterTable now.");
+  }
+
+  // ------------------------------ Unsupported methods ---------------------------------------------
+
+  @Override
+  public List<String> listViews(String databaseName) throws CatalogException {
+    return Collections.emptyList();
+  }
+
+  @Override
+  public CatalogPartition getPartition(
+      ObjectPath tablePath, CatalogPartitionSpec partitionSpec
+  ) throws CatalogException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean partitionExists(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws CatalogException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void createPartition(
+      ObjectPath tablePath, CatalogPartitionSpec partitionSpec, CatalogPartition partition, boolean ignoreIfExists
+  ) throws CatalogException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void dropPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec, boolean ignoreIfNotExists)
+      throws CatalogException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void alterPartition(
+      ObjectPath tablePath, CatalogPartitionSpec partitionSpec, CatalogPartition newPartition, boolean ignoreIfNotExists
+  ) throws CatalogException {

Review comment:
       We prefer two options for formatting argument lists. Either aligned with the first argument:
   
   ```java
   public void alterPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec, CatalogPartition newPartition,
                              boolean ignoreIfNotExists) throws CatalogException {
     ...
   }
   ```
   
   Or, indented by 2 indents (4 spaces) and aligned with that position:
   ```java
   public void alterPartition(
       ObjectPath tablePath, CatalogPartitionSpec partitionSpec, CatalogPartition newPartition, boolean ignoreIfNotExists)
       throws CatalogException {
     ...
   }
   ```
   
   `throws` can be on the next line, indented to the same place.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453112583



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java
##########
@@ -0,0 +1,241 @@
+/*
+ * 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.File;
+import java.util.Map;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.types.Types;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Test;
+
+public class TestFlinkCatalogDatabase extends FlinkCatalogTestBase {
+
+  public TestFlinkCatalogDatabase(String catalogName, String[] baseNamepace) {
+    super(catalogName, baseNamepace);
+  }
+
+  @After
+  public void clean() {
+    sql("DROP TABLE IF EXISTS %s.tl", flinkIdentifier);
+    sql("DROP DATABASE IF EXISTS %s", flinkIdentifier);
+  }
+
+  @Test
+  public void testCreateNamespace() {
+    Assert.assertFalse(
+        "Database should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+
+    Assert.assertTrue("Database should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+  }
+
+  @Test
+  public void testDefaultDatabase() {
+    sql("USE CATALOG %s", catalogName);
+
+    Assert.assertEquals("Should use the current catalog", tEnv.getCurrentCatalog(), catalogName);
+    Assert.assertEquals("Should use the configured default namespace", tEnv.getCurrentDatabase(), "default");
+  }
+
+  @Test
+  public void testDropEmptyDatabase() {
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("DROP DATABASE %s", flinkIdentifier);
+
+    Assert.assertFalse(
+        "Namespace should have been dropped",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+  }
+
+  @Test
+  public void testDropNonEmptyNamespace() {
+    Assume.assumeFalse("Hadoop catalog throws IOException: Directory is not empty.", isHadoopCatalog);

Review comment:
       This sounds like a bug in the Hadoop catalog. Can we fix it instead of ignoring this test case?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453105170



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java
##########
@@ -0,0 +1,508 @@
+/*
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.AbstractCatalog;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogDatabase;
+import org.apache.flink.table.catalog.CatalogDatabaseImpl;
+import org.apache.flink.table.catalog.CatalogFunction;
+import org.apache.flink.table.catalog.CatalogPartition;
+import org.apache.flink.table.catalog.CatalogPartitionSpec;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+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.DatabaseNotEmptyException;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
+import org.apache.flink.table.catalog.exceptions.FunctionNotExistException;
+import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.catalog.stats.CatalogColumnStatistics;
+import org.apache.flink.table.catalog.stats.CatalogTableStatistics;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.util.StringUtils;
+import org.apache.iceberg.CachingCatalog;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+
+/**
+ * A Flink Catalog implementation that wraps an Iceberg {@link Catalog}.
+ * <p>
+ * The mapping between Flink database and Iceberg namespace:
+ * Supplying a base namespace for a given catalog, so if you have a catalog that supports a 2-level namespace, you
+ * would supply the first level in the catalog configuration and the second level would be exposed as Flink databases.
+ * <p>
+ * The Iceberg table manages its partitions by itself. The partition of the Iceberg table is independent of the
+ * partition of Flink.
+ */
+public class FlinkCatalog extends AbstractCatalog {
+
+  private final Catalog originalCatalog;
+  private final Catalog icebergCatalog;
+  private final String[] baseNamespace;
+  private final SupportsNamespaces asNamespaceCatalog;
+
+  public FlinkCatalog(
+      String catalogName,
+      String defaultDatabase,
+      String[] baseNamespace,
+      Catalog icebergCatalog,
+      boolean cacheEnabled) {
+    super(catalogName, defaultDatabase);
+    this.originalCatalog = icebergCatalog;
+    this.icebergCatalog = cacheEnabled ? CachingCatalog.wrap(icebergCatalog) : icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    if (icebergCatalog instanceof SupportsNamespaces) {
+      asNamespaceCatalog = (SupportsNamespaces) icebergCatalog;
+    } else {
+      asNamespaceCatalog = null;
+    }
+  }
+
+  @Override
+  public void open() throws CatalogException {
+  }
+
+  @Override
+  public void close() throws CatalogException {
+    if (originalCatalog instanceof Closeable) {
+      try {
+        ((Closeable) originalCatalog).close();
+      } catch (IOException e) {
+        throw new CatalogException(e);
+      }
+    }
+  }
+
+  private Namespace toNamespace(String database) {
+    String[] namespace = new String[baseNamespace.length + 1];
+    System.arraycopy(baseNamespace, 0, namespace, 0, baseNamespace.length);
+    namespace[baseNamespace.length] = database;
+    return Namespace.of(namespace);
+  }
+
+  private TableIdentifier toIdentifier(ObjectPath path) {
+    return TableIdentifier.of(toNamespace(path.getDatabaseName()), path.getObjectName());
+  }
+
+  @Override
+  public List<String> listDatabases() throws CatalogException {
+    if (asNamespaceCatalog == null) {
+      return Collections.singletonList(getDefaultDatabase());
+    }
+
+    return listAllNamespaces(Namespace.empty()).stream()
+        .map(n -> n.level(n.levels().length - 1))
+        .collect(Collectors.toList());
+  }
+
+  private List<Namespace> listAllNamespaces(Namespace namespace) {
+    if (asNamespaceCatalog == null) {
+      throw new RuntimeException("The asNamespaceCatalog should not be null.");
+    }
+
+    String[] levels = namespace.levels();
+    if (levels.length == baseNamespace.length + 1) {
+      return Collections.singletonList(namespace);
+    }
+    if (levels.length < baseNamespace.length + 1) {

Review comment:
       Nit: we like to add empty lines after control flow blocks (between the last `}` and this `if`).




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453106430



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java
##########
@@ -0,0 +1,508 @@
+/*
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.AbstractCatalog;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogDatabase;
+import org.apache.flink.table.catalog.CatalogDatabaseImpl;
+import org.apache.flink.table.catalog.CatalogFunction;
+import org.apache.flink.table.catalog.CatalogPartition;
+import org.apache.flink.table.catalog.CatalogPartitionSpec;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+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.DatabaseNotEmptyException;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
+import org.apache.flink.table.catalog.exceptions.FunctionNotExistException;
+import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.catalog.stats.CatalogColumnStatistics;
+import org.apache.flink.table.catalog.stats.CatalogTableStatistics;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.util.StringUtils;
+import org.apache.iceberg.CachingCatalog;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+
+/**
+ * A Flink Catalog implementation that wraps an Iceberg {@link Catalog}.
+ * <p>
+ * The mapping between Flink database and Iceberg namespace:
+ * Supplying a base namespace for a given catalog, so if you have a catalog that supports a 2-level namespace, you
+ * would supply the first level in the catalog configuration and the second level would be exposed as Flink databases.
+ * <p>
+ * The Iceberg table manages its partitions by itself. The partition of the Iceberg table is independent of the
+ * partition of Flink.
+ */
+public class FlinkCatalog extends AbstractCatalog {
+
+  private final Catalog originalCatalog;
+  private final Catalog icebergCatalog;
+  private final String[] baseNamespace;
+  private final SupportsNamespaces asNamespaceCatalog;
+
+  public FlinkCatalog(
+      String catalogName,
+      String defaultDatabase,
+      String[] baseNamespace,
+      Catalog icebergCatalog,
+      boolean cacheEnabled) {
+    super(catalogName, defaultDatabase);
+    this.originalCatalog = icebergCatalog;
+    this.icebergCatalog = cacheEnabled ? CachingCatalog.wrap(icebergCatalog) : icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    if (icebergCatalog instanceof SupportsNamespaces) {
+      asNamespaceCatalog = (SupportsNamespaces) icebergCatalog;
+    } else {
+      asNamespaceCatalog = null;
+    }
+  }
+
+  @Override
+  public void open() throws CatalogException {
+  }
+
+  @Override
+  public void close() throws CatalogException {
+    if (originalCatalog instanceof Closeable) {
+      try {
+        ((Closeable) originalCatalog).close();
+      } catch (IOException e) {
+        throw new CatalogException(e);
+      }
+    }
+  }
+
+  private Namespace toNamespace(String database) {
+    String[] namespace = new String[baseNamespace.length + 1];
+    System.arraycopy(baseNamespace, 0, namespace, 0, baseNamespace.length);
+    namespace[baseNamespace.length] = database;
+    return Namespace.of(namespace);
+  }
+
+  private TableIdentifier toIdentifier(ObjectPath path) {
+    return TableIdentifier.of(toNamespace(path.getDatabaseName()), path.getObjectName());
+  }
+
+  @Override
+  public List<String> listDatabases() throws CatalogException {
+    if (asNamespaceCatalog == null) {
+      return Collections.singletonList(getDefaultDatabase());
+    }
+
+    return listAllNamespaces(Namespace.empty()).stream()
+        .map(n -> n.level(n.levels().length - 1))
+        .collect(Collectors.toList());
+  }
+
+  private List<Namespace> listAllNamespaces(Namespace namespace) {
+    if (asNamespaceCatalog == null) {
+      throw new RuntimeException("The asNamespaceCatalog should not be null.");
+    }
+
+    String[] levels = namespace.levels();
+    if (levels.length == baseNamespace.length + 1) {
+      return Collections.singletonList(namespace);
+    }
+    if (levels.length < baseNamespace.length + 1) {
+      for (int i = 0; i < levels.length; i++) {
+        if (!baseNamespace[i].equals(levels[i])) {
+          return Collections.emptyList();
+        }
+      }
+      List<Namespace> ret = new ArrayList<>();
+      asNamespaceCatalog.listNamespaces(namespace).forEach(n -> ret.addAll(listAllNamespaces(n)));
+      return ret;
+    }
+    return Collections.emptyList();
+  }
+
+  @Override
+  public CatalogDatabase getDatabase(String databaseName) throws DatabaseNotExistException, CatalogException {
+    if (asNamespaceCatalog == null) {
+      if (!getDefaultDatabase().equals(databaseName)) {
+        throw new DatabaseNotExistException(getName(), databaseName);
+      } else {
+        return new CatalogDatabaseImpl(new HashMap<>(), "");
+      }
+    } else {
+      try {
+        Map<String, String> metadata =
+            new HashMap<>(asNamespaceCatalog.loadNamespaceMetadata(toNamespace(databaseName)));

Review comment:
       Minor: We prefer using the factory methods in `Maps` instead of specific class names.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r454051737



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java
##########
@@ -21,10 +21,19 @@
 
 import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.types.FieldsDataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.utils.TypeConversions;
 import org.apache.iceberg.Schema;
 import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
 import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.TypeUtil;
 
+/**
+ * Converter between Flink types and Iceberg type.
+ * The conversion is not a 1:1 mapping that not allows back-and-forth conversion. So some information might get lost
+ * during the back-and-forth conversion.

Review comment:
       Yes




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453429303



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java
##########
@@ -0,0 +1,241 @@
+/*
+ * 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.File;
+import java.util.Map;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.types.Types;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Test;
+
+public class TestFlinkCatalogDatabase extends FlinkCatalogTestBase {
+
+  public TestFlinkCatalogDatabase(String catalogName, String[] baseNamepace) {
+    super(catalogName, baseNamepace);
+  }
+
+  @After
+  public void clean() {
+    sql("DROP TABLE IF EXISTS %s.tl", flinkIdentifier);
+    sql("DROP DATABASE IF EXISTS %s", flinkIdentifier);
+  }
+
+  @Test
+  public void testCreateNamespace() {
+    Assert.assertFalse(
+        "Database should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+
+    Assert.assertTrue("Database should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+  }
+
+  @Test
+  public void testDefaultDatabase() {
+    sql("USE CATALOG %s", catalogName);
+
+    Assert.assertEquals("Should use the current catalog", tEnv.getCurrentCatalog(), catalogName);
+    Assert.assertEquals("Should use the configured default namespace", tEnv.getCurrentDatabase(), "default");
+  }
+
+  @Test
+  public void testDropEmptyDatabase() {
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("DROP DATABASE %s", flinkIdentifier);
+
+    Assert.assertFalse(
+        "Namespace should have been dropped",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+  }
+
+  @Test
+  public void testDropNonEmptyNamespace() {
+    Assume.assumeFalse("Hadoop catalog throws IOException: Directory is not empty.", isHadoopCatalog);
+
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+
+    validationCatalog.createTable(
+        TableIdentifier.of(icebergNamespace, "tl"),
+        new Schema(Types.NestedField.optional(0, "id", Types.LongType.get())));
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+    Assert.assertTrue("Table should exist", validationCatalog.tableExists(TableIdentifier.of(icebergNamespace, "tl")));
+
+    AssertHelpers.assertThrowsCause(
+        "Should fail if trying to delete a non-empty database",
+        DatabaseNotEmptyException.class,
+        String.format("Database %s in catalog %s is not empty.", DATABASE, catalogName),
+        () -> sql("DROP DATABASE %s", flinkIdentifier));
+
+    sql("DROP TABLE %s.tl", flinkIdentifier);
+  }
+
+  @Test
+  public void testListTables() {
+    Assert.assertFalse(
+        "Namespace should not already exist",
+        validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    sql("CREATE DATABASE %s", flinkIdentifier);
+    sql("USE CATALOG %s", catalogName);
+    sql("USE %s", DATABASE);
+
+    Assert.assertTrue("Namespace should exist", validationNamespaceCatalog.namespaceExists(icebergNamespace));
+
+    Assert.assertEquals("Should not list any tables", 0, tEnv.listTables().length);

Review comment:
       In Flink 1.10, not support DDL `SHOW TABLES`. It is supported in 1.11.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r454051982



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java
##########
@@ -21,10 +21,19 @@
 
 import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.types.FieldsDataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.utils.TypeConversions;
 import org.apache.iceberg.Schema;
 import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
 import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.TypeUtil;
 
+/**
+ * Converter between Flink types and Iceberg type.
+ * The conversion is not a 1:1 mapping that not allows back-and-forth conversion. So some information might get lost
+ * during the back-and-forth conversion.

Review comment:
       Iceberg to Flink: will loss UUID.
   Flink to Iceberg: will loss precisions.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1182: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#discussion_r453400568



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java
##########
@@ -0,0 +1,508 @@
+/*
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.AbstractCatalog;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogDatabase;
+import org.apache.flink.table.catalog.CatalogDatabaseImpl;
+import org.apache.flink.table.catalog.CatalogFunction;
+import org.apache.flink.table.catalog.CatalogPartition;
+import org.apache.flink.table.catalog.CatalogPartitionSpec;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+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.DatabaseNotEmptyException;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
+import org.apache.flink.table.catalog.exceptions.FunctionNotExistException;
+import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.catalog.stats.CatalogColumnStatistics;
+import org.apache.flink.table.catalog.stats.CatalogTableStatistics;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.util.StringUtils;
+import org.apache.iceberg.CachingCatalog;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+
+/**
+ * A Flink Catalog implementation that wraps an Iceberg {@link Catalog}.
+ * <p>
+ * The mapping between Flink database and Iceberg namespace:
+ * Supplying a base namespace for a given catalog, so if you have a catalog that supports a 2-level namespace, you
+ * would supply the first level in the catalog configuration and the second level would be exposed as Flink databases.
+ * <p>
+ * The Iceberg table manages its partitions by itself. The partition of the Iceberg table is independent of the
+ * partition of Flink.
+ */
+public class FlinkCatalog extends AbstractCatalog {
+
+  private final Catalog originalCatalog;
+  private final Catalog icebergCatalog;
+  private final String[] baseNamespace;
+  private final SupportsNamespaces asNamespaceCatalog;
+
+  public FlinkCatalog(
+      String catalogName,
+      String defaultDatabase,
+      String[] baseNamespace,
+      Catalog icebergCatalog,
+      boolean cacheEnabled) {
+    super(catalogName, defaultDatabase);
+    this.originalCatalog = icebergCatalog;
+    this.icebergCatalog = cacheEnabled ? CachingCatalog.wrap(icebergCatalog) : icebergCatalog;
+    this.baseNamespace = baseNamespace;
+    if (icebergCatalog instanceof SupportsNamespaces) {
+      asNamespaceCatalog = (SupportsNamespaces) icebergCatalog;
+    } else {
+      asNamespaceCatalog = null;
+    }
+  }
+
+  @Override
+  public void open() throws CatalogException {
+  }
+
+  @Override
+  public void close() throws CatalogException {
+    if (originalCatalog instanceof Closeable) {
+      try {
+        ((Closeable) originalCatalog).close();
+      } catch (IOException e) {
+        throw new CatalogException(e);
+      }
+    }
+  }
+
+  private Namespace toNamespace(String database) {
+    String[] namespace = new String[baseNamespace.length + 1];
+    System.arraycopy(baseNamespace, 0, namespace, 0, baseNamespace.length);
+    namespace[baseNamespace.length] = database;
+    return Namespace.of(namespace);
+  }
+
+  private TableIdentifier toIdentifier(ObjectPath path) {
+    return TableIdentifier.of(toNamespace(path.getDatabaseName()), path.getObjectName());
+  }
+
+  @Override
+  public List<String> listDatabases() throws CatalogException {
+    if (asNamespaceCatalog == null) {
+      return Collections.singletonList(getDefaultDatabase());
+    }
+
+    return listAllNamespaces(Namespace.empty()).stream()
+        .map(n -> n.level(n.levels().length - 1))
+        .collect(Collectors.toList());
+  }
+
+  private List<Namespace> listAllNamespaces(Namespace namespace) {
+    if (asNamespaceCatalog == null) {
+      throw new RuntimeException("The asNamespaceCatalog should not be null.");
+    }
+
+    String[] levels = namespace.levels();
+    if (levels.length == baseNamespace.length + 1) {
+      return Collections.singletonList(namespace);
+    }
+    if (levels.length < baseNamespace.length + 1) {
+      for (int i = 0; i < levels.length; i++) {
+        if (!baseNamespace[i].equals(levels[i])) {
+          return Collections.emptyList();
+        }
+      }
+      List<Namespace> ret = new ArrayList<>();
+      asNamespaceCatalog.listNamespaces(namespace).forEach(n -> ret.addAll(listAllNamespaces(n)));
+      return ret;
+    }
+    return Collections.emptyList();
+  }
+
+  @Override
+  public CatalogDatabase getDatabase(String databaseName) throws DatabaseNotExistException, CatalogException {
+    if (asNamespaceCatalog == null) {
+      if (!getDefaultDatabase().equals(databaseName)) {
+        throw new DatabaseNotExistException(getName(), databaseName);
+      } else {
+        return new CatalogDatabaseImpl(new HashMap<>(), "");
+      }
+    } else {
+      try {
+        Map<String, String> metadata =
+            new HashMap<>(asNamespaceCatalog.loadNamespaceMetadata(toNamespace(databaseName)));
+        String comment = metadata.remove("comment");
+        return new CatalogDatabaseImpl(metadata, comment);
+      } catch (NoSuchNamespaceException e) {
+        throw new DatabaseNotExistException(getName(), databaseName, e);
+      }
+    }
+  }
+
+  @Override
+  public boolean databaseExists(String databaseName) throws CatalogException {
+    try {
+      getDatabase(databaseName);
+      return true;
+    } catch (DatabaseNotExistException ignore) {
+      return false;
+    }
+  }
+
+  @Override
+  public void createDatabase(String name, CatalogDatabase database, boolean ignoreIfExists)
+      throws DatabaseAlreadyExistException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      try {
+        asNamespaceCatalog.createNamespace(
+            toNamespace(name),
+            mergeComment(database.getProperties(), database.getComment()));
+      } catch (AlreadyExistsException e) {
+        if (!ignoreIfExists) {
+          throw new DatabaseAlreadyExistException(getName(), name, e);
+        }
+      }
+    } else {
+      throw new UnsupportedOperationException("Namespaces are not supported by catalog: " + getName());
+    }
+  }
+
+  private Map<String, String> mergeComment(Map<String, String> metadata, String comment) {
+    Map<String, String> ret = new HashMap<>(metadata);
+    if (metadata.containsKey("comment")) {
+      throw new CatalogException("Database properties should not contain key: 'comment'.");
+    }
+    if (!StringUtils.isNullOrWhitespaceOnly(comment)) {
+      ret.put("comment", comment);
+    }
+    return ret;
+  }
+
+  @Override
+  public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade)
+      throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      try {
+        boolean success = asNamespaceCatalog.dropNamespace(toNamespace(name));
+        if (!success && !ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name);
+        }
+      } catch (NoSuchNamespaceException e) {
+        if (!ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name, e);
+        }
+      } catch (NamespaceNotEmptyException e) {
+        throw new DatabaseNotEmptyException(getName(), name, e);
+      }
+    } else {
+      if (!ignoreIfNotExists) {
+        throw new DatabaseNotExistException(getName(), name);
+      }
+    }
+  }
+
+  @Override
+  public void alterDatabase(String name, CatalogDatabase newDatabase, boolean ignoreIfNotExists)
+      throws DatabaseNotExistException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      Namespace namespace = toNamespace(name);
+      Map<String, String> updates = Maps.newHashMap();
+      Set<String> removals = Sets.newHashSet();
+
+      try {
+        Map<String, String> oldOptions = asNamespaceCatalog.loadNamespaceMetadata(namespace);
+        Map<String, String> newOptions = mergeComment(newDatabase.getProperties(), newDatabase.getComment());
+
+        for (String key : oldOptions.keySet()) {
+          if (!newOptions.containsKey(key)) {
+            removals.add(key);
+          }
+        }
+
+        for (Map.Entry<String, String> entry : newOptions.entrySet()) {
+          if (!entry.getValue().equals(oldOptions.get(entry.getKey()))) {
+            updates.put(entry.getKey(), entry.getValue());
+          }
+        }
+
+        if (!updates.isEmpty()) {
+          asNamespaceCatalog.setProperties(namespace, updates);
+        }
+
+        if (!removals.isEmpty()) {
+          asNamespaceCatalog.removeProperties(namespace, removals);
+        }
+
+      } catch (org.apache.iceberg.exceptions.NoSuchNamespaceException e) {
+        if (!ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name, e);
+        }
+      }
+    } else {
+      if (!ignoreIfNotExists) {
+        throw new DatabaseNotExistException(getName(), name);
+      }
+    }
+  }
+
+  @Override
+  public List<String> listTables(String databaseName) throws DatabaseNotExistException, CatalogException {
+    try {
+      return icebergCatalog.listTables(toNamespace(databaseName)).stream()
+          .map(TableIdentifier::name)
+          .collect(Collectors.toList());
+    } catch (NoSuchNamespaceException e) {
+      throw new DatabaseNotExistException(getName(), databaseName, e);
+    }
+  }
+
+  @Override
+  public CatalogBaseTable getTable(ObjectPath tablePath) throws TableNotExistException, CatalogException {
+    try {
+      Table table = icebergCatalog.loadTable(toIdentifier(tablePath));
+      TableSchema tableSchema = FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(table.schema()));
+
+      // NOTE: We can not create a IcebergCatalogTable, because Flink optimizer may use CatalogTableImpl to copy a new
+      // catalog table.
+      // Let's re-loading table from Iceberg catalog when creating source/sink operators.
+      return new CatalogTableImpl(tableSchema, table.properties(), "");
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      throw new TableNotExistException(getName(), tablePath, e);
+    }
+  }
+
+  @Override
+  public boolean tableExists(ObjectPath tablePath) throws CatalogException {
+    return icebergCatalog.tableExists(toIdentifier(tablePath));
+  }
+
+  @Override
+  public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists)
+      throws TableNotExistException, CatalogException {
+    try {
+      icebergCatalog.dropTable(toIdentifier(tablePath));
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      throw new TableNotExistException(getName(), tablePath, e);
+    }
+  }
+
+  @Override
+  public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists)
+      throws TableNotExistException, TableAlreadyExistException, CatalogException {
+    try {
+      icebergCatalog.renameTable(
+          toIdentifier(tablePath),
+          toIdentifier(new ObjectPath(tablePath.getDatabaseName(), newTableName)));
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      throw new TableNotExistException(getName(), tablePath, e);
+    } catch (AlreadyExistsException e) {
+      throw new TableAlreadyExistException(getName(), tablePath, e);
+    }
+  }
+
+  /**
+   * TODO Implement DDL-string parser for PartitionSpec.
+   */
+  @Override
+  public void createTable(ObjectPath tablePath, CatalogBaseTable table, boolean ignoreIfExists)
+      throws CatalogException {
+    throw new UnsupportedOperationException("Not support createTable now.");
+  }
+
+  @Override
+  public void alterTable(ObjectPath tablePath, CatalogBaseTable newTable, boolean ignoreIfNotExists)
+      throws CatalogException {
+    throw new UnsupportedOperationException("Not support alterTable now.");
+  }
+
+  // ------------------------------ Unsupported methods ---------------------------------------------
+
+  @Override
+  public List<String> listViews(String databaseName) throws CatalogException {
+    return Collections.emptyList();
+  }
+
+  @Override
+  public CatalogPartition getPartition(
+      ObjectPath tablePath, CatalogPartitionSpec partitionSpec
+  ) throws CatalogException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean partitionExists(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws CatalogException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void createPartition(
+      ObjectPath tablePath, CatalogPartitionSpec partitionSpec, CatalogPartition partition, boolean ignoreIfExists
+  ) throws CatalogException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void dropPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec, boolean ignoreIfNotExists)
+      throws CatalogException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void alterPartition(
+      ObjectPath tablePath, CatalogPartitionSpec partitionSpec, CatalogPartition newPartition, boolean ignoreIfNotExists
+  ) throws CatalogException {

Review comment:
       Got it.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1182: Flink: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1182:
URL: https://github.com/apache/iceberg/pull/1182#issuecomment-661353500


   Thanks for the updates, @JingsongLi! This looks good to me. I'll merge it.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org