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: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
