nastra commented on code in PR #6428: URL: https://github.com/apache/iceberg/pull/6428#discussion_r1050439739
########## snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java: ########## @@ -0,0 +1,220 @@ +/* + * 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.snowflake; + +import java.io.Closeable; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.iceberg.BaseMetastoreCatalog; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SupportsNamespaces; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; +import org.apache.iceberg.hadoop.Configurable; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.jdbc.JdbcClientPool; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.snowflake.entities.SnowflakeSchema; +import org.apache.iceberg.snowflake.entities.SnowflakeTable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class SnowflakeCatalog extends BaseMetastoreCatalog + implements Closeable, SupportsNamespaces, Configurable<Object> { + + private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class); + + private Object conf; Review Comment: do we have a more specific type for the config? `Object` seems a little bit too generic ########## snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java: ########## @@ -0,0 +1,220 @@ +/* + * 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.snowflake; + +import java.io.Closeable; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.iceberg.BaseMetastoreCatalog; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SupportsNamespaces; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; +import org.apache.iceberg.hadoop.Configurable; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.jdbc.JdbcClientPool; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.snowflake.entities.SnowflakeSchema; +import org.apache.iceberg.snowflake.entities.SnowflakeTable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class SnowflakeCatalog extends BaseMetastoreCatalog + implements Closeable, SupportsNamespaces, Configurable<Object> { + + private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class); + + private Object conf; + private String catalogName = SnowflakeResources.DEFAULT_CATALOG_NAME; + private Map<String, String> catalogProperties = null; + private FileIO fileIO; + private SnowflakeClient snowflakeClient; + + public SnowflakeCatalog() {} + + @VisibleForTesting + void setSnowflakeClient(SnowflakeClient snowflakeClient) { + this.snowflakeClient = snowflakeClient; + } + + @VisibleForTesting + void setFileIO(FileIO fileIO) { + this.fileIO = fileIO; + } + + @Override + public List<TableIdentifier> listTables(Namespace namespace) { + LOG.debug("listTables with namespace: {}", namespace); + Preconditions.checkArgument( + namespace.length() <= SnowflakeResources.MAX_NAMESPACE_DEPTH, + "Snowflake doesn't support more than %s levels of namespace, got %s", + SnowflakeResources.MAX_NAMESPACE_DEPTH, + namespace); + + List<SnowflakeTable> sfTables = snowflakeClient.listIcebergTables(namespace); + + return sfTables.stream() + .map( + table -> + TableIdentifier.of(table.getDatabase(), table.getSchemaName(), table.getName())) + .collect(Collectors.toList()); + } + + @Override + public boolean dropTable(TableIdentifier identifier, boolean purge) { + throw new UnsupportedOperationException( + String.format("dropTable not supported; attempted for table '%s'", identifier)); + } + + @Override + public void renameTable(TableIdentifier from, TableIdentifier to) { + throw new UnsupportedOperationException( + String.format("renameTable not supported; attempted from '%s' to '%s'", from, to)); + } + + @Override + public void initialize(String name, Map<String, String> properties) { + catalogProperties = properties; + + if (name != null) { + this.catalogName = name; + } + + if (snowflakeClient == null) { + String uri = properties.get(CatalogProperties.URI); + Preconditions.checkNotNull(uri, "JDBC connection URI is required"); + + try { + // We'll ensure the expected JDBC driver implementation class is initialized through + // reflection + // regardless of which classloader ends up using this JdbcSnowflakeClient, but we'll only + // warn if the expected driver fails to load, since users may use repackaged or custom + // JDBC drivers for Snowflake communcation. + Class.forName(JdbcSnowflakeClient.EXPECTED_JDBC_IMPL); + } catch (ClassNotFoundException cnfe) { + LOG.warn( + "Failed to load expected JDBC SnowflakeDriver - if queries fail by failing" + + " to find a suitable driver for jdbc:snowflake:// URIs, you must add the Snowflake " + + " JDBC driver to your jars/packages", + cnfe); + } + + JdbcClientPool connectionPool = new JdbcClientPool(uri, properties); + snowflakeClient = new JdbcSnowflakeClient(connectionPool); + } + + if (fileIO == null) { + String fileIOImpl = SnowflakeResources.DEFAULT_FILE_IO_IMPL; + + if (catalogProperties.containsKey(CatalogProperties.FILE_IO_IMPL)) { + fileIOImpl = catalogProperties.get(CatalogProperties.FILE_IO_IMPL); + } + + fileIO = CatalogUtil.loadFileIO(fileIOImpl, catalogProperties, conf); + } + } + + @Override + public void close() { + snowflakeClient.close(); Review Comment: should we do a null check here in case the client wasn't initialized in the first place? Note that you might also want to use a `CloseableGroup closeableGroup` to close additional resources, such as `fileIO` ########## snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java: ########## @@ -0,0 +1,220 @@ +/* + * 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.snowflake; + +import java.io.Closeable; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.iceberg.BaseMetastoreCatalog; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SupportsNamespaces; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; +import org.apache.iceberg.hadoop.Configurable; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.jdbc.JdbcClientPool; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.snowflake.entities.SnowflakeSchema; +import org.apache.iceberg.snowflake.entities.SnowflakeTable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class SnowflakeCatalog extends BaseMetastoreCatalog + implements Closeable, SupportsNamespaces, Configurable<Object> { + + private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class); + + private Object conf; + private String catalogName = SnowflakeResources.DEFAULT_CATALOG_NAME; + private Map<String, String> catalogProperties = null; + private FileIO fileIO; + private SnowflakeClient snowflakeClient; + + public SnowflakeCatalog() {} + + @VisibleForTesting + void setSnowflakeClient(SnowflakeClient snowflakeClient) { + this.snowflakeClient = snowflakeClient; + } + + @VisibleForTesting + void setFileIO(FileIO fileIO) { + this.fileIO = fileIO; + } + + @Override + public List<TableIdentifier> listTables(Namespace namespace) { + LOG.debug("listTables with namespace: {}", namespace); + Preconditions.checkArgument( + namespace.length() <= SnowflakeResources.MAX_NAMESPACE_DEPTH, + "Snowflake doesn't support more than %s levels of namespace, got %s", + SnowflakeResources.MAX_NAMESPACE_DEPTH, + namespace); + + List<SnowflakeTable> sfTables = snowflakeClient.listIcebergTables(namespace); + + return sfTables.stream() + .map( + table -> + TableIdentifier.of(table.getDatabase(), table.getSchemaName(), table.getName())) + .collect(Collectors.toList()); + } + + @Override + public boolean dropTable(TableIdentifier identifier, boolean purge) { + throw new UnsupportedOperationException( + String.format("dropTable not supported; attempted for table '%s'", identifier)); + } + + @Override + public void renameTable(TableIdentifier from, TableIdentifier to) { + throw new UnsupportedOperationException( + String.format("renameTable not supported; attempted from '%s' to '%s'", from, to)); + } + + @Override + public void initialize(String name, Map<String, String> properties) { + catalogProperties = properties; + + if (name != null) { Review Comment: it seems a bit weird to check whether `name` / `snowflakeClient` / `fileIO` have/haven't been set. Can you please elaborate why this is necessary? Typically the `initialize()` should be called exactly once and its purpose is to initialize everything that's necessary ########## snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java: ########## @@ -0,0 +1,220 @@ +/* + * 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.snowflake; + +import java.io.Closeable; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.iceberg.BaseMetastoreCatalog; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SupportsNamespaces; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; +import org.apache.iceberg.hadoop.Configurable; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.jdbc.JdbcClientPool; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.snowflake.entities.SnowflakeSchema; +import org.apache.iceberg.snowflake.entities.SnowflakeTable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class SnowflakeCatalog extends BaseMetastoreCatalog + implements Closeable, SupportsNamespaces, Configurable<Object> { + + private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class); + + private Object conf; + private String catalogName = SnowflakeResources.DEFAULT_CATALOG_NAME; + private Map<String, String> catalogProperties = null; + private FileIO fileIO; + private SnowflakeClient snowflakeClient; + + public SnowflakeCatalog() {} + + @VisibleForTesting + void setSnowflakeClient(SnowflakeClient snowflakeClient) { + this.snowflakeClient = snowflakeClient; + } + + @VisibleForTesting + void setFileIO(FileIO fileIO) { + this.fileIO = fileIO; + } + + @Override + public List<TableIdentifier> listTables(Namespace namespace) { + LOG.debug("listTables with namespace: {}", namespace); + Preconditions.checkArgument( + namespace.length() <= SnowflakeResources.MAX_NAMESPACE_DEPTH, + "Snowflake doesn't support more than %s levels of namespace, got %s", + SnowflakeResources.MAX_NAMESPACE_DEPTH, + namespace); + + List<SnowflakeTable> sfTables = snowflakeClient.listIcebergTables(namespace); + + return sfTables.stream() + .map( + table -> + TableIdentifier.of(table.getDatabase(), table.getSchemaName(), table.getName())) + .collect(Collectors.toList()); + } + + @Override + public boolean dropTable(TableIdentifier identifier, boolean purge) { + throw new UnsupportedOperationException( + String.format("dropTable not supported; attempted for table '%s'", identifier)); + } + + @Override + public void renameTable(TableIdentifier from, TableIdentifier to) { + throw new UnsupportedOperationException( + String.format("renameTable not supported; attempted from '%s' to '%s'", from, to)); + } + + @Override + public void initialize(String name, Map<String, String> properties) { + catalogProperties = properties; + + if (name != null) { + this.catalogName = name; + } + + if (snowflakeClient == null) { + String uri = properties.get(CatalogProperties.URI); + Preconditions.checkNotNull(uri, "JDBC connection URI is required"); + + try { + // We'll ensure the expected JDBC driver implementation class is initialized through + // reflection + // regardless of which classloader ends up using this JdbcSnowflakeClient, but we'll only + // warn if the expected driver fails to load, since users may use repackaged or custom + // JDBC drivers for Snowflake communcation. + Class.forName(JdbcSnowflakeClient.EXPECTED_JDBC_IMPL); + } catch (ClassNotFoundException cnfe) { + LOG.warn( + "Failed to load expected JDBC SnowflakeDriver - if queries fail by failing" + + " to find a suitable driver for jdbc:snowflake:// URIs, you must add the Snowflake " + + " JDBC driver to your jars/packages", + cnfe); + } + + JdbcClientPool connectionPool = new JdbcClientPool(uri, properties); + snowflakeClient = new JdbcSnowflakeClient(connectionPool); + } + + if (fileIO == null) { + String fileIOImpl = SnowflakeResources.DEFAULT_FILE_IO_IMPL; + + if (catalogProperties.containsKey(CatalogProperties.FILE_IO_IMPL)) { + fileIOImpl = catalogProperties.get(CatalogProperties.FILE_IO_IMPL); + } + + fileIO = CatalogUtil.loadFileIO(fileIOImpl, catalogProperties, conf); + } + } + + @Override + public void close() { + snowflakeClient.close(); + } + + @Override + public void createNamespace(Namespace namespace, Map<String, String> metadata) { + throw new UnsupportedOperationException( + String.format("createNamespace not supported; attempted for namespace '%s'", namespace)); + } + + @Override + public List<Namespace> listNamespaces(Namespace namespace) { + LOG.debug("listNamespaces with namespace: {}", namespace); + Preconditions.checkArgument( Review Comment: same question as above: should this maybe go into the client? ########## build.gradle: ########## @@ -696,6 +696,26 @@ project(':iceberg-dell') { } } +project(':iceberg-snowflake') { + test { + useJUnitPlatform() + } + + dependencies { + implementation project(':iceberg-core') + implementation project(':iceberg-common') + implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow') + implementation project(':iceberg-aws') + implementation "com.fasterxml.jackson.core:jackson-databind" + implementation "com.fasterxml.jackson.core:jackson-core" + implementation "commons-dbutils:commons-dbutils:1.7" + + runtimeOnly("net.snowflake:snowflake-jdbc:3.13.22") Review Comment: the version of this and `commons-dbutils` above should go into `versions.props` ########## snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java: ########## @@ -0,0 +1,220 @@ +/* + * 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.snowflake; + +import java.io.Closeable; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.iceberg.BaseMetastoreCatalog; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SupportsNamespaces; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; +import org.apache.iceberg.hadoop.Configurable; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.jdbc.JdbcClientPool; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.snowflake.entities.SnowflakeSchema; +import org.apache.iceberg.snowflake.entities.SnowflakeTable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class SnowflakeCatalog extends BaseMetastoreCatalog + implements Closeable, SupportsNamespaces, Configurable<Object> { + + private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class); + + private Object conf; + private String catalogName = SnowflakeResources.DEFAULT_CATALOG_NAME; + private Map<String, String> catalogProperties = null; + private FileIO fileIO; + private SnowflakeClient snowflakeClient; + + public SnowflakeCatalog() {} + + @VisibleForTesting + void setSnowflakeClient(SnowflakeClient snowflakeClient) { + this.snowflakeClient = snowflakeClient; + } + + @VisibleForTesting + void setFileIO(FileIO fileIO) { + this.fileIO = fileIO; + } + + @Override + public List<TableIdentifier> listTables(Namespace namespace) { + LOG.debug("listTables with namespace: {}", namespace); + Preconditions.checkArgument( + namespace.length() <= SnowflakeResources.MAX_NAMESPACE_DEPTH, + "Snowflake doesn't support more than %s levels of namespace, got %s", + SnowflakeResources.MAX_NAMESPACE_DEPTH, + namespace); + + List<SnowflakeTable> sfTables = snowflakeClient.listIcebergTables(namespace); + + return sfTables.stream() + .map( + table -> + TableIdentifier.of(table.getDatabase(), table.getSchemaName(), table.getName())) + .collect(Collectors.toList()); + } + + @Override + public boolean dropTable(TableIdentifier identifier, boolean purge) { + throw new UnsupportedOperationException( + String.format("dropTable not supported; attempted for table '%s'", identifier)); + } + + @Override + public void renameTable(TableIdentifier from, TableIdentifier to) { + throw new UnsupportedOperationException( + String.format("renameTable not supported; attempted from '%s' to '%s'", from, to)); + } + + @Override + public void initialize(String name, Map<String, String> properties) { + catalogProperties = properties; + + if (name != null) { + this.catalogName = name; + } + + if (snowflakeClient == null) { + String uri = properties.get(CatalogProperties.URI); + Preconditions.checkNotNull(uri, "JDBC connection URI is required"); + + try { + // We'll ensure the expected JDBC driver implementation class is initialized through + // reflection + // regardless of which classloader ends up using this JdbcSnowflakeClient, but we'll only + // warn if the expected driver fails to load, since users may use repackaged or custom + // JDBC drivers for Snowflake communcation. + Class.forName(JdbcSnowflakeClient.EXPECTED_JDBC_IMPL); + } catch (ClassNotFoundException cnfe) { + LOG.warn( + "Failed to load expected JDBC SnowflakeDriver - if queries fail by failing" + + " to find a suitable driver for jdbc:snowflake:// URIs, you must add the Snowflake " + + " JDBC driver to your jars/packages", + cnfe); + } + + JdbcClientPool connectionPool = new JdbcClientPool(uri, properties); + snowflakeClient = new JdbcSnowflakeClient(connectionPool); + } + + if (fileIO == null) { + String fileIOImpl = SnowflakeResources.DEFAULT_FILE_IO_IMPL; + + if (catalogProperties.containsKey(CatalogProperties.FILE_IO_IMPL)) { + fileIOImpl = catalogProperties.get(CatalogProperties.FILE_IO_IMPL); + } + + fileIO = CatalogUtil.loadFileIO(fileIOImpl, catalogProperties, conf); + } + } + + @Override + public void close() { + snowflakeClient.close(); + } + + @Override + public void createNamespace(Namespace namespace, Map<String, String> metadata) { + throw new UnsupportedOperationException( + String.format("createNamespace not supported; attempted for namespace '%s'", namespace)); + } + + @Override + public List<Namespace> listNamespaces(Namespace namespace) { + LOG.debug("listNamespaces with namespace: {}", namespace); + Preconditions.checkArgument( + namespace.length() <= SnowflakeResources.MAX_NAMESPACE_DEPTH - 1, + "Snowflake doesn't support more than %s levels of namespace, tried to list under %s", + SnowflakeResources.MAX_NAMESPACE_DEPTH, + namespace); + List<SnowflakeSchema> sfSchemas = snowflakeClient.listSchemas(namespace); + + List<Namespace> namespaceList = + sfSchemas.stream() + .map(schema -> Namespace.of(schema.getDatabase(), schema.getName())) + .collect(Collectors.toList()); + return namespaceList; + } + + @Override + public Map<String, String> loadNamespaceMetadata(Namespace namespace) + throws NoSuchNamespaceException { + LOG.debug("loadNamespaceMetadata with namespace: {}", namespace); + Map<String, String> nameSpaceMetadata = Maps.newHashMap(); + nameSpaceMetadata.put("name", namespace.toString()); + return nameSpaceMetadata; + } + + @Override + public boolean dropNamespace(Namespace namespace) { + throw new UnsupportedOperationException( + String.format("dropNamespace not supported; attempted for namespace '%s'", namespace)); + } + + @Override + public boolean setProperties(Namespace namespace, Map<String, String> properties) { + throw new UnsupportedOperationException( + String.format("setProperties not supported; attempted for namespace '%s'", namespace)); + } + + @Override + public boolean removeProperties(Namespace namespace, Set<String> properties) { + throw new UnsupportedOperationException( + String.format("removeProperties not supported; attempted for namespace '%s'", namespace)); + } + + @Override + protected TableOperations newTableOps(TableIdentifier tableIdentifier) { + Preconditions.checkArgument( + tableIdentifier.namespace().length() <= SnowflakeResources.MAX_NAMESPACE_DEPTH, + "Snowflake doesn't support more than %s levels of namespace, got %s", + SnowflakeResources.MAX_NAMESPACE_DEPTH, + tableIdentifier); + + return new SnowflakeTableOperations( + snowflakeClient, fileIO, catalogProperties, catalogName, tableIdentifier); + } + + @Override + protected String defaultWarehouseLocation(TableIdentifier tableIdentifier) { + return null; Review Comment: should this throw an exception rather than returning null maybe? ########## snowflake/src/test/java/org/apache/iceberg/snowflake/SnowflakeCatalogTest.java: ########## @@ -0,0 +1,231 @@ +/* + * 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.snowflake; + +import java.util.List; +import java.util.Map; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableMetadataParser; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.snowflake.entities.SnowflakeTableMetadata; +import org.apache.iceberg.types.Types; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class SnowflakeCatalogTest { + + static final String TEST_CATALOG_NAME = "slushLog"; + private SnowflakeCatalog catalog; + + @Before + public void before() { + catalog = new SnowflakeCatalog(); + + FakeSnowflakeClient client = new FakeSnowflakeClient(); + client.addTable( + "DB_1", + "SCHEMA_1", + "TAB_1", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"s3://tab1/metadata/v3.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_1", + "SCHEMA_1", + "TAB_2", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"s3://tab2/metadata/v1.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_2", + "SCHEMA_2", + "TAB_3", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"azure://myaccount.blob.core.windows.net/mycontainer/tab3/metadata/v334.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_2", + "SCHEMA_2", + "TAB_4", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"azure://myaccount.blob.core.windows.net/mycontainer/tab4/metadata/v323.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_3", + "SCHEMA_3", + "TAB_5", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"gcs://tab5/metadata/v793.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_3", + "SCHEMA_4", + "TAB_6", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"gcs://tab6/metadata/v123.metadata.json\",\"status\":\"success\"}")); + + catalog.setSnowflakeClient(client); + + InMemoryFileIO fakeFileIO = new InMemoryFileIO(); + + Schema schema = + new Schema( + Types.NestedField.required(1, "x", Types.StringType.get(), "comment1"), + Types.NestedField.required(2, "y", Types.StringType.get(), "comment2")); + PartitionSpec partitionSpec = + PartitionSpec.builderFor(schema).identity("x").withSpecId(1000).build(); + fakeFileIO.addFile( + "s3://tab1/metadata/v3.metadata.json", + TableMetadataParser.toJson( + TableMetadata.newTableMetadata( + schema, partitionSpec, "s3://tab1/", ImmutableMap.<String, String>of())) + .getBytes()); + fakeFileIO.addFile( + "wasbs://mycontai...@myaccount.blob.core.windows.net/tab3/metadata/v334.metadata.json", + TableMetadataParser.toJson( + TableMetadata.newTableMetadata( + schema, + partitionSpec, + "wasbs://mycontai...@myaccount.blob.core.windows.net/tab1/", + ImmutableMap.<String, String>of())) + .getBytes()); + fakeFileIO.addFile( + "gs://tab5/metadata/v793.metadata.json", + TableMetadataParser.toJson( + TableMetadata.newTableMetadata( + schema, partitionSpec, "gs://tab5/", ImmutableMap.<String, String>of())) + .getBytes()); + + catalog.setFileIO(fakeFileIO); + + Map<String, String> properties = Maps.newHashMap(); + catalog.initialize(TEST_CATALOG_NAME, properties); + } + + @Test + public void testListNamespace() { + List<Namespace> namespaces = catalog.listNamespaces(); + Assert.assertEquals( + Lists.newArrayList( + Namespace.of("DB_1", "SCHEMA_1"), + Namespace.of("DB_2", "SCHEMA_2"), + Namespace.of("DB_3", "SCHEMA_3"), + Namespace.of("DB_3", "SCHEMA_4")), + namespaces); + } + + @Test + public void testListNamespaceWithinDB() { + String dbName = "DB_1"; + List<Namespace> namespaces = catalog.listNamespaces(Namespace.of(dbName)); + Assert.assertEquals(Lists.newArrayList(Namespace.of(dbName, "SCHEMA_1")), namespaces); + } + + @Test + public void testListNamespaceWithinNonExistentDB() { + // Existence check for nonexistent parent namespaces is optional in the SupportsNamespaces + // interface. + String dbName = "NONEXISTENT_DB"; + Assert.assertThrows(RuntimeException.class, () -> catalog.listNamespaces(Namespace.of(dbName))); + } + + @Test + public void testListNamespaceWithinSchema() { + // No "sub-namespaces" beyond database.schema; invalid to try to list namespaces given + // a database.schema. + String dbName = "DB_3"; + String schemaName = "SCHEMA_4"; + Assert.assertThrows( Review Comment: same as above. Also please add a `.hasMessage(..)` check to make sure we're getting the right message here. ########## snowflake/src/test/java/org/apache/iceberg/snowflake/SnowflakeCatalogTest.java: ########## @@ -0,0 +1,231 @@ +/* + * 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.snowflake; + +import java.util.List; +import java.util.Map; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableMetadataParser; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.snowflake.entities.SnowflakeTableMetadata; +import org.apache.iceberg.types.Types; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class SnowflakeCatalogTest { + + static final String TEST_CATALOG_NAME = "slushLog"; + private SnowflakeCatalog catalog; + + @Before + public void before() { + catalog = new SnowflakeCatalog(); + + FakeSnowflakeClient client = new FakeSnowflakeClient(); + client.addTable( + "DB_1", + "SCHEMA_1", + "TAB_1", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"s3://tab1/metadata/v3.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_1", + "SCHEMA_1", + "TAB_2", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"s3://tab2/metadata/v1.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_2", + "SCHEMA_2", + "TAB_3", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"azure://myaccount.blob.core.windows.net/mycontainer/tab3/metadata/v334.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_2", + "SCHEMA_2", + "TAB_4", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"azure://myaccount.blob.core.windows.net/mycontainer/tab4/metadata/v323.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_3", + "SCHEMA_3", + "TAB_5", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"gcs://tab5/metadata/v793.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_3", + "SCHEMA_4", + "TAB_6", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"gcs://tab6/metadata/v123.metadata.json\",\"status\":\"success\"}")); + + catalog.setSnowflakeClient(client); + + InMemoryFileIO fakeFileIO = new InMemoryFileIO(); + + Schema schema = + new Schema( + Types.NestedField.required(1, "x", Types.StringType.get(), "comment1"), + Types.NestedField.required(2, "y", Types.StringType.get(), "comment2")); + PartitionSpec partitionSpec = + PartitionSpec.builderFor(schema).identity("x").withSpecId(1000).build(); + fakeFileIO.addFile( + "s3://tab1/metadata/v3.metadata.json", + TableMetadataParser.toJson( + TableMetadata.newTableMetadata( + schema, partitionSpec, "s3://tab1/", ImmutableMap.<String, String>of())) + .getBytes()); + fakeFileIO.addFile( + "wasbs://mycontai...@myaccount.blob.core.windows.net/tab3/metadata/v334.metadata.json", + TableMetadataParser.toJson( + TableMetadata.newTableMetadata( + schema, + partitionSpec, + "wasbs://mycontai...@myaccount.blob.core.windows.net/tab1/", + ImmutableMap.<String, String>of())) + .getBytes()); + fakeFileIO.addFile( + "gs://tab5/metadata/v793.metadata.json", + TableMetadataParser.toJson( + TableMetadata.newTableMetadata( + schema, partitionSpec, "gs://tab5/", ImmutableMap.<String, String>of())) + .getBytes()); + + catalog.setFileIO(fakeFileIO); + + Map<String, String> properties = Maps.newHashMap(); + catalog.initialize(TEST_CATALOG_NAME, properties); + } + + @Test + public void testListNamespace() { + List<Namespace> namespaces = catalog.listNamespaces(); + Assert.assertEquals( + Lists.newArrayList( + Namespace.of("DB_1", "SCHEMA_1"), + Namespace.of("DB_2", "SCHEMA_2"), + Namespace.of("DB_3", "SCHEMA_3"), + Namespace.of("DB_3", "SCHEMA_4")), + namespaces); + } + + @Test + public void testListNamespaceWithinDB() { + String dbName = "DB_1"; + List<Namespace> namespaces = catalog.listNamespaces(Namespace.of(dbName)); + Assert.assertEquals(Lists.newArrayList(Namespace.of(dbName, "SCHEMA_1")), namespaces); + } + + @Test + public void testListNamespaceWithinNonExistentDB() { + // Existence check for nonexistent parent namespaces is optional in the SupportsNamespaces + // interface. + String dbName = "NONEXISTENT_DB"; + Assert.assertThrows(RuntimeException.class, () -> catalog.listNamespaces(Namespace.of(dbName))); + } + + @Test + public void testListNamespaceWithinSchema() { + // No "sub-namespaces" beyond database.schema; invalid to try to list namespaces given + // a database.schema. + String dbName = "DB_3"; + String schemaName = "SCHEMA_4"; + Assert.assertThrows( + IllegalArgumentException.class, + () -> catalog.listNamespaces(Namespace.of(dbName, schemaName))); + } + + @Test + public void testListTables() { + List<TableIdentifier> tables = catalog.listTables(Namespace.empty()); + Assert.assertEquals( + Lists.newArrayList( + TableIdentifier.of("DB_1", "SCHEMA_1", "TAB_1"), + TableIdentifier.of("DB_1", "SCHEMA_1", "TAB_2"), + TableIdentifier.of("DB_2", "SCHEMA_2", "TAB_3"), + TableIdentifier.of("DB_2", "SCHEMA_2", "TAB_4"), + TableIdentifier.of("DB_3", "SCHEMA_3", "TAB_5"), + TableIdentifier.of("DB_3", "SCHEMA_4", "TAB_6")), + tables); + } + + @Test + public void testListTablesWithinDB() { + String dbName = "DB_1"; + List<TableIdentifier> tables = catalog.listTables(Namespace.of(dbName)); + Assert.assertEquals( + Lists.newArrayList( + TableIdentifier.of("DB_1", "SCHEMA_1", "TAB_1"), + TableIdentifier.of("DB_1", "SCHEMA_1", "TAB_2")), + tables); + } + + @Test + public void testListTablesWithinNonexistentDB() { + String dbName = "NONEXISTENT_DB"; + Assert.assertThrows(RuntimeException.class, () -> catalog.listTables(Namespace.of(dbName))); + } + + @Test + public void testListTablesWithinSchema() { + String dbName = "DB_2"; + String schemaName = "SCHEMA_2"; + List<TableIdentifier> tables = catalog.listTables(Namespace.of(dbName, schemaName)); + Assert.assertEquals( + Lists.newArrayList( + TableIdentifier.of("DB_2", "SCHEMA_2", "TAB_3"), + TableIdentifier.of("DB_2", "SCHEMA_2", "TAB_4")), + tables); + } + + @Test + public void testListTablesWithinNonexistentSchema() { + String dbName = "DB_2"; + String schemaName = "NONEXISTENT_DB"; + Assert.assertThrows( Review Comment: same as above. Also please add a `.hasMessage(..)` check to make sure we're getting the right message here. ########## snowflake/src/main/java/org/apache/iceberg/snowflake/entities/SnowflakeTable.java: ########## @@ -0,0 +1,85 @@ +/* + * 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.snowflake.entities; + +import java.util.List; +import org.apache.commons.dbutils.ResultSetHandler; +import org.apache.iceberg.relocated.com.google.common.base.Objects; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +public class SnowflakeTable { + private String databaseName; + private String schemaName; + private String name; + + public SnowflakeTable(String databaseName, String schemaName, String name) { Review Comment: what's the reason for having this class here? I was thinking whether it would make sense to directly convert this to a `TableIdentifier` via the `ResultSetHandler` in this class? Wouldn't that save us from having to convert from `SnowflakeTable` to `TableIdentifier`? ########## snowflake/src/test/java/org/apache/iceberg/snowflake/SnowflakeCatalogTest.java: ########## @@ -0,0 +1,231 @@ +/* + * 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.snowflake; + +import java.util.List; +import java.util.Map; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableMetadataParser; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.snowflake.entities.SnowflakeTableMetadata; +import org.apache.iceberg.types.Types; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class SnowflakeCatalogTest { + + static final String TEST_CATALOG_NAME = "slushLog"; + private SnowflakeCatalog catalog; + + @Before + public void before() { + catalog = new SnowflakeCatalog(); + + FakeSnowflakeClient client = new FakeSnowflakeClient(); + client.addTable( + "DB_1", + "SCHEMA_1", + "TAB_1", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"s3://tab1/metadata/v3.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_1", + "SCHEMA_1", + "TAB_2", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"s3://tab2/metadata/v1.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_2", + "SCHEMA_2", + "TAB_3", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"azure://myaccount.blob.core.windows.net/mycontainer/tab3/metadata/v334.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_2", + "SCHEMA_2", + "TAB_4", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"azure://myaccount.blob.core.windows.net/mycontainer/tab4/metadata/v323.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_3", + "SCHEMA_3", + "TAB_5", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"gcs://tab5/metadata/v793.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_3", + "SCHEMA_4", + "TAB_6", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"gcs://tab6/metadata/v123.metadata.json\",\"status\":\"success\"}")); + + catalog.setSnowflakeClient(client); + + InMemoryFileIO fakeFileIO = new InMemoryFileIO(); + + Schema schema = + new Schema( + Types.NestedField.required(1, "x", Types.StringType.get(), "comment1"), + Types.NestedField.required(2, "y", Types.StringType.get(), "comment2")); + PartitionSpec partitionSpec = + PartitionSpec.builderFor(schema).identity("x").withSpecId(1000).build(); + fakeFileIO.addFile( + "s3://tab1/metadata/v3.metadata.json", + TableMetadataParser.toJson( + TableMetadata.newTableMetadata( + schema, partitionSpec, "s3://tab1/", ImmutableMap.<String, String>of())) + .getBytes()); + fakeFileIO.addFile( + "wasbs://mycontai...@myaccount.blob.core.windows.net/tab3/metadata/v334.metadata.json", + TableMetadataParser.toJson( + TableMetadata.newTableMetadata( + schema, + partitionSpec, + "wasbs://mycontai...@myaccount.blob.core.windows.net/tab1/", + ImmutableMap.<String, String>of())) + .getBytes()); + fakeFileIO.addFile( + "gs://tab5/metadata/v793.metadata.json", + TableMetadataParser.toJson( + TableMetadata.newTableMetadata( + schema, partitionSpec, "gs://tab5/", ImmutableMap.<String, String>of())) + .getBytes()); + + catalog.setFileIO(fakeFileIO); + + Map<String, String> properties = Maps.newHashMap(); + catalog.initialize(TEST_CATALOG_NAME, properties); + } + + @Test + public void testListNamespace() { + List<Namespace> namespaces = catalog.listNamespaces(); + Assert.assertEquals( + Lists.newArrayList( + Namespace.of("DB_1", "SCHEMA_1"), + Namespace.of("DB_2", "SCHEMA_2"), + Namespace.of("DB_3", "SCHEMA_3"), + Namespace.of("DB_3", "SCHEMA_4")), + namespaces); + } + + @Test + public void testListNamespaceWithinDB() { + String dbName = "DB_1"; + List<Namespace> namespaces = catalog.listNamespaces(Namespace.of(dbName)); + Assert.assertEquals(Lists.newArrayList(Namespace.of(dbName, "SCHEMA_1")), namespaces); + } + + @Test + public void testListNamespaceWithinNonExistentDB() { + // Existence check for nonexistent parent namespaces is optional in the SupportsNamespaces + // interface. + String dbName = "NONEXISTENT_DB"; + Assert.assertThrows(RuntimeException.class, () -> catalog.listNamespaces(Namespace.of(dbName))); + } + + @Test + public void testListNamespaceWithinSchema() { + // No "sub-namespaces" beyond database.schema; invalid to try to list namespaces given + // a database.schema. + String dbName = "DB_3"; + String schemaName = "SCHEMA_4"; + Assert.assertThrows( + IllegalArgumentException.class, + () -> catalog.listNamespaces(Namespace.of(dbName, schemaName))); + } + + @Test + public void testListTables() { + List<TableIdentifier> tables = catalog.listTables(Namespace.empty()); + Assert.assertEquals( + Lists.newArrayList( + TableIdentifier.of("DB_1", "SCHEMA_1", "TAB_1"), + TableIdentifier.of("DB_1", "SCHEMA_1", "TAB_2"), + TableIdentifier.of("DB_2", "SCHEMA_2", "TAB_3"), + TableIdentifier.of("DB_2", "SCHEMA_2", "TAB_4"), + TableIdentifier.of("DB_3", "SCHEMA_3", "TAB_5"), + TableIdentifier.of("DB_3", "SCHEMA_4", "TAB_6")), + tables); + } + + @Test + public void testListTablesWithinDB() { + String dbName = "DB_1"; + List<TableIdentifier> tables = catalog.listTables(Namespace.of(dbName)); + Assert.assertEquals( + Lists.newArrayList( + TableIdentifier.of("DB_1", "SCHEMA_1", "TAB_1"), + TableIdentifier.of("DB_1", "SCHEMA_1", "TAB_2")), + tables); + } + + @Test + public void testListTablesWithinNonexistentDB() { + String dbName = "NONEXISTENT_DB"; + Assert.assertThrows(RuntimeException.class, () -> catalog.listTables(Namespace.of(dbName))); + } + + @Test + public void testListTablesWithinSchema() { + String dbName = "DB_2"; + String schemaName = "SCHEMA_2"; + List<TableIdentifier> tables = catalog.listTables(Namespace.of(dbName, schemaName)); + Assert.assertEquals( + Lists.newArrayList( + TableIdentifier.of("DB_2", "SCHEMA_2", "TAB_3"), + TableIdentifier.of("DB_2", "SCHEMA_2", "TAB_4")), + tables); + } + + @Test + public void testListTablesWithinNonexistentSchema() { + String dbName = "DB_2"; + String schemaName = "NONEXISTENT_DB"; + Assert.assertThrows( + RuntimeException.class, () -> catalog.listTables(Namespace.of(dbName, schemaName))); + } + + @Test + public void testLoadS3Table() { + Table table = catalog.loadTable(TableIdentifier.of(Namespace.of("DB_1", "SCHEMA_1"), "TAB_1")); + Assert.assertEquals(table.location(), "s3://tab1/"); + } + + @Test + public void testLoadAzureTable() { + Table table = catalog.loadTable(TableIdentifier.of(Namespace.of("DB_2", "SCHEMA_2"), "TAB_3")); + Assert.assertEquals( + table.location(), "wasbs://mycontai...@myaccount.blob.core.windows.net/tab1/"); Review Comment: `assertEquals(expected, actual)` is the signature, so I think the params should be turned around :) However, it would be great to switch this (and potentially other places above) to `Assertions.assertThat(table.location().isEqualTo(...)` as that is more fluent to read ########## snowflake/src/test/java/org/apache/iceberg/snowflake/SnowflakeCatalogTest.java: ########## @@ -0,0 +1,231 @@ +/* + * 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.snowflake; + +import java.util.List; +import java.util.Map; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableMetadataParser; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.snowflake.entities.SnowflakeTableMetadata; +import org.apache.iceberg.types.Types; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class SnowflakeCatalogTest { + + static final String TEST_CATALOG_NAME = "slushLog"; + private SnowflakeCatalog catalog; + + @Before + public void before() { + catalog = new SnowflakeCatalog(); + + FakeSnowflakeClient client = new FakeSnowflakeClient(); + client.addTable( + "DB_1", + "SCHEMA_1", + "TAB_1", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"s3://tab1/metadata/v3.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_1", + "SCHEMA_1", + "TAB_2", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"s3://tab2/metadata/v1.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_2", + "SCHEMA_2", + "TAB_3", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"azure://myaccount.blob.core.windows.net/mycontainer/tab3/metadata/v334.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_2", + "SCHEMA_2", + "TAB_4", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"azure://myaccount.blob.core.windows.net/mycontainer/tab4/metadata/v323.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_3", + "SCHEMA_3", + "TAB_5", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"gcs://tab5/metadata/v793.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_3", + "SCHEMA_4", + "TAB_6", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"gcs://tab6/metadata/v123.metadata.json\",\"status\":\"success\"}")); + + catalog.setSnowflakeClient(client); + + InMemoryFileIO fakeFileIO = new InMemoryFileIO(); + + Schema schema = + new Schema( + Types.NestedField.required(1, "x", Types.StringType.get(), "comment1"), + Types.NestedField.required(2, "y", Types.StringType.get(), "comment2")); + PartitionSpec partitionSpec = + PartitionSpec.builderFor(schema).identity("x").withSpecId(1000).build(); + fakeFileIO.addFile( + "s3://tab1/metadata/v3.metadata.json", + TableMetadataParser.toJson( + TableMetadata.newTableMetadata( + schema, partitionSpec, "s3://tab1/", ImmutableMap.<String, String>of())) + .getBytes()); + fakeFileIO.addFile( + "wasbs://mycontai...@myaccount.blob.core.windows.net/tab3/metadata/v334.metadata.json", + TableMetadataParser.toJson( + TableMetadata.newTableMetadata( + schema, + partitionSpec, + "wasbs://mycontai...@myaccount.blob.core.windows.net/tab1/", + ImmutableMap.<String, String>of())) + .getBytes()); + fakeFileIO.addFile( + "gs://tab5/metadata/v793.metadata.json", + TableMetadataParser.toJson( + TableMetadata.newTableMetadata( + schema, partitionSpec, "gs://tab5/", ImmutableMap.<String, String>of())) + .getBytes()); + + catalog.setFileIO(fakeFileIO); + + Map<String, String> properties = Maps.newHashMap(); + catalog.initialize(TEST_CATALOG_NAME, properties); + } + + @Test + public void testListNamespace() { + List<Namespace> namespaces = catalog.listNamespaces(); + Assert.assertEquals( + Lists.newArrayList( + Namespace.of("DB_1", "SCHEMA_1"), + Namespace.of("DB_2", "SCHEMA_2"), + Namespace.of("DB_3", "SCHEMA_3"), + Namespace.of("DB_3", "SCHEMA_4")), + namespaces); + } + + @Test + public void testListNamespaceWithinDB() { + String dbName = "DB_1"; + List<Namespace> namespaces = catalog.listNamespaces(Namespace.of(dbName)); + Assert.assertEquals(Lists.newArrayList(Namespace.of(dbName, "SCHEMA_1")), namespaces); + } + + @Test + public void testListNamespaceWithinNonExistentDB() { + // Existence check for nonexistent parent namespaces is optional in the SupportsNamespaces + // interface. + String dbName = "NONEXISTENT_DB"; + Assert.assertThrows(RuntimeException.class, () -> catalog.listNamespaces(Namespace.of(dbName))); + } + + @Test + public void testListNamespaceWithinSchema() { + // No "sub-namespaces" beyond database.schema; invalid to try to list namespaces given + // a database.schema. + String dbName = "DB_3"; + String schemaName = "SCHEMA_4"; + Assert.assertThrows( + IllegalArgumentException.class, + () -> catalog.listNamespaces(Namespace.of(dbName, schemaName))); + } + + @Test + public void testListTables() { + List<TableIdentifier> tables = catalog.listTables(Namespace.empty()); + Assert.assertEquals( + Lists.newArrayList( + TableIdentifier.of("DB_1", "SCHEMA_1", "TAB_1"), + TableIdentifier.of("DB_1", "SCHEMA_1", "TAB_2"), + TableIdentifier.of("DB_2", "SCHEMA_2", "TAB_3"), + TableIdentifier.of("DB_2", "SCHEMA_2", "TAB_4"), + TableIdentifier.of("DB_3", "SCHEMA_3", "TAB_5"), + TableIdentifier.of("DB_3", "SCHEMA_4", "TAB_6")), + tables); + } + + @Test + public void testListTablesWithinDB() { + String dbName = "DB_1"; + List<TableIdentifier> tables = catalog.listTables(Namespace.of(dbName)); + Assert.assertEquals( + Lists.newArrayList( + TableIdentifier.of("DB_1", "SCHEMA_1", "TAB_1"), + TableIdentifier.of("DB_1", "SCHEMA_1", "TAB_2")), + tables); + } + + @Test + public void testListTablesWithinNonexistentDB() { + String dbName = "NONEXISTENT_DB"; + Assert.assertThrows(RuntimeException.class, () -> catalog.listTables(Namespace.of(dbName))); Review Comment: same as above. Also please add a `.hasMessage(..)` check to make sure we're getting the right message here. ########## snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java: ########## @@ -0,0 +1,220 @@ +/* + * 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.snowflake; + +import java.io.Closeable; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.iceberg.BaseMetastoreCatalog; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SupportsNamespaces; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; +import org.apache.iceberg.hadoop.Configurable; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.jdbc.JdbcClientPool; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.snowflake.entities.SnowflakeSchema; +import org.apache.iceberg.snowflake.entities.SnowflakeTable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class SnowflakeCatalog extends BaseMetastoreCatalog + implements Closeable, SupportsNamespaces, Configurable<Object> { + + private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class); + + private Object conf; + private String catalogName = SnowflakeResources.DEFAULT_CATALOG_NAME; + private Map<String, String> catalogProperties = null; + private FileIO fileIO; + private SnowflakeClient snowflakeClient; + + public SnowflakeCatalog() {} + + @VisibleForTesting + void setSnowflakeClient(SnowflakeClient snowflakeClient) { + this.snowflakeClient = snowflakeClient; + } + + @VisibleForTesting + void setFileIO(FileIO fileIO) { + this.fileIO = fileIO; + } + + @Override + public List<TableIdentifier> listTables(Namespace namespace) { + LOG.debug("listTables with namespace: {}", namespace); + Preconditions.checkArgument( + namespace.length() <= SnowflakeResources.MAX_NAMESPACE_DEPTH, + "Snowflake doesn't support more than %s levels of namespace, got %s", + SnowflakeResources.MAX_NAMESPACE_DEPTH, + namespace); + + List<SnowflakeTable> sfTables = snowflakeClient.listIcebergTables(namespace); + + return sfTables.stream() + .map( + table -> + TableIdentifier.of(table.getDatabase(), table.getSchemaName(), table.getName())) + .collect(Collectors.toList()); + } + + @Override + public boolean dropTable(TableIdentifier identifier, boolean purge) { + throw new UnsupportedOperationException( + String.format("dropTable not supported; attempted for table '%s'", identifier)); + } + + @Override + public void renameTable(TableIdentifier from, TableIdentifier to) { + throw new UnsupportedOperationException( + String.format("renameTable not supported; attempted from '%s' to '%s'", from, to)); + } + + @Override + public void initialize(String name, Map<String, String> properties) { + catalogProperties = properties; + + if (name != null) { + this.catalogName = name; + } + + if (snowflakeClient == null) { + String uri = properties.get(CatalogProperties.URI); + Preconditions.checkNotNull(uri, "JDBC connection URI is required"); + + try { + // We'll ensure the expected JDBC driver implementation class is initialized through + // reflection + // regardless of which classloader ends up using this JdbcSnowflakeClient, but we'll only + // warn if the expected driver fails to load, since users may use repackaged or custom + // JDBC drivers for Snowflake communcation. + Class.forName(JdbcSnowflakeClient.EXPECTED_JDBC_IMPL); + } catch (ClassNotFoundException cnfe) { + LOG.warn( + "Failed to load expected JDBC SnowflakeDriver - if queries fail by failing" + + " to find a suitable driver for jdbc:snowflake:// URIs, you must add the Snowflake " + + " JDBC driver to your jars/packages", + cnfe); + } + + JdbcClientPool connectionPool = new JdbcClientPool(uri, properties); + snowflakeClient = new JdbcSnowflakeClient(connectionPool); + } + + if (fileIO == null) { + String fileIOImpl = SnowflakeResources.DEFAULT_FILE_IO_IMPL; + + if (catalogProperties.containsKey(CatalogProperties.FILE_IO_IMPL)) { + fileIOImpl = catalogProperties.get(CatalogProperties.FILE_IO_IMPL); + } + + fileIO = CatalogUtil.loadFileIO(fileIOImpl, catalogProperties, conf); + } + } + + @Override + public void close() { + snowflakeClient.close(); Review Comment: So you might end up having the following code in `initialize(...)`: ``` this.closeableGroup = new CloseableGroup(); .... closeableGroup.addCloseable(snowflakeClient); ... closeableGroup.addCloseable(fileIO); ... closeableGroup.setSuppressCloseFailure(true); ``` and in `close()` you would simply call ``` if (null != closeableGroup) { closeableGroup.close(); } ``` ########## snowflake/src/main/java/org/apache/iceberg/snowflake/entities/SnowflakeTable.java: ########## @@ -0,0 +1,85 @@ +/* + * 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.snowflake.entities; + +import java.util.List; +import org.apache.commons.dbutils.ResultSetHandler; +import org.apache.iceberg.relocated.com.google.common.base.Objects; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +public class SnowflakeTable { + private String databaseName; + private String schemaName; + private String name; + + public SnowflakeTable(String databaseName, String schemaName, String name) { Review Comment: the same applies for `SnowflakeSchema`. Couldn't we directly convert/use `Namespace` via the `ResultSetHandler`? ########## snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java: ########## @@ -0,0 +1,163 @@ +/* + * 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.snowflake; + +import java.sql.SQLException; +import java.util.List; +import org.apache.commons.dbutils.QueryRunner; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.jdbc.JdbcClientPool; +import org.apache.iceberg.jdbc.UncheckedInterruptedException; +import org.apache.iceberg.jdbc.UncheckedSQLException; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.snowflake.entities.SnowflakeSchema; +import org.apache.iceberg.snowflake.entities.SnowflakeTable; +import org.apache.iceberg.snowflake.entities.SnowflakeTableMetadata; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This implementation of SnowflakeClient builds on top of Snowflake's JDBC driver to interact with + * Snowflake's Iceberg-aware resource model. Despite using JDBC libraries, the resource model is + * derived from Snowflake's own first-class support for Iceberg tables as opposed to using an opaque + * JDBC layer to store Iceberg metadata itself in an Iceberg-agnostic database. + * + * <p>This thus differs from the JdbcCatalog in that Snowflake's service provides the source of + * truth of Iceberg metadata, rather than serving as a storage layer for a client-defined Iceberg + * resource model. + */ +public class JdbcSnowflakeClient implements SnowflakeClient { + public static final String EXPECTED_JDBC_IMPL = "net.snowflake.client.jdbc.SnowflakeDriver"; + + private static final Logger LOG = LoggerFactory.getLogger(JdbcSnowflakeClient.class); + private final JdbcClientPool connectionPool; + private QueryRunner queryRunner = new QueryRunner(true); Review Comment: nit: maybe init this only in the constructor ########## snowflake/src/test/java/org/apache/iceberg/snowflake/SnowflakeCatalogTest.java: ########## @@ -0,0 +1,231 @@ +/* + * 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.snowflake; + +import java.util.List; +import java.util.Map; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableMetadataParser; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.snowflake.entities.SnowflakeTableMetadata; +import org.apache.iceberg.types.Types; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class SnowflakeCatalogTest { + + static final String TEST_CATALOG_NAME = "slushLog"; + private SnowflakeCatalog catalog; + + @Before + public void before() { + catalog = new SnowflakeCatalog(); + + FakeSnowflakeClient client = new FakeSnowflakeClient(); + client.addTable( + "DB_1", + "SCHEMA_1", + "TAB_1", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"s3://tab1/metadata/v3.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_1", + "SCHEMA_1", + "TAB_2", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"s3://tab2/metadata/v1.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_2", + "SCHEMA_2", + "TAB_3", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"azure://myaccount.blob.core.windows.net/mycontainer/tab3/metadata/v334.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_2", + "SCHEMA_2", + "TAB_4", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"azure://myaccount.blob.core.windows.net/mycontainer/tab4/metadata/v323.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_3", + "SCHEMA_3", + "TAB_5", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"gcs://tab5/metadata/v793.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_3", + "SCHEMA_4", + "TAB_6", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"gcs://tab6/metadata/v123.metadata.json\",\"status\":\"success\"}")); + + catalog.setSnowflakeClient(client); Review Comment: as mentioned in comments on `SnowflakeCatalog`, we should probably be calling one of the `initialize` methods here as that is typically how a catalog is being initialized in Iceberg ########## snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeResources.java: ########## @@ -0,0 +1,29 @@ +/* + * 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.snowflake; + +final class SnowflakeResources { Review Comment: is this class required? would it maybe make more sense to move this into an existing one (`SnowflakeCatalog` for example)? ########## snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java: ########## @@ -0,0 +1,220 @@ +/* + * 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.snowflake; + +import java.io.Closeable; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.iceberg.BaseMetastoreCatalog; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SupportsNamespaces; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; +import org.apache.iceberg.hadoop.Configurable; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.jdbc.JdbcClientPool; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.snowflake.entities.SnowflakeSchema; +import org.apache.iceberg.snowflake.entities.SnowflakeTable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class SnowflakeCatalog extends BaseMetastoreCatalog + implements Closeable, SupportsNamespaces, Configurable<Object> { + + private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class); + + private Object conf; + private String catalogName = SnowflakeResources.DEFAULT_CATALOG_NAME; + private Map<String, String> catalogProperties = null; + private FileIO fileIO; + private SnowflakeClient snowflakeClient; + + public SnowflakeCatalog() {} + + @VisibleForTesting + void setSnowflakeClient(SnowflakeClient snowflakeClient) { + this.snowflakeClient = snowflakeClient; + } + + @VisibleForTesting + void setFileIO(FileIO fileIO) { + this.fileIO = fileIO; + } + + @Override + public List<TableIdentifier> listTables(Namespace namespace) { + LOG.debug("listTables with namespace: {}", namespace); + Preconditions.checkArgument( + namespace.length() <= SnowflakeResources.MAX_NAMESPACE_DEPTH, + "Snowflake doesn't support more than %s levels of namespace, got %s", + SnowflakeResources.MAX_NAMESPACE_DEPTH, + namespace); + + List<SnowflakeTable> sfTables = snowflakeClient.listIcebergTables(namespace); + + return sfTables.stream() + .map( + table -> + TableIdentifier.of(table.getDatabase(), table.getSchemaName(), table.getName())) + .collect(Collectors.toList()); + } + + @Override + public boolean dropTable(TableIdentifier identifier, boolean purge) { + throw new UnsupportedOperationException( + String.format("dropTable not supported; attempted for table '%s'", identifier)); + } + + @Override + public void renameTable(TableIdentifier from, TableIdentifier to) { + throw new UnsupportedOperationException( + String.format("renameTable not supported; attempted from '%s' to '%s'", from, to)); + } + + @Override + public void initialize(String name, Map<String, String> properties) { + catalogProperties = properties; + + if (name != null) { + this.catalogName = name; + } + + if (snowflakeClient == null) { + String uri = properties.get(CatalogProperties.URI); + Preconditions.checkNotNull(uri, "JDBC connection URI is required"); + + try { + // We'll ensure the expected JDBC driver implementation class is initialized through + // reflection + // regardless of which classloader ends up using this JdbcSnowflakeClient, but we'll only + // warn if the expected driver fails to load, since users may use repackaged or custom + // JDBC drivers for Snowflake communcation. + Class.forName(JdbcSnowflakeClient.EXPECTED_JDBC_IMPL); + } catch (ClassNotFoundException cnfe) { + LOG.warn( + "Failed to load expected JDBC SnowflakeDriver - if queries fail by failing" + + " to find a suitable driver for jdbc:snowflake:// URIs, you must add the Snowflake " + + " JDBC driver to your jars/packages", + cnfe); + } + + JdbcClientPool connectionPool = new JdbcClientPool(uri, properties); + snowflakeClient = new JdbcSnowflakeClient(connectionPool); + } + + if (fileIO == null) { + String fileIOImpl = SnowflakeResources.DEFAULT_FILE_IO_IMPL; + + if (catalogProperties.containsKey(CatalogProperties.FILE_IO_IMPL)) { + fileIOImpl = catalogProperties.get(CatalogProperties.FILE_IO_IMPL); + } + + fileIO = CatalogUtil.loadFileIO(fileIOImpl, catalogProperties, conf); + } + } + + @Override + public void close() { + snowflakeClient.close(); + } + + @Override + public void createNamespace(Namespace namespace, Map<String, String> metadata) { + throw new UnsupportedOperationException( + String.format("createNamespace not supported; attempted for namespace '%s'", namespace)); + } + + @Override + public List<Namespace> listNamespaces(Namespace namespace) { + LOG.debug("listNamespaces with namespace: {}", namespace); + Preconditions.checkArgument( + namespace.length() <= SnowflakeResources.MAX_NAMESPACE_DEPTH - 1, + "Snowflake doesn't support more than %s levels of namespace, tried to list under %s", + SnowflakeResources.MAX_NAMESPACE_DEPTH, + namespace); + List<SnowflakeSchema> sfSchemas = snowflakeClient.listSchemas(namespace); + + List<Namespace> namespaceList = + sfSchemas.stream() + .map(schema -> Namespace.of(schema.getDatabase(), schema.getName())) + .collect(Collectors.toList()); + return namespaceList; + } + + @Override + public Map<String, String> loadNamespaceMetadata(Namespace namespace) + throws NoSuchNamespaceException { + LOG.debug("loadNamespaceMetadata with namespace: {}", namespace); + Map<String, String> nameSpaceMetadata = Maps.newHashMap(); + nameSpaceMetadata.put("name", namespace.toString()); Review Comment: I don't think this behavior here is correct. If Namespace properties are not supported, then this should probably just return an empty immutable map ########## snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java: ########## @@ -0,0 +1,220 @@ +/* + * 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.snowflake; + +import java.io.Closeable; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.iceberg.BaseMetastoreCatalog; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SupportsNamespaces; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; +import org.apache.iceberg.hadoop.Configurable; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.jdbc.JdbcClientPool; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.snowflake.entities.SnowflakeSchema; +import org.apache.iceberg.snowflake.entities.SnowflakeTable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class SnowflakeCatalog extends BaseMetastoreCatalog + implements Closeable, SupportsNamespaces, Configurable<Object> { + + private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class); + + private Object conf; + private String catalogName = SnowflakeResources.DEFAULT_CATALOG_NAME; + private Map<String, String> catalogProperties = null; + private FileIO fileIO; + private SnowflakeClient snowflakeClient; + + public SnowflakeCatalog() {} + + @VisibleForTesting + void setSnowflakeClient(SnowflakeClient snowflakeClient) { + this.snowflakeClient = snowflakeClient; + } + + @VisibleForTesting + void setFileIO(FileIO fileIO) { + this.fileIO = fileIO; + } + + @Override + public List<TableIdentifier> listTables(Namespace namespace) { + LOG.debug("listTables with namespace: {}", namespace); + Preconditions.checkArgument( Review Comment: should this check potentially go into the `snowflakeClient`? ########## snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeClient.java: ########## @@ -0,0 +1,44 @@ +/* + * 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.snowflake; + +import java.io.Closeable; +import java.util.List; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.snowflake.entities.SnowflakeSchema; +import org.apache.iceberg.snowflake.entities.SnowflakeTable; +import org.apache.iceberg.snowflake.entities.SnowflakeTableMetadata; + +/** + * This interface abstracts out the underlying communication protocols for contacting Snowflake to + * obtain the various resource representations defined under "entities". Classes using this + * interface should minimize assumptions about whether an underlying client uses e.g. REST, JDBC or + * other underlying libraries/protocols. + */ +public interface SnowflakeClient extends Closeable { + List<SnowflakeSchema> listSchemas(Namespace namespace); + + List<SnowflakeTable> listIcebergTables(Namespace namespace); + + SnowflakeTableMetadata getTableMetadata(TableIdentifier tableIdentifier); + + @Override Review Comment: are we overriding close() only to remove the exception from the signature? If so, then I think it makes more sense to change it to `SnowflakeClient extends AutoCloseable`. With the `close()` method of `AutoCloseable` one can decide to not throw an exception. ########## snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java: ########## @@ -0,0 +1,220 @@ +/* + * 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.snowflake; + +import java.io.Closeable; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.iceberg.BaseMetastoreCatalog; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SupportsNamespaces; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; +import org.apache.iceberg.hadoop.Configurable; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.jdbc.JdbcClientPool; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.snowflake.entities.SnowflakeSchema; +import org.apache.iceberg.snowflake.entities.SnowflakeTable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class SnowflakeCatalog extends BaseMetastoreCatalog + implements Closeable, SupportsNamespaces, Configurable<Object> { + + private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class); + + private Object conf; + private String catalogName = SnowflakeResources.DEFAULT_CATALOG_NAME; + private Map<String, String> catalogProperties = null; + private FileIO fileIO; + private SnowflakeClient snowflakeClient; + + public SnowflakeCatalog() {} + + @VisibleForTesting + void setSnowflakeClient(SnowflakeClient snowflakeClient) { + this.snowflakeClient = snowflakeClient; + } + + @VisibleForTesting + void setFileIO(FileIO fileIO) { + this.fileIO = fileIO; + } + + @Override + public List<TableIdentifier> listTables(Namespace namespace) { + LOG.debug("listTables with namespace: {}", namespace); + Preconditions.checkArgument( + namespace.length() <= SnowflakeResources.MAX_NAMESPACE_DEPTH, + "Snowflake doesn't support more than %s levels of namespace, got %s", + SnowflakeResources.MAX_NAMESPACE_DEPTH, + namespace); + + List<SnowflakeTable> sfTables = snowflakeClient.listIcebergTables(namespace); + + return sfTables.stream() + .map( + table -> + TableIdentifier.of(table.getDatabase(), table.getSchemaName(), table.getName())) + .collect(Collectors.toList()); + } + + @Override + public boolean dropTable(TableIdentifier identifier, boolean purge) { + throw new UnsupportedOperationException( + String.format("dropTable not supported; attempted for table '%s'", identifier)); + } + + @Override + public void renameTable(TableIdentifier from, TableIdentifier to) { + throw new UnsupportedOperationException( + String.format("renameTable not supported; attempted from '%s' to '%s'", from, to)); + } + + @Override + public void initialize(String name, Map<String, String> properties) { + catalogProperties = properties; + + if (name != null) { + this.catalogName = name; + } + + if (snowflakeClient == null) { + String uri = properties.get(CatalogProperties.URI); + Preconditions.checkNotNull(uri, "JDBC connection URI is required"); + + try { + // We'll ensure the expected JDBC driver implementation class is initialized through + // reflection + // regardless of which classloader ends up using this JdbcSnowflakeClient, but we'll only + // warn if the expected driver fails to load, since users may use repackaged or custom + // JDBC drivers for Snowflake communcation. + Class.forName(JdbcSnowflakeClient.EXPECTED_JDBC_IMPL); + } catch (ClassNotFoundException cnfe) { + LOG.warn( + "Failed to load expected JDBC SnowflakeDriver - if queries fail by failing" + + " to find a suitable driver for jdbc:snowflake:// URIs, you must add the Snowflake " + + " JDBC driver to your jars/packages", + cnfe); + } + + JdbcClientPool connectionPool = new JdbcClientPool(uri, properties); + snowflakeClient = new JdbcSnowflakeClient(connectionPool); + } + + if (fileIO == null) { + String fileIOImpl = SnowflakeResources.DEFAULT_FILE_IO_IMPL; + + if (catalogProperties.containsKey(CatalogProperties.FILE_IO_IMPL)) { + fileIOImpl = catalogProperties.get(CatalogProperties.FILE_IO_IMPL); + } + + fileIO = CatalogUtil.loadFileIO(fileIOImpl, catalogProperties, conf); + } + } + + @Override + public void close() { + snowflakeClient.close(); + } + + @Override + public void createNamespace(Namespace namespace, Map<String, String> metadata) { + throw new UnsupportedOperationException( + String.format("createNamespace not supported; attempted for namespace '%s'", namespace)); + } + + @Override + public List<Namespace> listNamespaces(Namespace namespace) { + LOG.debug("listNamespaces with namespace: {}", namespace); + Preconditions.checkArgument( + namespace.length() <= SnowflakeResources.MAX_NAMESPACE_DEPTH - 1, + "Snowflake doesn't support more than %s levels of namespace, tried to list under %s", + SnowflakeResources.MAX_NAMESPACE_DEPTH, + namespace); + List<SnowflakeSchema> sfSchemas = snowflakeClient.listSchemas(namespace); + + List<Namespace> namespaceList = + sfSchemas.stream() + .map(schema -> Namespace.of(schema.getDatabase(), schema.getName())) + .collect(Collectors.toList()); + return namespaceList; + } + + @Override + public Map<String, String> loadNamespaceMetadata(Namespace namespace) + throws NoSuchNamespaceException { + LOG.debug("loadNamespaceMetadata with namespace: {}", namespace); + Map<String, String> nameSpaceMetadata = Maps.newHashMap(); + nameSpaceMetadata.put("name", namespace.toString()); + return nameSpaceMetadata; + } + + @Override + public boolean dropNamespace(Namespace namespace) { + throw new UnsupportedOperationException( + String.format("dropNamespace not supported; attempted for namespace '%s'", namespace)); + } + + @Override + public boolean setProperties(Namespace namespace, Map<String, String> properties) { + throw new UnsupportedOperationException( + String.format("setProperties not supported; attempted for namespace '%s'", namespace)); + } + + @Override + public boolean removeProperties(Namespace namespace, Set<String> properties) { + throw new UnsupportedOperationException( + String.format("removeProperties not supported; attempted for namespace '%s'", namespace)); + } + + @Override + protected TableOperations newTableOps(TableIdentifier tableIdentifier) { + Preconditions.checkArgument( Review Comment: this check shouldn't be done here. All it should be doing in this method is returning a new `SnowflakeTableOperations` instance ########## snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java: ########## @@ -0,0 +1,163 @@ +/* + * 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.snowflake; + +import java.sql.SQLException; +import java.util.List; +import org.apache.commons.dbutils.QueryRunner; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.jdbc.JdbcClientPool; +import org.apache.iceberg.jdbc.UncheckedInterruptedException; +import org.apache.iceberg.jdbc.UncheckedSQLException; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.snowflake.entities.SnowflakeSchema; +import org.apache.iceberg.snowflake.entities.SnowflakeTable; +import org.apache.iceberg.snowflake.entities.SnowflakeTableMetadata; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This implementation of SnowflakeClient builds on top of Snowflake's JDBC driver to interact with + * Snowflake's Iceberg-aware resource model. Despite using JDBC libraries, the resource model is + * derived from Snowflake's own first-class support for Iceberg tables as opposed to using an opaque + * JDBC layer to store Iceberg metadata itself in an Iceberg-agnostic database. + * + * <p>This thus differs from the JdbcCatalog in that Snowflake's service provides the source of + * truth of Iceberg metadata, rather than serving as a storage layer for a client-defined Iceberg + * resource model. + */ +public class JdbcSnowflakeClient implements SnowflakeClient { + public static final String EXPECTED_JDBC_IMPL = "net.snowflake.client.jdbc.SnowflakeDriver"; + + private static final Logger LOG = LoggerFactory.getLogger(JdbcSnowflakeClient.class); + private final JdbcClientPool connectionPool; + private QueryRunner queryRunner = new QueryRunner(true); + + JdbcSnowflakeClient(JdbcClientPool conn) { + connectionPool = conn; + } + + @VisibleForTesting + void setQueryRunner(QueryRunner queryRunner) { + this.queryRunner = queryRunner; + } + + @Override + public List<SnowflakeSchema> listSchemas(Namespace namespace) { + StringBuilder baseQuery = new StringBuilder("SHOW SCHEMAS"); + Object[] queryParams = null; + if (namespace == null || namespace.isEmpty()) { + // for empty or null namespace search for all schemas at account level where the user + // has access to list. + baseQuery.append(" IN ACCOUNT"); + } else { + // otherwise restrict listing of schema within the database. + baseQuery.append(" IN DATABASE IDENTIFIER(?)"); + queryParams = new Object[] {namespace.level(SnowflakeResources.NAMESPACE_DB_LEVEL - 1)}; + } + + final String finalQuery = baseQuery.toString(); + final Object[] finalQueryParams = queryParams; + List<SnowflakeSchema> schemas; + try { + schemas = + connectionPool.run( + conn -> + queryRunner.query( + conn, finalQuery, SnowflakeSchema.createHandler(), finalQueryParams)); + } catch (SQLException e) { + throw new UncheckedSQLException( + e, + "Failed to list schemas for namespace %s", + namespace != null ? namespace.toString() : ""); + } catch (InterruptedException e) { + throw new UncheckedInterruptedException(e, "Interrupted while listing schemas"); + } + return schemas; + } + + @Override + public List<SnowflakeTable> listIcebergTables(Namespace namespace) { + StringBuilder baseQuery = new StringBuilder("SHOW ICEBERG TABLES"); + Object[] queryParams = null; + if (namespace.length() == SnowflakeResources.MAX_NAMESPACE_DEPTH) { + // For two level namespace, search for iceberg tables within the given schema. + baseQuery.append(" IN SCHEMA IDENTIFIER(?)"); + queryParams = + new Object[] { + String.format( + "%s.%s", + namespace.level(SnowflakeResources.NAMESPACE_DB_LEVEL - 1), + namespace.level(SnowflakeResources.NAMESPACE_SCHEMA_LEVEL - 1)) + }; + } else if (namespace.length() == SnowflakeResources.NAMESPACE_DB_LEVEL) { + // For one level namespace, search for iceberg tables within the given database. + baseQuery.append(" IN DATABASE IDENTIFIER(?)"); + queryParams = new Object[] {namespace.level(SnowflakeResources.NAMESPACE_DB_LEVEL - 1)}; + } else { + // For empty or db level namespace, search at account level. + baseQuery.append(" IN ACCOUNT"); + } + + final String finalQuery = baseQuery.toString(); + final Object[] finalQueryParams = queryParams; + List<SnowflakeTable> tables; + try { + tables = + connectionPool.run( + conn -> + queryRunner.query( + conn, finalQuery, SnowflakeTable.createHandler(), finalQueryParams)); + } catch (SQLException e) { + throw new UncheckedSQLException( + e, "Failed to list tables for namespace %s", namespace.toString()); + } catch (InterruptedException e) { + throw new UncheckedInterruptedException(e, "Interrupted while listing tables"); + } + return tables; + } + + @Override + public SnowflakeTableMetadata getTableMetadata(TableIdentifier tableIdentifier) { + SnowflakeTableMetadata tableMeta; + try { + final String finalQuery = "SELECT SYSTEM$GET_ICEBERG_TABLE_INFORMATION(?) AS METADATA"; + tableMeta = + connectionPool.run( + conn -> + queryRunner.query( + conn, + finalQuery, + SnowflakeTableMetadata.createHandler(), + tableIdentifier.toString())); + } catch (SQLException e) { + throw new UncheckedSQLException( + e, "Failed to get table metadata for %s", tableIdentifier.toString()); + } catch (InterruptedException e) { + throw new UncheckedInterruptedException(e, "Interrupted while getting table metadata"); + } + return tableMeta; + } + + @Override + public void close() { + connectionPool.close(); Review Comment: should this have a null check? ########## snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java: ########## @@ -0,0 +1,163 @@ +/* + * 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.snowflake; + +import java.sql.SQLException; +import java.util.List; +import org.apache.commons.dbutils.QueryRunner; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.jdbc.JdbcClientPool; +import org.apache.iceberg.jdbc.UncheckedInterruptedException; +import org.apache.iceberg.jdbc.UncheckedSQLException; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.snowflake.entities.SnowflakeSchema; +import org.apache.iceberg.snowflake.entities.SnowflakeTable; +import org.apache.iceberg.snowflake.entities.SnowflakeTableMetadata; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This implementation of SnowflakeClient builds on top of Snowflake's JDBC driver to interact with + * Snowflake's Iceberg-aware resource model. Despite using JDBC libraries, the resource model is + * derived from Snowflake's own first-class support for Iceberg tables as opposed to using an opaque + * JDBC layer to store Iceberg metadata itself in an Iceberg-agnostic database. + * + * <p>This thus differs from the JdbcCatalog in that Snowflake's service provides the source of + * truth of Iceberg metadata, rather than serving as a storage layer for a client-defined Iceberg + * resource model. + */ +public class JdbcSnowflakeClient implements SnowflakeClient { + public static final String EXPECTED_JDBC_IMPL = "net.snowflake.client.jdbc.SnowflakeDriver"; + + private static final Logger LOG = LoggerFactory.getLogger(JdbcSnowflakeClient.class); + private final JdbcClientPool connectionPool; + private QueryRunner queryRunner = new QueryRunner(true); + + JdbcSnowflakeClient(JdbcClientPool conn) { + connectionPool = conn; Review Comment: should this have a `Preconditions.checkArgument(null != conn, "...")`? ########## snowflake/src/test/java/org/apache/iceberg/snowflake/SnowflakeCatalogTest.java: ########## @@ -0,0 +1,231 @@ +/* + * 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.snowflake; + +import java.util.List; +import java.util.Map; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableMetadataParser; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.snowflake.entities.SnowflakeTableMetadata; +import org.apache.iceberg.types.Types; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class SnowflakeCatalogTest { + + static final String TEST_CATALOG_NAME = "slushLog"; + private SnowflakeCatalog catalog; + + @Before + public void before() { + catalog = new SnowflakeCatalog(); + + FakeSnowflakeClient client = new FakeSnowflakeClient(); + client.addTable( + "DB_1", + "SCHEMA_1", + "TAB_1", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"s3://tab1/metadata/v3.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_1", + "SCHEMA_1", + "TAB_2", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"s3://tab2/metadata/v1.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_2", + "SCHEMA_2", + "TAB_3", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"azure://myaccount.blob.core.windows.net/mycontainer/tab3/metadata/v334.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_2", + "SCHEMA_2", + "TAB_4", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"azure://myaccount.blob.core.windows.net/mycontainer/tab4/metadata/v323.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_3", + "SCHEMA_3", + "TAB_5", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"gcs://tab5/metadata/v793.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_3", + "SCHEMA_4", + "TAB_6", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"gcs://tab6/metadata/v123.metadata.json\",\"status\":\"success\"}")); + + catalog.setSnowflakeClient(client); + + InMemoryFileIO fakeFileIO = new InMemoryFileIO(); + + Schema schema = + new Schema( + Types.NestedField.required(1, "x", Types.StringType.get(), "comment1"), + Types.NestedField.required(2, "y", Types.StringType.get(), "comment2")); + PartitionSpec partitionSpec = + PartitionSpec.builderFor(schema).identity("x").withSpecId(1000).build(); + fakeFileIO.addFile( + "s3://tab1/metadata/v3.metadata.json", + TableMetadataParser.toJson( + TableMetadata.newTableMetadata( + schema, partitionSpec, "s3://tab1/", ImmutableMap.<String, String>of())) + .getBytes()); + fakeFileIO.addFile( + "wasbs://mycontai...@myaccount.blob.core.windows.net/tab3/metadata/v334.metadata.json", + TableMetadataParser.toJson( + TableMetadata.newTableMetadata( + schema, + partitionSpec, + "wasbs://mycontai...@myaccount.blob.core.windows.net/tab1/", + ImmutableMap.<String, String>of())) + .getBytes()); + fakeFileIO.addFile( + "gs://tab5/metadata/v793.metadata.json", + TableMetadataParser.toJson( + TableMetadata.newTableMetadata( + schema, partitionSpec, "gs://tab5/", ImmutableMap.<String, String>of())) + .getBytes()); + + catalog.setFileIO(fakeFileIO); + + Map<String, String> properties = Maps.newHashMap(); + catalog.initialize(TEST_CATALOG_NAME, properties); + } + + @Test + public void testListNamespace() { + List<Namespace> namespaces = catalog.listNamespaces(); + Assert.assertEquals( + Lists.newArrayList( + Namespace.of("DB_1", "SCHEMA_1"), + Namespace.of("DB_2", "SCHEMA_2"), + Namespace.of("DB_3", "SCHEMA_3"), + Namespace.of("DB_3", "SCHEMA_4")), + namespaces); + } + + @Test + public void testListNamespaceWithinDB() { + String dbName = "DB_1"; + List<Namespace> namespaces = catalog.listNamespaces(Namespace.of(dbName)); + Assert.assertEquals(Lists.newArrayList(Namespace.of(dbName, "SCHEMA_1")), namespaces); + } + + @Test + public void testListNamespaceWithinNonExistentDB() { + // Existence check for nonexistent parent namespaces is optional in the SupportsNamespaces + // interface. + String dbName = "NONEXISTENT_DB"; + Assert.assertThrows(RuntimeException.class, () -> catalog.listNamespaces(Namespace.of(dbName))); Review Comment: given that this is new code being added, it would be great if we could switch all assertions to AssertJ as that makes it later easier if we decide to move from Junit4 to Junit5. This check in particular would then be something like `Assertions.assertThatThrownBy(() -> catalog.listNamespaces(Namespace.of(dbName))).isInstanceOf(RuntimeException.class).hasMessage(...)`. Note that it's generally good practive to also assert on the message to make sure the right exception with the right message is thrown ########## snowflake/src/test/java/org/apache/iceberg/snowflake/SnowflakeCatalogTest.java: ########## @@ -0,0 +1,231 @@ +/* + * 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.snowflake; + +import java.util.List; +import java.util.Map; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableMetadataParser; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.snowflake.entities.SnowflakeTableMetadata; +import org.apache.iceberg.types.Types; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class SnowflakeCatalogTest { + + static final String TEST_CATALOG_NAME = "slushLog"; + private SnowflakeCatalog catalog; + + @Before + public void before() { + catalog = new SnowflakeCatalog(); + + FakeSnowflakeClient client = new FakeSnowflakeClient(); + client.addTable( + "DB_1", + "SCHEMA_1", + "TAB_1", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"s3://tab1/metadata/v3.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_1", + "SCHEMA_1", + "TAB_2", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"s3://tab2/metadata/v1.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_2", + "SCHEMA_2", + "TAB_3", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"azure://myaccount.blob.core.windows.net/mycontainer/tab3/metadata/v334.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_2", + "SCHEMA_2", + "TAB_4", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"azure://myaccount.blob.core.windows.net/mycontainer/tab4/metadata/v323.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_3", + "SCHEMA_3", + "TAB_5", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"gcs://tab5/metadata/v793.metadata.json\",\"status\":\"success\"}")); + client.addTable( + "DB_3", + "SCHEMA_4", + "TAB_6", + SnowflakeTableMetadata.parseJson( + "{\"metadataLocation\":\"gcs://tab6/metadata/v123.metadata.json\",\"status\":\"success\"}")); + + catalog.setSnowflakeClient(client); + + InMemoryFileIO fakeFileIO = new InMemoryFileIO(); + + Schema schema = + new Schema( + Types.NestedField.required(1, "x", Types.StringType.get(), "comment1"), + Types.NestedField.required(2, "y", Types.StringType.get(), "comment2")); + PartitionSpec partitionSpec = + PartitionSpec.builderFor(schema).identity("x").withSpecId(1000).build(); + fakeFileIO.addFile( + "s3://tab1/metadata/v3.metadata.json", + TableMetadataParser.toJson( + TableMetadata.newTableMetadata( + schema, partitionSpec, "s3://tab1/", ImmutableMap.<String, String>of())) + .getBytes()); + fakeFileIO.addFile( + "wasbs://mycontai...@myaccount.blob.core.windows.net/tab3/metadata/v334.metadata.json", + TableMetadataParser.toJson( + TableMetadata.newTableMetadata( + schema, + partitionSpec, + "wasbs://mycontai...@myaccount.blob.core.windows.net/tab1/", + ImmutableMap.<String, String>of())) + .getBytes()); + fakeFileIO.addFile( + "gs://tab5/metadata/v793.metadata.json", + TableMetadataParser.toJson( + TableMetadata.newTableMetadata( + schema, partitionSpec, "gs://tab5/", ImmutableMap.<String, String>of())) + .getBytes()); + + catalog.setFileIO(fakeFileIO); + + Map<String, String> properties = Maps.newHashMap(); + catalog.initialize(TEST_CATALOG_NAME, properties); + } + + @Test + public void testListNamespace() { + List<Namespace> namespaces = catalog.listNamespaces(); + Assert.assertEquals( Review Comment: could be simplified to ``` Assertions.assertThat(catalog.listNamespaces()) .containsExactly( Namespace.of("DB_1", "SCHEMA_1"), Namespace.of("DB_2", "SCHEMA_2"), Namespace.of("DB_3", "SCHEMA_3"), Namespace.of("DB_3", "SCHEMA_4")); ``` ########## snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java: ########## @@ -0,0 +1,220 @@ +/* + * 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.snowflake; + +import java.io.Closeable; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.iceberg.BaseMetastoreCatalog; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SupportsNamespaces; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; +import org.apache.iceberg.hadoop.Configurable; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.jdbc.JdbcClientPool; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.snowflake.entities.SnowflakeSchema; +import org.apache.iceberg.snowflake.entities.SnowflakeTable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class SnowflakeCatalog extends BaseMetastoreCatalog + implements Closeable, SupportsNamespaces, Configurable<Object> { + + private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class); + + private Object conf; + private String catalogName = SnowflakeResources.DEFAULT_CATALOG_NAME; + private Map<String, String> catalogProperties = null; + private FileIO fileIO; + private SnowflakeClient snowflakeClient; + + public SnowflakeCatalog() {} + + @VisibleForTesting + void setSnowflakeClient(SnowflakeClient snowflakeClient) { + this.snowflakeClient = snowflakeClient; + } + + @VisibleForTesting + void setFileIO(FileIO fileIO) { + this.fileIO = fileIO; + } + + @Override + public List<TableIdentifier> listTables(Namespace namespace) { + LOG.debug("listTables with namespace: {}", namespace); + Preconditions.checkArgument( + namespace.length() <= SnowflakeResources.MAX_NAMESPACE_DEPTH, + "Snowflake doesn't support more than %s levels of namespace, got %s", + SnowflakeResources.MAX_NAMESPACE_DEPTH, + namespace); + + List<SnowflakeTable> sfTables = snowflakeClient.listIcebergTables(namespace); + + return sfTables.stream() + .map( + table -> + TableIdentifier.of(table.getDatabase(), table.getSchemaName(), table.getName())) + .collect(Collectors.toList()); + } + + @Override + public boolean dropTable(TableIdentifier identifier, boolean purge) { + throw new UnsupportedOperationException( + String.format("dropTable not supported; attempted for table '%s'", identifier)); + } + + @Override + public void renameTable(TableIdentifier from, TableIdentifier to) { + throw new UnsupportedOperationException( + String.format("renameTable not supported; attempted from '%s' to '%s'", from, to)); + } + + @Override + public void initialize(String name, Map<String, String> properties) { + catalogProperties = properties; + + if (name != null) { Review Comment: I think what would make sense in such a case is to have an alternative `initialize(String name, SnowflakeClient snowflakeClient, FileIO fileIO, Map<String, String> properties)` method. The default `initialize(String name, Map<String, String> properties)` method would then call this alternative one. We have a similar pattern in the `NessieCatalog`. Then I think you could remove the `setSnowflakeClient` / `setFileIO` methods and always call one of the `initialize` methods -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org