You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2022/12/15 03:10:18 UTC

[GitHub] [iceberg] dennishuo opened a new pull request, #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

dennishuo opened a new pull request, #6428:
URL: https://github.com/apache/iceberg/pull/6428

   This read-only implementation of the Catalog interface, initially built on top of the [Snowflake JDBC driver](https://docs.snowflake.com/en/user-guide/jdbc.html) for the connection layer, enables engines like Spark using the Iceberg Java SDK to be able to consume [Snowflake-managed Iceberg Tables](https://www.snowflake.com/blog/iceberg-tables-powering-open-standards-with-snowflake-innovations/) via Iceberg Catalog interfaces.
   
   Example, assuming a Snowflake account with a database `iot_data` containing a schema `public` and a managed Iceberg table `sensor_test_results`:
   
       spark-shell --conf spark.sql.catalog.snowlog=org.apache.iceberg.spark.SparkCatalog \
           --conf spark.sql.catalog.snowlog.catalog-impl=org.apache.iceberg.snowflake.SnowflakeCatalog \
           --conf spark.sql.catalog.snowlog.uri="jdbc:snowflake://$ACCOUNT.snowflakecomputing.com" \
           ....
       scala> spark.sessionState.catalogManager.setCurrentCatalog("snowlog");
       scala> spark.sql("show namespaces in iot_data").show(false);
       scala> spark.sql("select * from iot_data.public.sensor_test_results limit 10").show(false);
   
   Note that the involvement of a JDBC driver is only incidental, and functionality is different from the `JdbcCatalog` - here, Snowflake itself manages manifest/metadata files and table/snapshot metadata, and this catalog layer facilitates the coordination of metadata-file locations and discovery of the latest table snapshot versions without resorting to file-listing or "directory-name"-listing (for listTables or listNamespaces) like the HadoopCatalog.


-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1050257006


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

Review Comment:
   Good suggestion! I recall skimming over it when looking at the JdbcCatalog's unittest, but saw most of the test cases were for (currently) unsupported functionality. Just for completeness though I went ahead and plugged it in (which helped identify the need to specify `test { useJUnitPlatform() }` in build.gradle to successfully inherit the tests) and verified that they all fail due to depending on at least `buildTable/createNamespace` to set up test data, as @sfc-gh-mparmar mentioned.
   
   One possibility if we want to split out read-only variations even for temporary scenarios, would be to abstract out the mutations into helper methods in the `CatalogTests` base class, and only test cases that explicitly test mutations would call the catalog directly, letting read-only stuff do the mutations in their fake in-memory data model.
   
   It seems like the `supports*` test methods are also pointing towards possibly wanting to extract a more formalized way to define differences in supported functionality between catalogs, perhaps more fine-grained than what things like e.g. `SupportsNamespaces` java interfaces provide.
   
   Is there any work in progress to do something for Catalog interfaces analogous to the [HCFS efforts for HadoopFileSystem](https://cwiki.apache.org/confluence/display/HADOOP2/HCFS) ? 
   
   Seems like CatalogTests may have already been written with the analogous [AbstractFSContractTestBase](https://github.com/apache/hadoop/blob/03cfc852791c14fad39db4e5b14104a276c08e59/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContractTestBase.java) in mind, where we could move towards analogous ["contract" definitions](https://github.com/apache/hadoop/blob/03cfc852791c14fad39db4e5b14104a276c08e59/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/LocalFSContract.java) and [contract config specs](https://github.com/apache/hadoop/blob/03cfc852791c14fad39db4e5b14104a276c08e59/hadoop-common-project/hadoop-common/src/test/resources/contract/localfs.xml)



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] nastra commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
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://mycontainer@myaccount.blob.core.windows.net/tab3/metadata/v334.metadata.json",
+        TableMetadataParser.toJson(
+                TableMetadata.newTableMetadata(
+                    schema,
+                    partitionSpec,
+                    "wasbs://mycontainer@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://mycontainer@myaccount.blob.core.windows.net/tab3/metadata/v334.metadata.json",
+        TableMetadataParser.toJson(
+                TableMetadata.newTableMetadata(
+                    schema,
+                    partitionSpec,
+                    "wasbs://mycontainer@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://mycontainer@myaccount.blob.core.windows.net/tab3/metadata/v334.metadata.json",
+        TableMetadataParser.toJson(
+                TableMetadata.newTableMetadata(
+                    schema,
+                    partitionSpec,
+                    "wasbs://mycontainer@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://mycontainer@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://mycontainer@myaccount.blob.core.windows.net/tab3/metadata/v334.metadata.json",
+        TableMetadataParser.toJson(
+                TableMetadata.newTableMetadata(
+                    schema,
+                    partitionSpec,
+                    "wasbs://mycontainer@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://mycontainer@myaccount.blob.core.windows.net/tab3/metadata/v334.metadata.json",
+        TableMetadataParser.toJson(
+                TableMetadata.newTableMetadata(
+                    schema,
+                    partitionSpec,
+                    "wasbs://mycontainer@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://mycontainer@myaccount.blob.core.windows.net/tab3/metadata/v334.metadata.json",
+        TableMetadataParser.toJson(
+                TableMetadata.newTableMetadata(
+                    schema,
+                    partitionSpec,
+                    "wasbs://mycontainer@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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051245397


##########
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:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1052728003


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/entities/SnowflakeIdentifier.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class SnowflakeIdentifier {
+  public enum Type {
+    ROOT,
+    DATABASE,
+    SCHEMA,
+    TABLE
+  }
+
+  private String databaseName;
+  private String schemaName;
+  private String tableName;
+
+  protected SnowflakeIdentifier(String databaseName, String schemaName, String tableName) {

Review Comment:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1056621897


##########
snowflake/src/test/java/org/apache/iceberg/snowflake/FakeSnowflakeClient.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.jdbc.UncheckedSQLException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class FakeSnowflakeClient implements SnowflakeClient {

Review Comment:
   Yeah, it'd definitely be nice to set up a real integration test in the future. It'll take some planning though since setup is pretty heavyweight between setting up cloud storage credentials alongside the snowflake account and configuration/feature-enablement.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1056628417


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/NamespaceHelpers.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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 org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class NamespaceHelpers {
+  private static final int MAX_NAMESPACE_DEPTH = 2;
+  private static final int NAMESPACE_ROOT_LEVEL = 0;
+  private static final int NAMESPACE_DB_LEVEL = 1;
+  private static final int NAMESPACE_SCHEMA_LEVEL = 2;
+
+  private static final Logger LOG = LoggerFactory.getLogger(NamespaceHelpers.class);
+
+  private NamespaceHelpers() {}
+
+  /**
+   * Converts a Namespace into a SnowflakeIdentifier representing ROOT, a DATABASE, or a SCHEMA.
+   *
+   * @throws IllegalArgumentException if the namespace is not a supported depth.
+   */
+  public static SnowflakeIdentifier toSnowflakeIdentifier(Namespace namespace) {

Review Comment:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1052973867


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeTableOperations.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.Map;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.apache.iceberg.snowflake.entities.SnowflakeTableMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class SnowflakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeTableOperations.class);
+  private final String catalogName;
+
+  private final FileIO fileIO;
+  private final TableIdentifier tableIdentifier;
+  private final SnowflakeIdentifier snowflakeIdentifierForTable;
+
+  private final SnowflakeClient snowflakeClient;
+
+  private final Map<String, String> catalogProperties;

Review Comment:
   Removed, rolled just `catalogName` into `fullTableName`



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1054042049


##########
snowflake/src/test/java/org/apache/iceberg/snowflake/SnowflakeCatalogTest.java:
##########
@@ -0,0 +1,277 @@
+/*
+ * 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.IOException;
+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.Maps;
+import org.apache.iceberg.snowflake.entities.SnowflakeTableMetadata;
+import org.apache.iceberg.types.Types;
+import org.assertj.core.api.Assertions;
+import org.junit.Before;
+import org.junit.Test;
+
+public class SnowflakeCatalogTest {
+
+  static final String TEST_CATALOG_NAME = "slushLog";
+  private SnowflakeCatalog catalog;
+  private FakeSnowflakeClient fakeClient;
+  private InMemoryFileIO fakeFileIO;
+  private Map<String, String> properties;
+
+  @Before
+  public void before() {
+    catalog = new SnowflakeCatalog();
+
+    fakeClient = new FakeSnowflakeClient();
+    fakeClient.addTable(
+        "DB_1",
+        "SCHEMA_1",
+        "TAB_1",
+        SnowflakeTableMetadata.parseJson(
+            "{\"metadataLocation\":\"s3://tab1/metadata/v3.metadata.json\",\"status\":\"success\"}"));
+    fakeClient.addTable(
+        "DB_1",
+        "SCHEMA_1",
+        "TAB_2",
+        SnowflakeTableMetadata.parseJson(
+            "{\"metadataLocation\":\"s3://tab2/metadata/v1.metadata.json\",\"status\":\"success\"}"));
+    fakeClient.addTable(
+        "DB_2",
+        "SCHEMA_2",
+        "TAB_3",
+        SnowflakeTableMetadata.parseJson(
+            "{\"metadataLocation\":\"azure://myaccount.blob.core.windows.net/mycontainer/tab3/metadata/v334.metadata.json\",\"status\":\"success\"}"));
+    fakeClient.addTable(
+        "DB_2",
+        "SCHEMA_2",
+        "TAB_4",
+        SnowflakeTableMetadata.parseJson(
+            "{\"metadataLocation\":\"azure://myaccount.blob.core.windows.net/mycontainer/tab4/metadata/v323.metadata.json\",\"status\":\"success\"}"));
+    fakeClient.addTable(
+        "DB_3",
+        "SCHEMA_3",
+        "TAB_5",
+        SnowflakeTableMetadata.parseJson(
+            "{\"metadataLocation\":\"gcs://tab5/metadata/v793.metadata.json\",\"status\":\"success\"}"));
+    fakeClient.addTable(
+        "DB_3",
+        "SCHEMA_4",
+        "TAB_6",
+        SnowflakeTableMetadata.parseJson(
+            "{\"metadataLocation\":\"gcs://tab6/metadata/v123.metadata.json\",\"status\":\"success\"}"));
+
+    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://mycontainer@myaccount.blob.core.windows.net/tab3/metadata/v334.metadata.json",
+        TableMetadataParser.toJson(
+                TableMetadata.newTableMetadata(
+                    schema,
+                    partitionSpec,
+                    "wasbs://mycontainer@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());
+
+    properties = Maps.newHashMap();
+    catalog.initialize(TEST_CATALOG_NAME, fakeClient, fakeFileIO, properties);
+  }
+
+  @Test
+  public void testInitializeNullClient() {
+    Assertions.assertThatExceptionOfType(IllegalArgumentException.class)
+        .isThrownBy(() -> catalog.initialize(TEST_CATALOG_NAME, null, fakeFileIO, properties))
+        .withMessageContaining("snowflakeClient must be non-null");
+  }
+
+  @Test
+  public void testInitializeNullFileIO() {
+    Assertions.assertThatExceptionOfType(IllegalArgumentException.class)
+        .isThrownBy(() -> catalog.initialize(TEST_CATALOG_NAME, fakeClient, null, properties))
+        .withMessageContaining("fileIO must be non-null");
+  }
+
+  @Test
+  public void testListNamespace() {
+    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"));
+  }
+
+  @Test
+  public void testListNamespaceWithinDB() {
+    String dbName = "DB_1";
+    Assertions.assertThat(catalog.listNamespaces(Namespace.of(dbName)))
+        .containsExactly(Namespace.of(dbName, "SCHEMA_1"));
+  }
+
+  @Test
+  public void testListNamespaceWithinNonExistentDB() {
+    // Existence check for nonexistent parent namespaces is optional in the SupportsNamespaces
+    // interface.
+    String dbName = "NONEXISTENT_DB";
+    Assertions.assertThatExceptionOfType(RuntimeException.class)
+        .isThrownBy(() -> catalog.listNamespaces(Namespace.of(dbName)))
+        .withMessageContaining("does not exist")
+        .withMessageContaining(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";
+    Assertions.assertThatExceptionOfType(IllegalArgumentException.class)
+        .isThrownBy(() -> catalog.listNamespaces(Namespace.of(dbName, schemaName)))
+        .withMessageContaining("level")
+        .withMessageContaining("DB_3.SCHEMA_4");
+  }
+
+  @Test
+  public void testListTables() {
+    Assertions.assertThat(catalog.listTables(Namespace.empty()))
+        .containsExactly(
+            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"));
+  }
+
+  @Test
+  public void testListTablesWithinDB() {
+    String dbName = "DB_1";
+    Assertions.assertThat(catalog.listTables(Namespace.of(dbName)))
+        .containsExactly(
+            TableIdentifier.of("DB_1", "SCHEMA_1", "TAB_1"),
+            TableIdentifier.of("DB_1", "SCHEMA_1", "TAB_2"));
+  }
+
+  @Test
+  public void testListTablesWithinNonexistentDB() {
+    String dbName = "NONEXISTENT_DB";
+    Assertions.assertThatExceptionOfType(RuntimeException.class)
+        .isThrownBy(() -> catalog.listTables(Namespace.of(dbName)))
+        .withMessageContaining("does not exist")
+        .withMessageContaining(dbName);
+  }
+
+  @Test
+  public void testListTablesWithinSchema() {
+    String dbName = "DB_2";
+    String schemaName = "SCHEMA_2";
+    Assertions.assertThat(catalog.listTables(Namespace.of(dbName, schemaName)))
+        .containsExactly(
+            TableIdentifier.of("DB_2", "SCHEMA_2", "TAB_3"),
+            TableIdentifier.of("DB_2", "SCHEMA_2", "TAB_4"));
+  }
+
+  @Test
+  public void testListTablesWithinNonexistentSchema() {
+    String dbName = "DB_2";
+    String schemaName = "NONEXISTENT_SCHEMA";
+    Assertions.assertThatExceptionOfType(RuntimeException.class)
+        .isThrownBy(() -> catalog.listTables(Namespace.of(dbName, schemaName)))
+        .withMessageContaining("does not exist")
+        .withMessageContaining("DB_2.NONEXISTENT_SCHEMA");
+  }
+
+  @Test
+  public void testLoadS3Table() {
+    Table table = catalog.loadTable(TableIdentifier.of(Namespace.of("DB_1", "SCHEMA_1"), "TAB_1"));
+    Assertions.assertThat(table.location()).isEqualTo("s3://tab1/");
+  }
+
+  @Test
+  public void testLoadAzureTable() {
+    Table table = catalog.loadTable(TableIdentifier.of(Namespace.of("DB_2", "SCHEMA_2"), "TAB_3"));
+    Assertions.assertThat(table.location())
+        .isEqualTo("wasbs://mycontainer@myaccount.blob.core.windows.net/tab1/");
+  }
+
+  @Test
+  public void testLoadGcsTable() {
+    Table table = catalog.loadTable(TableIdentifier.of(Namespace.of("DB_3", "SCHEMA_3"), "TAB_5"));
+    Assertions.assertThat(table.location()).isEqualTo("gs://tab5/");
+  }
+
+  @Test
+  public void testLoadTableWithMalformedTableIdentifier() {
+    Assertions.assertThatExceptionOfType(IllegalArgumentException.class)
+        .isThrownBy(
+            () ->
+                catalog.loadTable(
+                    TableIdentifier.of(Namespace.of("DB_1", "SCHEMA_1", "BAD_NS_LEVEL"), "TAB_1")))
+        .withMessageContaining("level")
+        .withMessageContaining("DB_1.SCHEMA_1.BAD_NS_LEVEL");
+    Assertions.assertThatExceptionOfType(IllegalArgumentException.class)
+        .isThrownBy(
+            () -> catalog.loadTable(TableIdentifier.of(Namespace.of("DB_WITHOUT_SCHEMA"), "TAB_1")))
+        .withMessageContaining("level")
+        .withMessageContaining("DB_WITHOUT_SCHEMA.TAB_1");
+  }
+
+  @Test
+  public void testCloseBeforeInitialize() throws IOException {
+    catalog = new SnowflakeCatalog();
+    catalog.close();

Review Comment:
   Good point, added assertion that despite this no-exception close() call, we didn't close the injected `fakeClient`.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1066449738


##########
snowflake/src/test/java/org/apache/iceberg/snowflake/NamespaceHelpersTest.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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 org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+public class NamespaceHelpersTest {
+  @Test
+  public void testToSnowflakeIdentifierRoot() {
+    Assertions.assertThat(NamespaceHelpers.toSnowflakeIdentifier(Namespace.empty()))
+        .isEqualTo(SnowflakeIdentifier.ofRoot());
+  }
+
+  @Test
+  public void testToSnowflakeIdentifierDatabase() {
+    Assertions.assertThat(NamespaceHelpers.toSnowflakeIdentifier(Namespace.of("DB1")))
+        .isEqualTo(SnowflakeIdentifier.ofDatabase("DB1"));
+  }
+
+  @Test
+  public void testToSnowflakeIdentifierSchema() {
+    Assertions.assertThat(NamespaceHelpers.toSnowflakeIdentifier(Namespace.of("DB1", "SCHEMA1")))
+        .isEqualTo(SnowflakeIdentifier.ofSchema("DB1", "SCHEMA1"));
+  }
+
+  @Test
+  public void testToSnowflakeIdentifierMaxNamespaceLevel() {
+    Assertions.assertThatExceptionOfType(IllegalArgumentException.class)
+        .isThrownBy(
+            () ->
+                NamespaceHelpers.toSnowflakeIdentifier(
+                    Namespace.of("DB1", "SCHEMA1", "THIRD_NS_LVL")))
+        .withMessageContaining("max namespace level");
+  }
+
+  @Test
+  public void testToSnowflakeIdentifierTable() {
+    Assertions.assertThat(
+            NamespaceHelpers.toSnowflakeIdentifier(TableIdentifier.of("DB1", "SCHEMA1", "TABLE1")))
+        .isEqualTo(SnowflakeIdentifier.ofTable("DB1", "SCHEMA1", "TABLE1"));
+  }
+
+  @Test
+  public void testToSnowflakeIdentifierTableBadNamespace() {
+    Assertions.assertThatExceptionOfType(IllegalArgumentException.class)
+        .isThrownBy(
+            () ->
+                NamespaceHelpers.toSnowflakeIdentifier(
+                    TableIdentifier.of(Namespace.of("DB1_WITHOUT_SCHEMA"), "TABLE1")))
+        .withMessageContaining("must be at the SCHEMA level");
+  }
+
+  @Test
+  public void testToIcebergNamespaceRoot() {
+    Assertions.assertThat(NamespaceHelpers.toIcebergNamespace(SnowflakeIdentifier.ofRoot()))

Review Comment:
   Good point about round trips; I do normally have a tendency to write single large test cases just for convenience, so I was trying to do better about following https://www.oreilly.com/library/view/java-extreme-programming/0596003870/ch04s05.html :)
   
   But I agree the right granularity is flexible depending on the scenarios. I went ahead and made all the "success" test cases be combined round-trip tests instead, and kept "failure" cases separate.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1063850430


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.hadoop.HadoopFileIO";
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    LOG.debug("listTables with namespace: {}", namespace);
+    SnowflakeIdentifier scope = NamespaceHelpers.getSnowflakeIdentifierForNamespace(namespace);
+    Preconditions.checkArgument(
+        scope.getType() == SnowflakeIdentifier.Type.ROOT
+            || scope.getType() == SnowflakeIdentifier.Type.DATABASE
+            || scope.getType() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at ROOT, DATABASE, or SCHEMA level; got %s from namespace %s",
+        scope,
+        namespace);
+
+    List<SnowflakeIdentifier> sfTables = snowflakeClient.listIcebergTables(scope);
+
+    return sfTables.stream()
+        .map(
+            table ->
+                TableIdentifier.of(
+                    table.getDatabaseName(), table.getSchemaName(), table.getTableName()))
+        .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) {
+    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);
+
+    String fileIOImpl = DEFAULT_FILE_IO_IMPL;
+    if (properties.containsKey(CatalogProperties.FILE_IO_IMPL)) {
+      fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL);
+    }
+
+    initialize(
+        name,
+        new JdbcSnowflakeClient(connectionPool),
+        CatalogUtil.loadFileIO(fileIOImpl, properties, conf),
+        properties);
+  }
+
+  /**
+   * Initialize using caller-supplied SnowflakeClient and FileIO.
+   *
+   * @param name The name of the catalog, defaults to "snowflake_catalog"
+   * @param snowflakeClient The client encapsulating network communication with Snowflake
+   * @param fileIO The {@link FileIO} to use for table operations
+   * @param properties The catalog options to use and propagate to dependencies
+   */
+  @SuppressWarnings("checkstyle:HiddenField")
+  public void initialize(
+      String name, SnowflakeClient snowflakeClient, FileIO fileIO, Map<String, String> properties) {
+    Preconditions.checkArgument(null != snowflakeClient, "snowflakeClient must be non-null");
+    Preconditions.checkArgument(null != fileIO, "fileIO must be non-null");
+    this.catalogName = name == null ? DEFAULT_CATALOG_NAME : name;
+    this.snowflakeClient = snowflakeClient;
+    this.fileIO = fileIO;
+    this.catalogProperties = properties;
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(snowflakeClient);
+    closeableGroup.addCloseable(fileIO);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (null != closeableGroup) {
+      closeableGroup.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);
+    SnowflakeIdentifier scope = NamespaceHelpers.getSnowflakeIdentifierForNamespace(namespace);
+    Preconditions.checkArgument(
+        scope.getType() == SnowflakeIdentifier.Type.ROOT
+            || scope.getType() == SnowflakeIdentifier.Type.DATABASE,
+        "listNamespaces must be at either ROOT or DATABASE level; got %s from namespace %s",
+        scope,
+        namespace);
+    List<SnowflakeIdentifier> sfSchemas = snowflakeClient.listSchemas(scope);
+
+    List<Namespace> namespaceList =
+        sfSchemas.stream()
+            .map(
+                schema -> {
+                  Preconditions.checkState(
+                      schema.getType() == SnowflakeIdentifier.Type.SCHEMA,
+                      "Got identifier of type %s from listSchemas for %s",
+                      schema.getType(),
+                      namespace);
+                  return Namespace.of(schema.getDatabaseName(), schema.getSchemaName());
+                })
+            .collect(Collectors.toList());
+    return namespaceList;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace)
+      throws NoSuchNamespaceException {
+    LOG.debug("loadNamespaceMetadata with namespace: {}", namespace);
+    return ImmutableMap.of();
+  }
+
+  @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) {
+    return new SnowflakeTableOperations(

Review Comment:
   Eventually we would probably update the underlying FileIO to address the cross bucket issues, but that isn't particularly common for a single table at this point.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1063900869


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java:
##########
@@ -0,0 +1,386 @@
+/*
+ * 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.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+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.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+/**
+ * This implementation of SnowflakeClient builds on top of Snowflake's JDBC driver to interact with
+ * Snowflake's Iceberg-aware resource model.
+ */
+class JdbcSnowflakeClient implements SnowflakeClient {
+  static final String EXPECTED_JDBC_IMPL = "net.snowflake.client.jdbc.SnowflakeDriver";
+
+  @FunctionalInterface
+  interface ResultSetParser<T> {
+    T parse(ResultSet rs) throws SQLException;
+  }
+
+  /**
+   * This class wraps the basic boilerplate of setting up PreparedStatements and applying a
+   * ResultSetParser to translate a ResultSet into parsed objects. Allows easily injecting
+   * subclasses for debugging/testing purposes.
+   */
+  static class QueryHarness {
+    public <T> T query(Connection conn, String sql, ResultSetParser<T> parser, String... args)
+        throws SQLException {
+      try (PreparedStatement statement = conn.prepareStatement(sql)) {
+        if (args != null) {
+          for (int i = 0; i < args.length; ++i) {
+            statement.setString(i + 1, args[i]);
+          }
+        }
+
+        try (ResultSet rs = statement.executeQuery()) {
+          return parser.parse(rs);
+        }
+      }
+    }
+  }
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Database identifiers,
+   * containing "name" (representing databaseName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> DATABASE_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> databases = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("name");
+          databases.add(SnowflakeIdentifier.ofDatabase(databaseName));
+        }
+        return databases;
+      };
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Schema identifiers,
+   * containing "database_name" and "name" (representing schemaName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> SCHEMA_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> schemas = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("database_name");
+          String schemaName = rs.getString("name");
+          schemas.add(SnowflakeIdentifier.ofSchema(databaseName, schemaName));
+        }
+        return schemas;
+      };
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Table identifiers,
+   * containing "database_name", "schema_name", and "name" (representing tableName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> TABLE_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> tables = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("database_name");
+          String schemaName = rs.getString("schema_name");
+          String tableName = rs.getString("name");
+          tables.add(SnowflakeIdentifier.ofTable(databaseName, schemaName, tableName));
+        }
+        return tables;
+      };
+
+  /**
+   * Expects to handle ResultSets representing a single record holding Snowflake Iceberg metadata.
+   */
+  public static final ResultSetParser<SnowflakeTableMetadata> TABLE_METADATA_RESULT_SET_HANDLER =
+      rs -> {
+        if (!rs.next()) {
+          return null;
+        }
+
+        String rawJsonVal = rs.getString("METADATA");
+        return SnowflakeTableMetadata.parseJson(rawJsonVal);
+      };
+
+  private final JdbcClientPool connectionPool;
+  private QueryHarness queryHarness;
+
+  JdbcSnowflakeClient(JdbcClientPool conn) {
+    Preconditions.checkArgument(null != conn, "JdbcClientPool must be non-null");
+    connectionPool = conn;
+    queryHarness = new QueryHarness();
+  }
+
+  @VisibleForTesting
+  void setQueryHarness(QueryHarness queryHarness) {
+    this.queryHarness = queryHarness;
+  }
+
+  /**
+   * For rare cases where PreparedStatements aren't supported for user-supplied identifiers intended
+   * for use in special LIKE clauses, we can sanitize by "broadening" the identifier with
+   * single-character wildcards and manually post-filter client-side.
+   *
+   * <p>Note: This sanitization approach intentionally "broadens" the scope of matching results;
+   * callers must be able to handle this method returning an all-wildcard expression; i.e. the
+   * caller must treat the usage of the LIKE clause as only an optional optimization, and should
+   * post-filter for correctness as if the LIKE clause wasn't present in the query at all.
+   */
+  @VisibleForTesting
+  String sanitizeIdentifierWithWildcardForLikeClause(String identifier) {
+    // Restrict identifiers to the "Unquoted object identifiers" synax documented at
+    // https://docs.snowflake.com/en/sql-reference/identifiers-syntax.html
+    //
+    // Use a strict allowlist of characters, replace everything *not* matching the character set
+    // with "_", which is used as a single-character wildcard in Snowflake.
+    String sanitized = identifier.replaceAll("[^a-zA-Z0-9_$]", "_");
+    if (sanitized.startsWith("$")) {
+      sanitized = "_" + sanitized.substring(1);
+    }
+    return sanitized;
+  }
+
+  @Override
+  public boolean databaseExists(SnowflakeIdentifier database) {
+    Preconditions.checkArgument(
+        database.type() == SnowflakeIdentifier.Type.DATABASE,
+        "databaseExists requires a DATABASE identifier, got '%s'",
+        database);
+
+    // Due to current limitations in PreparedStatement parameters for the LIKE clause in
+    // SHOW DATABASES queries, we'll use a fairly limited allowlist for identifier characters,
+    // using wildcards for non-allowed characters, and post-filter for matching.
+    final String finalQuery =
+        String.format(
+            "SHOW DATABASES LIKE '%s' IN ACCOUNT",
+            sanitizeIdentifierWithWildcardForLikeClause(database.databaseName()));
+    List<SnowflakeIdentifier> databases;
+    try {
+      databases =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn, finalQuery, DATABASE_RESULT_SET_HANDLER, (String[]) null));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to check if database '%s' exists", database);
+    } catch (InterruptedException e) {
+      throw new UncheckedInterruptedException(
+          e, "Interrupted while checking if database '%s' exists", database);
+    }
+
+    // Filter to handle the edge case of '_' appearing as a wildcard that can't be remapped the way
+    // it can for predicates in SELECT statements.
+    databases.removeIf(db -> !database.databaseName().equalsIgnoreCase(db.databaseName()));
+    return !databases.isEmpty();
+  }
+
+  @Override
+  public boolean schemaExists(SnowflakeIdentifier schema) {
+    Preconditions.checkArgument(
+        schema.type() == SnowflakeIdentifier.Type.SCHEMA,
+        "schemaExists requires a SCHEMA identifier, got '%s'",
+        schema);
+
+    if (!databaseExists(SnowflakeIdentifier.ofDatabase(schema.databaseName()))) {
+      return false;
+    }
+
+    // Due to current limitations in PreparedStatement parameters for the LIKE clause in
+    // SHOW SCHEMAS queries, we'll use a fairly limited allowlist for identifier characters,
+    // using wildcards for non-allowed characters, and post-filter for matching.
+    final String finalQuery =
+        String.format(
+            "SHOW SCHEMAS LIKE '%s' IN DATABASE IDENTIFIER(?)",
+            sanitizeIdentifierWithWildcardForLikeClause(schema.schemaName()));
+    List<SnowflakeIdentifier> schemas;
+    try {
+      schemas =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn,
+                      finalQuery,
+                      SCHEMA_RESULT_SET_HANDLER,
+                      new String[] {schema.databaseName()}));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to check if schema '%s' exists", schema);
+    } catch (InterruptedException e) {
+      throw new UncheckedInterruptedException(
+          e, "Interrupted while checking if schema '%s' exists", schema);
+    }
+
+    // Filter to handle the edge case of '_' appearing as a wildcard that can't be remapped the way
+    // it can for predicates in SELECT statements.
+    schemas.removeIf(sc -> !schema.schemaName().equalsIgnoreCase(sc.schemaName()));
+    return !schemas.isEmpty();
+  }
+
+  @Override
+  public List<SnowflakeIdentifier> listDatabases() {
+    List<SnowflakeIdentifier> databases;
+    try {
+      databases =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn,
+                      "SHOW DATABASES IN ACCOUNT",
+                      DATABASE_RESULT_SET_HANDLER,
+                      (String[]) null));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to list databases");
+    } catch (InterruptedException e) {
+      throw new UncheckedInterruptedException(e, "Interrupted while listing databases");
+    }
+    databases.forEach(
+        db ->
+            Preconditions.checkState(
+                db.type() == SnowflakeIdentifier.Type.DATABASE,
+                "Expected DATABASE, got identifier '%s'",
+                db));
+    return databases;
+  }
+
+  @Override
+  public List<SnowflakeIdentifier> listSchemas(SnowflakeIdentifier scope) {
+    StringBuilder baseQuery = new StringBuilder("SHOW SCHEMAS");
+    String[] queryParams = null;
+    switch (scope.type()) {
+      case ROOT:
+        // account-level listing
+        baseQuery.append(" IN ACCOUNT");
+        break;
+      case DATABASE:
+        // database-level listing
+        baseQuery.append(" IN DATABASE IDENTIFIER(?)");
+        queryParams = new String[] {scope.toIdentifierString()};
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format("Unsupported scope type for listSchemas: %s", scope));
+    }
+
+    final String finalQuery = baseQuery.toString();
+    final String[] finalQueryParams = queryParams;
+    List<SnowflakeIdentifier> schemas;
+    try {
+      schemas =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn, finalQuery, SCHEMA_RESULT_SET_HANDLER, finalQueryParams));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to list schemas for scope '%s'", scope);
+    } catch (InterruptedException e) {
+      throw new UncheckedInterruptedException(
+          e, "Interrupted while listing schemas for scope '%s'", scope);
+    }
+    schemas.forEach(
+        schema ->
+            Preconditions.checkState(
+                schema.type() == SnowflakeIdentifier.Type.SCHEMA,
+                "Expected SCHEMA, got identifier '%s' for scope '%s'",
+                schema,
+                scope));
+    return schemas;
+  }
+
+  @Override
+  public List<SnowflakeIdentifier> listIcebergTables(SnowflakeIdentifier scope) {
+    StringBuilder baseQuery = new StringBuilder("SHOW ICEBERG TABLES");

Review Comment:
   Right, this is a current design decision on the backend -- this catalog only lists "managed" Iceberg Tables right now, not External Iceberg Tables. Naturally, as currently Private Preview features we'll be happy to gather customer feedback accordingly.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1066454040


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeTableOperations.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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 org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class SnowflakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeTableOperations.class);
+
+  private final FileIO fileIO;
+  private final TableIdentifier tableIdentifier;
+  private final SnowflakeIdentifier snowflakeIdentifierForTable;
+  private final String fullTableName;
+
+  private final SnowflakeClient snowflakeClient;
+
+  protected SnowflakeTableOperations(
+      SnowflakeClient snowflakeClient,
+      FileIO fileIO,
+      String catalogName,
+      TableIdentifier tableIdentifier) {
+    this.snowflakeClient = snowflakeClient;
+    this.fileIO = fileIO;
+    this.tableIdentifier = tableIdentifier;
+    this.snowflakeIdentifierForTable = NamespaceHelpers.toSnowflakeIdentifier(tableIdentifier);
+    this.fullTableName = String.format("%s.%s", catalogName, tableIdentifier);
+  }
+
+  @Override
+  public void doRefresh() {
+    LOG.debug("Getting metadata location for table {}", tableIdentifier);
+    String location = loadTableMetadataLocation();
+    Preconditions.checkState(
+        location != null && !location.isEmpty(),
+        "Got null or empty location %s for table %s",

Review Comment:
   I've gotten into the habit of including the actual value of whether it's null or empty in these kinds of combined checks especially in IllegalStateExceptions because it's often useful when debug logs are passed around secondhand to better deduce what actually went wrong (e.g. failure to initialize something at all vs single missing parameter or faulty parsing)



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#issuecomment-1382648326

   @jackye1995 Thanks for the heads up! Looks like merging to head fixed it.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1052718809


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.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.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+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

Review Comment:
   Trimmed down



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1052726741


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/entities/SnowflakeIdentifier.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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;

Review Comment:
   Good point, moved up to snowflake package



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051682458


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.hadoop.HadoopFileIO";
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    LOG.debug("listTables with namespace: {}", namespace);
+    SnowflakeIdentifier scope = NamespaceHelpers.getSnowflakeIdentifierForNamespace(namespace);
+    Preconditions.checkArgument(
+        scope.getType() == SnowflakeIdentifier.Type.ROOT
+            || scope.getType() == SnowflakeIdentifier.Type.DATABASE
+            || scope.getType() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at ROOT, DATABASE, or SCHEMA level; got %s from namespace %s",
+        scope,
+        namespace);
+
+    List<SnowflakeIdentifier> sfTables = snowflakeClient.listIcebergTables(scope);
+
+    return sfTables.stream()
+        .map(
+            table ->
+                TableIdentifier.of(
+                    table.getDatabaseName(), table.getSchemaName(), table.getTableName()))
+        .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) {
+    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

Review Comment:
   Looks like line wrapping was auto-formatted.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] nastra commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1052987764


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/NamespaceHelpers.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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 org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class NamespaceHelpers {
+  private static final int MAX_NAMESPACE_DEPTH = 2;
+  private static final int NAMESPACE_ROOT_LEVEL = 0;
+  private static final int NAMESPACE_DB_LEVEL = 1;
+  private static final int NAMESPACE_SCHEMA_LEVEL = 2;
+
+  private static final Logger LOG = LoggerFactory.getLogger(NamespaceHelpers.class);
+
+  /**
+   * Converts a Namespace into a SnowflakeIdentifier representing ROOT, a DATABASE, or a SCHEMA.
+   *
+   * @throws IllegalArgumentException if the namespace is not a supported depth.
+   */
+  public static SnowflakeIdentifier getSnowflakeIdentifierForNamespace(Namespace namespace) {
+    SnowflakeIdentifier identifier = null;
+    switch (namespace.length()) {
+      case NAMESPACE_ROOT_LEVEL:
+        identifier = SnowflakeIdentifier.ofRoot();
+        break;
+      case NAMESPACE_DB_LEVEL:
+        identifier = SnowflakeIdentifier.ofDatabase(namespace.level(NAMESPACE_DB_LEVEL - 1));
+        break;
+      case NAMESPACE_SCHEMA_LEVEL:
+        identifier =
+            SnowflakeIdentifier.ofSchema(
+                namespace.level(NAMESPACE_DB_LEVEL - 1),
+                namespace.level(NAMESPACE_SCHEMA_LEVEL - 1));
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format(
+                "Snowflake max namespace level is %d, got namespace '%s'",
+                MAX_NAMESPACE_DEPTH, namespace));
+    }
+    LOG.debug("getSnowflakeIdentifierForNamespace({}) -> {}", namespace, identifier);
+    return identifier;
+  }
+
+  /**
+   * Converts a TableIdentifier into a SnowflakeIdentifier of type TABLE; the identifier must have
+   * exactly the right namespace depth to represent a fully-qualified Snowflake table identifier.
+   */
+  public static SnowflakeIdentifier getSnowflakeIdentifierForTableIdentifier(
+      TableIdentifier identifier) {
+    SnowflakeIdentifier namespaceScope = getSnowflakeIdentifierForNamespace(identifier.namespace());
+    Preconditions.checkArgument(
+        namespaceScope.getType() == SnowflakeIdentifier.Type.SCHEMA,
+        "Namespace portion of '%s' must be at the SCHEMA level, got namespaceScope '%s'",
+        identifier,
+        namespaceScope);
+    SnowflakeIdentifier ret =
+        SnowflakeIdentifier.ofTable(
+            namespaceScope.getDatabaseName(), namespaceScope.getSchemaName(), identifier.name());
+    LOG.debug("getSnowflakeIdentifierForTableIdentifier({}) -> {}", identifier, ret);
+    return ret;
+  }
+
+  private NamespaceHelpers() {}

Review Comment:
   nit: probably better to move this further up



##########
snowflake/src/test/java/org/apache/iceberg/snowflake/SnowflakeCatalogTest.java:
##########
@@ -0,0 +1,277 @@
+/*
+ * 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.IOException;
+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.Maps;
+import org.apache.iceberg.snowflake.entities.SnowflakeTableMetadata;
+import org.apache.iceberg.types.Types;
+import org.assertj.core.api.Assertions;
+import org.junit.Before;
+import org.junit.Test;
+
+public class SnowflakeCatalogTest {
+
+  static final String TEST_CATALOG_NAME = "slushLog";
+  private SnowflakeCatalog catalog;
+  private FakeSnowflakeClient fakeClient;
+  private InMemoryFileIO fakeFileIO;
+  private Map<String, String> properties;
+
+  @Before
+  public void before() {
+    catalog = new SnowflakeCatalog();
+
+    fakeClient = new FakeSnowflakeClient();
+    fakeClient.addTable(
+        "DB_1",
+        "SCHEMA_1",
+        "TAB_1",
+        SnowflakeTableMetadata.parseJson(
+            "{\"metadataLocation\":\"s3://tab1/metadata/v3.metadata.json\",\"status\":\"success\"}"));
+    fakeClient.addTable(
+        "DB_1",
+        "SCHEMA_1",
+        "TAB_2",
+        SnowflakeTableMetadata.parseJson(
+            "{\"metadataLocation\":\"s3://tab2/metadata/v1.metadata.json\",\"status\":\"success\"}"));
+    fakeClient.addTable(
+        "DB_2",
+        "SCHEMA_2",
+        "TAB_3",
+        SnowflakeTableMetadata.parseJson(
+            "{\"metadataLocation\":\"azure://myaccount.blob.core.windows.net/mycontainer/tab3/metadata/v334.metadata.json\",\"status\":\"success\"}"));
+    fakeClient.addTable(
+        "DB_2",
+        "SCHEMA_2",
+        "TAB_4",
+        SnowflakeTableMetadata.parseJson(
+            "{\"metadataLocation\":\"azure://myaccount.blob.core.windows.net/mycontainer/tab4/metadata/v323.metadata.json\",\"status\":\"success\"}"));
+    fakeClient.addTable(
+        "DB_3",
+        "SCHEMA_3",
+        "TAB_5",
+        SnowflakeTableMetadata.parseJson(
+            "{\"metadataLocation\":\"gcs://tab5/metadata/v793.metadata.json\",\"status\":\"success\"}"));
+    fakeClient.addTable(
+        "DB_3",
+        "SCHEMA_4",
+        "TAB_6",
+        SnowflakeTableMetadata.parseJson(
+            "{\"metadataLocation\":\"gcs://tab6/metadata/v123.metadata.json\",\"status\":\"success\"}"));
+
+    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://mycontainer@myaccount.blob.core.windows.net/tab3/metadata/v334.metadata.json",
+        TableMetadataParser.toJson(
+                TableMetadata.newTableMetadata(
+                    schema,
+                    partitionSpec,
+                    "wasbs://mycontainer@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());
+
+    properties = Maps.newHashMap();
+    catalog.initialize(TEST_CATALOG_NAME, fakeClient, fakeFileIO, properties);
+  }
+
+  @Test
+  public void testInitializeNullClient() {
+    Assertions.assertThatExceptionOfType(IllegalArgumentException.class)
+        .isThrownBy(() -> catalog.initialize(TEST_CATALOG_NAME, null, fakeFileIO, properties))
+        .withMessageContaining("snowflakeClient must be non-null");
+  }
+
+  @Test
+  public void testInitializeNullFileIO() {
+    Assertions.assertThatExceptionOfType(IllegalArgumentException.class)
+        .isThrownBy(() -> catalog.initialize(TEST_CATALOG_NAME, fakeClient, null, properties))
+        .withMessageContaining("fileIO must be non-null");
+  }
+
+  @Test
+  public void testListNamespace() {
+    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"));
+  }
+
+  @Test
+  public void testListNamespaceWithinDB() {
+    String dbName = "DB_1";
+    Assertions.assertThat(catalog.listNamespaces(Namespace.of(dbName)))
+        .containsExactly(Namespace.of(dbName, "SCHEMA_1"));
+  }
+
+  @Test
+  public void testListNamespaceWithinNonExistentDB() {
+    // Existence check for nonexistent parent namespaces is optional in the SupportsNamespaces
+    // interface.
+    String dbName = "NONEXISTENT_DB";
+    Assertions.assertThatExceptionOfType(RuntimeException.class)
+        .isThrownBy(() -> catalog.listNamespaces(Namespace.of(dbName)))
+        .withMessageContaining("does not exist")
+        .withMessageContaining(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";
+    Assertions.assertThatExceptionOfType(IllegalArgumentException.class)
+        .isThrownBy(() -> catalog.listNamespaces(Namespace.of(dbName, schemaName)))
+        .withMessageContaining("level")
+        .withMessageContaining("DB_3.SCHEMA_4");
+  }
+
+  @Test
+  public void testListTables() {
+    Assertions.assertThat(catalog.listTables(Namespace.empty()))
+        .containsExactly(
+            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"));
+  }
+
+  @Test
+  public void testListTablesWithinDB() {
+    String dbName = "DB_1";
+    Assertions.assertThat(catalog.listTables(Namespace.of(dbName)))
+        .containsExactly(
+            TableIdentifier.of("DB_1", "SCHEMA_1", "TAB_1"),
+            TableIdentifier.of("DB_1", "SCHEMA_1", "TAB_2"));
+  }
+
+  @Test
+  public void testListTablesWithinNonexistentDB() {
+    String dbName = "NONEXISTENT_DB";
+    Assertions.assertThatExceptionOfType(RuntimeException.class)
+        .isThrownBy(() -> catalog.listTables(Namespace.of(dbName)))
+        .withMessageContaining("does not exist")
+        .withMessageContaining(dbName);
+  }
+
+  @Test
+  public void testListTablesWithinSchema() {
+    String dbName = "DB_2";
+    String schemaName = "SCHEMA_2";
+    Assertions.assertThat(catalog.listTables(Namespace.of(dbName, schemaName)))
+        .containsExactly(
+            TableIdentifier.of("DB_2", "SCHEMA_2", "TAB_3"),
+            TableIdentifier.of("DB_2", "SCHEMA_2", "TAB_4"));
+  }
+
+  @Test
+  public void testListTablesWithinNonexistentSchema() {
+    String dbName = "DB_2";
+    String schemaName = "NONEXISTENT_SCHEMA";
+    Assertions.assertThatExceptionOfType(RuntimeException.class)
+        .isThrownBy(() -> catalog.listTables(Namespace.of(dbName, schemaName)))
+        .withMessageContaining("does not exist")
+        .withMessageContaining("DB_2.NONEXISTENT_SCHEMA");
+  }
+
+  @Test
+  public void testLoadS3Table() {
+    Table table = catalog.loadTable(TableIdentifier.of(Namespace.of("DB_1", "SCHEMA_1"), "TAB_1"));
+    Assertions.assertThat(table.location()).isEqualTo("s3://tab1/");
+  }
+
+  @Test
+  public void testLoadAzureTable() {
+    Table table = catalog.loadTable(TableIdentifier.of(Namespace.of("DB_2", "SCHEMA_2"), "TAB_3"));
+    Assertions.assertThat(table.location())
+        .isEqualTo("wasbs://mycontainer@myaccount.blob.core.windows.net/tab1/");
+  }
+
+  @Test
+  public void testLoadGcsTable() {
+    Table table = catalog.loadTable(TableIdentifier.of(Namespace.of("DB_3", "SCHEMA_3"), "TAB_5"));
+    Assertions.assertThat(table.location()).isEqualTo("gs://tab5/");
+  }
+
+  @Test
+  public void testLoadTableWithMalformedTableIdentifier() {
+    Assertions.assertThatExceptionOfType(IllegalArgumentException.class)
+        .isThrownBy(
+            () ->
+                catalog.loadTable(
+                    TableIdentifier.of(Namespace.of("DB_1", "SCHEMA_1", "BAD_NS_LEVEL"), "TAB_1")))
+        .withMessageContaining("level")
+        .withMessageContaining("DB_1.SCHEMA_1.BAD_NS_LEVEL");
+    Assertions.assertThatExceptionOfType(IllegalArgumentException.class)
+        .isThrownBy(
+            () -> catalog.loadTable(TableIdentifier.of(Namespace.of("DB_WITHOUT_SCHEMA"), "TAB_1")))
+        .withMessageContaining("level")
+        .withMessageContaining("DB_WITHOUT_SCHEMA.TAB_1");
+  }
+
+  @Test
+  public void testCloseBeforeInitialize() throws IOException {
+    catalog = new SnowflakeCatalog();
+    catalog.close();

Review Comment:
   I think it would still be good to add some assertion here as this looks weird without any assertion



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1056617707


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java:
##########
@@ -0,0 +1,362 @@
+/*
+ * 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.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+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.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+/**
+ * This implementation of SnowflakeClient builds on top of Snowflake's JDBC driver to interact with
+ * Snowflake's Iceberg-aware resource model.
+ */
+class JdbcSnowflakeClient implements SnowflakeClient {
+  public static final String EXPECTED_JDBC_IMPL = "net.snowflake.client.jdbc.SnowflakeDriver";
+
+  @FunctionalInterface
+  interface ResultSetParser<T> {
+    T parse(ResultSet rs) throws SQLException;
+  }
+
+  /**
+   * This class wraps the basic boilerplate of setting up PreparedStatements and applying a
+   * ResultSetParser to translate a ResultSet into parsed objects. Allows easily injecting
+   * subclasses for debugging/testing purposes.
+   */
+  static class QueryHarness {
+    public <T> T query(Connection conn, String sql, ResultSetParser<T> parser, String... args)
+        throws SQLException {
+      try (PreparedStatement statement = conn.prepareStatement(sql)) {
+        if (args != null) {
+          for (int i = 0; i < args.length; ++i) {
+            statement.setString(i + 1, args[i]);
+          }
+        }
+
+        try (ResultSet rs = statement.executeQuery()) {
+          return parser.parse(rs);
+        }
+      }
+    }
+  }
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Database identifiers,
+   * containing "name" (representing databaseName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> DATABASE_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> databases = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("name");
+          databases.add(SnowflakeIdentifier.ofDatabase(databaseName));
+        }
+        return databases;
+      };
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Schema identifiers,
+   * containing "database_name" and "name" (representing schemaName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> SCHEMA_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> schemas = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("database_name");
+          String schemaName = rs.getString("name");
+          schemas.add(SnowflakeIdentifier.ofSchema(databaseName, schemaName));
+        }
+        return schemas;
+      };
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Table identifiers,
+   * containing "database_name", "schema_name", and "name" (representing tableName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> TABLE_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> tables = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("database_name");
+          String schemaName = rs.getString("schema_name");
+          String tableName = rs.getString("name");
+          tables.add(SnowflakeIdentifier.ofTable(databaseName, schemaName, tableName));
+        }
+        return tables;
+      };
+
+  /**
+   * Expects to handle ResultSets representing a single record holding Snowflake Iceberg metadata.
+   */
+  public static final ResultSetParser<SnowflakeTableMetadata> TABLE_METADATA_RESULT_SET_HANDLER =
+      rs -> {
+        if (!rs.next()) {
+          return null;
+        }
+
+        String rawJsonVal = rs.getString("METADATA");
+        return SnowflakeTableMetadata.parseJson(rawJsonVal);
+      };
+
+  private final JdbcClientPool connectionPool;
+  private QueryHarness queryHarness;
+
+  JdbcSnowflakeClient(JdbcClientPool conn) {
+    Preconditions.checkArgument(null != conn, "JdbcClientPool must be non-null");
+    connectionPool = conn;
+    queryHarness = new QueryHarness();
+  }
+
+  @VisibleForTesting
+  void setQueryHarness(QueryHarness queryHarness) {
+    this.queryHarness = queryHarness;
+  }
+
+  /**
+   * For rare cases where PreparedStatements aren't supported for user-supplied identifiers intended
+   * for use in special LIKE clauses, we can sanitize by "broadening" the identifier with
+   * single-character wildcards and manually post-filter client-side.
+   *
+   * <p>Note: This sanitization approach intentionally "broadens" the scope of matching results;
+   * callers must be able to handle this method returning an all-wildcard expression; i.e. the
+   * caller must treat the usage of the LIKE clause as only an optional optimization, and should
+   * post-filter for correctness as if the LIKE clause wasn't present in the query at all.
+   */
+  @VisibleForTesting
+  String sanitizeIdentifierWithWildcardForLikeClause(String identifier) {
+    // Restrict identifiers to the "Unquoted object identifiers" synax documented at
+    // https://docs.snowflake.com/en/sql-reference/identifiers-syntax.html
+    //
+    // Use a strict allowlist of characters, replace everything *not* matching the character set
+    // with "_", which is used as a single-character wildcard in Snowflake.
+    String sanitized = identifier.replaceAll("[^a-zA-Z0-9_$]", "_");
+    if (sanitized.startsWith("$")) {
+      sanitized = "_" + sanitized.substring(1);
+    }
+    return sanitized;
+  }
+
+  @Override
+  public boolean databaseExists(SnowflakeIdentifier database) {
+    Preconditions.checkArgument(
+        database.type() == SnowflakeIdentifier.Type.DATABASE,
+        "databaseExists requires a DATABASE identifier, got '%s'",
+        database);
+
+    // Due to current limitations in PreparedStatement parameters for the LIKE clause in
+    // SHOW DATABASES queries, we'll use a fairly limited allowlist for identifier characters,
+    // using wildcards for non-allowed characters, and post-filter for matching.
+    final String finalQuery =
+        String.format(
+            "SHOW DATABASES LIKE '%s' IN ACCOUNT",
+            sanitizeIdentifierWithWildcardForLikeClause(database.databaseName()));
+    List<SnowflakeIdentifier> databases;
+    try {
+      databases =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn, finalQuery, DATABASE_RESULT_SET_HANDLER, (String[]) null));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to check if database exists");

Review Comment:
   Done



##########
snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java:
##########
@@ -0,0 +1,362 @@
+/*
+ * 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.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+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.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+/**
+ * This implementation of SnowflakeClient builds on top of Snowflake's JDBC driver to interact with
+ * Snowflake's Iceberg-aware resource model.
+ */
+class JdbcSnowflakeClient implements SnowflakeClient {
+  public static final String EXPECTED_JDBC_IMPL = "net.snowflake.client.jdbc.SnowflakeDriver";
+
+  @FunctionalInterface
+  interface ResultSetParser<T> {
+    T parse(ResultSet rs) throws SQLException;
+  }
+
+  /**
+   * This class wraps the basic boilerplate of setting up PreparedStatements and applying a
+   * ResultSetParser to translate a ResultSet into parsed objects. Allows easily injecting
+   * subclasses for debugging/testing purposes.
+   */
+  static class QueryHarness {
+    public <T> T query(Connection conn, String sql, ResultSetParser<T> parser, String... args)
+        throws SQLException {
+      try (PreparedStatement statement = conn.prepareStatement(sql)) {
+        if (args != null) {
+          for (int i = 0; i < args.length; ++i) {
+            statement.setString(i + 1, args[i]);
+          }
+        }
+
+        try (ResultSet rs = statement.executeQuery()) {
+          return parser.parse(rs);
+        }
+      }
+    }
+  }
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Database identifiers,
+   * containing "name" (representing databaseName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> DATABASE_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> databases = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("name");
+          databases.add(SnowflakeIdentifier.ofDatabase(databaseName));
+        }
+        return databases;
+      };
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Schema identifiers,
+   * containing "database_name" and "name" (representing schemaName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> SCHEMA_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> schemas = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("database_name");
+          String schemaName = rs.getString("name");
+          schemas.add(SnowflakeIdentifier.ofSchema(databaseName, schemaName));
+        }
+        return schemas;
+      };
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Table identifiers,
+   * containing "database_name", "schema_name", and "name" (representing tableName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> TABLE_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> tables = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("database_name");
+          String schemaName = rs.getString("schema_name");
+          String tableName = rs.getString("name");
+          tables.add(SnowflakeIdentifier.ofTable(databaseName, schemaName, tableName));
+        }
+        return tables;
+      };
+
+  /**
+   * Expects to handle ResultSets representing a single record holding Snowflake Iceberg metadata.
+   */
+  public static final ResultSetParser<SnowflakeTableMetadata> TABLE_METADATA_RESULT_SET_HANDLER =
+      rs -> {
+        if (!rs.next()) {
+          return null;
+        }
+
+        String rawJsonVal = rs.getString("METADATA");
+        return SnowflakeTableMetadata.parseJson(rawJsonVal);
+      };
+
+  private final JdbcClientPool connectionPool;
+  private QueryHarness queryHarness;
+
+  JdbcSnowflakeClient(JdbcClientPool conn) {
+    Preconditions.checkArgument(null != conn, "JdbcClientPool must be non-null");
+    connectionPool = conn;
+    queryHarness = new QueryHarness();
+  }
+
+  @VisibleForTesting
+  void setQueryHarness(QueryHarness queryHarness) {
+    this.queryHarness = queryHarness;
+  }
+
+  /**
+   * For rare cases where PreparedStatements aren't supported for user-supplied identifiers intended
+   * for use in special LIKE clauses, we can sanitize by "broadening" the identifier with
+   * single-character wildcards and manually post-filter client-side.
+   *
+   * <p>Note: This sanitization approach intentionally "broadens" the scope of matching results;
+   * callers must be able to handle this method returning an all-wildcard expression; i.e. the
+   * caller must treat the usage of the LIKE clause as only an optional optimization, and should
+   * post-filter for correctness as if the LIKE clause wasn't present in the query at all.
+   */
+  @VisibleForTesting
+  String sanitizeIdentifierWithWildcardForLikeClause(String identifier) {
+    // Restrict identifiers to the "Unquoted object identifiers" synax documented at
+    // https://docs.snowflake.com/en/sql-reference/identifiers-syntax.html
+    //
+    // Use a strict allowlist of characters, replace everything *not* matching the character set
+    // with "_", which is used as a single-character wildcard in Snowflake.
+    String sanitized = identifier.replaceAll("[^a-zA-Z0-9_$]", "_");
+    if (sanitized.startsWith("$")) {
+      sanitized = "_" + sanitized.substring(1);
+    }
+    return sanitized;
+  }
+
+  @Override
+  public boolean databaseExists(SnowflakeIdentifier database) {
+    Preconditions.checkArgument(
+        database.type() == SnowflakeIdentifier.Type.DATABASE,
+        "databaseExists requires a DATABASE identifier, got '%s'",
+        database);
+
+    // Due to current limitations in PreparedStatement parameters for the LIKE clause in
+    // SHOW DATABASES queries, we'll use a fairly limited allowlist for identifier characters,
+    // using wildcards for non-allowed characters, and post-filter for matching.
+    final String finalQuery =
+        String.format(
+            "SHOW DATABASES LIKE '%s' IN ACCOUNT",
+            sanitizeIdentifierWithWildcardForLikeClause(database.databaseName()));
+    List<SnowflakeIdentifier> databases;
+    try {
+      databases =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn, finalQuery, DATABASE_RESULT_SET_HANDLER, (String[]) null));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to check if database exists");
+    } catch (InterruptedException e) {
+      throw new UncheckedInterruptedException(e, "Interrupted while checking if database exists");
+    }
+
+    // Filter to handle the edge case of '_' appearing as a wildcard that can't be remapped the way
+    // it can for predicates in SELECT statements.
+    databases.removeIf(db -> !database.databaseName().equalsIgnoreCase(db.databaseName()));
+    return !databases.isEmpty();
+  }
+
+  @Override
+  public boolean schemaExists(SnowflakeIdentifier schema) {
+    Preconditions.checkArgument(
+        schema.type() == SnowflakeIdentifier.Type.SCHEMA,
+        "schemaExists requires a SCHEMA identifier, got '%s'",
+        schema);
+
+    if (!databaseExists(SnowflakeIdentifier.ofDatabase(schema.databaseName()))) {
+      return false;
+    }
+
+    // Due to current limitations in PreparedStatement parameters for the LIKE clause in
+    // SHOW SCHEMAS queries, we'll use a fairly limited allowlist for identifier characters,
+    // using wildcards for non-allowed characters, and post-filter for matching.
+    final String finalQuery =
+        String.format(
+            "SHOW SCHEMAS LIKE '%s' IN DATABASE IDENTIFIER(?)",
+            sanitizeIdentifierWithWildcardForLikeClause(schema.schemaName()));
+    List<SnowflakeIdentifier> schemas;
+    try {
+      schemas =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn,
+                      finalQuery,
+                      SCHEMA_RESULT_SET_HANDLER,
+                      new String[] {schema.databaseName()}));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to check if schema exists");

Review Comment:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1052897164


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeTableOperations.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.Map;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.apache.iceberg.snowflake.entities.SnowflakeTableMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class SnowflakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeTableOperations.class);
+  private final String catalogName;
+
+  private final FileIO fileIO;
+  private final TableIdentifier tableIdentifier;
+  private final SnowflakeIdentifier snowflakeIdentifierForTable;
+
+  private final SnowflakeClient snowflakeClient;
+
+  private final Map<String, String> catalogProperties;
+
+  protected SnowflakeTableOperations(
+      SnowflakeClient snowflakeClient,
+      FileIO fileIO,
+      Map<String, String> properties,
+      String catalogName,
+      TableIdentifier tableIdentifier) {
+    this.snowflakeClient = snowflakeClient;
+    this.fileIO = fileIO;
+    this.catalogProperties = properties;
+    this.catalogName = catalogName;
+    this.tableIdentifier = tableIdentifier;
+    this.snowflakeIdentifierForTable =
+        NamespaceHelpers.getSnowflakeIdentifierForTableIdentifier(tableIdentifier);
+  }
+
+  @Override
+  public void doRefresh() {
+    LOG.debug("Getting metadata location for table {}", tableIdentifier);
+    String location = getTableMetadataLocation();
+    Preconditions.checkState(
+        location != null && !location.isEmpty(),
+        "Got null or empty location %s for table %s",
+        location,
+        tableIdentifier);
+    refreshFromMetadataLocation(location);
+  }
+
+  @Override
+  public FileIO io() {
+    return fileIO;
+  }
+
+  @Override
+  protected String tableName() {
+    return tableIdentifier.toString();
+  }
+
+  private String getTableMetadataLocation() {
+    SnowflakeTableMetadata metadata = snowflakeClient.getTableMetadata(snowflakeIdentifierForTable);
+
+    if (metadata == null) {
+      throw new NoSuchTableException("Cannot find table %s", snowflakeIdentifierForTable);
+    }
+    if (!metadata.getStatus().equals("success")) {

Review Comment:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051681587


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.hadoop.HadoopFileIO";
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    LOG.debug("listTables with namespace: {}", namespace);
+    SnowflakeIdentifier scope = NamespaceHelpers.getSnowflakeIdentifierForNamespace(namespace);
+    Preconditions.checkArgument(
+        scope.getType() == SnowflakeIdentifier.Type.ROOT
+            || scope.getType() == SnowflakeIdentifier.Type.DATABASE
+            || scope.getType() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at ROOT, DATABASE, or SCHEMA level; got %s from namespace %s",
+        scope,
+        namespace);
+
+    List<SnowflakeIdentifier> sfTables = snowflakeClient.listIcebergTables(scope);
+
+    return sfTables.stream()
+        .map(
+            table ->
+                TableIdentifier.of(
+                    table.getDatabaseName(), table.getSchemaName(), table.getTableName()))
+        .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) {
+    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);
+
+    String fileIOImpl = DEFAULT_FILE_IO_IMPL;
+    if (properties.containsKey(CatalogProperties.FILE_IO_IMPL)) {
+      fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL);
+    }
+
+    initialize(
+        name,
+        new JdbcSnowflakeClient(connectionPool),
+        CatalogUtil.loadFileIO(fileIOImpl, properties, conf),
+        properties);
+  }
+
+  /**
+   * Initialize using caller-supplied SnowflakeClient and FileIO.
+   *
+   * @param name The name of the catalog, defaults to "snowflake_catalog"
+   * @param snowflakeClient The client encapsulating network communication with Snowflake
+   * @param fileIO The {@link FileIO} to use for table operations
+   * @param properties The catalog options to use and propagate to dependencies
+   */
+  @SuppressWarnings("checkstyle:HiddenField")
+  public void initialize(
+      String name, SnowflakeClient snowflakeClient, FileIO fileIO, Map<String, String> properties) {
+    Preconditions.checkArgument(null != snowflakeClient, "snowflakeClient must be non-null");
+    Preconditions.checkArgument(null != fileIO, "fileIO must be non-null");
+    this.catalogName = name == null ? DEFAULT_CATALOG_NAME : name;
+    this.snowflakeClient = snowflakeClient;
+    this.fileIO = fileIO;
+    this.catalogProperties = properties;
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(snowflakeClient);
+    closeableGroup.addCloseable(fileIO);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (null != closeableGroup) {
+      closeableGroup.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);
+    SnowflakeIdentifier scope = NamespaceHelpers.getSnowflakeIdentifierForNamespace(namespace);
+    Preconditions.checkArgument(
+        scope.getType() == SnowflakeIdentifier.Type.ROOT
+            || scope.getType() == SnowflakeIdentifier.Type.DATABASE,
+        "listNamespaces must be at either ROOT or DATABASE level; got %s from namespace %s",
+        scope,
+        namespace);
+    List<SnowflakeIdentifier> sfSchemas = snowflakeClient.listSchemas(scope);
+
+    List<Namespace> namespaceList =
+        sfSchemas.stream()
+            .map(
+                schema -> {
+                  Preconditions.checkState(
+                      schema.getType() == SnowflakeIdentifier.Type.SCHEMA,
+                      "Got identifier of type %s from listSchemas for %s",
+                      schema.getType(),
+                      namespace);
+                  return Namespace.of(schema.getDatabaseName(), schema.getSchemaName());
+                })
+            .collect(Collectors.toList());
+    return namespaceList;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace)
+      throws NoSuchNamespaceException {
+    LOG.debug("loadNamespaceMetadata with namespace: {}", namespace);
+    return ImmutableMap.of();

Review Comment:
   Snowflake schemas do not have metadata associated with them?
   
   Also, a namespace exists if this returns without throwing an exception. Is that the intended behavior?



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051681975


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeClient.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.snowflake;
+
+import java.io.Closeable;
+import java.util.List;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+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 {
+  /**
+   * Lists all Snowflake schemas within a given scope. Returned SnowflakeIdentifiers must have
+   * getType() == SnowflakeIdentifier.Type.SCHEMA.
+   *
+   * @param scope The scope in which to list, which may be ROOT or a single DATABASE.
+   */
+  List<SnowflakeIdentifier> listSchemas(SnowflakeIdentifier scope);
+
+  /**
+   * Lists all Snowflake Iceberg tables within a given scope. Returned SnowflakeIdentifiers must
+   * have getType() == SnowflakeIdentifier.Type.TABLE.
+   *
+   * @param scope The scope in which to list, which may be ROOT, a DATABASE, or a SCHEMA.
+   */
+  List<SnowflakeIdentifier> listIcebergTables(SnowflakeIdentifier scope);
+
+  /**
+   * Returns Snowflake-level metadata containing locations to more detailed metadata.
+   *
+   * @param tableIdentifier The fully-qualified identifier that must be of type
+   *     SnowflakeIdentifier.Type.TABLE.
+   */
+  SnowflakeTableMetadata getTableMetadata(SnowflakeIdentifier tableIdentifier);

Review Comment:
   Style: `get` in a method name.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1052732576


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/entities/SnowflakeTableMetadata.java:
##########
@@ -0,0 +1,158 @@
+/*
+ * 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 com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.commons.dbutils.ResultSetHandler;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+public class SnowflakeTableMetadata {
+  public static final Pattern SNOWFLAKE_AZURE_PATTERN =
+      Pattern.compile("azure://([^/]+)/([^/]+)/(.*)");
+
+  private String snowflakeMetadataLocation;
+  private String icebergMetadataLocation;
+  private String status;
+
+  // Note: Since not all sources will necessarily come from a raw JSON representation, this raw
+  // JSON should only be considered a convenient debugging field. Equality of two
+  // SnowflakeTableMetadata instances should not depend on equality of this field.
+  private String rawJsonVal;
+
+  public SnowflakeTableMetadata(
+      String snowflakeMetadataLocation,
+      String icebergMetadataLocation,
+      String status,
+      String rawJsonVal) {
+    this.snowflakeMetadataLocation = snowflakeMetadataLocation;
+    this.icebergMetadataLocation = icebergMetadataLocation;
+    this.status = status;
+    this.rawJsonVal = rawJsonVal;
+  }
+
+  /** Storage location of table metadata in Snowflake's path syntax. */
+  public String getSnowflakeMetadataLocation() {
+    return snowflakeMetadataLocation;
+  }
+
+  /** Storage location of table metadata in Iceberg's path syntax. */
+  public String getIcebergMetadataLocation() {
+    return icebergMetadataLocation;
+  }
+
+  public String getStatus() {
+    return status;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    } else if (!(o instanceof SnowflakeTableMetadata)) {
+      return false;
+    }
+
+    // Only consider parsed fields, not the raw JSON that may or may not be the original source of
+    // this instance.
+    SnowflakeTableMetadata that = (SnowflakeTableMetadata) o;
+    return Objects.equal(this.snowflakeMetadataLocation, that.snowflakeMetadataLocation)
+        && Objects.equal(this.icebergMetadataLocation, that.icebergMetadataLocation)
+        && Objects.equal(this.status, that.status);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(snowflakeMetadataLocation, icebergMetadataLocation, status);
+  }
+
+  @Override
+  public String toString() {
+    return String.format(
+        "snowflakeMetadataLocation: '%s', icebergMetadataLocation: '%s', status: '%s",
+        snowflakeMetadataLocation, icebergMetadataLocation, status);
+  }
+
+  /**
+   * Translates from Snowflake's path syntax to Iceberg's path syntax for paths matching known
+   * non-compatible Snowflake paths. Throws IllegalArgumentException if the prefix of the
+   * snowflakeLocation is a known non-compatible path syntax but fails to match the expected path
+   * components for a successful translation.
+   */
+  public static String getIcebergLocationFromSnowflakeLocation(String snowflakeLocation) {
+    if (snowflakeLocation.startsWith("azure://")) {
+      // Convert from expected path of the form:
+      // azure://account.blob.core.windows.net/container/volumepath
+      // to:
+      // wasbs://container@account.blob.core.windows.net/volumepath
+      Matcher matcher = SNOWFLAKE_AZURE_PATTERN.matcher(snowflakeLocation);
+      Preconditions.checkArgument(
+          matcher.matches(),
+          "Location '%s' failed to match pattern '%s'",
+          snowflakeLocation,
+          SNOWFLAKE_AZURE_PATTERN);
+      return String.format(
+          "wasbs://%s@%s/%s", matcher.group(2), matcher.group(1), matcher.group(3));
+    } else if (snowflakeLocation.startsWith("gcs://")) {
+      // Convert from expected path of the form:
+      // gcs://bucket/path
+      // to:
+      // gs://bucket/path
+      return "gs" + snowflakeLocation.substring(3);
+    }
+    return snowflakeLocation;

Review Comment:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1052731182


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/entities/SnowflakeIdentifier.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class SnowflakeIdentifier {
+  public enum Type {
+    ROOT,
+    DATABASE,
+    SCHEMA,
+    TABLE
+  }
+
+  private String databaseName;
+  private String schemaName;
+  private String tableName;
+
+  protected SnowflakeIdentifier(String databaseName, String schemaName, String tableName) {
+    this.databaseName = databaseName;
+    this.schemaName = schemaName;
+    this.tableName = tableName;
+  }
+
+  public static SnowflakeIdentifier ofRoot() {
+    return new SnowflakeIdentifier(null, null, null);
+  }
+
+  public static SnowflakeIdentifier ofDatabase(String databaseName) {
+    Preconditions.checkArgument(null != databaseName, "databaseName must be non-null");
+    return new SnowflakeIdentifier(databaseName, null, null);
+  }
+
+  public static SnowflakeIdentifier ofSchema(String databaseName, String schemaName) {
+    Preconditions.checkArgument(null != databaseName, "databaseName must be non-null");
+    Preconditions.checkArgument(null != schemaName, "schemaName must be non-null");
+    return new SnowflakeIdentifier(databaseName, schemaName, null);
+  }
+
+  public static SnowflakeIdentifier ofTable(
+      String databaseName, String schemaName, String tableName) {
+    Preconditions.checkArgument(null != databaseName, "databaseName must be non-null");
+    Preconditions.checkArgument(null != schemaName, "schemaName must be non-null");
+    Preconditions.checkArgument(null != tableName, "tableName must be non-null");
+    return new SnowflakeIdentifier(databaseName, schemaName, tableName);
+  }
+
+  /**
+   * If type is TABLE, expect non-null databaseName, schemaName, and tableName. If type is SCHEMA,
+   * expect non-null databaseName and schemaName. If type is DATABASE, expect non-null databaseName.
+   * If type is ROOT, expect all of databaseName, schemaName, and tableName to be null.
+   */
+  public Type getType() {
+    if (null != tableName) {
+      return Type.TABLE;
+    } else if (null != schemaName) {
+      return Type.SCHEMA;
+    } else if (null != databaseName) {
+      return Type.DATABASE;
+    } else {
+      return Type.ROOT;
+    }
+  }
+
+  public String getTableName() {

Review Comment:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051657208


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.hadoop.HadoopFileIO";
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    LOG.debug("listTables with namespace: {}", namespace);
+    SnowflakeIdentifier scope = NamespaceHelpers.getSnowflakeIdentifierForNamespace(namespace);
+    Preconditions.checkArgument(
+        scope.getType() == SnowflakeIdentifier.Type.ROOT
+            || scope.getType() == SnowflakeIdentifier.Type.DATABASE
+            || scope.getType() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at ROOT, DATABASE, or SCHEMA level; got %s from namespace %s",
+        scope,
+        namespace);
+
+    List<SnowflakeIdentifier> sfTables = snowflakeClient.listIcebergTables(scope);
+
+    return sfTables.stream()
+        .map(
+            table ->
+                TableIdentifier.of(
+                    table.getDatabaseName(), table.getSchemaName(), table.getTableName()))
+        .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) {
+    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);
+
+    String fileIOImpl = DEFAULT_FILE_IO_IMPL;
+    if (properties.containsKey(CatalogProperties.FILE_IO_IMPL)) {
+      fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL);
+    }
+
+    initialize(
+        name,
+        new JdbcSnowflakeClient(connectionPool),
+        CatalogUtil.loadFileIO(fileIOImpl, properties, conf),
+        properties);
+  }
+
+  /**
+   * Initialize using caller-supplied SnowflakeClient and FileIO.
+   *
+   * @param name The name of the catalog, defaults to "snowflake_catalog"
+   * @param snowflakeClient The client encapsulating network communication with Snowflake
+   * @param fileIO The {@link FileIO} to use for table operations
+   * @param properties The catalog options to use and propagate to dependencies
+   */
+  @SuppressWarnings("checkstyle:HiddenField")
+  public void initialize(

Review Comment:
   Does this need to be public?



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051656395


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.hadoop.HadoopFileIO";
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    LOG.debug("listTables with namespace: {}", namespace);
+    SnowflakeIdentifier scope = NamespaceHelpers.getSnowflakeIdentifierForNamespace(namespace);
+    Preconditions.checkArgument(
+        scope.getType() == SnowflakeIdentifier.Type.ROOT
+            || scope.getType() == SnowflakeIdentifier.Type.DATABASE
+            || scope.getType() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at ROOT, DATABASE, or SCHEMA level; got %s from namespace %s",
+        scope,
+        namespace);
+
+    List<SnowflakeIdentifier> sfTables = snowflakeClient.listIcebergTables(scope);
+
+    return sfTables.stream()
+        .map(
+            table ->
+                TableIdentifier.of(

Review Comment:
   You might want to consider moving the conversion from from snowflake identifier to that class so you can just call `snowflakeIdentifier.asIcebergIdentifier()` rather than building it here (and everywhere else the conversion is necessary).



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051655803


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.hadoop.HadoopFileIO";

Review Comment:
   This should probably default to ResolvingFileIO



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051662809


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.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.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+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

Review Comment:
   I don't feel like we need to describe the difference between this and the JDBC Catalog.  Just a clear explanation of how this catalog operates.  



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051662280


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeTableOperations.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.Map;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.apache.iceberg.snowflake.entities.SnowflakeTableMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class SnowflakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeTableOperations.class);
+  private final String catalogName;
+
+  private final FileIO fileIO;
+  private final TableIdentifier tableIdentifier;
+  private final SnowflakeIdentifier snowflakeIdentifierForTable;
+
+  private final SnowflakeClient snowflakeClient;
+
+  private final Map<String, String> catalogProperties;

Review Comment:
   This doesn't appear to be used anywhere



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1054042400


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/NamespaceHelpers.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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 org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class NamespaceHelpers {
+  private static final int MAX_NAMESPACE_DEPTH = 2;
+  private static final int NAMESPACE_ROOT_LEVEL = 0;
+  private static final int NAMESPACE_DB_LEVEL = 1;
+  private static final int NAMESPACE_SCHEMA_LEVEL = 2;
+
+  private static final Logger LOG = LoggerFactory.getLogger(NamespaceHelpers.class);
+
+  /**
+   * Converts a Namespace into a SnowflakeIdentifier representing ROOT, a DATABASE, or a SCHEMA.
+   *
+   * @throws IllegalArgumentException if the namespace is not a supported depth.
+   */
+  public static SnowflakeIdentifier getSnowflakeIdentifierForNamespace(Namespace namespace) {
+    SnowflakeIdentifier identifier = null;
+    switch (namespace.length()) {
+      case NAMESPACE_ROOT_LEVEL:
+        identifier = SnowflakeIdentifier.ofRoot();
+        break;
+      case NAMESPACE_DB_LEVEL:
+        identifier = SnowflakeIdentifier.ofDatabase(namespace.level(NAMESPACE_DB_LEVEL - 1));
+        break;
+      case NAMESPACE_SCHEMA_LEVEL:
+        identifier =
+            SnowflakeIdentifier.ofSchema(
+                namespace.level(NAMESPACE_DB_LEVEL - 1),
+                namespace.level(NAMESPACE_SCHEMA_LEVEL - 1));
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format(
+                "Snowflake max namespace level is %d, got namespace '%s'",
+                MAX_NAMESPACE_DEPTH, namespace));
+    }
+    LOG.debug("getSnowflakeIdentifierForNamespace({}) -> {}", namespace, identifier);
+    return identifier;
+  }
+
+  /**
+   * Converts a TableIdentifier into a SnowflakeIdentifier of type TABLE; the identifier must have
+   * exactly the right namespace depth to represent a fully-qualified Snowflake table identifier.
+   */
+  public static SnowflakeIdentifier getSnowflakeIdentifierForTableIdentifier(
+      TableIdentifier identifier) {
+    SnowflakeIdentifier namespaceScope = getSnowflakeIdentifierForNamespace(identifier.namespace());
+    Preconditions.checkArgument(
+        namespaceScope.getType() == SnowflakeIdentifier.Type.SCHEMA,
+        "Namespace portion of '%s' must be at the SCHEMA level, got namespaceScope '%s'",
+        identifier,
+        namespaceScope);
+    SnowflakeIdentifier ret =
+        SnowflakeIdentifier.ofTable(
+            namespaceScope.getDatabaseName(), namespaceScope.getSchemaName(), identifier.name());
+    LOG.debug("getSnowflakeIdentifierForTableIdentifier({}) -> {}", identifier, ret);
+    return ret;
+  }
+
+  private NamespaceHelpers() {}

Review Comment:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1052975968


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.hadoop.HadoopFileIO";
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    LOG.debug("listTables with namespace: {}", namespace);
+    SnowflakeIdentifier scope = NamespaceHelpers.getSnowflakeIdentifierForNamespace(namespace);
+    Preconditions.checkArgument(
+        scope.getType() == SnowflakeIdentifier.Type.ROOT
+            || scope.getType() == SnowflakeIdentifier.Type.DATABASE
+            || scope.getType() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at ROOT, DATABASE, or SCHEMA level; got %s from namespace %s",

Review Comment:
   No, tables are schema-level only even though Snowflake's built-ins do default to recursive-listing behaviors (e.g. `show iceberg tables in datatabase foo`). Changed to only support listing tables at SCHEMA level.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1063817856


##########
core/src/test/java/org/apache/iceberg/io/InMemoryFileIO.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.io;
+
+import java.util.Map;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class InMemoryFileIO implements FileIO {

Review Comment:
   I think @nastra was suggesting that we just take the in-memory implementation as a separate PR and get that committed before this PR, not change the packaging and follow up with a PR to move it.  Then you can just rebase this PR on top.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1056616708


##########
snowflake/src/test/java/org/apache/iceberg/snowflake/JdbcSnowflakeClientTest.java:
##########
@@ -0,0 +1,620 @@
+/*
+ * 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 static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+import org.apache.iceberg.ClientPool;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.jdbc.UncheckedInterruptedException;
+import org.apache.iceberg.jdbc.UncheckedSQLException;
+import org.assertj.core.api.Assertions;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.ArgumentMatchers;
+import org.mockito.Mock;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.junit.MockitoJUnitRunner;
+import org.mockito.stubbing.Answer;
+
+@RunWith(MockitoJUnitRunner.class)
+public class JdbcSnowflakeClientTest {
+  @Mock private Connection mockConnection;
+  @Mock private JdbcClientPool mockClientPool;
+  @Mock private JdbcSnowflakeClient.QueryHarness mockQueryHarness;
+  @Mock private ResultSet mockResultSet;
+
+  private JdbcSnowflakeClient snowflakeClient;
+
+  @Before
+  public void before() throws SQLException, InterruptedException {
+    snowflakeClient = new JdbcSnowflakeClient(mockClientPool);
+    snowflakeClient.setQueryHarness(mockQueryHarness);
+
+    doAnswer(
+            new Answer() {
+              @Override
+              public Object answer(InvocationOnMock invocation) throws Throwable {
+                return ((ClientPool.Action) invocation.getArguments()[0]).run(mockConnection);
+              }
+            })
+        .when(mockClientPool)
+        .run(any(ClientPool.Action.class));
+    doAnswer(
+            new Answer() {
+              @Override
+              public Object answer(InvocationOnMock invocation) throws Throwable {
+                return ((JdbcSnowflakeClient.ResultSetParser) invocation.getArguments()[2])
+                    .parse(mockResultSet);
+              }
+            })
+        .when(mockQueryHarness)
+        .query(
+            any(Connection.class),
+            any(String.class),
+            any(JdbcSnowflakeClient.ResultSetParser.class),
+            ArgumentMatchers.<String>any());
+  }
+
+  @Test
+  public void testNullClientPoolInConstructor() {
+    Assertions.assertThatExceptionOfType(IllegalArgumentException.class)
+        .isThrownBy(() -> new JdbcSnowflakeClient(null))
+        .withMessageContaining("JdbcClientPool must be non-null");
+  }
+
+  @Test
+  public void testDatabaseExists() throws SQLException {
+    when(mockResultSet.next()).thenReturn(true).thenReturn(false);
+    when(mockResultSet.getString("name")).thenReturn("DB_1");
+
+    Assertions.assertThat(snowflakeClient.databaseExists(SnowflakeIdentifier.ofDatabase("DB_1")))
+        .isTrue();
+
+    verify(mockQueryHarness)
+        .query(
+            eq(mockConnection),
+            eq("SHOW DATABASES LIKE 'DB_1' IN ACCOUNT"),
+            any(JdbcSnowflakeClient.ResultSetParser.class),
+            eq((String[]) null));

Review Comment:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1053850427


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.hadoop.HadoopFileIO";
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    LOG.debug("listTables with namespace: {}", namespace);
+    SnowflakeIdentifier scope = NamespaceHelpers.getSnowflakeIdentifierForNamespace(namespace);
+    Preconditions.checkArgument(
+        scope.getType() == SnowflakeIdentifier.Type.ROOT
+            || scope.getType() == SnowflakeIdentifier.Type.DATABASE
+            || scope.getType() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at ROOT, DATABASE, or SCHEMA level; got %s from namespace %s",
+        scope,
+        namespace);
+
+    List<SnowflakeIdentifier> sfTables = snowflakeClient.listIcebergTables(scope);

Review Comment:
   Also updated behavior of listNamespaces at the root level to only return the list of databases/1-level namespaces instead of behaving as if it's a recursive listing of 2-level namespaces.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051682638


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeTableOperations.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.Map;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.apache.iceberg.snowflake.entities.SnowflakeTableMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class SnowflakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeTableOperations.class);
+  private final String catalogName;
+
+  private final FileIO fileIO;
+  private final TableIdentifier tableIdentifier;
+  private final SnowflakeIdentifier snowflakeIdentifierForTable;
+
+  private final SnowflakeClient snowflakeClient;
+
+  private final Map<String, String> catalogProperties;
+
+  protected SnowflakeTableOperations(
+      SnowflakeClient snowflakeClient,
+      FileIO fileIO,
+      Map<String, String> properties,
+      String catalogName,
+      TableIdentifier tableIdentifier) {
+    this.snowflakeClient = snowflakeClient;
+    this.fileIO = fileIO;
+    this.catalogProperties = properties;
+    this.catalogName = catalogName;
+    this.tableIdentifier = tableIdentifier;
+    this.snowflakeIdentifierForTable =
+        NamespaceHelpers.getSnowflakeIdentifierForTableIdentifier(tableIdentifier);
+  }
+
+  @Override
+  public void doRefresh() {
+    LOG.debug("Getting metadata location for table {}", tableIdentifier);
+    String location = getTableMetadataLocation();
+    Preconditions.checkState(
+        location != null && !location.isEmpty(),
+        "Got null or empty location %s for table %s",
+        location,
+        tableIdentifier);
+    refreshFromMetadataLocation(location);
+  }
+
+  @Override
+  public FileIO io() {
+    return fileIO;
+  }
+
+  @Override
+  protected String tableName() {
+    return tableIdentifier.toString();
+  }
+
+  private String getTableMetadataLocation() {
+    SnowflakeTableMetadata metadata = snowflakeClient.getTableMetadata(snowflakeIdentifierForTable);
+
+    if (metadata == null) {
+      throw new NoSuchTableException("Cannot find table %s", snowflakeIdentifierForTable);
+    }
+    if (!metadata.getStatus().equals("success")) {

Review Comment:
   Missing empty lines in this method.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051681930


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeClient.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.snowflake;
+
+import java.io.Closeable;
+import java.util.List;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+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 {

Review Comment:
   Agreed.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1056623516


##########
snowflake/src/test/java/org/apache/iceberg/snowflake/FakeSnowflakeClient.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.jdbc.UncheckedSQLException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class FakeSnowflakeClient implements SnowflakeClient {
+  // In-memory lookup by database/schema/tableName to table metadata.
+  private Map<String, Map<String, Map<String, SnowflakeTableMetadata>>> databases =
+      Maps.newTreeMap();
+  private boolean closed = false;
+
+  public FakeSnowflakeClient() {}
+
+  /**
+   * Also adds parent database/schema if they don't already exist. If the tableName already exists
+   * under the given database/schema, the value is replaced with the provided metadata.
+   */
+  public void addTable(

Review Comment:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1052700505


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.hadoop.HadoopFileIO";

Review Comment:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1052739871


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.hadoop.HadoopFileIO";
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    LOG.debug("listTables with namespace: {}", namespace);
+    SnowflakeIdentifier scope = NamespaceHelpers.getSnowflakeIdentifierForNamespace(namespace);
+    Preconditions.checkArgument(
+        scope.getType() == SnowflakeIdentifier.Type.ROOT
+            || scope.getType() == SnowflakeIdentifier.Type.DATABASE
+            || scope.getType() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at ROOT, DATABASE, or SCHEMA level; got %s from namespace %s",
+        scope,
+        namespace);
+
+    List<SnowflakeIdentifier> sfTables = snowflakeClient.listIcebergTables(scope);
+
+    return sfTables.stream()
+        .map(
+            table ->
+                TableIdentifier.of(
+                    table.getDatabaseName(), table.getSchemaName(), table.getTableName()))
+        .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) {
+    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);
+
+    String fileIOImpl = DEFAULT_FILE_IO_IMPL;
+    if (properties.containsKey(CatalogProperties.FILE_IO_IMPL)) {
+      fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL);
+    }
+
+    initialize(
+        name,
+        new JdbcSnowflakeClient(connectionPool),
+        CatalogUtil.loadFileIO(fileIOImpl, properties, conf),
+        properties);
+  }
+
+  /**
+   * Initialize using caller-supplied SnowflakeClient and FileIO.
+   *
+   * @param name The name of the catalog, defaults to "snowflake_catalog"
+   * @param snowflakeClient The client encapsulating network communication with Snowflake
+   * @param fileIO The {@link FileIO} to use for table operations
+   * @param properties The catalog options to use and propagate to dependencies
+   */
+  @SuppressWarnings("checkstyle:HiddenField")
+  public void initialize(
+      String name, SnowflakeClient snowflakeClient, FileIO fileIO, Map<String, String> properties) {
+    Preconditions.checkArgument(null != snowflakeClient, "snowflakeClient must be non-null");
+    Preconditions.checkArgument(null != fileIO, "fileIO must be non-null");
+    this.catalogName = name == null ? DEFAULT_CATALOG_NAME : name;
+    this.snowflakeClient = snowflakeClient;
+    this.fileIO = fileIO;
+    this.catalogProperties = properties;
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(snowflakeClient);
+    closeableGroup.addCloseable(fileIO);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (null != closeableGroup) {
+      closeableGroup.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);
+    SnowflakeIdentifier scope = NamespaceHelpers.getSnowflakeIdentifierForNamespace(namespace);
+    Preconditions.checkArgument(
+        scope.getType() == SnowflakeIdentifier.Type.ROOT
+            || scope.getType() == SnowflakeIdentifier.Type.DATABASE,
+        "listNamespaces must be at either ROOT or DATABASE level; got %s from namespace %s",
+        scope,
+        namespace);
+    List<SnowflakeIdentifier> sfSchemas = snowflakeClient.listSchemas(scope);
+
+    List<Namespace> namespaceList =
+        sfSchemas.stream()
+            .map(
+                schema -> {
+                  Preconditions.checkState(
+                      schema.getType() == SnowflakeIdentifier.Type.SCHEMA,
+                      "Got identifier of type %s from listSchemas for %s",
+                      schema.getType(),
+                      namespace);
+                  return Namespace.of(schema.getDatabaseName(), schema.getSchemaName());
+                })
+            .collect(Collectors.toList());
+    return namespaceList;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace)
+      throws NoSuchNamespaceException {
+    LOG.debug("loadNamespaceMetadata with namespace: {}", namespace);

Review Comment:
   Mostly have found method-call level debug logs useful when a top-level call results in multiple possibly non-obvious underlying calls (e.g. when reading a table goes through newTableOps -> doRefresh, etc) but I agree these top-level calls that mostly correspond one-to-one with user commands probably don't benefit much from these. Removed from this class.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1052903155


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeTableOperations.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.Map;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.apache.iceberg.snowflake.entities.SnowflakeTableMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class SnowflakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeTableOperations.class);
+  private final String catalogName;
+
+  private final FileIO fileIO;
+  private final TableIdentifier tableIdentifier;
+  private final SnowflakeIdentifier snowflakeIdentifierForTable;
+
+  private final SnowflakeClient snowflakeClient;
+
+  private final Map<String, String> catalogProperties;
+
+  protected SnowflakeTableOperations(
+      SnowflakeClient snowflakeClient,
+      FileIO fileIO,
+      Map<String, String> properties,
+      String catalogName,
+      TableIdentifier tableIdentifier) {
+    this.snowflakeClient = snowflakeClient;
+    this.fileIO = fileIO;
+    this.catalogProperties = properties;
+    this.catalogName = catalogName;
+    this.tableIdentifier = tableIdentifier;
+    this.snowflakeIdentifierForTable =
+        NamespaceHelpers.getSnowflakeIdentifierForTableIdentifier(tableIdentifier);
+  }
+
+  @Override
+  public void doRefresh() {
+    LOG.debug("Getting metadata location for table {}", tableIdentifier);
+    String location = getTableMetadataLocation();
+    Preconditions.checkState(
+        location != null && !location.isEmpty(),
+        "Got null or empty location %s for table %s",
+        location,
+        tableIdentifier);
+    refreshFromMetadataLocation(location);
+  }
+
+  @Override
+  public FileIO io() {
+    return fileIO;
+  }
+
+  @Override
+  protected String tableName() {
+    return tableIdentifier.toString();

Review Comment:
   Done. Also added a test case since it's not  necessarily obvious at first glance whether all the plumbing is correct, but it's a bit messy since `tableName()` only has `protected` access in `BaseMetastoreTableOperations`. I ended up exposing a package-private version of the same thing that calls through. I guess if it's only used for logging purposes anyways though in BaseMetastoreTableOperations it might be okay to leave out the test case? Or if it's not too ugly we can keep the test case.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1063835005


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeClient.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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;
+
+/**
+ * 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.
+ */
+interface SnowflakeClient extends Closeable {
+
+  /** Returns true if the database exists, false otherwise. */
+  boolean databaseExists(SnowflakeIdentifier database);
+
+  /** Returns true if the schema and its parent database exists, false otherwise. */
+  boolean schemaExists(SnowflakeIdentifier schema);
+
+  /** Lists all Snowflake databases within the currently configured account. */
+  List<SnowflakeIdentifier> listDatabases();
+
+  /**
+   * Lists all Snowflake schemas within a given scope. Returned SnowflakeIdentifiers must have
+   * type() == SnowflakeIdentifier.Type.SCHEMA.
+   *
+   * @param scope The scope in which to list, which may be ROOT or a single DATABASE.

Review Comment:
   This seems to imply that if you pass `ROOT` type identifier that it would recursively list the schemas across all databases.  Listing schemas (similar to listing tables) should only return the current level.  I believe this should be limited to `DATABASE`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1063872853


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java:
##########
@@ -0,0 +1,386 @@
+/*
+ * 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.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+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.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+/**
+ * This implementation of SnowflakeClient builds on top of Snowflake's JDBC driver to interact with
+ * Snowflake's Iceberg-aware resource model.
+ */
+class JdbcSnowflakeClient implements SnowflakeClient {
+  static final String EXPECTED_JDBC_IMPL = "net.snowflake.client.jdbc.SnowflakeDriver";
+
+  @FunctionalInterface
+  interface ResultSetParser<T> {
+    T parse(ResultSet rs) throws SQLException;
+  }
+
+  /**
+   * This class wraps the basic boilerplate of setting up PreparedStatements and applying a
+   * ResultSetParser to translate a ResultSet into parsed objects. Allows easily injecting
+   * subclasses for debugging/testing purposes.
+   */
+  static class QueryHarness {
+    public <T> T query(Connection conn, String sql, ResultSetParser<T> parser, String... args)
+        throws SQLException {
+      try (PreparedStatement statement = conn.prepareStatement(sql)) {
+        if (args != null) {
+          for (int i = 0; i < args.length; ++i) {
+            statement.setString(i + 1, args[i]);
+          }
+        }
+
+        try (ResultSet rs = statement.executeQuery()) {
+          return parser.parse(rs);
+        }
+      }
+    }
+  }
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Database identifiers,
+   * containing "name" (representing databaseName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> DATABASE_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> databases = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("name");
+          databases.add(SnowflakeIdentifier.ofDatabase(databaseName));
+        }
+        return databases;
+      };
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Schema identifiers,
+   * containing "database_name" and "name" (representing schemaName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> SCHEMA_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> schemas = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("database_name");
+          String schemaName = rs.getString("name");
+          schemas.add(SnowflakeIdentifier.ofSchema(databaseName, schemaName));
+        }
+        return schemas;
+      };
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Table identifiers,
+   * containing "database_name", "schema_name", and "name" (representing tableName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> TABLE_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> tables = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("database_name");
+          String schemaName = rs.getString("schema_name");
+          String tableName = rs.getString("name");
+          tables.add(SnowflakeIdentifier.ofTable(databaseName, schemaName, tableName));
+        }
+        return tables;
+      };
+
+  /**
+   * Expects to handle ResultSets representing a single record holding Snowflake Iceberg metadata.
+   */
+  public static final ResultSetParser<SnowflakeTableMetadata> TABLE_METADATA_RESULT_SET_HANDLER =
+      rs -> {
+        if (!rs.next()) {
+          return null;
+        }
+
+        String rawJsonVal = rs.getString("METADATA");
+        return SnowflakeTableMetadata.parseJson(rawJsonVal);
+      };
+
+  private final JdbcClientPool connectionPool;
+  private QueryHarness queryHarness;
+
+  JdbcSnowflakeClient(JdbcClientPool conn) {
+    Preconditions.checkArgument(null != conn, "JdbcClientPool must be non-null");
+    connectionPool = conn;
+    queryHarness = new QueryHarness();
+  }
+
+  @VisibleForTesting
+  void setQueryHarness(QueryHarness queryHarness) {
+    this.queryHarness = queryHarness;
+  }
+
+  /**
+   * For rare cases where PreparedStatements aren't supported for user-supplied identifiers intended
+   * for use in special LIKE clauses, we can sanitize by "broadening" the identifier with
+   * single-character wildcards and manually post-filter client-side.
+   *
+   * <p>Note: This sanitization approach intentionally "broadens" the scope of matching results;
+   * callers must be able to handle this method returning an all-wildcard expression; i.e. the
+   * caller must treat the usage of the LIKE clause as only an optional optimization, and should
+   * post-filter for correctness as if the LIKE clause wasn't present in the query at all.
+   */
+  @VisibleForTesting
+  String sanitizeIdentifierWithWildcardForLikeClause(String identifier) {
+    // Restrict identifiers to the "Unquoted object identifiers" synax documented at
+    // https://docs.snowflake.com/en/sql-reference/identifiers-syntax.html
+    //
+    // Use a strict allowlist of characters, replace everything *not* matching the character set
+    // with "_", which is used as a single-character wildcard in Snowflake.
+    String sanitized = identifier.replaceAll("[^a-zA-Z0-9_$]", "_");
+    if (sanitized.startsWith("$")) {
+      sanitized = "_" + sanitized.substring(1);
+    }
+    return sanitized;
+  }
+
+  @Override
+  public boolean databaseExists(SnowflakeIdentifier database) {
+    Preconditions.checkArgument(
+        database.type() == SnowflakeIdentifier.Type.DATABASE,
+        "databaseExists requires a DATABASE identifier, got '%s'",
+        database);
+
+    // Due to current limitations in PreparedStatement parameters for the LIKE clause in
+    // SHOW DATABASES queries, we'll use a fairly limited allowlist for identifier characters,
+    // using wildcards for non-allowed characters, and post-filter for matching.
+    final String finalQuery =
+        String.format(
+            "SHOW DATABASES LIKE '%s' IN ACCOUNT",
+            sanitizeIdentifierWithWildcardForLikeClause(database.databaseName()));
+    List<SnowflakeIdentifier> databases;
+    try {
+      databases =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn, finalQuery, DATABASE_RESULT_SET_HANDLER, (String[]) null));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to check if database '%s' exists", database);
+    } catch (InterruptedException e) {
+      throw new UncheckedInterruptedException(
+          e, "Interrupted while checking if database '%s' exists", database);
+    }
+
+    // Filter to handle the edge case of '_' appearing as a wildcard that can't be remapped the way
+    // it can for predicates in SELECT statements.
+    databases.removeIf(db -> !database.databaseName().equalsIgnoreCase(db.databaseName()));
+    return !databases.isEmpty();
+  }
+
+  @Override
+  public boolean schemaExists(SnowflakeIdentifier schema) {
+    Preconditions.checkArgument(
+        schema.type() == SnowflakeIdentifier.Type.SCHEMA,
+        "schemaExists requires a SCHEMA identifier, got '%s'",
+        schema);
+
+    if (!databaseExists(SnowflakeIdentifier.ofDatabase(schema.databaseName()))) {
+      return false;
+    }
+
+    // Due to current limitations in PreparedStatement parameters for the LIKE clause in
+    // SHOW SCHEMAS queries, we'll use a fairly limited allowlist for identifier characters,
+    // using wildcards for non-allowed characters, and post-filter for matching.
+    final String finalQuery =
+        String.format(
+            "SHOW SCHEMAS LIKE '%s' IN DATABASE IDENTIFIER(?)",
+            sanitizeIdentifierWithWildcardForLikeClause(schema.schemaName()));
+    List<SnowflakeIdentifier> schemas;
+    try {
+      schemas =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn,
+                      finalQuery,
+                      SCHEMA_RESULT_SET_HANDLER,
+                      new String[] {schema.databaseName()}));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to check if schema '%s' exists", schema);
+    } catch (InterruptedException e) {
+      throw new UncheckedInterruptedException(
+          e, "Interrupted while checking if schema '%s' exists", schema);
+    }
+
+    // Filter to handle the edge case of '_' appearing as a wildcard that can't be remapped the way
+    // it can for predicates in SELECT statements.
+    schemas.removeIf(sc -> !schema.schemaName().equalsIgnoreCase(sc.schemaName()));
+    return !schemas.isEmpty();
+  }
+
+  @Override
+  public List<SnowflakeIdentifier> listDatabases() {
+    List<SnowflakeIdentifier> databases;
+    try {
+      databases =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn,
+                      "SHOW DATABASES IN ACCOUNT",
+                      DATABASE_RESULT_SET_HANDLER,
+                      (String[]) null));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to list databases");
+    } catch (InterruptedException e) {
+      throw new UncheckedInterruptedException(e, "Interrupted while listing databases");
+    }
+    databases.forEach(
+        db ->
+            Preconditions.checkState(
+                db.type() == SnowflakeIdentifier.Type.DATABASE,
+                "Expected DATABASE, got identifier '%s'",
+                db));
+    return databases;
+  }
+
+  @Override
+  public List<SnowflakeIdentifier> listSchemas(SnowflakeIdentifier scope) {
+    StringBuilder baseQuery = new StringBuilder("SHOW SCHEMAS");
+    String[] queryParams = null;
+    switch (scope.type()) {
+      case ROOT:
+        // account-level listing
+        baseQuery.append(" IN ACCOUNT");
+        break;
+      case DATABASE:
+        // database-level listing
+        baseQuery.append(" IN DATABASE IDENTIFIER(?)");
+        queryParams = new String[] {scope.toIdentifierString()};
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format("Unsupported scope type for listSchemas: %s", scope));
+    }
+
+    final String finalQuery = baseQuery.toString();
+    final String[] finalQueryParams = queryParams;
+    List<SnowflakeIdentifier> schemas;
+    try {
+      schemas =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn, finalQuery, SCHEMA_RESULT_SET_HANDLER, finalQueryParams));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to list schemas for scope '%s'", scope);
+    } catch (InterruptedException e) {
+      throw new UncheckedInterruptedException(
+          e, "Interrupted while listing schemas for scope '%s'", scope);
+    }
+    schemas.forEach(
+        schema ->
+            Preconditions.checkState(
+                schema.type() == SnowflakeIdentifier.Type.SCHEMA,
+                "Expected SCHEMA, got identifier '%s' for scope '%s'",
+                schema,
+                scope));
+    return schemas;
+  }
+
+  @Override
+  public List<SnowflakeIdentifier> listIcebergTables(SnowflakeIdentifier scope) {
+    StringBuilder baseQuery = new StringBuilder("SHOW ICEBERG TABLES");

Review Comment:
   @dennishuo I tried running this catalog implementation with Spark and this statement doesn't appear to show external iceberg tables for me.  I can run:
   
   ```
   SHOW ICEBERG TABLES IN SCHEMA <DB>.<SCHEMA>;
   ```
   
   And there are no results, but running:
   
   ```
   SHOW EXTERNAL TABLES IN SCHEMA <DB>.<SCHEMA>;
   ```
   
   Correctly shows the Iceberg tables and I can see their storage and metadata locations.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1063872853


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java:
##########
@@ -0,0 +1,386 @@
+/*
+ * 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.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+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.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+/**
+ * This implementation of SnowflakeClient builds on top of Snowflake's JDBC driver to interact with
+ * Snowflake's Iceberg-aware resource model.
+ */
+class JdbcSnowflakeClient implements SnowflakeClient {
+  static final String EXPECTED_JDBC_IMPL = "net.snowflake.client.jdbc.SnowflakeDriver";
+
+  @FunctionalInterface
+  interface ResultSetParser<T> {
+    T parse(ResultSet rs) throws SQLException;
+  }
+
+  /**
+   * This class wraps the basic boilerplate of setting up PreparedStatements and applying a
+   * ResultSetParser to translate a ResultSet into parsed objects. Allows easily injecting
+   * subclasses for debugging/testing purposes.
+   */
+  static class QueryHarness {
+    public <T> T query(Connection conn, String sql, ResultSetParser<T> parser, String... args)
+        throws SQLException {
+      try (PreparedStatement statement = conn.prepareStatement(sql)) {
+        if (args != null) {
+          for (int i = 0; i < args.length; ++i) {
+            statement.setString(i + 1, args[i]);
+          }
+        }
+
+        try (ResultSet rs = statement.executeQuery()) {
+          return parser.parse(rs);
+        }
+      }
+    }
+  }
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Database identifiers,
+   * containing "name" (representing databaseName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> DATABASE_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> databases = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("name");
+          databases.add(SnowflakeIdentifier.ofDatabase(databaseName));
+        }
+        return databases;
+      };
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Schema identifiers,
+   * containing "database_name" and "name" (representing schemaName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> SCHEMA_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> schemas = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("database_name");
+          String schemaName = rs.getString("name");
+          schemas.add(SnowflakeIdentifier.ofSchema(databaseName, schemaName));
+        }
+        return schemas;
+      };
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Table identifiers,
+   * containing "database_name", "schema_name", and "name" (representing tableName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> TABLE_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> tables = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("database_name");
+          String schemaName = rs.getString("schema_name");
+          String tableName = rs.getString("name");
+          tables.add(SnowflakeIdentifier.ofTable(databaseName, schemaName, tableName));
+        }
+        return tables;
+      };
+
+  /**
+   * Expects to handle ResultSets representing a single record holding Snowflake Iceberg metadata.
+   */
+  public static final ResultSetParser<SnowflakeTableMetadata> TABLE_METADATA_RESULT_SET_HANDLER =
+      rs -> {
+        if (!rs.next()) {
+          return null;
+        }
+
+        String rawJsonVal = rs.getString("METADATA");
+        return SnowflakeTableMetadata.parseJson(rawJsonVal);
+      };
+
+  private final JdbcClientPool connectionPool;
+  private QueryHarness queryHarness;
+
+  JdbcSnowflakeClient(JdbcClientPool conn) {
+    Preconditions.checkArgument(null != conn, "JdbcClientPool must be non-null");
+    connectionPool = conn;
+    queryHarness = new QueryHarness();
+  }
+
+  @VisibleForTesting
+  void setQueryHarness(QueryHarness queryHarness) {
+    this.queryHarness = queryHarness;
+  }
+
+  /**
+   * For rare cases where PreparedStatements aren't supported for user-supplied identifiers intended
+   * for use in special LIKE clauses, we can sanitize by "broadening" the identifier with
+   * single-character wildcards and manually post-filter client-side.
+   *
+   * <p>Note: This sanitization approach intentionally "broadens" the scope of matching results;
+   * callers must be able to handle this method returning an all-wildcard expression; i.e. the
+   * caller must treat the usage of the LIKE clause as only an optional optimization, and should
+   * post-filter for correctness as if the LIKE clause wasn't present in the query at all.
+   */
+  @VisibleForTesting
+  String sanitizeIdentifierWithWildcardForLikeClause(String identifier) {
+    // Restrict identifiers to the "Unquoted object identifiers" synax documented at
+    // https://docs.snowflake.com/en/sql-reference/identifiers-syntax.html
+    //
+    // Use a strict allowlist of characters, replace everything *not* matching the character set
+    // with "_", which is used as a single-character wildcard in Snowflake.
+    String sanitized = identifier.replaceAll("[^a-zA-Z0-9_$]", "_");
+    if (sanitized.startsWith("$")) {
+      sanitized = "_" + sanitized.substring(1);
+    }
+    return sanitized;
+  }
+
+  @Override
+  public boolean databaseExists(SnowflakeIdentifier database) {
+    Preconditions.checkArgument(
+        database.type() == SnowflakeIdentifier.Type.DATABASE,
+        "databaseExists requires a DATABASE identifier, got '%s'",
+        database);
+
+    // Due to current limitations in PreparedStatement parameters for the LIKE clause in
+    // SHOW DATABASES queries, we'll use a fairly limited allowlist for identifier characters,
+    // using wildcards for non-allowed characters, and post-filter for matching.
+    final String finalQuery =
+        String.format(
+            "SHOW DATABASES LIKE '%s' IN ACCOUNT",
+            sanitizeIdentifierWithWildcardForLikeClause(database.databaseName()));
+    List<SnowflakeIdentifier> databases;
+    try {
+      databases =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn, finalQuery, DATABASE_RESULT_SET_HANDLER, (String[]) null));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to check if database '%s' exists", database);
+    } catch (InterruptedException e) {
+      throw new UncheckedInterruptedException(
+          e, "Interrupted while checking if database '%s' exists", database);
+    }
+
+    // Filter to handle the edge case of '_' appearing as a wildcard that can't be remapped the way
+    // it can for predicates in SELECT statements.
+    databases.removeIf(db -> !database.databaseName().equalsIgnoreCase(db.databaseName()));
+    return !databases.isEmpty();
+  }
+
+  @Override
+  public boolean schemaExists(SnowflakeIdentifier schema) {
+    Preconditions.checkArgument(
+        schema.type() == SnowflakeIdentifier.Type.SCHEMA,
+        "schemaExists requires a SCHEMA identifier, got '%s'",
+        schema);
+
+    if (!databaseExists(SnowflakeIdentifier.ofDatabase(schema.databaseName()))) {
+      return false;
+    }
+
+    // Due to current limitations in PreparedStatement parameters for the LIKE clause in
+    // SHOW SCHEMAS queries, we'll use a fairly limited allowlist for identifier characters,
+    // using wildcards for non-allowed characters, and post-filter for matching.
+    final String finalQuery =
+        String.format(
+            "SHOW SCHEMAS LIKE '%s' IN DATABASE IDENTIFIER(?)",
+            sanitizeIdentifierWithWildcardForLikeClause(schema.schemaName()));
+    List<SnowflakeIdentifier> schemas;
+    try {
+      schemas =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn,
+                      finalQuery,
+                      SCHEMA_RESULT_SET_HANDLER,
+                      new String[] {schema.databaseName()}));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to check if schema '%s' exists", schema);
+    } catch (InterruptedException e) {
+      throw new UncheckedInterruptedException(
+          e, "Interrupted while checking if schema '%s' exists", schema);
+    }
+
+    // Filter to handle the edge case of '_' appearing as a wildcard that can't be remapped the way
+    // it can for predicates in SELECT statements.
+    schemas.removeIf(sc -> !schema.schemaName().equalsIgnoreCase(sc.schemaName()));
+    return !schemas.isEmpty();
+  }
+
+  @Override
+  public List<SnowflakeIdentifier> listDatabases() {
+    List<SnowflakeIdentifier> databases;
+    try {
+      databases =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn,
+                      "SHOW DATABASES IN ACCOUNT",
+                      DATABASE_RESULT_SET_HANDLER,
+                      (String[]) null));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to list databases");
+    } catch (InterruptedException e) {
+      throw new UncheckedInterruptedException(e, "Interrupted while listing databases");
+    }
+    databases.forEach(
+        db ->
+            Preconditions.checkState(
+                db.type() == SnowflakeIdentifier.Type.DATABASE,
+                "Expected DATABASE, got identifier '%s'",
+                db));
+    return databases;
+  }
+
+  @Override
+  public List<SnowflakeIdentifier> listSchemas(SnowflakeIdentifier scope) {
+    StringBuilder baseQuery = new StringBuilder("SHOW SCHEMAS");
+    String[] queryParams = null;
+    switch (scope.type()) {
+      case ROOT:
+        // account-level listing
+        baseQuery.append(" IN ACCOUNT");
+        break;
+      case DATABASE:
+        // database-level listing
+        baseQuery.append(" IN DATABASE IDENTIFIER(?)");
+        queryParams = new String[] {scope.toIdentifierString()};
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format("Unsupported scope type for listSchemas: %s", scope));
+    }
+
+    final String finalQuery = baseQuery.toString();
+    final String[] finalQueryParams = queryParams;
+    List<SnowflakeIdentifier> schemas;
+    try {
+      schemas =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn, finalQuery, SCHEMA_RESULT_SET_HANDLER, finalQueryParams));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to list schemas for scope '%s'", scope);
+    } catch (InterruptedException e) {
+      throw new UncheckedInterruptedException(
+          e, "Interrupted while listing schemas for scope '%s'", scope);
+    }
+    schemas.forEach(
+        schema ->
+            Preconditions.checkState(
+                schema.type() == SnowflakeIdentifier.Type.SCHEMA,
+                "Expected SCHEMA, got identifier '%s' for scope '%s'",
+                schema,
+                scope));
+    return schemas;
+  }
+
+  @Override
+  public List<SnowflakeIdentifier> listIcebergTables(SnowflakeIdentifier scope) {
+    StringBuilder baseQuery = new StringBuilder("SHOW ICEBERG TABLES");

Review Comment:
   @dennishuo I tried running this catalog implementation with Spark and this statement doesn't appear to show external iceberg tables for me.  I can run:
   
   ```
   SHOW ICEBERG TABLES IN SCHEMA <DB>.<SCHEMA>;
   ```
   
   And there are no results, but running:
   
   ```
   SHOW TABLES IN SCHEMA <DB>.<SCHEMA>;
   ```
   
   Correctly shows the Iceberg tables and I can see their storage and metadata locations.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1063864362


##########
core/src/test/java/org/apache/iceberg/io/InMemoryFileIO.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.io;
+
+import java.util.Map;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class InMemoryFileIO implements FileIO {

Review Comment:
   Ah makes sense, just split out https://github.com/apache/iceberg/pull/6538



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] jackye1995 commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1069937682


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeTableMetadata.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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 com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+class SnowflakeTableMetadata {
+  public static final Pattern SNOWFLAKE_AZURE_PATTERN =
+      Pattern.compile("azure://([^/]+)/([^/]+)/(.*)");
+
+  private final String snowflakeMetadataLocation;
+  private final String icebergMetadataLocation;
+  private final String status;
+
+  // Note: Since not all sources will necessarily come from a raw JSON representation, this raw
+  // JSON should only be considered a convenient debugging field. Equality of two
+  // SnowflakeTableMetadata instances should not depend on equality of this field.
+  private final String rawJsonVal;
+
+  SnowflakeTableMetadata(
+      String snowflakeMetadataLocation,
+      String icebergMetadataLocation,
+      String status,
+      String rawJsonVal) {
+    this.snowflakeMetadataLocation = snowflakeMetadataLocation;
+    this.icebergMetadataLocation = icebergMetadataLocation;
+    this.status = status;
+    this.rawJsonVal = rawJsonVal;
+  }
+
+  /** Storage location of table metadata in Snowflake's path syntax. */
+  public String snowflakeMetadataLocation() {
+    return snowflakeMetadataLocation;
+  }
+
+  /** Storage location of table metadata in Iceberg's path syntax. */
+  public String icebergMetadataLocation() {
+    return icebergMetadataLocation;
+  }
+
+  public String getStatus() {
+    return status;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    } else if (!(o instanceof SnowflakeTableMetadata)) {
+      return false;
+    }
+
+    // Only consider parsed fields, not the raw JSON that may or may not be the original source of
+    // this instance.
+    SnowflakeTableMetadata that = (SnowflakeTableMetadata) o;
+    return Objects.equal(this.snowflakeMetadataLocation, that.snowflakeMetadataLocation)
+        && Objects.equal(this.icebergMetadataLocation, that.icebergMetadataLocation)
+        && Objects.equal(this.status, that.status);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(snowflakeMetadataLocation, icebergMetadataLocation, status);
+  }
+
+  @Override
+  public String toString() {
+    return String.format(
+        "snowflakeMetadataLocation: '%s', icebergMetadataLocation: '%s', status: '%s'",
+        snowflakeMetadataLocation, icebergMetadataLocation, status);
+  }
+
+  public String toDebugString() {
+    return String.format("%s, rawJsonVal: %s", toString(), rawJsonVal);
+  }
+
+  /**
+   * Translates from Snowflake's path syntax to Iceberg's path syntax for paths matching known
+   * non-compatible Snowflake paths. Throws IllegalArgumentException if the prefix of the
+   * snowflakeLocation is a known non-compatible path syntax but fails to match the expected path
+   * components for a successful translation.
+   */
+  public static String snowflakeLocationToIcebergLocation(String snowflakeLocation) {

Review Comment:
   Thanks for the clarification, I approved the changes, very excited to see this from Snowflake.
   
   >  I've been meaning to open a discussion to see if anyone has thought about maybe adding some ease-of-use hooks for last-mile automatic basic path translations right before they go to FileIO resolution.
   
   For HadoopFileIO, i think this is not really needed because typically we see our users already configured HDFS settings to map schemes to whatever file system implementations they would like to use.
   
   I think ResolvingFileIO to some extent already does this kind of translation to some extent, maybe we can extend its functionality at that front.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks merged pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks merged PR #6428:
URL: https://github.com/apache/iceberg/pull/6428


-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1066451225


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeTableMetadata.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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 com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+class SnowflakeTableMetadata {
+  public static final Pattern SNOWFLAKE_AZURE_PATTERN =
+      Pattern.compile("azure://([^/]+)/([^/]+)/(.*)");
+
+  private final String snowflakeMetadataLocation;
+  private final String icebergMetadataLocation;
+  private final String status;
+
+  // Note: Since not all sources will necessarily come from a raw JSON representation, this raw
+  // JSON should only be considered a convenient debugging field. Equality of two
+  // SnowflakeTableMetadata instances should not depend on equality of this field.
+  private final String rawJsonVal;
+
+  SnowflakeTableMetadata(
+      String snowflakeMetadataLocation,
+      String icebergMetadataLocation,
+      String status,
+      String rawJsonVal) {
+    this.snowflakeMetadataLocation = snowflakeMetadataLocation;
+    this.icebergMetadataLocation = icebergMetadataLocation;
+    this.status = status;
+    this.rawJsonVal = rawJsonVal;
+  }
+
+  /** Storage location of table metadata in Snowflake's path syntax. */
+  public String snowflakeMetadataLocation() {
+    return snowflakeMetadataLocation;
+  }
+
+  /** Storage location of table metadata in Iceberg's path syntax. */
+  public String icebergMetadataLocation() {
+    return icebergMetadataLocation;
+  }
+
+  public String getStatus() {
+    return status;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    } else if (!(o instanceof SnowflakeTableMetadata)) {
+      return false;
+    }
+
+    // Only consider parsed fields, not the raw JSON that may or may not be the original source of
+    // this instance.
+    SnowflakeTableMetadata that = (SnowflakeTableMetadata) o;
+    return Objects.equal(this.snowflakeMetadataLocation, that.snowflakeMetadataLocation)
+        && Objects.equal(this.icebergMetadataLocation, that.icebergMetadataLocation)
+        && Objects.equal(this.status, that.status);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(snowflakeMetadataLocation, icebergMetadataLocation, status);
+  }
+
+  @Override
+  public String toString() {
+    return String.format(
+        "snowflakeMetadataLocation: '%s', icebergMetadataLocation: '%s', status: '%s', rawJsonVal: %s",

Review Comment:
   Done. It's not too large for now, but I agree it's better split out, then toString() could be canonical regardless of source. 



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051651449


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/entities/SnowflakeIdentifier.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class SnowflakeIdentifier {

Review Comment:
   Please add Javadoc to describe the name collisions between the snowflake and iceberg terminology and why this identifier is used as opposed to the Iceberg TableIdentifier.  



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051656611


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.hadoop.HadoopFileIO";
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    LOG.debug("listTables with namespace: {}", namespace);
+    SnowflakeIdentifier scope = NamespaceHelpers.getSnowflakeIdentifierForNamespace(namespace);
+    Preconditions.checkArgument(
+        scope.getType() == SnowflakeIdentifier.Type.ROOT
+            || scope.getType() == SnowflakeIdentifier.Type.DATABASE
+            || scope.getType() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at ROOT, DATABASE, or SCHEMA level; got %s from namespace %s",
+        scope,
+        namespace);
+
+    List<SnowflakeIdentifier> sfTables = snowflakeClient.listIcebergTables(scope);
+
+    return sfTables.stream()
+        .map(
+            table ->
+                TableIdentifier.of(
+                    table.getDatabaseName(), table.getSchemaName(), table.getTableName()))
+        .collect(Collectors.toList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    throw new UnsupportedOperationException(
+        String.format("dropTable not supported; attempted for table '%s'", identifier));

Review Comment:
   There's no need to log what was attempted if it's not supported.  Just log something like `Snowflake catalog does not support dropping tables.` 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051657435


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeClient.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.snowflake;
+
+import java.io.Closeable;
+import java.util.List;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+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 {

Review Comment:
   I don't believe this needs to be public



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1052825989


##########
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')

Review Comment:
   Looks like this was vestigial from fiddling around with S3FileIO directly earlier, removed.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1056538981


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.io.ResolvingFileIO";

Review Comment:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1056539354


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.io.ResolvingFileIO";
+
+  static class FileIOFactory {
+    public FileIO newFileIO(String impl, Map<String, String> properties, Object hadoopConf) {
+      return CatalogUtil.loadFileIO(impl, properties, hadoopConf);
+    }
+  }
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIOFactory fileIOFactory;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    SnowflakeIdentifier scope = NamespaceHelpers.toSnowflakeIdentifier(namespace);
+    Preconditions.checkArgument(
+        scope.type() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at SCHEMA level; got %s from namespace %s",
+        scope,
+        namespace);
+
+    List<SnowflakeIdentifier> sfTables = snowflakeClient.listIcebergTables(scope);
+
+    return sfTables.stream()
+        .map(NamespaceHelpers::toIcebergTableIdentifier)
+        .collect(Collectors.toList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support dropTable");
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support renameTable");
+  }
+
+  @Override
+  public void initialize(String name, Map<String, String> properties) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");

Review Comment:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#issuecomment-1364368088

   @nastra Thanks, happy holidays to you too! I think I got to all your comments, but please let me know if I missed anything


-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1056615649


##########
snowflake/src/test/java/org/apache/iceberg/snowflake/JdbcSnowflakeClientTest.java:
##########
@@ -0,0 +1,620 @@
+/*
+ * 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 static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+import org.apache.iceberg.ClientPool;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.jdbc.UncheckedInterruptedException;
+import org.apache.iceberg.jdbc.UncheckedSQLException;
+import org.assertj.core.api.Assertions;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.ArgumentMatchers;
+import org.mockito.Mock;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.junit.MockitoJUnitRunner;
+import org.mockito.stubbing.Answer;
+
+@RunWith(MockitoJUnitRunner.class)
+public class JdbcSnowflakeClientTest {
+  @Mock private Connection mockConnection;
+  @Mock private JdbcClientPool mockClientPool;
+  @Mock private JdbcSnowflakeClient.QueryHarness mockQueryHarness;
+  @Mock private ResultSet mockResultSet;
+
+  private JdbcSnowflakeClient snowflakeClient;
+
+  @Before
+  public void before() throws SQLException, InterruptedException {
+    snowflakeClient = new JdbcSnowflakeClient(mockClientPool);
+    snowflakeClient.setQueryHarness(mockQueryHarness);
+
+    doAnswer(
+            new Answer() {

Review Comment:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1052972711


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeTableOperations.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.Map;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.apache.iceberg.snowflake.entities.SnowflakeTableMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class SnowflakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeTableOperations.class);
+  private final String catalogName;

Review Comment:
   Removed, rolled into `fullTableName` in constructor and then no longer held beyond constructor.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1063857287


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,254 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  private static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  private static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.io.ResolvingFileIO";
+
+  static class FileIOFactory {
+    public FileIO newFileIO(String impl, Map<String, String> properties, Object hadoopConf) {
+      return CatalogUtil.loadFileIO(impl, properties, hadoopConf);
+    }
+  }
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIOFactory fileIOFactory;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    SnowflakeIdentifier scope = NamespaceHelpers.toSnowflakeIdentifier(namespace);
+    Preconditions.checkArgument(
+        scope.type() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at SCHEMA level; got %s from namespace %s",
+        scope,
+        namespace);
+
+    List<SnowflakeIdentifier> sfTables = snowflakeClient.listIcebergTables(scope);
+
+    return sfTables.stream()
+        .map(NamespaceHelpers::toIcebergTableIdentifier)
+        .collect(Collectors.toList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support dropTable");
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support renameTable");
+  }
+
+  @Override
+  public void initialize(String name, Map<String, String> properties) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkArgument(null != 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 communication.
+      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);
+
+    initialize(name, new JdbcSnowflakeClient(connectionPool), new FileIOFactory(), properties);
+  }
+
+  /**
+   * Initialize using caller-supplied SnowflakeClient and FileIO.
+   *
+   * @param name The name of the catalog, defaults to "snowflake_catalog"
+   * @param snowflakeClient The client encapsulating network communication with Snowflake
+   * @param fileIOFactory The {@link FileIOFactory} to use to instantiate a new FileIO for each new
+   *     table operation
+   * @param properties The catalog options to use and propagate to dependencies
+   */
+  @SuppressWarnings("checkstyle:HiddenField")
+  void initialize(
+      String name,
+      SnowflakeClient snowflakeClient,
+      FileIOFactory fileIOFactory,
+      Map<String, String> properties) {
+    Preconditions.checkArgument(null != snowflakeClient, "snowflakeClient must be non-null");
+    Preconditions.checkArgument(null != fileIOFactory, "fileIOFactory must be non-null");
+    this.catalogName = name == null ? DEFAULT_CATALOG_NAME : name;
+    this.snowflakeClient = snowflakeClient;
+    this.fileIOFactory = fileIOFactory;
+    this.catalogProperties = properties;
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(snowflakeClient);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (null != closeableGroup) {
+      closeableGroup.close();
+    }
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support createNamespace");
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    SnowflakeIdentifier scope = NamespaceHelpers.toSnowflakeIdentifier(namespace);
+    List<SnowflakeIdentifier> results = null;
+    switch (scope.type()) {
+      case ROOT:
+        results = snowflakeClient.listDatabases();
+        break;
+      case DATABASE:
+        results = snowflakeClient.listSchemas(scope);
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format(
+                "listNamespaces must be at either ROOT or DATABASE level; got %s from namespace %s",
+                scope, namespace));
+    }
+
+    List<Namespace> namespaceList =
+        results.stream().map(NamespaceHelpers::toIcebergNamespace).collect(Collectors.toList());
+    return namespaceList;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace)
+      throws NoSuchNamespaceException {
+    SnowflakeIdentifier id = NamespaceHelpers.toSnowflakeIdentifier(namespace);
+    boolean namespaceExists;
+    switch (id.type()) {
+      case DATABASE:
+        namespaceExists = snowflakeClient.databaseExists(id);
+        break;
+      case SCHEMA:
+        namespaceExists = snowflakeClient.schemaExists(id);
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format(
+                "loadNamespaceMetadat must be at either DATABASE or SCHEMA level; got %s from namespace %s",
+                id, namespace));
+    }
+    if (namespaceExists) {
+      return ImmutableMap.of();
+    } else {
+      throw new NoSuchNamespaceException(
+          "Namespace '%s' with snowflake identifier '%s' doesn't exist", namespace, id);
+    }
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support dropNamespace");
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support setProperties");
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support removeProperties");
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    String fileIOImpl = DEFAULT_FILE_IO_IMPL;
+    if (catalogProperties.containsKey(CatalogProperties.FILE_IO_IMPL)) {
+      fileIOImpl = catalogProperties.get(CatalogProperties.FILE_IO_IMPL);
+    }
+
+    // Initialize a fresh FileIO for each TableOperations created, because some FileIO
+    // implementations such as S3FileIO can become bound to a single S3 bucket. Additionally,
+    // FileIO implementations often support only a finite set of one or more URI schemes (i.e.
+    // S3FileIO only supports s3/s3a/s3n, and even ResolvingFileIO only supports the combination
+    // of schemes registered for S3FileIO and HadoopFileIO). Individual catalogs may need to
+    // support tables across different cloud/storage providers with disjoint FileIO implementations.
+    FileIO fileIO = fileIOFactory.newFileIO(fileIOImpl, catalogProperties, conf);
+    closeableGroup.addCloseable(fileIO);
+    return new SnowflakeTableOperations(snowflakeClient, fileIO, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier tableIdentifier) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support defaultWarehouseLocation");
+  }
+
+  @Override
+  public void setConf(Object conf) {
+    this.conf = conf;
+  }
+
+  public Object getConf() {

Review Comment:
   Removed



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1063861242


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeTableMetadata.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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 com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+class SnowflakeTableMetadata {
+  public static final Pattern SNOWFLAKE_AZURE_PATTERN =
+      Pattern.compile("azure://([^/]+)/([^/]+)/(.*)");
+
+  private String snowflakeMetadataLocation;
+  private String icebergMetadataLocation;
+  private String status;

Review Comment:
   These can be final



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1063872853


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java:
##########
@@ -0,0 +1,386 @@
+/*
+ * 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.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+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.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+/**
+ * This implementation of SnowflakeClient builds on top of Snowflake's JDBC driver to interact with
+ * Snowflake's Iceberg-aware resource model.
+ */
+class JdbcSnowflakeClient implements SnowflakeClient {
+  static final String EXPECTED_JDBC_IMPL = "net.snowflake.client.jdbc.SnowflakeDriver";
+
+  @FunctionalInterface
+  interface ResultSetParser<T> {
+    T parse(ResultSet rs) throws SQLException;
+  }
+
+  /**
+   * This class wraps the basic boilerplate of setting up PreparedStatements and applying a
+   * ResultSetParser to translate a ResultSet into parsed objects. Allows easily injecting
+   * subclasses for debugging/testing purposes.
+   */
+  static class QueryHarness {
+    public <T> T query(Connection conn, String sql, ResultSetParser<T> parser, String... args)
+        throws SQLException {
+      try (PreparedStatement statement = conn.prepareStatement(sql)) {
+        if (args != null) {
+          for (int i = 0; i < args.length; ++i) {
+            statement.setString(i + 1, args[i]);
+          }
+        }
+
+        try (ResultSet rs = statement.executeQuery()) {
+          return parser.parse(rs);
+        }
+      }
+    }
+  }
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Database identifiers,
+   * containing "name" (representing databaseName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> DATABASE_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> databases = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("name");
+          databases.add(SnowflakeIdentifier.ofDatabase(databaseName));
+        }
+        return databases;
+      };
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Schema identifiers,
+   * containing "database_name" and "name" (representing schemaName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> SCHEMA_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> schemas = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("database_name");
+          String schemaName = rs.getString("name");
+          schemas.add(SnowflakeIdentifier.ofSchema(databaseName, schemaName));
+        }
+        return schemas;
+      };
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Table identifiers,
+   * containing "database_name", "schema_name", and "name" (representing tableName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> TABLE_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> tables = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("database_name");
+          String schemaName = rs.getString("schema_name");
+          String tableName = rs.getString("name");
+          tables.add(SnowflakeIdentifier.ofTable(databaseName, schemaName, tableName));
+        }
+        return tables;
+      };
+
+  /**
+   * Expects to handle ResultSets representing a single record holding Snowflake Iceberg metadata.
+   */
+  public static final ResultSetParser<SnowflakeTableMetadata> TABLE_METADATA_RESULT_SET_HANDLER =
+      rs -> {
+        if (!rs.next()) {
+          return null;
+        }
+
+        String rawJsonVal = rs.getString("METADATA");
+        return SnowflakeTableMetadata.parseJson(rawJsonVal);
+      };
+
+  private final JdbcClientPool connectionPool;
+  private QueryHarness queryHarness;
+
+  JdbcSnowflakeClient(JdbcClientPool conn) {
+    Preconditions.checkArgument(null != conn, "JdbcClientPool must be non-null");
+    connectionPool = conn;
+    queryHarness = new QueryHarness();
+  }
+
+  @VisibleForTesting
+  void setQueryHarness(QueryHarness queryHarness) {
+    this.queryHarness = queryHarness;
+  }
+
+  /**
+   * For rare cases where PreparedStatements aren't supported for user-supplied identifiers intended
+   * for use in special LIKE clauses, we can sanitize by "broadening" the identifier with
+   * single-character wildcards and manually post-filter client-side.
+   *
+   * <p>Note: This sanitization approach intentionally "broadens" the scope of matching results;
+   * callers must be able to handle this method returning an all-wildcard expression; i.e. the
+   * caller must treat the usage of the LIKE clause as only an optional optimization, and should
+   * post-filter for correctness as if the LIKE clause wasn't present in the query at all.
+   */
+  @VisibleForTesting
+  String sanitizeIdentifierWithWildcardForLikeClause(String identifier) {
+    // Restrict identifiers to the "Unquoted object identifiers" synax documented at
+    // https://docs.snowflake.com/en/sql-reference/identifiers-syntax.html
+    //
+    // Use a strict allowlist of characters, replace everything *not* matching the character set
+    // with "_", which is used as a single-character wildcard in Snowflake.
+    String sanitized = identifier.replaceAll("[^a-zA-Z0-9_$]", "_");
+    if (sanitized.startsWith("$")) {
+      sanitized = "_" + sanitized.substring(1);
+    }
+    return sanitized;
+  }
+
+  @Override
+  public boolean databaseExists(SnowflakeIdentifier database) {
+    Preconditions.checkArgument(
+        database.type() == SnowflakeIdentifier.Type.DATABASE,
+        "databaseExists requires a DATABASE identifier, got '%s'",
+        database);
+
+    // Due to current limitations in PreparedStatement parameters for the LIKE clause in
+    // SHOW DATABASES queries, we'll use a fairly limited allowlist for identifier characters,
+    // using wildcards for non-allowed characters, and post-filter for matching.
+    final String finalQuery =
+        String.format(
+            "SHOW DATABASES LIKE '%s' IN ACCOUNT",
+            sanitizeIdentifierWithWildcardForLikeClause(database.databaseName()));
+    List<SnowflakeIdentifier> databases;
+    try {
+      databases =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn, finalQuery, DATABASE_RESULT_SET_HANDLER, (String[]) null));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to check if database '%s' exists", database);
+    } catch (InterruptedException e) {
+      throw new UncheckedInterruptedException(
+          e, "Interrupted while checking if database '%s' exists", database);
+    }
+
+    // Filter to handle the edge case of '_' appearing as a wildcard that can't be remapped the way
+    // it can for predicates in SELECT statements.
+    databases.removeIf(db -> !database.databaseName().equalsIgnoreCase(db.databaseName()));
+    return !databases.isEmpty();
+  }
+
+  @Override
+  public boolean schemaExists(SnowflakeIdentifier schema) {
+    Preconditions.checkArgument(
+        schema.type() == SnowflakeIdentifier.Type.SCHEMA,
+        "schemaExists requires a SCHEMA identifier, got '%s'",
+        schema);
+
+    if (!databaseExists(SnowflakeIdentifier.ofDatabase(schema.databaseName()))) {
+      return false;
+    }
+
+    // Due to current limitations in PreparedStatement parameters for the LIKE clause in
+    // SHOW SCHEMAS queries, we'll use a fairly limited allowlist for identifier characters,
+    // using wildcards for non-allowed characters, and post-filter for matching.
+    final String finalQuery =
+        String.format(
+            "SHOW SCHEMAS LIKE '%s' IN DATABASE IDENTIFIER(?)",
+            sanitizeIdentifierWithWildcardForLikeClause(schema.schemaName()));
+    List<SnowflakeIdentifier> schemas;
+    try {
+      schemas =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn,
+                      finalQuery,
+                      SCHEMA_RESULT_SET_HANDLER,
+                      new String[] {schema.databaseName()}));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to check if schema '%s' exists", schema);
+    } catch (InterruptedException e) {
+      throw new UncheckedInterruptedException(
+          e, "Interrupted while checking if schema '%s' exists", schema);
+    }
+
+    // Filter to handle the edge case of '_' appearing as a wildcard that can't be remapped the way
+    // it can for predicates in SELECT statements.
+    schemas.removeIf(sc -> !schema.schemaName().equalsIgnoreCase(sc.schemaName()));
+    return !schemas.isEmpty();
+  }
+
+  @Override
+  public List<SnowflakeIdentifier> listDatabases() {
+    List<SnowflakeIdentifier> databases;
+    try {
+      databases =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn,
+                      "SHOW DATABASES IN ACCOUNT",
+                      DATABASE_RESULT_SET_HANDLER,
+                      (String[]) null));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to list databases");
+    } catch (InterruptedException e) {
+      throw new UncheckedInterruptedException(e, "Interrupted while listing databases");
+    }
+    databases.forEach(
+        db ->
+            Preconditions.checkState(
+                db.type() == SnowflakeIdentifier.Type.DATABASE,
+                "Expected DATABASE, got identifier '%s'",
+                db));
+    return databases;
+  }
+
+  @Override
+  public List<SnowflakeIdentifier> listSchemas(SnowflakeIdentifier scope) {
+    StringBuilder baseQuery = new StringBuilder("SHOW SCHEMAS");
+    String[] queryParams = null;
+    switch (scope.type()) {
+      case ROOT:
+        // account-level listing
+        baseQuery.append(" IN ACCOUNT");
+        break;
+      case DATABASE:
+        // database-level listing
+        baseQuery.append(" IN DATABASE IDENTIFIER(?)");
+        queryParams = new String[] {scope.toIdentifierString()};
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format("Unsupported scope type for listSchemas: %s", scope));
+    }
+
+    final String finalQuery = baseQuery.toString();
+    final String[] finalQueryParams = queryParams;
+    List<SnowflakeIdentifier> schemas;
+    try {
+      schemas =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn, finalQuery, SCHEMA_RESULT_SET_HANDLER, finalQueryParams));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to list schemas for scope '%s'", scope);
+    } catch (InterruptedException e) {
+      throw new UncheckedInterruptedException(
+          e, "Interrupted while listing schemas for scope '%s'", scope);
+    }
+    schemas.forEach(
+        schema ->
+            Preconditions.checkState(
+                schema.type() == SnowflakeIdentifier.Type.SCHEMA,
+                "Expected SCHEMA, got identifier '%s' for scope '%s'",
+                schema,
+                scope));
+    return schemas;
+  }
+
+  @Override
+  public List<SnowflakeIdentifier> listIcebergTables(SnowflakeIdentifier scope) {
+    StringBuilder baseQuery = new StringBuilder("SHOW ICEBERG TABLES");

Review Comment:
   @dennishuo I tried running this catalog implementation with Spark and this statement doesn't appear to show external iceberg tables for me.  I can run:
   
   ```
   SHOW ICEBERG TABLES IN SCHEMA <DB>.<SCHEMA>;
   ```
   
   And there are now results, but running:
   
   ```
   SHOW TABLES IN SCHEMA <DB>.<SCHEMA>;
   ```
   
   Correctly shows the Iceberg tables and I can see their storage and metadata locations.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] sfc-gh-mparmar commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
sfc-gh-mparmar commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1050236173


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

Review Comment:
   For the first phase, we are supporting a small subset (read-only) of capabilities  for the catalog which doesn't include the ability to create/modify namespaces, tables etc. so we can't repurpose CatalogTests in this case yet.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051230895


##########
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:
   Done. Much cleaner, thanks for the suggestion!



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051244552


##########
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:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051274675


##########
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:
   Yeah, this is a bit awkward at the moment just because we ideally wanted to create a clean `SnowflakeClient` abstraction layer that can be well-encapsulated from `SnowflakeCatalog` and Iceberg-specific business logic, but indeed the processing of Namespace levels unfortunately leaks into both.
   
   I'll see if refactoring out a SnowflakeIdentifier can move all Namespace-processing stuff into one place and hopefully also make this container class unnecessary.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#issuecomment-1356445009

   Interesting, not sure why it removed @danielcweeks when I re-requested review from @nastra (definitely wasn't intentional -- as far as I can tell my repository permissions don't even provide the ability to remove reviewer requests). I wonder if this is possibly another manifestation of https://github.com/community/community/discussions/8939


-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1070158086


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  private static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  private static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.io.ResolvingFileIO";
+
+  static class FileIOFactory {
+    public FileIO newFileIO(String impl, Map<String, String> properties, Object hadoopConf) {
+      return CatalogUtil.loadFileIO(impl, properties, hadoopConf);

Review Comment:
   This factory seems odd since it has no state that is passed through to the `loadFileIO` method. Couldn't this just call `loadFileIO` directly?



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1063858652


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java:
##########
@@ -0,0 +1,386 @@
+/*
+ * 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.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+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.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+/**
+ * This implementation of SnowflakeClient builds on top of Snowflake's JDBC driver to interact with
+ * Snowflake's Iceberg-aware resource model.
+ */
+class JdbcSnowflakeClient implements SnowflakeClient {
+  static final String EXPECTED_JDBC_IMPL = "net.snowflake.client.jdbc.SnowflakeDriver";
+
+  @FunctionalInterface
+  interface ResultSetParser<T> {
+    T parse(ResultSet rs) throws SQLException;
+  }
+
+  /**
+   * This class wraps the basic boilerplate of setting up PreparedStatements and applying a
+   * ResultSetParser to translate a ResultSet into parsed objects. Allows easily injecting
+   * subclasses for debugging/testing purposes.
+   */
+  static class QueryHarness {
+    public <T> T query(Connection conn, String sql, ResultSetParser<T> parser, String... args)
+        throws SQLException {
+      try (PreparedStatement statement = conn.prepareStatement(sql)) {
+        if (args != null) {
+          for (int i = 0; i < args.length; ++i) {
+            statement.setString(i + 1, args[i]);
+          }
+        }
+
+        try (ResultSet rs = statement.executeQuery()) {
+          return parser.parse(rs);
+        }
+      }
+    }
+  }
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Database identifiers,
+   * containing "name" (representing databaseName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> DATABASE_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> databases = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("name");
+          databases.add(SnowflakeIdentifier.ofDatabase(databaseName));
+        }
+        return databases;
+      };
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Schema identifiers,
+   * containing "database_name" and "name" (representing schemaName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> SCHEMA_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> schemas = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("database_name");
+          String schemaName = rs.getString("name");
+          schemas.add(SnowflakeIdentifier.ofSchema(databaseName, schemaName));
+        }
+        return schemas;
+      };
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Table identifiers,
+   * containing "database_name", "schema_name", and "name" (representing tableName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> TABLE_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> tables = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("database_name");
+          String schemaName = rs.getString("schema_name");
+          String tableName = rs.getString("name");
+          tables.add(SnowflakeIdentifier.ofTable(databaseName, schemaName, tableName));
+        }
+        return tables;
+      };
+
+  /**
+   * Expects to handle ResultSets representing a single record holding Snowflake Iceberg metadata.
+   */
+  public static final ResultSetParser<SnowflakeTableMetadata> TABLE_METADATA_RESULT_SET_HANDLER =
+      rs -> {
+        if (!rs.next()) {
+          return null;
+        }
+
+        String rawJsonVal = rs.getString("METADATA");
+        return SnowflakeTableMetadata.parseJson(rawJsonVal);
+      };
+
+  private final JdbcClientPool connectionPool;
+  private QueryHarness queryHarness;
+
+  JdbcSnowflakeClient(JdbcClientPool conn) {
+    Preconditions.checkArgument(null != conn, "JdbcClientPool must be non-null");
+    connectionPool = conn;
+    queryHarness = new QueryHarness();
+  }
+
+  @VisibleForTesting
+  void setQueryHarness(QueryHarness queryHarness) {
+    this.queryHarness = queryHarness;
+  }
+
+  /**
+   * For rare cases where PreparedStatements aren't supported for user-supplied identifiers intended
+   * for use in special LIKE clauses, we can sanitize by "broadening" the identifier with
+   * single-character wildcards and manually post-filter client-side.
+   *
+   * <p>Note: This sanitization approach intentionally "broadens" the scope of matching results;
+   * callers must be able to handle this method returning an all-wildcard expression; i.e. the
+   * caller must treat the usage of the LIKE clause as only an optional optimization, and should
+   * post-filter for correctness as if the LIKE clause wasn't present in the query at all.
+   */
+  @VisibleForTesting
+  String sanitizeIdentifierWithWildcardForLikeClause(String identifier) {
+    // Restrict identifiers to the "Unquoted object identifiers" synax documented at
+    // https://docs.snowflake.com/en/sql-reference/identifiers-syntax.html
+    //
+    // Use a strict allowlist of characters, replace everything *not* matching the character set
+    // with "_", which is used as a single-character wildcard in Snowflake.
+    String sanitized = identifier.replaceAll("[^a-zA-Z0-9_$]", "_");
+    if (sanitized.startsWith("$")) {
+      sanitized = "_" + sanitized.substring(1);
+    }
+    return sanitized;
+  }
+
+  @Override
+  public boolean databaseExists(SnowflakeIdentifier database) {
+    Preconditions.checkArgument(
+        database.type() == SnowflakeIdentifier.Type.DATABASE,
+        "databaseExists requires a DATABASE identifier, got '%s'",
+        database);
+
+    // Due to current limitations in PreparedStatement parameters for the LIKE clause in
+    // SHOW DATABASES queries, we'll use a fairly limited allowlist for identifier characters,
+    // using wildcards for non-allowed characters, and post-filter for matching.
+    final String finalQuery =
+        String.format(
+            "SHOW DATABASES LIKE '%s' IN ACCOUNT",
+            sanitizeIdentifierWithWildcardForLikeClause(database.databaseName()));
+    List<SnowflakeIdentifier> databases;
+    try {
+      databases =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn, finalQuery, DATABASE_RESULT_SET_HANDLER, (String[]) null));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to check if database '%s' exists", database);
+    } catch (InterruptedException e) {
+      throw new UncheckedInterruptedException(
+          e, "Interrupted while checking if database '%s' exists", database);
+    }
+
+    // Filter to handle the edge case of '_' appearing as a wildcard that can't be remapped the way
+    // it can for predicates in SELECT statements.
+    databases.removeIf(db -> !database.databaseName().equalsIgnoreCase(db.databaseName()));
+    return !databases.isEmpty();
+  }
+
+  @Override
+  public boolean schemaExists(SnowflakeIdentifier schema) {
+    Preconditions.checkArgument(
+        schema.type() == SnowflakeIdentifier.Type.SCHEMA,
+        "schemaExists requires a SCHEMA identifier, got '%s'",
+        schema);
+
+    if (!databaseExists(SnowflakeIdentifier.ofDatabase(schema.databaseName()))) {
+      return false;
+    }
+
+    // Due to current limitations in PreparedStatement parameters for the LIKE clause in
+    // SHOW SCHEMAS queries, we'll use a fairly limited allowlist for identifier characters,
+    // using wildcards for non-allowed characters, and post-filter for matching.
+    final String finalQuery =
+        String.format(
+            "SHOW SCHEMAS LIKE '%s' IN DATABASE IDENTIFIER(?)",
+            sanitizeIdentifierWithWildcardForLikeClause(schema.schemaName()));
+    List<SnowflakeIdentifier> schemas;
+    try {
+      schemas =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn,
+                      finalQuery,
+                      SCHEMA_RESULT_SET_HANDLER,
+                      new String[] {schema.databaseName()}));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to check if schema '%s' exists", schema);
+    } catch (InterruptedException e) {
+      throw new UncheckedInterruptedException(
+          e, "Interrupted while checking if schema '%s' exists", schema);
+    }
+
+    // Filter to handle the edge case of '_' appearing as a wildcard that can't be remapped the way
+    // it can for predicates in SELECT statements.
+    schemas.removeIf(sc -> !schema.schemaName().equalsIgnoreCase(sc.schemaName()));
+    return !schemas.isEmpty();
+  }
+
+  @Override
+  public List<SnowflakeIdentifier> listDatabases() {
+    List<SnowflakeIdentifier> databases;
+    try {
+      databases =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn,
+                      "SHOW DATABASES IN ACCOUNT",
+                      DATABASE_RESULT_SET_HANDLER,
+                      (String[]) null));

Review Comment:
   This is vararg, so you can just leave the param off rather than casting a null.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1065123282


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java:
##########
@@ -0,0 +1,386 @@
+/*
+ * 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.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+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.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+/**
+ * This implementation of SnowflakeClient builds on top of Snowflake's JDBC driver to interact with
+ * Snowflake's Iceberg-aware resource model.
+ */
+class JdbcSnowflakeClient implements SnowflakeClient {
+  static final String EXPECTED_JDBC_IMPL = "net.snowflake.client.jdbc.SnowflakeDriver";
+
+  @FunctionalInterface
+  interface ResultSetParser<T> {
+    T parse(ResultSet rs) throws SQLException;
+  }
+
+  /**
+   * This class wraps the basic boilerplate of setting up PreparedStatements and applying a
+   * ResultSetParser to translate a ResultSet into parsed objects. Allows easily injecting
+   * subclasses for debugging/testing purposes.
+   */
+  static class QueryHarness {
+    public <T> T query(Connection conn, String sql, ResultSetParser<T> parser, String... args)
+        throws SQLException {
+      try (PreparedStatement statement = conn.prepareStatement(sql)) {
+        if (args != null) {
+          for (int i = 0; i < args.length; ++i) {
+            statement.setString(i + 1, args[i]);
+          }
+        }
+
+        try (ResultSet rs = statement.executeQuery()) {
+          return parser.parse(rs);
+        }
+      }
+    }
+  }
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Database identifiers,
+   * containing "name" (representing databaseName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> DATABASE_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> databases = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("name");
+          databases.add(SnowflakeIdentifier.ofDatabase(databaseName));
+        }
+        return databases;
+      };
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Schema identifiers,
+   * containing "database_name" and "name" (representing schemaName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> SCHEMA_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> schemas = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("database_name");
+          String schemaName = rs.getString("name");
+          schemas.add(SnowflakeIdentifier.ofSchema(databaseName, schemaName));
+        }
+        return schemas;
+      };
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Table identifiers,
+   * containing "database_name", "schema_name", and "name" (representing tableName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> TABLE_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> tables = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("database_name");
+          String schemaName = rs.getString("schema_name");
+          String tableName = rs.getString("name");
+          tables.add(SnowflakeIdentifier.ofTable(databaseName, schemaName, tableName));
+        }
+        return tables;
+      };
+
+  /**
+   * Expects to handle ResultSets representing a single record holding Snowflake Iceberg metadata.
+   */
+  public static final ResultSetParser<SnowflakeTableMetadata> TABLE_METADATA_RESULT_SET_HANDLER =
+      rs -> {
+        if (!rs.next()) {
+          return null;
+        }
+
+        String rawJsonVal = rs.getString("METADATA");
+        return SnowflakeTableMetadata.parseJson(rawJsonVal);
+      };
+
+  private final JdbcClientPool connectionPool;
+  private QueryHarness queryHarness;
+
+  JdbcSnowflakeClient(JdbcClientPool conn) {
+    Preconditions.checkArgument(null != conn, "JdbcClientPool must be non-null");
+    connectionPool = conn;
+    queryHarness = new QueryHarness();
+  }
+
+  @VisibleForTesting
+  void setQueryHarness(QueryHarness queryHarness) {
+    this.queryHarness = queryHarness;
+  }
+
+  /**
+   * For rare cases where PreparedStatements aren't supported for user-supplied identifiers intended
+   * for use in special LIKE clauses, we can sanitize by "broadening" the identifier with
+   * single-character wildcards and manually post-filter client-side.
+   *
+   * <p>Note: This sanitization approach intentionally "broadens" the scope of matching results;
+   * callers must be able to handle this method returning an all-wildcard expression; i.e. the
+   * caller must treat the usage of the LIKE clause as only an optional optimization, and should
+   * post-filter for correctness as if the LIKE clause wasn't present in the query at all.
+   */
+  @VisibleForTesting
+  String sanitizeIdentifierWithWildcardForLikeClause(String identifier) {
+    // Restrict identifiers to the "Unquoted object identifiers" synax documented at
+    // https://docs.snowflake.com/en/sql-reference/identifiers-syntax.html
+    //
+    // Use a strict allowlist of characters, replace everything *not* matching the character set
+    // with "_", which is used as a single-character wildcard in Snowflake.
+    String sanitized = identifier.replaceAll("[^a-zA-Z0-9_$]", "_");
+    if (sanitized.startsWith("$")) {
+      sanitized = "_" + sanitized.substring(1);
+    }
+    return sanitized;
+  }
+
+  @Override
+  public boolean databaseExists(SnowflakeIdentifier database) {
+    Preconditions.checkArgument(
+        database.type() == SnowflakeIdentifier.Type.DATABASE,
+        "databaseExists requires a DATABASE identifier, got '%s'",
+        database);
+
+    // Due to current limitations in PreparedStatement parameters for the LIKE clause in
+    // SHOW DATABASES queries, we'll use a fairly limited allowlist for identifier characters,
+    // using wildcards for non-allowed characters, and post-filter for matching.
+    final String finalQuery =
+        String.format(
+            "SHOW DATABASES LIKE '%s' IN ACCOUNT",
+            sanitizeIdentifierWithWildcardForLikeClause(database.databaseName()));
+    List<SnowflakeIdentifier> databases;
+    try {
+      databases =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn, finalQuery, DATABASE_RESULT_SET_HANDLER, (String[]) null));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to check if database '%s' exists", database);
+    } catch (InterruptedException e) {
+      throw new UncheckedInterruptedException(
+          e, "Interrupted while checking if database '%s' exists", database);
+    }
+
+    // Filter to handle the edge case of '_' appearing as a wildcard that can't be remapped the way
+    // it can for predicates in SELECT statements.
+    databases.removeIf(db -> !database.databaseName().equalsIgnoreCase(db.databaseName()));
+    return !databases.isEmpty();
+  }
+
+  @Override
+  public boolean schemaExists(SnowflakeIdentifier schema) {
+    Preconditions.checkArgument(
+        schema.type() == SnowflakeIdentifier.Type.SCHEMA,
+        "schemaExists requires a SCHEMA identifier, got '%s'",
+        schema);
+
+    if (!databaseExists(SnowflakeIdentifier.ofDatabase(schema.databaseName()))) {
+      return false;
+    }
+
+    // Due to current limitations in PreparedStatement parameters for the LIKE clause in
+    // SHOW SCHEMAS queries, we'll use a fairly limited allowlist for identifier characters,
+    // using wildcards for non-allowed characters, and post-filter for matching.
+    final String finalQuery =
+        String.format(
+            "SHOW SCHEMAS LIKE '%s' IN DATABASE IDENTIFIER(?)",
+            sanitizeIdentifierWithWildcardForLikeClause(schema.schemaName()));
+    List<SnowflakeIdentifier> schemas;
+    try {
+      schemas =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn,
+                      finalQuery,
+                      SCHEMA_RESULT_SET_HANDLER,
+                      new String[] {schema.databaseName()}));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to check if schema '%s' exists", schema);
+    } catch (InterruptedException e) {
+      throw new UncheckedInterruptedException(
+          e, "Interrupted while checking if schema '%s' exists", schema);
+    }
+
+    // Filter to handle the edge case of '_' appearing as a wildcard that can't be remapped the way
+    // it can for predicates in SELECT statements.
+    schemas.removeIf(sc -> !schema.schemaName().equalsIgnoreCase(sc.schemaName()));
+    return !schemas.isEmpty();
+  }
+
+  @Override
+  public List<SnowflakeIdentifier> listDatabases() {
+    List<SnowflakeIdentifier> databases;
+    try {
+      databases =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn,
+                      "SHOW DATABASES IN ACCOUNT",
+                      DATABASE_RESULT_SET_HANDLER,
+                      (String[]) null));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to list databases");
+    } catch (InterruptedException e) {
+      throw new UncheckedInterruptedException(e, "Interrupted while listing databases");
+    }
+    databases.forEach(
+        db ->
+            Preconditions.checkState(
+                db.type() == SnowflakeIdentifier.Type.DATABASE,
+                "Expected DATABASE, got identifier '%s'",
+                db));
+    return databases;
+  }
+
+  @Override
+  public List<SnowflakeIdentifier> listSchemas(SnowflakeIdentifier scope) {
+    StringBuilder baseQuery = new StringBuilder("SHOW SCHEMAS");
+    String[] queryParams = null;
+    switch (scope.type()) {
+      case ROOT:
+        // account-level listing
+        baseQuery.append(" IN ACCOUNT");
+        break;
+      case DATABASE:
+        // database-level listing
+        baseQuery.append(" IN DATABASE IDENTIFIER(?)");
+        queryParams = new String[] {scope.toIdentifierString()};
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format("Unsupported scope type for listSchemas: %s", scope));
+    }
+
+    final String finalQuery = baseQuery.toString();
+    final String[] finalQueryParams = queryParams;
+    List<SnowflakeIdentifier> schemas;
+    try {
+      schemas =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn, finalQuery, SCHEMA_RESULT_SET_HANDLER, finalQueryParams));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to list schemas for scope '%s'", scope);
+    } catch (InterruptedException e) {
+      throw new UncheckedInterruptedException(
+          e, "Interrupted while listing schemas for scope '%s'", scope);
+    }
+    schemas.forEach(
+        schema ->
+            Preconditions.checkState(
+                schema.type() == SnowflakeIdentifier.Type.SCHEMA,
+                "Expected SCHEMA, got identifier '%s' for scope '%s'",
+                schema,
+                scope));
+    return schemas;
+  }
+
+  @Override
+  public List<SnowflakeIdentifier> listIcebergTables(SnowflakeIdentifier scope) {
+    StringBuilder baseQuery = new StringBuilder("SHOW ICEBERG TABLES");

Review Comment:
   I think this is ok for now and we can revisit once external tables goes GA.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#issuecomment-1376496316

   Thanks for the note! Successfully merged main. I'll await @nastra 's review


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051231396


##########
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://mycontainer@myaccount.blob.core.windows.net/tab3/metadata/v334.metadata.json",
+        TableMetadataParser.toJson(
+                TableMetadata.newTableMetadata(
+                    schema,
+                    partitionSpec,
+                    "wasbs://mycontainer@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:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#issuecomment-1356015002

   @nastra Thanks for the thorough review and suggestions! Finished applying all your suggestions, including fully converting to `assertj`/`Assertions` and refactoring out the Namespace<->SnowflakeIdentifier parsing to better encapsulate all the argument-checking/parsing into one place.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051679882


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/entities/SnowflakeIdentifier.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class SnowflakeIdentifier {
+  public enum Type {
+    ROOT,
+    DATABASE,
+    SCHEMA,
+    TABLE
+  }
+
+  private String databaseName;
+  private String schemaName;
+  private String tableName;
+
+  protected SnowflakeIdentifier(String databaseName, String schemaName, String tableName) {
+    this.databaseName = databaseName;
+    this.schemaName = schemaName;
+    this.tableName = tableName;
+  }
+
+  public static SnowflakeIdentifier ofRoot() {
+    return new SnowflakeIdentifier(null, null, null);
+  }
+
+  public static SnowflakeIdentifier ofDatabase(String databaseName) {
+    Preconditions.checkArgument(null != databaseName, "databaseName must be non-null");
+    return new SnowflakeIdentifier(databaseName, null, null);
+  }
+
+  public static SnowflakeIdentifier ofSchema(String databaseName, String schemaName) {
+    Preconditions.checkArgument(null != databaseName, "databaseName must be non-null");
+    Preconditions.checkArgument(null != schemaName, "schemaName must be non-null");
+    return new SnowflakeIdentifier(databaseName, schemaName, null);
+  }
+
+  public static SnowflakeIdentifier ofTable(
+      String databaseName, String schemaName, String tableName) {
+    Preconditions.checkArgument(null != databaseName, "databaseName must be non-null");
+    Preconditions.checkArgument(null != schemaName, "schemaName must be non-null");
+    Preconditions.checkArgument(null != tableName, "tableName must be non-null");
+    return new SnowflakeIdentifier(databaseName, schemaName, tableName);
+  }
+
+  /**
+   * If type is TABLE, expect non-null databaseName, schemaName, and tableName. If type is SCHEMA,
+   * expect non-null databaseName and schemaName. If type is DATABASE, expect non-null databaseName.
+   * If type is ROOT, expect all of databaseName, schemaName, and tableName to be null.
+   */
+  public Type getType() {
+    if (null != tableName) {
+      return Type.TABLE;
+    } else if (null != schemaName) {
+      return Type.SCHEMA;
+    } else if (null != databaseName) {
+      return Type.DATABASE;
+    } else {
+      return Type.ROOT;
+    }
+  }
+
+  public String getTableName() {

Review Comment:
   Iceberg does not use `get` in method names. Either it isn't helpful and can be omitted, or it should be replaced with a more specific verb. It's arguable that no one will touch this code besides Snowflake so it is okay, but I'd still recommend conforming to the project style.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051680116


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/NamespaceHelpers.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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 org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class NamespaceHelpers {
+  private static final int MAX_NAMESPACE_DEPTH = 2;
+  private static final int NAMESPACE_ROOT_LEVEL = 0;
+  private static final int NAMESPACE_DB_LEVEL = 1;
+  private static final int NAMESPACE_SCHEMA_LEVEL = 2;
+
+  private static final Logger LOG = LoggerFactory.getLogger(NamespaceHelpers.class);
+
+  /**
+   * Converts a Namespace into a SnowflakeIdentifier representing ROOT, a DATABASE, or a SCHEMA.
+   *
+   * @throws IllegalArgumentException if the namespace is not a supported depth.
+   */
+  public static SnowflakeIdentifier getSnowflakeIdentifierForNamespace(Namespace namespace) {

Review Comment:
   Style: `get` in method name.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051658860


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.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.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+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);

Review Comment:
   `LOG` isn't used



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051682764


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeTableOperations.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.Map;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.apache.iceberg.snowflake.entities.SnowflakeTableMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class SnowflakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeTableOperations.class);
+  private final String catalogName;
+
+  private final FileIO fileIO;
+  private final TableIdentifier tableIdentifier;
+  private final SnowflakeIdentifier snowflakeIdentifierForTable;
+
+  private final SnowflakeClient snowflakeClient;
+
+  private final Map<String, String> catalogProperties;
+
+  protected SnowflakeTableOperations(
+      SnowflakeClient snowflakeClient,
+      FileIO fileIO,
+      Map<String, String> properties,
+      String catalogName,
+      TableIdentifier tableIdentifier) {
+    this.snowflakeClient = snowflakeClient;
+    this.fileIO = fileIO;
+    this.catalogProperties = properties;
+    this.catalogName = catalogName;
+    this.tableIdentifier = tableIdentifier;
+    this.snowflakeIdentifierForTable =
+        NamespaceHelpers.getSnowflakeIdentifierForTableIdentifier(tableIdentifier);
+  }
+
+  @Override
+  public void doRefresh() {
+    LOG.debug("Getting metadata location for table {}", tableIdentifier);
+    String location = getTableMetadataLocation();
+    Preconditions.checkState(
+        location != null && !location.isEmpty(),
+        "Got null or empty location %s for table %s",
+        location,
+        tableIdentifier);
+    refreshFromMetadataLocation(location);
+  }
+
+  @Override
+  public FileIO io() {
+    return fileIO;
+  }
+
+  @Override
+  protected String tableName() {
+    return tableIdentifier.toString();

Review Comment:
   This should include the catalog name, but the Iceberg identifier won't have it.



##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeTableOperations.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.Map;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.apache.iceberg.snowflake.entities.SnowflakeTableMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class SnowflakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeTableOperations.class);
+  private final String catalogName;
+
+  private final FileIO fileIO;
+  private final TableIdentifier tableIdentifier;
+  private final SnowflakeIdentifier snowflakeIdentifierForTable;
+
+  private final SnowflakeClient snowflakeClient;
+
+  private final Map<String, String> catalogProperties;

Review Comment:
   I think this is needed by `tableName`.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051681027


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/entities/SnowflakeIdentifier.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class SnowflakeIdentifier {
+  public enum Type {
+    ROOT,
+    DATABASE,
+    SCHEMA,
+    TABLE
+  }
+
+  private String databaseName;
+  private String schemaName;
+  private String tableName;
+
+  protected SnowflakeIdentifier(String databaseName, String schemaName, String tableName) {
+    this.databaseName = databaseName;
+    this.schemaName = schemaName;
+    this.tableName = tableName;
+  }
+
+  public static SnowflakeIdentifier ofRoot() {
+    return new SnowflakeIdentifier(null, null, null);
+  }
+
+  public static SnowflakeIdentifier ofDatabase(String databaseName) {
+    Preconditions.checkArgument(null != databaseName, "databaseName must be non-null");
+    return new SnowflakeIdentifier(databaseName, null, null);
+  }
+
+  public static SnowflakeIdentifier ofSchema(String databaseName, String schemaName) {
+    Preconditions.checkArgument(null != databaseName, "databaseName must be non-null");
+    Preconditions.checkArgument(null != schemaName, "schemaName must be non-null");
+    return new SnowflakeIdentifier(databaseName, schemaName, null);
+  }
+
+  public static SnowflakeIdentifier ofTable(
+      String databaseName, String schemaName, String tableName) {
+    Preconditions.checkArgument(null != databaseName, "databaseName must be non-null");
+    Preconditions.checkArgument(null != schemaName, "schemaName must be non-null");
+    Preconditions.checkArgument(null != tableName, "tableName must be non-null");
+    return new SnowflakeIdentifier(databaseName, schemaName, tableName);
+  }
+
+  /**
+   * If type is TABLE, expect non-null databaseName, schemaName, and tableName. If type is SCHEMA,
+   * expect non-null databaseName and schemaName. If type is DATABASE, expect non-null databaseName.
+   * If type is ROOT, expect all of databaseName, schemaName, and tableName to be null.
+   */
+  public Type getType() {
+    if (null != tableName) {
+      return Type.TABLE;
+    } else if (null != schemaName) {
+      return Type.SCHEMA;
+    } else if (null != databaseName) {
+      return Type.DATABASE;
+    } else {
+      return Type.ROOT;
+    }
+  }
+
+  public String getTableName() {
+    return tableName;
+  }
+
+  public String getDatabaseName() {
+    return databaseName;
+  }
+
+  public String getSchemaName() {
+    return schemaName;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    } else if (!(o instanceof SnowflakeIdentifier)) {
+      return false;
+    }
+
+    SnowflakeIdentifier that = (SnowflakeIdentifier) o;
+    return Objects.equal(this.databaseName, that.databaseName)
+        && Objects.equal(this.schemaName, that.schemaName)
+        && Objects.equal(this.tableName, that.tableName);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(databaseName, schemaName, tableName);
+  }
+
+  /** Returns this identifier as a String suitable for use in a Snowflake IDENTIFIER param. */
+  public String toIdentifierString() {
+    switch (getType()) {
+      case TABLE:
+        return String.format("%s.%s.%s", databaseName, schemaName, tableName);
+      case SCHEMA:
+        return String.format("%s.%s", databaseName, schemaName);
+      case DATABASE:
+        return databaseName;
+      default:
+        return "";
+    }
+  }
+
+  @Override
+  public String toString() {
+    return String.format("%s: '%s'", getType(), toIdentifierString());
+  }
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Schema identifiers,
+   * containing "database_name" and "name" (representing schemaName).
+   */
+  public static ResultSetHandler<List<SnowflakeIdentifier>> createSchemaHandler() {
+    return rs -> {

Review Comment:
   Can these handlers be static since they don't change based on context?



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051680729


##########
snowflake/src/test/java/org/apache/iceberg/snowflake/InMemoryFileIO.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.Map;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InMemoryInputFile;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class InMemoryFileIO implements FileIO {

Review Comment:
   Isn't there already an in-memory FileIO? If not, can we add this in a more generic location so others can use it?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1052895586


##########
snowflake/src/test/java/org/apache/iceberg/snowflake/InMemoryFileIO.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.Map;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InMemoryInputFile;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class InMemoryFileIO implements FileIO {

Review Comment:
   Not as far as I can tell - looks like InMemoryInputFile and InMemoryOutputFile were introduced in https://github.com/apache/iceberg/commit/ed54952b51a6cdc7d2350b8223c276421845a8e2 without an accompanying FileIO implementation. I went ahead and moved this into `core/src/test/java/org/apache/iceberg/io/`, expanded the functionality to behave as one would expected with both `InMemoryInputFile` and `InMemoryOutputFile`, and added its own little unittest.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1054040108


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.hadoop.HadoopFileIO";
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    LOG.debug("listTables with namespace: {}", namespace);
+    SnowflakeIdentifier scope = NamespaceHelpers.getSnowflakeIdentifierForNamespace(namespace);
+    Preconditions.checkArgument(
+        scope.getType() == SnowflakeIdentifier.Type.ROOT
+            || scope.getType() == SnowflakeIdentifier.Type.DATABASE
+            || scope.getType() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at ROOT, DATABASE, or SCHEMA level; got %s from namespace %s",
+        scope,
+        namespace);
+
+    List<SnowflakeIdentifier> sfTables = snowflakeClient.listIcebergTables(scope);
+
+    return sfTables.stream()
+        .map(
+            table ->
+                TableIdentifier.of(
+                    table.getDatabaseName(), table.getSchemaName(), table.getTableName()))
+        .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) {
+    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);
+
+    String fileIOImpl = DEFAULT_FILE_IO_IMPL;
+    if (properties.containsKey(CatalogProperties.FILE_IO_IMPL)) {
+      fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL);
+    }
+
+    initialize(
+        name,
+        new JdbcSnowflakeClient(connectionPool),
+        CatalogUtil.loadFileIO(fileIOImpl, properties, conf),
+        properties);
+  }
+
+  /**
+   * Initialize using caller-supplied SnowflakeClient and FileIO.
+   *
+   * @param name The name of the catalog, defaults to "snowflake_catalog"
+   * @param snowflakeClient The client encapsulating network communication with Snowflake
+   * @param fileIO The {@link FileIO} to use for table operations
+   * @param properties The catalog options to use and propagate to dependencies
+   */
+  @SuppressWarnings("checkstyle:HiddenField")
+  public void initialize(
+      String name, SnowflakeClient snowflakeClient, FileIO fileIO, Map<String, String> properties) {
+    Preconditions.checkArgument(null != snowflakeClient, "snowflakeClient must be non-null");
+    Preconditions.checkArgument(null != fileIO, "fileIO must be non-null");
+    this.catalogName = name == null ? DEFAULT_CATALOG_NAME : name;
+    this.snowflakeClient = snowflakeClient;
+    this.fileIO = fileIO;
+    this.catalogProperties = properties;
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(snowflakeClient);
+    closeableGroup.addCloseable(fileIO);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (null != closeableGroup) {
+      closeableGroup.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);
+    SnowflakeIdentifier scope = NamespaceHelpers.getSnowflakeIdentifierForNamespace(namespace);
+    Preconditions.checkArgument(
+        scope.getType() == SnowflakeIdentifier.Type.ROOT
+            || scope.getType() == SnowflakeIdentifier.Type.DATABASE,
+        "listNamespaces must be at either ROOT or DATABASE level; got %s from namespace %s",
+        scope,
+        namespace);
+    List<SnowflakeIdentifier> sfSchemas = snowflakeClient.listSchemas(scope);
+
+    List<Namespace> namespaceList =
+        sfSchemas.stream()
+            .map(
+                schema -> {
+                  Preconditions.checkState(
+                      schema.getType() == SnowflakeIdentifier.Type.SCHEMA,
+                      "Got identifier of type %s from listSchemas for %s",
+                      schema.getType(),
+                      namespace);
+                  return Namespace.of(schema.getDatabaseName(), schema.getSchemaName());
+                })
+            .collect(Collectors.toList());
+    return namespaceList;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace)
+      throws NoSuchNamespaceException {
+    LOG.debug("loadNamespaceMetadata with namespace: {}", namespace);
+    return ImmutableMap.of();

Review Comment:
   For now, no metadata that would make sense to expose as Iceberg namespace metadata. We'll likely expand this in the future once there's a plan in place for more comprehensive Iceberg-relevant metadata.
   
   Good catch on the default `SupportsNamespaces` impl relying on this for `namespaceExists` - I went ahead and added functionality to do existence checks for Database/Schema where appropriate, along with corresponding unittests.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1052707608


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.hadoop.HadoopFileIO";
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    LOG.debug("listTables with namespace: {}", namespace);
+    SnowflakeIdentifier scope = NamespaceHelpers.getSnowflakeIdentifierForNamespace(namespace);
+    Preconditions.checkArgument(
+        scope.getType() == SnowflakeIdentifier.Type.ROOT
+            || scope.getType() == SnowflakeIdentifier.Type.DATABASE
+            || scope.getType() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at ROOT, DATABASE, or SCHEMA level; got %s from namespace %s",
+        scope,
+        namespace);
+
+    List<SnowflakeIdentifier> sfTables = snowflakeClient.listIcebergTables(scope);
+
+    return sfTables.stream()
+        .map(
+            table ->
+                TableIdentifier.of(
+                    table.getDatabaseName(), table.getSchemaName(), table.getTableName()))
+        .collect(Collectors.toList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    throw new UnsupportedOperationException(
+        String.format("dropTable not supported; attempted for table '%s'", identifier));

Review Comment:
   Done.



##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.hadoop.HadoopFileIO";
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    LOG.debug("listTables with namespace: {}", namespace);
+    SnowflakeIdentifier scope = NamespaceHelpers.getSnowflakeIdentifierForNamespace(namespace);
+    Preconditions.checkArgument(
+        scope.getType() == SnowflakeIdentifier.Type.ROOT
+            || scope.getType() == SnowflakeIdentifier.Type.DATABASE
+            || scope.getType() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at ROOT, DATABASE, or SCHEMA level; got %s from namespace %s",
+        scope,
+        namespace);
+
+    List<SnowflakeIdentifier> sfTables = snowflakeClient.listIcebergTables(scope);
+
+    return sfTables.stream()
+        .map(
+            table ->
+                TableIdentifier.of(
+                    table.getDatabaseName(), table.getSchemaName(), table.getTableName()))
+        .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));

Review Comment:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1052898357


##########
snowflake/src/test/java/org/apache/iceberg/snowflake/SnowflakeCatalogTest.java:
##########
@@ -0,0 +1,277 @@
+/*
+ * 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.IOException;
+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.Maps;
+import org.apache.iceberg.snowflake.entities.SnowflakeTableMetadata;
+import org.apache.iceberg.types.Types;
+import org.assertj.core.api.Assertions;
+import org.junit.Before;
+import org.junit.Test;
+
+public class SnowflakeCatalogTest {
+
+  static final String TEST_CATALOG_NAME = "slushLog";
+  private SnowflakeCatalog catalog;
+  private FakeSnowflakeClient fakeClient;
+  private InMemoryFileIO fakeFileIO;
+  private Map<String, String> properties;
+
+  @Before
+  public void before() {
+    catalog = new SnowflakeCatalog();
+
+    fakeClient = new FakeSnowflakeClient();
+    fakeClient.addTable(
+        "DB_1",
+        "SCHEMA_1",
+        "TAB_1",
+        SnowflakeTableMetadata.parseJson(
+            "{\"metadataLocation\":\"s3://tab1/metadata/v3.metadata.json\",\"status\":\"success\"}"));
+    fakeClient.addTable(
+        "DB_1",
+        "SCHEMA_1",
+        "TAB_2",
+        SnowflakeTableMetadata.parseJson(
+            "{\"metadataLocation\":\"s3://tab2/metadata/v1.metadata.json\",\"status\":\"success\"}"));
+    fakeClient.addTable(
+        "DB_2",
+        "SCHEMA_2",
+        "TAB_3",
+        SnowflakeTableMetadata.parseJson(
+            "{\"metadataLocation\":\"azure://myaccount.blob.core.windows.net/mycontainer/tab3/metadata/v334.metadata.json\",\"status\":\"success\"}"));
+    fakeClient.addTable(
+        "DB_2",
+        "SCHEMA_2",
+        "TAB_4",
+        SnowflakeTableMetadata.parseJson(
+            "{\"metadataLocation\":\"azure://myaccount.blob.core.windows.net/mycontainer/tab4/metadata/v323.metadata.json\",\"status\":\"success\"}"));
+    fakeClient.addTable(
+        "DB_3",
+        "SCHEMA_3",
+        "TAB_5",
+        SnowflakeTableMetadata.parseJson(
+            "{\"metadataLocation\":\"gcs://tab5/metadata/v793.metadata.json\",\"status\":\"success\"}"));
+    fakeClient.addTable(
+        "DB_3",
+        "SCHEMA_4",
+        "TAB_6",
+        SnowflakeTableMetadata.parseJson(
+            "{\"metadataLocation\":\"gcs://tab6/metadata/v123.metadata.json\",\"status\":\"success\"}"));
+
+    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://mycontainer@myaccount.blob.core.windows.net/tab3/metadata/v334.metadata.json",
+        TableMetadataParser.toJson(
+                TableMetadata.newTableMetadata(
+                    schema,
+                    partitionSpec,
+                    "wasbs://mycontainer@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());
+
+    properties = Maps.newHashMap();
+    catalog.initialize(TEST_CATALOG_NAME, fakeClient, fakeFileIO, properties);
+  }
+
+  @Test
+  public void testInitializeNullClient() {
+    Assertions.assertThatExceptionOfType(IllegalArgumentException.class)
+        .isThrownBy(() -> catalog.initialize(TEST_CATALOG_NAME, null, fakeFileIO, properties))
+        .withMessageContaining("snowflakeClient must be non-null");
+  }
+
+  @Test
+  public void testInitializeNullFileIO() {
+    Assertions.assertThatExceptionOfType(IllegalArgumentException.class)
+        .isThrownBy(() -> catalog.initialize(TEST_CATALOG_NAME, fakeClient, null, properties))
+        .withMessageContaining("fileIO must be non-null");
+  }
+
+  @Test
+  public void testListNamespace() {
+    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"));
+  }
+
+  @Test
+  public void testListNamespaceWithinDB() {
+    String dbName = "DB_1";
+    Assertions.assertThat(catalog.listNamespaces(Namespace.of(dbName)))
+        .containsExactly(Namespace.of(dbName, "SCHEMA_1"));
+  }
+
+  @Test
+  public void testListNamespaceWithinNonExistentDB() {
+    // Existence check for nonexistent parent namespaces is optional in the SupportsNamespaces
+    // interface.
+    String dbName = "NONEXISTENT_DB";
+    Assertions.assertThatExceptionOfType(RuntimeException.class)
+        .isThrownBy(() -> catalog.listNamespaces(Namespace.of(dbName)))
+        .withMessageContaining("does not exist")
+        .withMessageContaining(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";
+    Assertions.assertThatExceptionOfType(IllegalArgumentException.class)
+        .isThrownBy(() -> catalog.listNamespaces(Namespace.of(dbName, schemaName)))
+        .withMessageContaining("level")
+        .withMessageContaining("DB_3.SCHEMA_4");
+  }
+
+  @Test
+  public void testListTables() {
+    Assertions.assertThat(catalog.listTables(Namespace.empty()))
+        .containsExactly(
+            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"));
+  }
+
+  @Test
+  public void testListTablesWithinDB() {
+    String dbName = "DB_1";
+    Assertions.assertThat(catalog.listTables(Namespace.of(dbName)))
+        .containsExactly(
+            TableIdentifier.of("DB_1", "SCHEMA_1", "TAB_1"),
+            TableIdentifier.of("DB_1", "SCHEMA_1", "TAB_2"));
+  }
+
+  @Test
+  public void testListTablesWithinNonexistentDB() {
+    String dbName = "NONEXISTENT_DB";
+    Assertions.assertThatExceptionOfType(RuntimeException.class)
+        .isThrownBy(() -> catalog.listTables(Namespace.of(dbName)))
+        .withMessageContaining("does not exist")
+        .withMessageContaining(dbName);
+  }
+
+  @Test
+  public void testListTablesWithinSchema() {
+    String dbName = "DB_2";
+    String schemaName = "SCHEMA_2";
+    Assertions.assertThat(catalog.listTables(Namespace.of(dbName, schemaName)))
+        .containsExactly(
+            TableIdentifier.of("DB_2", "SCHEMA_2", "TAB_3"),
+            TableIdentifier.of("DB_2", "SCHEMA_2", "TAB_4"));
+  }
+
+  @Test
+  public void testListTablesWithinNonexistentSchema() {
+    String dbName = "DB_2";
+    String schemaName = "NONEXISTENT_SCHEMA";
+    Assertions.assertThatExceptionOfType(RuntimeException.class)
+        .isThrownBy(() -> catalog.listTables(Namespace.of(dbName, schemaName)))
+        .withMessageContaining("does not exist")
+        .withMessageContaining("DB_2.NONEXISTENT_SCHEMA");
+  }
+
+  @Test
+  public void testLoadS3Table() {
+    Table table = catalog.loadTable(TableIdentifier.of(Namespace.of("DB_1", "SCHEMA_1"), "TAB_1"));
+    Assertions.assertThat(table.location()).isEqualTo("s3://tab1/");
+  }
+
+  @Test
+  public void testLoadAzureTable() {
+    Table table = catalog.loadTable(TableIdentifier.of(Namespace.of("DB_2", "SCHEMA_2"), "TAB_3"));
+    Assertions.assertThat(table.location())
+        .isEqualTo("wasbs://mycontainer@myaccount.blob.core.windows.net/tab1/");
+  }
+
+  @Test
+  public void testLoadGcsTable() {
+    Table table = catalog.loadTable(TableIdentifier.of(Namespace.of("DB_3", "SCHEMA_3"), "TAB_5"));
+    Assertions.assertThat(table.location()).isEqualTo("gs://tab5/");
+  }
+
+  @Test
+  public void testLoadTableWithMalformedTableIdentifier() {
+    Assertions.assertThatExceptionOfType(IllegalArgumentException.class)
+        .isThrownBy(
+            () ->
+                catalog.loadTable(
+                    TableIdentifier.of(Namespace.of("DB_1", "SCHEMA_1", "BAD_NS_LEVEL"), "TAB_1")))
+        .withMessageContaining("level")
+        .withMessageContaining("DB_1.SCHEMA_1.BAD_NS_LEVEL");
+    Assertions.assertThatExceptionOfType(IllegalArgumentException.class)
+        .isThrownBy(
+            () -> catalog.loadTable(TableIdentifier.of(Namespace.of("DB_WITHOUT_SCHEMA"), "TAB_1")))
+        .withMessageContaining("level")
+        .withMessageContaining("DB_WITHOUT_SCHEMA.TAB_1");
+  }
+
+  @Test
+  public void testCloseBeforeInitialize() throws IOException {
+    catalog = new SnowflakeCatalog();
+    catalog.close();

Review Comment:
   Testing that it's safe to call close() before initialize() without unexpected exceptions; the thought is that a caller might decide to abort using a particular catalog for whatever reason without initializing them, and might have some auto-close helpers, for example.
   
   Added a comment to explain here.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1052967517


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.hadoop.HadoopFileIO";
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    LOG.debug("listTables with namespace: {}", namespace);
+    SnowflakeIdentifier scope = NamespaceHelpers.getSnowflakeIdentifierForNamespace(namespace);
+    Preconditions.checkArgument(
+        scope.getType() == SnowflakeIdentifier.Type.ROOT
+            || scope.getType() == SnowflakeIdentifier.Type.DATABASE
+            || scope.getType() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at ROOT, DATABASE, or SCHEMA level; got %s from namespace %s",
+        scope,
+        namespace);
+
+    List<SnowflakeIdentifier> sfTables = snowflakeClient.listIcebergTables(scope);
+
+    return sfTables.stream()
+        .map(
+            table ->
+                TableIdentifier.of(
+                    table.getDatabaseName(), table.getSchemaName(), table.getTableName()))
+        .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) {
+    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);
+
+    String fileIOImpl = DEFAULT_FILE_IO_IMPL;
+    if (properties.containsKey(CatalogProperties.FILE_IO_IMPL)) {
+      fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL);
+    }
+
+    initialize(
+        name,
+        new JdbcSnowflakeClient(connectionPool),
+        CatalogUtil.loadFileIO(fileIOImpl, properties, conf),
+        properties);
+  }
+
+  /**
+   * Initialize using caller-supplied SnowflakeClient and FileIO.
+   *
+   * @param name The name of the catalog, defaults to "snowflake_catalog"
+   * @param snowflakeClient The client encapsulating network communication with Snowflake
+   * @param fileIO The {@link FileIO} to use for table operations
+   * @param properties The catalog options to use and propagate to dependencies
+   */
+  @SuppressWarnings("checkstyle:HiddenField")
+  public void initialize(
+      String name, SnowflakeClient snowflakeClient, FileIO fileIO, Map<String, String> properties) {
+    Preconditions.checkArgument(null != snowflakeClient, "snowflakeClient must be non-null");
+    Preconditions.checkArgument(null != fileIO, "fileIO must be non-null");
+    this.catalogName = name == null ? DEFAULT_CATALOG_NAME : name;
+    this.snowflakeClient = snowflakeClient;
+    this.fileIO = fileIO;
+    this.catalogProperties = properties;
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(snowflakeClient);
+    closeableGroup.addCloseable(fileIO);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (null != closeableGroup) {
+      closeableGroup.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);
+    SnowflakeIdentifier scope = NamespaceHelpers.getSnowflakeIdentifierForNamespace(namespace);
+    Preconditions.checkArgument(
+        scope.getType() == SnowflakeIdentifier.Type.ROOT
+            || scope.getType() == SnowflakeIdentifier.Type.DATABASE,
+        "listNamespaces must be at either ROOT or DATABASE level; got %s from namespace %s",
+        scope,
+        namespace);
+    List<SnowflakeIdentifier> sfSchemas = snowflakeClient.listSchemas(scope);
+
+    List<Namespace> namespaceList =
+        sfSchemas.stream()
+            .map(
+                schema -> {
+                  Preconditions.checkState(
+                      schema.getType() == SnowflakeIdentifier.Type.SCHEMA,
+                      "Got identifier of type %s from listSchemas for %s",
+                      schema.getType(),
+                      namespace);
+                  return Namespace.of(schema.getDatabaseName(), schema.getSchemaName());
+                })
+            .collect(Collectors.toList());
+    return namespaceList;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace)
+      throws NoSuchNamespaceException {
+    LOG.debug("loadNamespaceMetadata with namespace: {}", namespace);
+    return ImmutableMap.of();
+  }
+
+  @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) {
+    return new SnowflakeTableOperations(

Review Comment:
   Interesting, I wonder if this would also be a bigger problem down the road due to `io()` being a TableOperations-level method, in case tables themselves start having files in multiple buckets. It seems ideally either FIleIOs are always a URI-authority-level "resolving" FileIO the same way Hadoop's `FileSystem.get` or `Path.getFileSystem` are based on URI authority, or we'd want TableOperations to normally deal with one level higher abstraction where they'd supply some kind of FileIOFactory that can be used to resolve different FileIO instances even if, e.g. metadata.json/manifest-lists/manifests are in different buckets or even different cloud providers.
   
   For now I just pulled out a SnowflakeCatalog.FileIOFactory placeholder to maintain test-injectability and made it instantiate a new FileIO for every call to newTableOps.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1052715842


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.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.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+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);

Review Comment:
   Removed



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1052716567


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/NamespaceHelpers.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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 org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class NamespaceHelpers {

Review Comment:
   Changed to package-private



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051265807


##########
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:
   Looks like now with CloseableGroup it's probably not worth worrying about the exception in the signature; went ahead and removed this override.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051656685


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.hadoop.HadoopFileIO";
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    LOG.debug("listTables with namespace: {}", namespace);
+    SnowflakeIdentifier scope = NamespaceHelpers.getSnowflakeIdentifierForNamespace(namespace);
+    Preconditions.checkArgument(
+        scope.getType() == SnowflakeIdentifier.Type.ROOT
+            || scope.getType() == SnowflakeIdentifier.Type.DATABASE
+            || scope.getType() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at ROOT, DATABASE, or SCHEMA level; got %s from namespace %s",
+        scope,
+        namespace);
+
+    List<SnowflakeIdentifier> sfTables = snowflakeClient.listIcebergTables(scope);
+
+    return sfTables.stream()
+        .map(
+            table ->
+                TableIdentifier.of(
+                    table.getDatabaseName(), table.getSchemaName(), table.getTableName()))
+        .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));

Review Comment:
   Same here, just log that the operation is not supported.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1052700261


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/entities/SnowflakeIdentifier.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class SnowflakeIdentifier {

Review Comment:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1052972040


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.hadoop.HadoopFileIO";
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    LOG.debug("listTables with namespace: {}", namespace);
+    SnowflakeIdentifier scope = NamespaceHelpers.getSnowflakeIdentifierForNamespace(namespace);
+    Preconditions.checkArgument(
+        scope.getType() == SnowflakeIdentifier.Type.ROOT
+            || scope.getType() == SnowflakeIdentifier.Type.DATABASE
+            || scope.getType() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at ROOT, DATABASE, or SCHEMA level; got %s from namespace %s",
+        scope,
+        namespace);
+
+    List<SnowflakeIdentifier> sfTables = snowflakeClient.listIcebergTables(scope);

Review Comment:
   Ah, good to know. Updated precondition check and unittests.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051661784


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/NamespaceHelpers.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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 org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class NamespaceHelpers {

Review Comment:
   I don't think this needs to be public



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051681776


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.hadoop.HadoopFileIO";
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    LOG.debug("listTables with namespace: {}", namespace);
+    SnowflakeIdentifier scope = NamespaceHelpers.getSnowflakeIdentifierForNamespace(namespace);
+    Preconditions.checkArgument(
+        scope.getType() == SnowflakeIdentifier.Type.ROOT
+            || scope.getType() == SnowflakeIdentifier.Type.DATABASE
+            || scope.getType() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at ROOT, DATABASE, or SCHEMA level; got %s from namespace %s",
+        scope,
+        namespace);
+
+    List<SnowflakeIdentifier> sfTables = snowflakeClient.listIcebergTables(scope);
+
+    return sfTables.stream()
+        .map(
+            table ->
+                TableIdentifier.of(
+                    table.getDatabaseName(), table.getSchemaName(), table.getTableName()))
+        .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) {
+    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);
+
+    String fileIOImpl = DEFAULT_FILE_IO_IMPL;
+    if (properties.containsKey(CatalogProperties.FILE_IO_IMPL)) {
+      fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL);
+    }
+
+    initialize(
+        name,
+        new JdbcSnowflakeClient(connectionPool),
+        CatalogUtil.loadFileIO(fileIOImpl, properties, conf),
+        properties);
+  }
+
+  /**
+   * Initialize using caller-supplied SnowflakeClient and FileIO.
+   *
+   * @param name The name of the catalog, defaults to "snowflake_catalog"
+   * @param snowflakeClient The client encapsulating network communication with Snowflake
+   * @param fileIO The {@link FileIO} to use for table operations
+   * @param properties The catalog options to use and propagate to dependencies
+   */
+  @SuppressWarnings("checkstyle:HiddenField")
+  public void initialize(
+      String name, SnowflakeClient snowflakeClient, FileIO fileIO, Map<String, String> properties) {
+    Preconditions.checkArgument(null != snowflakeClient, "snowflakeClient must be non-null");
+    Preconditions.checkArgument(null != fileIO, "fileIO must be non-null");
+    this.catalogName = name == null ? DEFAULT_CATALOG_NAME : name;
+    this.snowflakeClient = snowflakeClient;
+    this.fileIO = fileIO;
+    this.catalogProperties = properties;
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(snowflakeClient);
+    closeableGroup.addCloseable(fileIO);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (null != closeableGroup) {
+      closeableGroup.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);
+    SnowflakeIdentifier scope = NamespaceHelpers.getSnowflakeIdentifierForNamespace(namespace);
+    Preconditions.checkArgument(
+        scope.getType() == SnowflakeIdentifier.Type.ROOT
+            || scope.getType() == SnowflakeIdentifier.Type.DATABASE,
+        "listNamespaces must be at either ROOT or DATABASE level; got %s from namespace %s",
+        scope,
+        namespace);
+    List<SnowflakeIdentifier> sfSchemas = snowflakeClient.listSchemas(scope);
+
+    List<Namespace> namespaceList =
+        sfSchemas.stream()
+            .map(
+                schema -> {
+                  Preconditions.checkState(
+                      schema.getType() == SnowflakeIdentifier.Type.SCHEMA,
+                      "Got identifier of type %s from listSchemas for %s",
+                      schema.getType(),
+                      namespace);
+                  return Namespace.of(schema.getDatabaseName(), schema.getSchemaName());
+                })
+            .collect(Collectors.toList());
+    return namespaceList;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace)
+      throws NoSuchNamespaceException {
+    LOG.debug("loadNamespaceMetadata with namespace: {}", namespace);
+    return ImmutableMap.of();
+  }
+
+  @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) {
+    return new SnowflakeTableOperations(

Review Comment:
   I'm not sure that I agree here. You can't assume that tables share a catalog-level FileIO, but the tables don't necessarily need a separate instance. We reuse the FileIO in other catalogs if the config doesn't change based on the table.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051681423


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.hadoop.HadoopFileIO";
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    LOG.debug("listTables with namespace: {}", namespace);
+    SnowflakeIdentifier scope = NamespaceHelpers.getSnowflakeIdentifierForNamespace(namespace);
+    Preconditions.checkArgument(
+        scope.getType() == SnowflakeIdentifier.Type.ROOT
+            || scope.getType() == SnowflakeIdentifier.Type.DATABASE
+            || scope.getType() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at ROOT, DATABASE, or SCHEMA level; got %s from namespace %s",
+        scope,
+        namespace);
+
+    List<SnowflakeIdentifier> sfTables = snowflakeClient.listIcebergTables(scope);
+
+    return sfTables.stream()
+        .map(
+            table ->
+                TableIdentifier.of(
+                    table.getDatabaseName(), table.getSchemaName(), table.getTableName()))
+        .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) {
+    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);
+
+    String fileIOImpl = DEFAULT_FILE_IO_IMPL;
+    if (properties.containsKey(CatalogProperties.FILE_IO_IMPL)) {
+      fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL);
+    }
+
+    initialize(
+        name,
+        new JdbcSnowflakeClient(connectionPool),
+        CatalogUtil.loadFileIO(fileIOImpl, properties, conf),
+        properties);
+  }
+
+  /**
+   * Initialize using caller-supplied SnowflakeClient and FileIO.
+   *
+   * @param name The name of the catalog, defaults to "snowflake_catalog"
+   * @param snowflakeClient The client encapsulating network communication with Snowflake
+   * @param fileIO The {@link FileIO} to use for table operations
+   * @param properties The catalog options to use and propagate to dependencies
+   */
+  @SuppressWarnings("checkstyle:HiddenField")
+  public void initialize(
+      String name, SnowflakeClient snowflakeClient, FileIO fileIO, Map<String, String> properties) {
+    Preconditions.checkArgument(null != snowflakeClient, "snowflakeClient must be non-null");
+    Preconditions.checkArgument(null != fileIO, "fileIO must be non-null");
+    this.catalogName = name == null ? DEFAULT_CATALOG_NAME : name;
+    this.snowflakeClient = snowflakeClient;
+    this.fileIO = fileIO;
+    this.catalogProperties = properties;
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(snowflakeClient);
+    closeableGroup.addCloseable(fileIO);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (null != closeableGroup) {
+      closeableGroup.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);
+    SnowflakeIdentifier scope = NamespaceHelpers.getSnowflakeIdentifierForNamespace(namespace);
+    Preconditions.checkArgument(
+        scope.getType() == SnowflakeIdentifier.Type.ROOT
+            || scope.getType() == SnowflakeIdentifier.Type.DATABASE,
+        "listNamespaces must be at either ROOT or DATABASE level; got %s from namespace %s",
+        scope,
+        namespace);
+    List<SnowflakeIdentifier> sfSchemas = snowflakeClient.listSchemas(scope);
+
+    List<Namespace> namespaceList =
+        sfSchemas.stream()
+            .map(
+                schema -> {
+                  Preconditions.checkState(
+                      schema.getType() == SnowflakeIdentifier.Type.SCHEMA,
+                      "Got identifier of type %s from listSchemas for %s",
+                      schema.getType(),
+                      namespace);
+                  return Namespace.of(schema.getDatabaseName(), schema.getSchemaName());
+                })
+            .collect(Collectors.toList());
+    return namespaceList;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace)
+      throws NoSuchNamespaceException {
+    LOG.debug("loadNamespaceMetadata with namespace: {}", namespace);

Review Comment:
   We don't typically debug individual calls like this. What is the value here?



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1050257006


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

Review Comment:
   Good suggestion! I recall skimming over it when looking at the JdbcCatalog's unittest, but saw most of the test cases were for (currently) unsupported functionality. Just for completeness though I went ahead and plugged it in (which helped identify the need to specify `test { useJUnitPlatform() }` in build.gradle to successfully inherit the tests) and verified that they all fail due to depending on at least `buildTable/createNamespace` to set up test data.
   
   One possibility if we want to split out read-only variations even for temporary scenarios, would be to abstract out the mutations into helper methods in the `CatalogTests` base class, and only test cases that explicitly test mutations would call the catalog directly, letting read-only stuff do the mutations in their fake in-memory data model.
   
   It seems like the `supports*` test methods are also pointing towards possibly wanting to extract a more formalized way to define differences in supported functionality between catalogs, perhaps more fine-grained than what things like e.g. `SupportsNamespaces` java interfaces provide.
   
   Is there any work in progress to do something for Catalog interfaces analogous to the [HCFS efforts for HadoopFileSystem](https://cwiki.apache.org/confluence/display/HADOOP2/HCFS) ? 
   
   Seems like CatalogTests may have already been written with the analogous [AbstractFSContractTestBase](https://github.com/apache/hadoop/blob/03cfc852791c14fad39db4e5b14104a276c08e59/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContractTestBase.java) in mind, where we could move towards analogous ["contract" definitions](https://github.com/apache/hadoop/blob/03cfc852791c14fad39db4e5b14104a276c08e59/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/LocalFSContract.java) and [contract config specs](https://github.com/apache/hadoop/blob/03cfc852791c14fad39db4e5b14104a276c08e59/hadoop-common-project/hadoop-common/src/test/resources/contract/localfs.xml)



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] manisin commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
manisin commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051059938


##########
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:
   The decision to use Object rather than HadoopConfig is to avoid runtime time dependency on hadoop packages. This usage pattern is inline with other catalogs like GlueCatalog.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051230270


##########
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:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051267959


##########
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://mycontainer@myaccount.blob.core.windows.net/tab3/metadata/v334.metadata.json",
+        TableMetadataParser.toJson(
+                TableMetadata.newTableMetadata(
+                    schema,
+                    partitionSpec,
+                    "wasbs://mycontainer@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:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1056619399


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/NamespaceHelpers.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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 org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class NamespaceHelpers {
+  private static final int MAX_NAMESPACE_DEPTH = 2;
+  private static final int NAMESPACE_ROOT_LEVEL = 0;
+  private static final int NAMESPACE_DB_LEVEL = 1;
+  private static final int NAMESPACE_SCHEMA_LEVEL = 2;
+
+  private static final Logger LOG = LoggerFactory.getLogger(NamespaceHelpers.class);
+
+  private NamespaceHelpers() {}
+
+  /**
+   * Converts a Namespace into a SnowflakeIdentifier representing ROOT, a DATABASE, or a SCHEMA.
+   *
+   * @throws IllegalArgumentException if the namespace is not a supported depth.
+   */
+  public static SnowflakeIdentifier toSnowflakeIdentifier(Namespace namespace) {
+    SnowflakeIdentifier identifier = null;
+    switch (namespace.length()) {
+      case NAMESPACE_ROOT_LEVEL:
+        identifier = SnowflakeIdentifier.ofRoot();
+        break;
+      case NAMESPACE_DB_LEVEL:
+        identifier = SnowflakeIdentifier.ofDatabase(namespace.level(NAMESPACE_DB_LEVEL - 1));
+        break;
+      case NAMESPACE_SCHEMA_LEVEL:
+        identifier =
+            SnowflakeIdentifier.ofSchema(
+                namespace.level(NAMESPACE_DB_LEVEL - 1),
+                namespace.level(NAMESPACE_SCHEMA_LEVEL - 1));
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format(
+                "Snowflake max namespace level is %d, got namespace '%s'",
+                MAX_NAMESPACE_DEPTH, namespace));
+    }
+    LOG.debug("toSnowflakeIdentifier({}) -> {}", namespace, identifier);
+    return identifier;
+  }
+
+  /**
+   * Converts a TableIdentifier into a SnowflakeIdentifier of type TABLE; the identifier must have
+   * exactly the right namespace depth to represent a fully-qualified Snowflake table identifier.
+   */
+  public static SnowflakeIdentifier toSnowflakeIdentifier(TableIdentifier identifier) {
+    SnowflakeIdentifier namespaceScope = toSnowflakeIdentifier(identifier.namespace());
+    Preconditions.checkArgument(
+        namespaceScope.type() == SnowflakeIdentifier.Type.SCHEMA,
+        "Namespace portion of '%s' must be at the SCHEMA level, got namespaceScope '%s'",
+        identifier,
+        namespaceScope);
+    SnowflakeIdentifier ret =
+        SnowflakeIdentifier.ofTable(
+            namespaceScope.databaseName(), namespaceScope.schemaName(), identifier.name());
+    LOG.debug("toSnowflakeIdentifier({}) -> {}", identifier, ret);
+    return ret;
+  }
+
+  /**
+   * Converts a SnowflakeIdentifier of type ROOT, DATABASE, or SCHEMA into an equivalent Iceberg
+   * Namespace; throws IllegalArgumentException if not an appropriate type.
+   */
+  public static Namespace toIcebergNamespace(SnowflakeIdentifier identifier) {
+    Namespace namespace = null;
+    switch (identifier.type()) {
+      case ROOT:
+        namespace = Namespace.of();
+        break;
+      case DATABASE:
+        namespace = Namespace.of(identifier.databaseName());
+        break;
+      case SCHEMA:
+        namespace = Namespace.of(identifier.databaseName(), identifier.schemaName());
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format("Cannot convert identifier '%s' to Namespace", identifier));
+    }
+    LOG.debug("toIcebergNamespace({}) -> {}", identifier, namespace);

Review Comment:
   Mostly the debug-level logging was useful while developing, but probably you're right no one else would really gain anything from it in the future. Removed all the debug-level logging.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1056617977


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/NamespaceHelpers.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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 org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class NamespaceHelpers {
+  private static final int MAX_NAMESPACE_DEPTH = 2;
+  private static final int NAMESPACE_ROOT_LEVEL = 0;
+  private static final int NAMESPACE_DB_LEVEL = 1;
+  private static final int NAMESPACE_SCHEMA_LEVEL = 2;
+
+  private static final Logger LOG = LoggerFactory.getLogger(NamespaceHelpers.class);
+
+  private NamespaceHelpers() {}
+
+  /**
+   * Converts a Namespace into a SnowflakeIdentifier representing ROOT, a DATABASE, or a SCHEMA.
+   *
+   * @throws IllegalArgumentException if the namespace is not a supported depth.
+   */
+  public static SnowflakeIdentifier toSnowflakeIdentifier(Namespace namespace) {
+    SnowflakeIdentifier identifier = null;
+    switch (namespace.length()) {
+      case NAMESPACE_ROOT_LEVEL:
+        identifier = SnowflakeIdentifier.ofRoot();
+        break;
+      case NAMESPACE_DB_LEVEL:
+        identifier = SnowflakeIdentifier.ofDatabase(namespace.level(NAMESPACE_DB_LEVEL - 1));
+        break;
+      case NAMESPACE_SCHEMA_LEVEL:
+        identifier =
+            SnowflakeIdentifier.ofSchema(
+                namespace.level(NAMESPACE_DB_LEVEL - 1),
+                namespace.level(NAMESPACE_SCHEMA_LEVEL - 1));
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format(
+                "Snowflake max namespace level is %d, got namespace '%s'",
+                MAX_NAMESPACE_DEPTH, namespace));
+    }
+    LOG.debug("toSnowflakeIdentifier({}) -> {}", namespace, identifier);
+    return identifier;
+  }
+
+  /**
+   * Converts a TableIdentifier into a SnowflakeIdentifier of type TABLE; the identifier must have
+   * exactly the right namespace depth to represent a fully-qualified Snowflake table identifier.
+   */
+  public static SnowflakeIdentifier toSnowflakeIdentifier(TableIdentifier identifier) {
+    SnowflakeIdentifier namespaceScope = toSnowflakeIdentifier(identifier.namespace());
+    Preconditions.checkArgument(
+        namespaceScope.type() == SnowflakeIdentifier.Type.SCHEMA,
+        "Namespace portion of '%s' must be at the SCHEMA level, got namespaceScope '%s'",
+        identifier,
+        namespaceScope);
+    SnowflakeIdentifier ret =
+        SnowflakeIdentifier.ofTable(
+            namespaceScope.databaseName(), namespaceScope.schemaName(), identifier.name());
+    LOG.debug("toSnowflakeIdentifier({}) -> {}", identifier, ret);
+    return ret;
+  }
+
+  /**
+   * Converts a SnowflakeIdentifier of type ROOT, DATABASE, or SCHEMA into an equivalent Iceberg
+   * Namespace; throws IllegalArgumentException if not an appropriate type.
+   */
+  public static Namespace toIcebergNamespace(SnowflakeIdentifier identifier) {
+    Namespace namespace = null;
+    switch (identifier.type()) {
+      case ROOT:
+        namespace = Namespace.of();

Review Comment:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1056622011


##########
snowflake/src/test/java/org/apache/iceberg/snowflake/SnowflakeCatalogTest.java:
##########
@@ -0,0 +1,310 @@
+/*
+ * 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.IOException;
+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.io.FileIO;
+import org.apache.iceberg.io.InMemoryFileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Types;
+import org.assertj.core.api.Assertions;
+import org.junit.Before;
+import org.junit.Test;
+
+public class SnowflakeCatalogTest {
+
+  static final String TEST_CATALOG_NAME = "slushLog";

Review Comment:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1056640133


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.io.ResolvingFileIO";
+
+  static class FileIOFactory {
+    public FileIO newFileIO(String impl, Map<String, String> properties, Object hadoopConf) {
+      return CatalogUtil.loadFileIO(impl, properties, hadoopConf);
+    }
+  }
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIOFactory fileIOFactory;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    SnowflakeIdentifier scope = NamespaceHelpers.toSnowflakeIdentifier(namespace);
+    Preconditions.checkArgument(
+        scope.type() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at SCHEMA level; got %s from namespace %s",
+        scope,
+        namespace);
+
+    List<SnowflakeIdentifier> sfTables = snowflakeClient.listIcebergTables(scope);
+
+    return sfTables.stream()
+        .map(NamespaceHelpers::toIcebergTableIdentifier)
+        .collect(Collectors.toList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support dropTable");
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support renameTable");
+  }
+
+  @Override
+  public void initialize(String name, Map<String, String> properties) {
+    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 communication.
+      Class.forName(JdbcSnowflakeClient.EXPECTED_JDBC_IMPL);
+    } catch (ClassNotFoundException cnfe) {

Review Comment:
   The most likely case would be people who bring their own fatjar with the whole thing repackaged - as long as they specify their repackaged impl, it should work fine.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1056638166


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.io.ResolvingFileIO";
+
+  static class FileIOFactory {
+    public FileIO newFileIO(String impl, Map<String, String> properties, Object hadoopConf) {
+      return CatalogUtil.loadFileIO(impl, properties, hadoopConf);
+    }
+  }
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIOFactory fileIOFactory;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    SnowflakeIdentifier scope = NamespaceHelpers.toSnowflakeIdentifier(namespace);
+    Preconditions.checkArgument(
+        scope.type() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at SCHEMA level; got %s from namespace %s",
+        scope,
+        namespace);
+
+    List<SnowflakeIdentifier> sfTables = snowflakeClient.listIcebergTables(scope);
+
+    return sfTables.stream()
+        .map(NamespaceHelpers::toIcebergTableIdentifier)
+        .collect(Collectors.toList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support dropTable");
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support renameTable");
+  }
+
+  @Override
+  public void initialize(String name, Map<String, String> properties) {
+    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 communication.
+      Class.forName(JdbcSnowflakeClient.EXPECTED_JDBC_IMPL);
+    } catch (ClassNotFoundException cnfe) {

Review Comment:
   I feel like we should just fail here if the JDBC driver isn't able to load.  Why would we continue to initialize if you don't have the expected jdbc driver?  Are there other known drivers that are compatible and would load?



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1063898773


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java:
##########
@@ -0,0 +1,386 @@
+/*
+ * 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.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+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.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+/**
+ * This implementation of SnowflakeClient builds on top of Snowflake's JDBC driver to interact with
+ * Snowflake's Iceberg-aware resource model.
+ */
+class JdbcSnowflakeClient implements SnowflakeClient {
+  static final String EXPECTED_JDBC_IMPL = "net.snowflake.client.jdbc.SnowflakeDriver";
+
+  @FunctionalInterface
+  interface ResultSetParser<T> {
+    T parse(ResultSet rs) throws SQLException;
+  }
+
+  /**
+   * This class wraps the basic boilerplate of setting up PreparedStatements and applying a
+   * ResultSetParser to translate a ResultSet into parsed objects. Allows easily injecting
+   * subclasses for debugging/testing purposes.
+   */
+  static class QueryHarness {
+    public <T> T query(Connection conn, String sql, ResultSetParser<T> parser, String... args)
+        throws SQLException {
+      try (PreparedStatement statement = conn.prepareStatement(sql)) {
+        if (args != null) {
+          for (int i = 0; i < args.length; ++i) {
+            statement.setString(i + 1, args[i]);
+          }
+        }
+
+        try (ResultSet rs = statement.executeQuery()) {
+          return parser.parse(rs);
+        }
+      }
+    }
+  }
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Database identifiers,
+   * containing "name" (representing databaseName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> DATABASE_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> databases = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("name");
+          databases.add(SnowflakeIdentifier.ofDatabase(databaseName));
+        }
+        return databases;
+      };
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Schema identifiers,
+   * containing "database_name" and "name" (representing schemaName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> SCHEMA_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> schemas = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("database_name");
+          String schemaName = rs.getString("name");
+          schemas.add(SnowflakeIdentifier.ofSchema(databaseName, schemaName));
+        }
+        return schemas;
+      };
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Table identifiers,
+   * containing "database_name", "schema_name", and "name" (representing tableName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> TABLE_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> tables = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("database_name");
+          String schemaName = rs.getString("schema_name");
+          String tableName = rs.getString("name");
+          tables.add(SnowflakeIdentifier.ofTable(databaseName, schemaName, tableName));
+        }
+        return tables;
+      };
+
+  /**
+   * Expects to handle ResultSets representing a single record holding Snowflake Iceberg metadata.
+   */
+  public static final ResultSetParser<SnowflakeTableMetadata> TABLE_METADATA_RESULT_SET_HANDLER =
+      rs -> {
+        if (!rs.next()) {
+          return null;
+        }
+
+        String rawJsonVal = rs.getString("METADATA");
+        return SnowflakeTableMetadata.parseJson(rawJsonVal);
+      };
+
+  private final JdbcClientPool connectionPool;
+  private QueryHarness queryHarness;
+
+  JdbcSnowflakeClient(JdbcClientPool conn) {
+    Preconditions.checkArgument(null != conn, "JdbcClientPool must be non-null");
+    connectionPool = conn;
+    queryHarness = new QueryHarness();
+  }
+
+  @VisibleForTesting
+  void setQueryHarness(QueryHarness queryHarness) {
+    this.queryHarness = queryHarness;
+  }
+
+  /**
+   * For rare cases where PreparedStatements aren't supported for user-supplied identifiers intended
+   * for use in special LIKE clauses, we can sanitize by "broadening" the identifier with
+   * single-character wildcards and manually post-filter client-side.
+   *
+   * <p>Note: This sanitization approach intentionally "broadens" the scope of matching results;
+   * callers must be able to handle this method returning an all-wildcard expression; i.e. the
+   * caller must treat the usage of the LIKE clause as only an optional optimization, and should
+   * post-filter for correctness as if the LIKE clause wasn't present in the query at all.
+   */
+  @VisibleForTesting
+  String sanitizeIdentifierWithWildcardForLikeClause(String identifier) {
+    // Restrict identifiers to the "Unquoted object identifiers" synax documented at
+    // https://docs.snowflake.com/en/sql-reference/identifiers-syntax.html
+    //
+    // Use a strict allowlist of characters, replace everything *not* matching the character set
+    // with "_", which is used as a single-character wildcard in Snowflake.
+    String sanitized = identifier.replaceAll("[^a-zA-Z0-9_$]", "_");
+    if (sanitized.startsWith("$")) {
+      sanitized = "_" + sanitized.substring(1);
+    }
+    return sanitized;
+  }
+
+  @Override
+  public boolean databaseExists(SnowflakeIdentifier database) {
+    Preconditions.checkArgument(
+        database.type() == SnowflakeIdentifier.Type.DATABASE,
+        "databaseExists requires a DATABASE identifier, got '%s'",
+        database);
+
+    // Due to current limitations in PreparedStatement parameters for the LIKE clause in
+    // SHOW DATABASES queries, we'll use a fairly limited allowlist for identifier characters,
+    // using wildcards for non-allowed characters, and post-filter for matching.
+    final String finalQuery =
+        String.format(
+            "SHOW DATABASES LIKE '%s' IN ACCOUNT",
+            sanitizeIdentifierWithWildcardForLikeClause(database.databaseName()));
+    List<SnowflakeIdentifier> databases;
+    try {
+      databases =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn, finalQuery, DATABASE_RESULT_SET_HANDLER, (String[]) null));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to check if database '%s' exists", database);
+    } catch (InterruptedException e) {
+      throw new UncheckedInterruptedException(
+          e, "Interrupted while checking if database '%s' exists", database);
+    }
+
+    // Filter to handle the edge case of '_' appearing as a wildcard that can't be remapped the way
+    // it can for predicates in SELECT statements.
+    databases.removeIf(db -> !database.databaseName().equalsIgnoreCase(db.databaseName()));
+    return !databases.isEmpty();
+  }
+
+  @Override
+  public boolean schemaExists(SnowflakeIdentifier schema) {
+    Preconditions.checkArgument(
+        schema.type() == SnowflakeIdentifier.Type.SCHEMA,
+        "schemaExists requires a SCHEMA identifier, got '%s'",
+        schema);
+
+    if (!databaseExists(SnowflakeIdentifier.ofDatabase(schema.databaseName()))) {
+      return false;
+    }
+
+    // Due to current limitations in PreparedStatement parameters for the LIKE clause in
+    // SHOW SCHEMAS queries, we'll use a fairly limited allowlist for identifier characters,
+    // using wildcards for non-allowed characters, and post-filter for matching.
+    final String finalQuery =
+        String.format(
+            "SHOW SCHEMAS LIKE '%s' IN DATABASE IDENTIFIER(?)",
+            sanitizeIdentifierWithWildcardForLikeClause(schema.schemaName()));
+    List<SnowflakeIdentifier> schemas;
+    try {
+      schemas =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn,
+                      finalQuery,
+                      SCHEMA_RESULT_SET_HANDLER,
+                      new String[] {schema.databaseName()}));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to check if schema '%s' exists", schema);
+    } catch (InterruptedException e) {
+      throw new UncheckedInterruptedException(
+          e, "Interrupted while checking if schema '%s' exists", schema);
+    }
+
+    // Filter to handle the edge case of '_' appearing as a wildcard that can't be remapped the way
+    // it can for predicates in SELECT statements.
+    schemas.removeIf(sc -> !schema.schemaName().equalsIgnoreCase(sc.schemaName()));
+    return !schemas.isEmpty();
+  }
+
+  @Override
+  public List<SnowflakeIdentifier> listDatabases() {
+    List<SnowflakeIdentifier> databases;
+    try {
+      databases =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn,
+                      "SHOW DATABASES IN ACCOUNT",
+                      DATABASE_RESULT_SET_HANDLER,
+                      (String[]) null));

Review Comment:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1063904193


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeTableMetadata.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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 com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+class SnowflakeTableMetadata {
+  public static final Pattern SNOWFLAKE_AZURE_PATTERN =
+      Pattern.compile("azure://([^/]+)/([^/]+)/(.*)");
+
+  private String snowflakeMetadataLocation;
+  private String icebergMetadataLocation;
+  private String status;
+
+  // Note: Since not all sources will necessarily come from a raw JSON representation, this raw
+  // JSON should only be considered a convenient debugging field. Equality of two
+  // SnowflakeTableMetadata instances should not depend on equality of this field.
+  private String rawJsonVal;

Review Comment:
   Added to toString



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] jackye1995 commented on pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#issuecomment-1382624709

   Looks like some CI tests are failing? Could you check? Maybe need to rebase.


-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051251362


##########
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:
   Decided on null check in the constructor instead as a checkArgument.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051681274


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.hadoop.HadoopFileIO";
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    LOG.debug("listTables with namespace: {}", namespace);
+    SnowflakeIdentifier scope = NamespaceHelpers.getSnowflakeIdentifierForNamespace(namespace);
+    Preconditions.checkArgument(
+        scope.getType() == SnowflakeIdentifier.Type.ROOT
+            || scope.getType() == SnowflakeIdentifier.Type.DATABASE
+            || scope.getType() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at ROOT, DATABASE, or SCHEMA level; got %s from namespace %s",

Review Comment:
   Does Snowflake allow tables to exist directly in a database or account root? If not, then those identifiers should not be supported here. This is not a recursive listing.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051679710


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/entities/SnowflakeIdentifier.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class SnowflakeIdentifier {
+  public enum Type {
+    ROOT,
+    DATABASE,
+    SCHEMA,
+    TABLE
+  }
+
+  private String databaseName;
+  private String schemaName;
+  private String tableName;
+
+  protected SnowflakeIdentifier(String databaseName, String schemaName, String tableName) {
+    this.databaseName = databaseName;
+    this.schemaName = schemaName;
+    this.tableName = tableName;
+  }
+
+  public static SnowflakeIdentifier ofRoot() {
+    return new SnowflakeIdentifier(null, null, null);
+  }
+
+  public static SnowflakeIdentifier ofDatabase(String databaseName) {
+    Preconditions.checkArgument(null != databaseName, "databaseName must be non-null");
+    return new SnowflakeIdentifier(databaseName, null, null);
+  }
+
+  public static SnowflakeIdentifier ofSchema(String databaseName, String schemaName) {
+    Preconditions.checkArgument(null != databaseName, "databaseName must be non-null");
+    Preconditions.checkArgument(null != schemaName, "schemaName must be non-null");
+    return new SnowflakeIdentifier(databaseName, schemaName, null);
+  }
+
+  public static SnowflakeIdentifier ofTable(
+      String databaseName, String schemaName, String tableName) {
+    Preconditions.checkArgument(null != databaseName, "databaseName must be non-null");
+    Preconditions.checkArgument(null != schemaName, "schemaName must be non-null");
+    Preconditions.checkArgument(null != tableName, "tableName must be non-null");
+    return new SnowflakeIdentifier(databaseName, schemaName, tableName);
+  }
+
+  /**
+   * If type is TABLE, expect non-null databaseName, schemaName, and tableName. If type is SCHEMA,
+   * expect non-null databaseName and schemaName. If type is DATABASE, expect non-null databaseName.
+   * If type is ROOT, expect all of databaseName, schemaName, and tableName to be null.
+   */
+  public Type getType() {

Review Comment:
   If the static factory methods are used, why not just set this as an instance field passed by those 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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051682219


##########
spark/v3.1/build.gradle:
##########
@@ -213,6 +213,9 @@ project(':iceberg-spark:iceberg-spark-runtime-3.1_2.12') {
     implementation(project(':iceberg-nessie')) {
       exclude group: 'com.google.code.findbugs', module: 'jsr305'
     }
+    implementation (project(':iceberg-snowflake')) {
+      exclude group: 'net.snowflake' , module: 'snowflake-jdbc'

Review Comment:
   Before we can include this in runtime Jars, we need to validate the file diff between the resulting Jars, update LICENSE/NOTICE files, and also take a look at the dependency tree to make sure it isn't too broad.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1052903155


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeTableOperations.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.Map;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.apache.iceberg.snowflake.entities.SnowflakeTableMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class SnowflakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeTableOperations.class);
+  private final String catalogName;
+
+  private final FileIO fileIO;
+  private final TableIdentifier tableIdentifier;
+  private final SnowflakeIdentifier snowflakeIdentifierForTable;
+
+  private final SnowflakeClient snowflakeClient;
+
+  private final Map<String, String> catalogProperties;
+
+  protected SnowflakeTableOperations(
+      SnowflakeClient snowflakeClient,
+      FileIO fileIO,
+      Map<String, String> properties,
+      String catalogName,
+      TableIdentifier tableIdentifier) {
+    this.snowflakeClient = snowflakeClient;
+    this.fileIO = fileIO;
+    this.catalogProperties = properties;
+    this.catalogName = catalogName;
+    this.tableIdentifier = tableIdentifier;
+    this.snowflakeIdentifierForTable =
+        NamespaceHelpers.getSnowflakeIdentifierForTableIdentifier(tableIdentifier);
+  }
+
+  @Override
+  public void doRefresh() {
+    LOG.debug("Getting metadata location for table {}", tableIdentifier);
+    String location = getTableMetadataLocation();
+    Preconditions.checkState(
+        location != null && !location.isEmpty(),
+        "Got null or empty location %s for table %s",
+        location,
+        tableIdentifier);
+    refreshFromMetadataLocation(location);
+  }
+
+  @Override
+  public FileIO io() {
+    return fileIO;
+  }
+
+  @Override
+  protected String tableName() {
+    return tableIdentifier.toString();

Review Comment:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051662251


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeTableOperations.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.Map;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.apache.iceberg.snowflake.entities.SnowflakeTableMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class SnowflakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeTableOperations.class);
+  private final String catalogName;

Review Comment:
   This doesn't appear to be used anywhere



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051679653


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/entities/SnowflakeIdentifier.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class SnowflakeIdentifier {
+  public enum Type {
+    ROOT,
+    DATABASE,
+    SCHEMA,
+    TABLE
+  }
+
+  private String databaseName;
+  private String schemaName;
+  private String tableName;
+
+  protected SnowflakeIdentifier(String databaseName, String schemaName, String tableName) {

Review Comment:
   I don't see classes that extend this. Can it be private instead of protected?



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051679094


##########
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')

Review Comment:
   What is used from `iceberg-aws`?



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] manisin commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
manisin commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051066674


##########
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:
   The intention here is that regardless of the client implementation, the Snowflake data model doesn't support more than 2 levels of namespace.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051241597


##########
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://mycontainer@myaccount.blob.core.windows.net/tab3/metadata/v334.metadata.json",
+        TableMetadataParser.toJson(
+                TableMetadata.newTableMetadata(
+                    schema,
+                    partitionSpec,
+                    "wasbs://mycontainer@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:
   Done. Switched to `Assertions.assertThatExceptionOfType(...).withMessageContaining(...)` here and elsewhere.



##########
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://mycontainer@myaccount.blob.core.windows.net/tab3/metadata/v334.metadata.json",
+        TableMetadataParser.toJson(
+                TableMetadata.newTableMetadata(
+                    schema,
+                    partitionSpec,
+                    "wasbs://mycontainer@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:
   Done



##########
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://mycontainer@myaccount.blob.core.windows.net/tab3/metadata/v334.metadata.json",
+        TableMetadataParser.toJson(
+                TableMetadata.newTableMetadata(
+                    schema,
+                    partitionSpec,
+                    "wasbs://mycontainer@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:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051241705


##########
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://mycontainer@myaccount.blob.core.windows.net/tab3/metadata/v334.metadata.json",
+        TableMetadataParser.toJson(
+                TableMetadata.newTableMetadata(
+                    schema,
+                    partitionSpec,
+                    "wasbs://mycontainer@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://mycontainer@myaccount.blob.core.windows.net/tab1/");

Review Comment:
   Done, switched to Assertions.assertThat



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] manisin commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
manisin commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051114634


##########
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:
   SnowflakeTable/SnowflakeSchema defines the entities in terms of snowflake's object model and is used to define the contract with the SnowflakeClient. We expect this to evolve independently of the iceberg's corresponding classes by allowing different behavior and members.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051264532


##########
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:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1056619704


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/NamespaceHelpers.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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 org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class NamespaceHelpers {
+  private static final int MAX_NAMESPACE_DEPTH = 2;
+  private static final int NAMESPACE_ROOT_LEVEL = 0;
+  private static final int NAMESPACE_DB_LEVEL = 1;
+  private static final int NAMESPACE_SCHEMA_LEVEL = 2;
+
+  private static final Logger LOG = LoggerFactory.getLogger(NamespaceHelpers.class);
+
+  private NamespaceHelpers() {}
+
+  /**
+   * Converts a Namespace into a SnowflakeIdentifier representing ROOT, a DATABASE, or a SCHEMA.
+   *
+   * @throws IllegalArgumentException if the namespace is not a supported depth.
+   */
+  public static SnowflakeIdentifier toSnowflakeIdentifier(Namespace namespace) {
+    SnowflakeIdentifier identifier = null;
+    switch (namespace.length()) {
+      case NAMESPACE_ROOT_LEVEL:
+        identifier = SnowflakeIdentifier.ofRoot();
+        break;
+      case NAMESPACE_DB_LEVEL:
+        identifier = SnowflakeIdentifier.ofDatabase(namespace.level(NAMESPACE_DB_LEVEL - 1));
+        break;
+      case NAMESPACE_SCHEMA_LEVEL:
+        identifier =
+            SnowflakeIdentifier.ofSchema(
+                namespace.level(NAMESPACE_DB_LEVEL - 1),
+                namespace.level(NAMESPACE_SCHEMA_LEVEL - 1));
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format(
+                "Snowflake max namespace level is %d, got namespace '%s'",
+                MAX_NAMESPACE_DEPTH, namespace));
+    }
+    LOG.debug("toSnowflakeIdentifier({}) -> {}", namespace, identifier);
+    return identifier;
+  }
+
+  /**
+   * Converts a TableIdentifier into a SnowflakeIdentifier of type TABLE; the identifier must have
+   * exactly the right namespace depth to represent a fully-qualified Snowflake table identifier.
+   */
+  public static SnowflakeIdentifier toSnowflakeIdentifier(TableIdentifier identifier) {
+    SnowflakeIdentifier namespaceScope = toSnowflakeIdentifier(identifier.namespace());
+    Preconditions.checkArgument(
+        namespaceScope.type() == SnowflakeIdentifier.Type.SCHEMA,
+        "Namespace portion of '%s' must be at the SCHEMA level, got namespaceScope '%s'",
+        identifier,
+        namespaceScope);
+    SnowflakeIdentifier ret =
+        SnowflakeIdentifier.ofTable(
+            namespaceScope.databaseName(), namespaceScope.schemaName(), identifier.name());
+    LOG.debug("toSnowflakeIdentifier({}) -> {}", identifier, ret);
+    return ret;
+  }
+
+  /**
+   * Converts a SnowflakeIdentifier of type ROOT, DATABASE, or SCHEMA into an equivalent Iceberg
+   * Namespace; throws IllegalArgumentException if not an appropriate type.
+   */
+  public static Namespace toIcebergNamespace(SnowflakeIdentifier identifier) {
+    Namespace namespace = null;
+    switch (identifier.type()) {
+      case ROOT:
+        namespace = Namespace.of();
+        break;
+      case DATABASE:
+        namespace = Namespace.of(identifier.databaseName());
+        break;
+      case SCHEMA:
+        namespace = Namespace.of(identifier.databaseName(), identifier.schemaName());
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format("Cannot convert identifier '%s' to Namespace", identifier));
+    }
+    LOG.debug("toIcebergNamespace({}) -> {}", identifier, namespace);
+    return namespace;
+  }
+
+  /**
+   * Converts a SnowflakeIdentifier to an equivalent Iceberg TableIdentifier; the identifier must be
+   * of type TABLE.
+   */
+  public static TableIdentifier toIcebergTableIdentifier(SnowflakeIdentifier identifier) {
+    Preconditions.checkArgument(
+        identifier.type() == SnowflakeIdentifier.Type.TABLE,
+        "SnowflakeIdentifier must be type TABLE, get '%s'",

Review Comment:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1056615091


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.io.ResolvingFileIO";
+
+  static class FileIOFactory {
+    public FileIO newFileIO(String impl, Map<String, String> properties, Object hadoopConf) {
+      return CatalogUtil.loadFileIO(impl, properties, hadoopConf);
+    }
+  }
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIOFactory fileIOFactory;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    SnowflakeIdentifier scope = NamespaceHelpers.toSnowflakeIdentifier(namespace);
+    Preconditions.checkArgument(
+        scope.type() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at SCHEMA level; got %s from namespace %s",
+        scope,
+        namespace);
+
+    List<SnowflakeIdentifier> sfTables = snowflakeClient.listIcebergTables(scope);
+
+    return sfTables.stream()
+        .map(NamespaceHelpers::toIcebergTableIdentifier)
+        .collect(Collectors.toList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support dropTable");
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support renameTable");
+  }
+
+  @Override
+  public void initialize(String name, Map<String, String> properties) {
+    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 communication.
+      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);
+
+    initialize(name, new JdbcSnowflakeClient(connectionPool), new FileIOFactory(), properties);
+  }
+
+  /**
+   * Initialize using caller-supplied SnowflakeClient and FileIO.
+   *
+   * @param name The name of the catalog, defaults to "snowflake_catalog"
+   * @param snowflakeClient The client encapsulating network communication with Snowflake
+   * @param fileIOFactory The {@link FileIOFactory} to use to instantiate a new FileIO for each new
+   *     table operation
+   * @param properties The catalog options to use and propagate to dependencies
+   */
+  @SuppressWarnings("checkstyle:HiddenField")
+  void initialize(
+      String name,
+      SnowflakeClient snowflakeClient,
+      FileIOFactory fileIOFactory,
+      Map<String, String> properties) {
+    Preconditions.checkArgument(null != snowflakeClient, "snowflakeClient must be non-null");
+    Preconditions.checkArgument(null != fileIOFactory, "fileIOFactory must be non-null");
+    this.catalogName = name == null ? DEFAULT_CATALOG_NAME : name;
+    this.snowflakeClient = snowflakeClient;
+    this.fileIOFactory = fileIOFactory;
+    this.catalogProperties = properties;
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(snowflakeClient);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (null != closeableGroup) {
+      closeableGroup.close();
+    }
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support createNamespace");
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    SnowflakeIdentifier scope = NamespaceHelpers.toSnowflakeIdentifier(namespace);
+    List<SnowflakeIdentifier> results = null;
+    switch (scope.type()) {
+      case ROOT:
+        results = snowflakeClient.listDatabases();
+        break;
+      case DATABASE:
+        results = snowflakeClient.listSchemas(scope);
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format(
+                "listNamespaces must be at either ROOT or DATABASE level; got %s from namespace %s",
+                scope, namespace));
+    }
+
+    List<Namespace> namespaceList =
+        results.stream()
+            .map(
+                result -> {
+                  Preconditions.checkState(
+                      result.type() == SnowflakeIdentifier.Type.SCHEMA
+                          || result.type() == SnowflakeIdentifier.Type.DATABASE,
+                      "Got identifier of type %s from listNamespaces for %s",
+                      result.type(),
+                      namespace);
+                  return NamespaceHelpers.toIcebergNamespace(result);
+                })
+            .collect(Collectors.toList());
+    return namespaceList;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace)
+      throws NoSuchNamespaceException {
+    SnowflakeIdentifier id = NamespaceHelpers.toSnowflakeIdentifier(namespace);
+    boolean namespaceExists;
+    switch (id.type()) {
+      case DATABASE:
+        namespaceExists = snowflakeClient.databaseExists(id);
+        break;
+      case SCHEMA:
+        namespaceExists = snowflakeClient.schemaExists(id);
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format(
+                "loadNamespaceMetadat must be at either DATABASE or SCHEMA level; got %s from namespace %s",
+                id, namespace));
+    }
+    if (namespaceExists) {
+      return ImmutableMap.of();
+    } else {
+      throw new NoSuchNamespaceException(
+          "Namespace '%s' with snowflake identifier '%s' doesn't exist", namespace, id);
+    }
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support dropNamespace");
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support setProperties");
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support removeProperties");
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    String fileIOImpl = DEFAULT_FILE_IO_IMPL;
+    if (catalogProperties.containsKey(CatalogProperties.FILE_IO_IMPL)) {
+      fileIOImpl = catalogProperties.get(CatalogProperties.FILE_IO_IMPL);
+    }
+    FileIO fileIO = fileIOFactory.newFileIO(fileIOImpl, catalogProperties, conf);

Review Comment:
   This was suggested by @danielcweeks in https://github.com/apache/iceberg/pull/6428#discussion_r1051657795 - added a comment to make the subtleties more clear. From my perspective it seems like either FileIO will need to adopt late-delegating behavior, or the Catalog interfaces will need to deal with a "meta-FileIO" interface that itself can late-delegate to different FileIO implementations.
   
   For now, a new FIleIO per TableOperation is probably fine as long as all tables live entirely in a single filesystem implementation and all its files share the same URI authority, but we'll need late-delegating behavior to support anything like multi-bucket/multi-filesystem tables.
   
   Comment added:
   
       // Initialize a fresh FileIO for each TableOperations created, because some FileIO
       // implementations such as S3FileIO can become bound to a single S3 bucket. Additionally,
       // FileIO implementations often support only a finite set of one or more URI schemes (i.e.
       // S3FileIO only supports s3/s3a/s3n, and even ResolvingFileIO only supports the combination
       // of schemes registered for S3FileIO and HadoopFileIO). Individual catalogs may need to
       // support tables across different cloud/storage providers with disjoint FileIO implementations.
   



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1056586064


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.io.ResolvingFileIO";
+
+  static class FileIOFactory {
+    public FileIO newFileIO(String impl, Map<String, String> properties, Object hadoopConf) {
+      return CatalogUtil.loadFileIO(impl, properties, hadoopConf);
+    }
+  }
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIOFactory fileIOFactory;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    SnowflakeIdentifier scope = NamespaceHelpers.toSnowflakeIdentifier(namespace);
+    Preconditions.checkArgument(
+        scope.type() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at SCHEMA level; got %s from namespace %s",
+        scope,
+        namespace);
+
+    List<SnowflakeIdentifier> sfTables = snowflakeClient.listIcebergTables(scope);
+
+    return sfTables.stream()
+        .map(NamespaceHelpers::toIcebergTableIdentifier)
+        .collect(Collectors.toList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support dropTable");
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support renameTable");
+  }
+
+  @Override
+  public void initialize(String name, Map<String, String> properties) {
+    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 communication.
+      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);
+
+    initialize(name, new JdbcSnowflakeClient(connectionPool), new FileIOFactory(), properties);
+  }
+
+  /**
+   * Initialize using caller-supplied SnowflakeClient and FileIO.
+   *
+   * @param name The name of the catalog, defaults to "snowflake_catalog"
+   * @param snowflakeClient The client encapsulating network communication with Snowflake
+   * @param fileIOFactory The {@link FileIOFactory} to use to instantiate a new FileIO for each new
+   *     table operation
+   * @param properties The catalog options to use and propagate to dependencies
+   */
+  @SuppressWarnings("checkstyle:HiddenField")
+  void initialize(
+      String name,
+      SnowflakeClient snowflakeClient,
+      FileIOFactory fileIOFactory,
+      Map<String, String> properties) {
+    Preconditions.checkArgument(null != snowflakeClient, "snowflakeClient must be non-null");
+    Preconditions.checkArgument(null != fileIOFactory, "fileIOFactory must be non-null");
+    this.catalogName = name == null ? DEFAULT_CATALOG_NAME : name;
+    this.snowflakeClient = snowflakeClient;
+    this.fileIOFactory = fileIOFactory;
+    this.catalogProperties = properties;
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(snowflakeClient);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (null != closeableGroup) {
+      closeableGroup.close();
+    }
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support createNamespace");
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    SnowflakeIdentifier scope = NamespaceHelpers.toSnowflakeIdentifier(namespace);
+    List<SnowflakeIdentifier> results = null;
+    switch (scope.type()) {
+      case ROOT:
+        results = snowflakeClient.listDatabases();
+        break;
+      case DATABASE:
+        results = snowflakeClient.listSchemas(scope);
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format(
+                "listNamespaces must be at either ROOT or DATABASE level; got %s from namespace %s",
+                scope, namespace));
+    }
+
+    List<Namespace> namespaceList =
+        results.stream()
+            .map(
+                result -> {
+                  Preconditions.checkState(

Review Comment:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1068815596


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeTableMetadata.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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 com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+class SnowflakeTableMetadata {
+  public static final Pattern SNOWFLAKE_AZURE_PATTERN =
+      Pattern.compile("azure://([^/]+)/([^/]+)/(.*)");
+
+  private final String snowflakeMetadataLocation;
+  private final String icebergMetadataLocation;
+  private final String status;
+
+  // Note: Since not all sources will necessarily come from a raw JSON representation, this raw
+  // JSON should only be considered a convenient debugging field. Equality of two
+  // SnowflakeTableMetadata instances should not depend on equality of this field.
+  private final String rawJsonVal;
+
+  SnowflakeTableMetadata(
+      String snowflakeMetadataLocation,
+      String icebergMetadataLocation,
+      String status,
+      String rawJsonVal) {
+    this.snowflakeMetadataLocation = snowflakeMetadataLocation;
+    this.icebergMetadataLocation = icebergMetadataLocation;
+    this.status = status;
+    this.rawJsonVal = rawJsonVal;
+  }
+
+  /** Storage location of table metadata in Snowflake's path syntax. */
+  public String snowflakeMetadataLocation() {
+    return snowflakeMetadataLocation;
+  }
+
+  /** Storage location of table metadata in Iceberg's path syntax. */
+  public String icebergMetadataLocation() {
+    return icebergMetadataLocation;
+  }
+
+  public String getStatus() {
+    return status;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    } else if (!(o instanceof SnowflakeTableMetadata)) {
+      return false;
+    }
+
+    // Only consider parsed fields, not the raw JSON that may or may not be the original source of
+    // this instance.
+    SnowflakeTableMetadata that = (SnowflakeTableMetadata) o;
+    return Objects.equal(this.snowflakeMetadataLocation, that.snowflakeMetadataLocation)
+        && Objects.equal(this.icebergMetadataLocation, that.icebergMetadataLocation)
+        && Objects.equal(this.status, that.status);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(snowflakeMetadataLocation, icebergMetadataLocation, status);
+  }
+
+  @Override
+  public String toString() {
+    return String.format(
+        "snowflakeMetadataLocation: '%s', icebergMetadataLocation: '%s', status: '%s'",
+        snowflakeMetadataLocation, icebergMetadataLocation, status);
+  }
+
+  public String toDebugString() {
+    return String.format("%s, rawJsonVal: %s", toString(), rawJsonVal);
+  }
+
+  /**
+   * Translates from Snowflake's path syntax to Iceberg's path syntax for paths matching known
+   * non-compatible Snowflake paths. Throws IllegalArgumentException if the prefix of the
+   * snowflakeLocation is a known non-compatible path syntax but fails to match the expected path
+   * components for a successful translation.
+   */
+  public static String snowflakeLocationToIcebergLocation(String snowflakeLocation) {

Review Comment:
   Right, in situations where Snowflake handles paths coming from externally produced sources, Snowflake tracks a canonical form of it and indeed handles s3a and s3n.
   
   Here, the conversion is for outbound paths produced by Snowflake, where the "s3://" prefix is already used natively, so only Azure and GCS need translation for "standard" scheme mappings accepted by e.g. HadoopFileSystem.
   
   This does remind me though - I've been meaning to open a discussion to see if anyone has thought about maybe adding some ease-of-use hooks for last-mile automatic basic path translations right before they go to FileIO resolution. For example, someone might want `s3://somebucket` paths to be rewritten to a `viewfs://` base path before letting HadoopFileIO automatically delegate to the right ViewFs impl. Or it could come up in cases where manifest files hold `s3://` paths but someone wants everything to go through a corresponding `dbfs://` mount point. Do you know of any prior discussions along those lines, and would it be worth opening an issue for broader input?



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1070164244


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  private static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  private static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.io.ResolvingFileIO";
+
+  static class FileIOFactory {
+    public FileIO newFileIO(String impl, Map<String, String> properties, Object hadoopConf) {
+      return CatalogUtil.loadFileIO(impl, properties, hadoopConf);

Review Comment:
   This was just extracted to preserve the flow of unittest setup to be able to return the pre-configured InMemoryFileIO fake instance rather than having dynamic classloading get an empty one. The alternative there would've been to introduce some static global state in InMemoryFileIO, but that seems prone to causing cross-test-case problems.
   
   I could add a comment or annotation here to explain its existence if that'd make it cleaner.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1063861471


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeTableMetadata.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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 com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+class SnowflakeTableMetadata {
+  public static final Pattern SNOWFLAKE_AZURE_PATTERN =
+      Pattern.compile("azure://([^/]+)/([^/]+)/(.*)");
+
+  private String snowflakeMetadataLocation;
+  private String icebergMetadataLocation;
+  private String status;
+
+  // Note: Since not all sources will necessarily come from a raw JSON representation, this raw
+  // JSON should only be considered a convenient debugging field. Equality of two
+  // SnowflakeTableMetadata instances should not depend on equality of this field.
+  private String rawJsonVal;

Review Comment:
   This actually isn't used anywhere.  Seems like you might want to either include it in the `toString` or just remove it until there's some use for it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1063872853


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java:
##########
@@ -0,0 +1,386 @@
+/*
+ * 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.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+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.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+/**
+ * This implementation of SnowflakeClient builds on top of Snowflake's JDBC driver to interact with
+ * Snowflake's Iceberg-aware resource model.
+ */
+class JdbcSnowflakeClient implements SnowflakeClient {
+  static final String EXPECTED_JDBC_IMPL = "net.snowflake.client.jdbc.SnowflakeDriver";
+
+  @FunctionalInterface
+  interface ResultSetParser<T> {
+    T parse(ResultSet rs) throws SQLException;
+  }
+
+  /**
+   * This class wraps the basic boilerplate of setting up PreparedStatements and applying a
+   * ResultSetParser to translate a ResultSet into parsed objects. Allows easily injecting
+   * subclasses for debugging/testing purposes.
+   */
+  static class QueryHarness {
+    public <T> T query(Connection conn, String sql, ResultSetParser<T> parser, String... args)
+        throws SQLException {
+      try (PreparedStatement statement = conn.prepareStatement(sql)) {
+        if (args != null) {
+          for (int i = 0; i < args.length; ++i) {
+            statement.setString(i + 1, args[i]);
+          }
+        }
+
+        try (ResultSet rs = statement.executeQuery()) {
+          return parser.parse(rs);
+        }
+      }
+    }
+  }
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Database identifiers,
+   * containing "name" (representing databaseName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> DATABASE_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> databases = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("name");
+          databases.add(SnowflakeIdentifier.ofDatabase(databaseName));
+        }
+        return databases;
+      };
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Schema identifiers,
+   * containing "database_name" and "name" (representing schemaName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> SCHEMA_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> schemas = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("database_name");
+          String schemaName = rs.getString("name");
+          schemas.add(SnowflakeIdentifier.ofSchema(databaseName, schemaName));
+        }
+        return schemas;
+      };
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Table identifiers,
+   * containing "database_name", "schema_name", and "name" (representing tableName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> TABLE_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> tables = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("database_name");
+          String schemaName = rs.getString("schema_name");
+          String tableName = rs.getString("name");
+          tables.add(SnowflakeIdentifier.ofTable(databaseName, schemaName, tableName));
+        }
+        return tables;
+      };
+
+  /**
+   * Expects to handle ResultSets representing a single record holding Snowflake Iceberg metadata.
+   */
+  public static final ResultSetParser<SnowflakeTableMetadata> TABLE_METADATA_RESULT_SET_HANDLER =
+      rs -> {
+        if (!rs.next()) {
+          return null;
+        }
+
+        String rawJsonVal = rs.getString("METADATA");
+        return SnowflakeTableMetadata.parseJson(rawJsonVal);
+      };
+
+  private final JdbcClientPool connectionPool;
+  private QueryHarness queryHarness;
+
+  JdbcSnowflakeClient(JdbcClientPool conn) {
+    Preconditions.checkArgument(null != conn, "JdbcClientPool must be non-null");
+    connectionPool = conn;
+    queryHarness = new QueryHarness();
+  }
+
+  @VisibleForTesting
+  void setQueryHarness(QueryHarness queryHarness) {
+    this.queryHarness = queryHarness;
+  }
+
+  /**
+   * For rare cases where PreparedStatements aren't supported for user-supplied identifiers intended
+   * for use in special LIKE clauses, we can sanitize by "broadening" the identifier with
+   * single-character wildcards and manually post-filter client-side.
+   *
+   * <p>Note: This sanitization approach intentionally "broadens" the scope of matching results;
+   * callers must be able to handle this method returning an all-wildcard expression; i.e. the
+   * caller must treat the usage of the LIKE clause as only an optional optimization, and should
+   * post-filter for correctness as if the LIKE clause wasn't present in the query at all.
+   */
+  @VisibleForTesting
+  String sanitizeIdentifierWithWildcardForLikeClause(String identifier) {
+    // Restrict identifiers to the "Unquoted object identifiers" synax documented at
+    // https://docs.snowflake.com/en/sql-reference/identifiers-syntax.html
+    //
+    // Use a strict allowlist of characters, replace everything *not* matching the character set
+    // with "_", which is used as a single-character wildcard in Snowflake.
+    String sanitized = identifier.replaceAll("[^a-zA-Z0-9_$]", "_");
+    if (sanitized.startsWith("$")) {
+      sanitized = "_" + sanitized.substring(1);
+    }
+    return sanitized;
+  }
+
+  @Override
+  public boolean databaseExists(SnowflakeIdentifier database) {
+    Preconditions.checkArgument(
+        database.type() == SnowflakeIdentifier.Type.DATABASE,
+        "databaseExists requires a DATABASE identifier, got '%s'",
+        database);
+
+    // Due to current limitations in PreparedStatement parameters for the LIKE clause in
+    // SHOW DATABASES queries, we'll use a fairly limited allowlist for identifier characters,
+    // using wildcards for non-allowed characters, and post-filter for matching.
+    final String finalQuery =
+        String.format(
+            "SHOW DATABASES LIKE '%s' IN ACCOUNT",
+            sanitizeIdentifierWithWildcardForLikeClause(database.databaseName()));
+    List<SnowflakeIdentifier> databases;
+    try {
+      databases =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn, finalQuery, DATABASE_RESULT_SET_HANDLER, (String[]) null));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to check if database '%s' exists", database);
+    } catch (InterruptedException e) {
+      throw new UncheckedInterruptedException(
+          e, "Interrupted while checking if database '%s' exists", database);
+    }
+
+    // Filter to handle the edge case of '_' appearing as a wildcard that can't be remapped the way
+    // it can for predicates in SELECT statements.
+    databases.removeIf(db -> !database.databaseName().equalsIgnoreCase(db.databaseName()));
+    return !databases.isEmpty();
+  }
+
+  @Override
+  public boolean schemaExists(SnowflakeIdentifier schema) {
+    Preconditions.checkArgument(
+        schema.type() == SnowflakeIdentifier.Type.SCHEMA,
+        "schemaExists requires a SCHEMA identifier, got '%s'",
+        schema);
+
+    if (!databaseExists(SnowflakeIdentifier.ofDatabase(schema.databaseName()))) {
+      return false;
+    }
+
+    // Due to current limitations in PreparedStatement parameters for the LIKE clause in
+    // SHOW SCHEMAS queries, we'll use a fairly limited allowlist for identifier characters,
+    // using wildcards for non-allowed characters, and post-filter for matching.
+    final String finalQuery =
+        String.format(
+            "SHOW SCHEMAS LIKE '%s' IN DATABASE IDENTIFIER(?)",
+            sanitizeIdentifierWithWildcardForLikeClause(schema.schemaName()));
+    List<SnowflakeIdentifier> schemas;
+    try {
+      schemas =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn,
+                      finalQuery,
+                      SCHEMA_RESULT_SET_HANDLER,
+                      new String[] {schema.databaseName()}));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to check if schema '%s' exists", schema);
+    } catch (InterruptedException e) {
+      throw new UncheckedInterruptedException(
+          e, "Interrupted while checking if schema '%s' exists", schema);
+    }
+
+    // Filter to handle the edge case of '_' appearing as a wildcard that can't be remapped the way
+    // it can for predicates in SELECT statements.
+    schemas.removeIf(sc -> !schema.schemaName().equalsIgnoreCase(sc.schemaName()));
+    return !schemas.isEmpty();
+  }
+
+  @Override
+  public List<SnowflakeIdentifier> listDatabases() {
+    List<SnowflakeIdentifier> databases;
+    try {
+      databases =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn,
+                      "SHOW DATABASES IN ACCOUNT",
+                      DATABASE_RESULT_SET_HANDLER,
+                      (String[]) null));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to list databases");
+    } catch (InterruptedException e) {
+      throw new UncheckedInterruptedException(e, "Interrupted while listing databases");
+    }
+    databases.forEach(
+        db ->
+            Preconditions.checkState(
+                db.type() == SnowflakeIdentifier.Type.DATABASE,
+                "Expected DATABASE, got identifier '%s'",
+                db));
+    return databases;
+  }
+
+  @Override
+  public List<SnowflakeIdentifier> listSchemas(SnowflakeIdentifier scope) {
+    StringBuilder baseQuery = new StringBuilder("SHOW SCHEMAS");
+    String[] queryParams = null;
+    switch (scope.type()) {
+      case ROOT:
+        // account-level listing
+        baseQuery.append(" IN ACCOUNT");
+        break;
+      case DATABASE:
+        // database-level listing
+        baseQuery.append(" IN DATABASE IDENTIFIER(?)");
+        queryParams = new String[] {scope.toIdentifierString()};
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format("Unsupported scope type for listSchemas: %s", scope));
+    }
+
+    final String finalQuery = baseQuery.toString();
+    final String[] finalQueryParams = queryParams;
+    List<SnowflakeIdentifier> schemas;
+    try {
+      schemas =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn, finalQuery, SCHEMA_RESULT_SET_HANDLER, finalQueryParams));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to list schemas for scope '%s'", scope);
+    } catch (InterruptedException e) {
+      throw new UncheckedInterruptedException(
+          e, "Interrupted while listing schemas for scope '%s'", scope);
+    }
+    schemas.forEach(
+        schema ->
+            Preconditions.checkState(
+                schema.type() == SnowflakeIdentifier.Type.SCHEMA,
+                "Expected SCHEMA, got identifier '%s' for scope '%s'",
+                schema,
+                scope));
+    return schemas;
+  }
+
+  @Override
+  public List<SnowflakeIdentifier> listIcebergTables(SnowflakeIdentifier scope) {
+    StringBuilder baseQuery = new StringBuilder("SHOW ICEBERG TABLES");

Review Comment:
   @dennishuo I tried running this catalog implementation with Spark and it appears that this statement doesn't appear to show external iceberg tables for me.  I can run:
   
   ```
   SHOW ICEBERG TABLES IN SCHEMA <DB>.<SCHEMA>;
   ```
   
   And there are now results, but running:
   
   ```
   SHOW TABLES IN SCHEMA <DB>.<SCHEMA>;
   ```
   
   Correctly shows the Iceberg tables and I can see their storage and metadata locations.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1063858154


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java:
##########
@@ -0,0 +1,386 @@
+/*
+ * 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.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+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.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+/**
+ * This implementation of SnowflakeClient builds on top of Snowflake's JDBC driver to interact with
+ * Snowflake's Iceberg-aware resource model.
+ */
+class JdbcSnowflakeClient implements SnowflakeClient {
+  static final String EXPECTED_JDBC_IMPL = "net.snowflake.client.jdbc.SnowflakeDriver";
+
+  @FunctionalInterface
+  interface ResultSetParser<T> {
+    T parse(ResultSet rs) throws SQLException;
+  }
+
+  /**
+   * This class wraps the basic boilerplate of setting up PreparedStatements and applying a
+   * ResultSetParser to translate a ResultSet into parsed objects. Allows easily injecting
+   * subclasses for debugging/testing purposes.
+   */
+  static class QueryHarness {
+    public <T> T query(Connection conn, String sql, ResultSetParser<T> parser, String... args)
+        throws SQLException {
+      try (PreparedStatement statement = conn.prepareStatement(sql)) {
+        if (args != null) {
+          for (int i = 0; i < args.length; ++i) {
+            statement.setString(i + 1, args[i]);
+          }
+        }
+
+        try (ResultSet rs = statement.executeQuery()) {
+          return parser.parse(rs);
+        }
+      }
+    }
+  }
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Database identifiers,
+   * containing "name" (representing databaseName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> DATABASE_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> databases = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("name");
+          databases.add(SnowflakeIdentifier.ofDatabase(databaseName));
+        }
+        return databases;
+      };
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Schema identifiers,
+   * containing "database_name" and "name" (representing schemaName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> SCHEMA_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> schemas = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("database_name");
+          String schemaName = rs.getString("name");
+          schemas.add(SnowflakeIdentifier.ofSchema(databaseName, schemaName));
+        }
+        return schemas;
+      };
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Table identifiers,
+   * containing "database_name", "schema_name", and "name" (representing tableName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> TABLE_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> tables = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("database_name");
+          String schemaName = rs.getString("schema_name");
+          String tableName = rs.getString("name");
+          tables.add(SnowflakeIdentifier.ofTable(databaseName, schemaName, tableName));
+        }
+        return tables;
+      };
+
+  /**
+   * Expects to handle ResultSets representing a single record holding Snowflake Iceberg metadata.
+   */
+  public static final ResultSetParser<SnowflakeTableMetadata> TABLE_METADATA_RESULT_SET_HANDLER =
+      rs -> {
+        if (!rs.next()) {
+          return null;
+        }
+
+        String rawJsonVal = rs.getString("METADATA");
+        return SnowflakeTableMetadata.parseJson(rawJsonVal);
+      };
+
+  private final JdbcClientPool connectionPool;
+  private QueryHarness queryHarness;
+
+  JdbcSnowflakeClient(JdbcClientPool conn) {
+    Preconditions.checkArgument(null != conn, "JdbcClientPool must be non-null");
+    connectionPool = conn;
+    queryHarness = new QueryHarness();
+  }
+
+  @VisibleForTesting
+  void setQueryHarness(QueryHarness queryHarness) {
+    this.queryHarness = queryHarness;
+  }
+
+  /**
+   * For rare cases where PreparedStatements aren't supported for user-supplied identifiers intended
+   * for use in special LIKE clauses, we can sanitize by "broadening" the identifier with
+   * single-character wildcards and manually post-filter client-side.
+   *
+   * <p>Note: This sanitization approach intentionally "broadens" the scope of matching results;
+   * callers must be able to handle this method returning an all-wildcard expression; i.e. the
+   * caller must treat the usage of the LIKE clause as only an optional optimization, and should
+   * post-filter for correctness as if the LIKE clause wasn't present in the query at all.
+   */
+  @VisibleForTesting
+  String sanitizeIdentifierWithWildcardForLikeClause(String identifier) {
+    // Restrict identifiers to the "Unquoted object identifiers" synax documented at
+    // https://docs.snowflake.com/en/sql-reference/identifiers-syntax.html
+    //
+    // Use a strict allowlist of characters, replace everything *not* matching the character set
+    // with "_", which is used as a single-character wildcard in Snowflake.
+    String sanitized = identifier.replaceAll("[^a-zA-Z0-9_$]", "_");
+    if (sanitized.startsWith("$")) {
+      sanitized = "_" + sanitized.substring(1);
+    }
+    return sanitized;
+  }
+
+  @Override
+  public boolean databaseExists(SnowflakeIdentifier database) {
+    Preconditions.checkArgument(
+        database.type() == SnowflakeIdentifier.Type.DATABASE,
+        "databaseExists requires a DATABASE identifier, got '%s'",
+        database);
+
+    // Due to current limitations in PreparedStatement parameters for the LIKE clause in
+    // SHOW DATABASES queries, we'll use a fairly limited allowlist for identifier characters,
+    // using wildcards for non-allowed characters, and post-filter for matching.
+    final String finalQuery =
+        String.format(
+            "SHOW DATABASES LIKE '%s' IN ACCOUNT",
+            sanitizeIdentifierWithWildcardForLikeClause(database.databaseName()));
+    List<SnowflakeIdentifier> databases;
+    try {
+      databases =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn, finalQuery, DATABASE_RESULT_SET_HANDLER, (String[]) null));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to check if database '%s' exists", database);
+    } catch (InterruptedException e) {
+      throw new UncheckedInterruptedException(
+          e, "Interrupted while checking if database '%s' exists", database);
+    }
+
+    // Filter to handle the edge case of '_' appearing as a wildcard that can't be remapped the way
+    // it can for predicates in SELECT statements.
+    databases.removeIf(db -> !database.databaseName().equalsIgnoreCase(db.databaseName()));
+    return !databases.isEmpty();
+  }
+
+  @Override
+  public boolean schemaExists(SnowflakeIdentifier schema) {
+    Preconditions.checkArgument(
+        schema.type() == SnowflakeIdentifier.Type.SCHEMA,
+        "schemaExists requires a SCHEMA identifier, got '%s'",
+        schema);
+
+    if (!databaseExists(SnowflakeIdentifier.ofDatabase(schema.databaseName()))) {
+      return false;
+    }
+
+    // Due to current limitations in PreparedStatement parameters for the LIKE clause in
+    // SHOW SCHEMAS queries, we'll use a fairly limited allowlist for identifier characters,
+    // using wildcards for non-allowed characters, and post-filter for matching.
+    final String finalQuery =
+        String.format(
+            "SHOW SCHEMAS LIKE '%s' IN DATABASE IDENTIFIER(?)",
+            sanitizeIdentifierWithWildcardForLikeClause(schema.schemaName()));
+    List<SnowflakeIdentifier> schemas;
+    try {
+      schemas =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn,
+                      finalQuery,
+                      SCHEMA_RESULT_SET_HANDLER,
+                      new String[] {schema.databaseName()}));

Review Comment:
   You shouldn't need to wrap this in an array since it's varargs.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] jackye1995 commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1068788303


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeTableMetadata.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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 com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+class SnowflakeTableMetadata {
+  public static final Pattern SNOWFLAKE_AZURE_PATTERN =
+      Pattern.compile("azure://([^/]+)/([^/]+)/(.*)");
+
+  private final String snowflakeMetadataLocation;
+  private final String icebergMetadataLocation;
+  private final String status;
+
+  // Note: Since not all sources will necessarily come from a raw JSON representation, this raw
+  // JSON should only be considered a convenient debugging field. Equality of two
+  // SnowflakeTableMetadata instances should not depend on equality of this field.
+  private final String rawJsonVal;
+
+  SnowflakeTableMetadata(
+      String snowflakeMetadataLocation,
+      String icebergMetadataLocation,
+      String status,
+      String rawJsonVal) {
+    this.snowflakeMetadataLocation = snowflakeMetadataLocation;
+    this.icebergMetadataLocation = icebergMetadataLocation;
+    this.status = status;
+    this.rawJsonVal = rawJsonVal;
+  }
+
+  /** Storage location of table metadata in Snowflake's path syntax. */
+  public String snowflakeMetadataLocation() {
+    return snowflakeMetadataLocation;
+  }
+
+  /** Storage location of table metadata in Iceberg's path syntax. */
+  public String icebergMetadataLocation() {
+    return icebergMetadataLocation;
+  }
+
+  public String getStatus() {
+    return status;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    } else if (!(o instanceof SnowflakeTableMetadata)) {
+      return false;
+    }
+
+    // Only consider parsed fields, not the raw JSON that may or may not be the original source of
+    // this instance.
+    SnowflakeTableMetadata that = (SnowflakeTableMetadata) o;
+    return Objects.equal(this.snowflakeMetadataLocation, that.snowflakeMetadataLocation)
+        && Objects.equal(this.icebergMetadataLocation, that.icebergMetadataLocation)
+        && Objects.equal(this.status, that.status);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(snowflakeMetadataLocation, icebergMetadataLocation, status);
+  }
+
+  @Override
+  public String toString() {
+    return String.format(
+        "snowflakeMetadataLocation: '%s', icebergMetadataLocation: '%s', status: '%s'",
+        snowflakeMetadataLocation, icebergMetadataLocation, status);
+  }
+
+  public String toDebugString() {
+    return String.format("%s, rawJsonVal: %s", toString(), rawJsonVal);
+  }
+
+  /**
+   * Translates from Snowflake's path syntax to Iceberg's path syntax for paths matching known
+   * non-compatible Snowflake paths. Throws IllegalArgumentException if the prefix of the
+   * snowflakeLocation is a known non-compatible path syntax but fails to match the expected path
+   * components for a successful translation.
+   */
+  public static String snowflakeLocationToIcebergLocation(String snowflakeLocation) {

Review Comment:
   I see Azure and GCP paths converted, what about the S3 ones? Are all combinations like s3a, s3n, etc. all compatible natively in Snowflake?



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1054089032


##########
spark/v3.1/build.gradle:
##########
@@ -213,6 +213,9 @@ project(':iceberg-spark:iceberg-spark-runtime-3.1_2.12') {
     implementation(project(':iceberg-nessie')) {
       exclude group: 'com.google.code.findbugs', module: 'jsr305'
     }
+    implementation (project(':iceberg-snowflake')) {
+      exclude group: 'net.snowflake' , module: 'snowflake-jdbc'

Review Comment:
   Now that we've removed dbutils entirely, I've confirmed that only the newly added classes are the sources added in this PR.
   
       [dhuo@SDP_DevVM-dhuo ~/Snowflake-Labs/iceberg]$ ls -l ./spark/v3.*/spark-runtime/build/libs/*dirty.jar
       -rw-rw-r-- 1 dhuo dhuo 26500156 Dec 20 23:53 ./spark/v3.1/spark-runtime/build/libs/iceberg-spark-runtime-3.1_2.12-b3a2842.dirty.jar
       -rw-rw-r-- 1 dhuo dhuo 26968503 Dec 20 23:53 ./spark/v3.2/spark-runtime/build/libs/iceberg-spark-runtime-3.2_2.12-b3a2842.dirty.jar
       -rw-rw-r-- 1 dhuo dhuo 27008357 Dec 20 23:53 ./spark/v3.3/spark-runtime/build/libs/iceberg-spark-runtime-3.3_2.12-b3a2842.dirty.jar
       [dhuo@SDP_DevVM-dhuo ~/Snowflake-Labs/iceberg]$ jar tf ./spark/v3.1/spark-runtime/build/libs/iceberg-spark-runtime-3.1_2.12-b3a2842.dirty.jar > sparkclasses-3.1.before.txt
       [dhuo@SDP_DevVM-dhuo ~/Snowflake-Labs/iceberg]$ jar tf ./spark/v3.2/spark-runtime/build/libs/iceberg-spark-runtime-3.2_2.12-b3a2842.dirty.jar > sparkclasses-3.2.before.txt
       [dhuo@SDP_DevVM-dhuo ~/Snowflake-Labs/iceberg]$ jar tf ./spark/v3.3/spark-runtime/build/libs/iceberg-spark-runtime-3.3_2.12-b3a2842.dirty.jar > sparkclasses-3.3.before.txt
       [dhuo@SDP_DevVM-dhuo ~/Snowflake-Labs/iceberg]$ ls -l ./spark/v3.*/spark-runtime/build/libs/*dirty.jar
       -rw-rw-r-- 1 dhuo dhuo 26524572 Dec 21 00:05 ./spark/v3.1/spark-runtime/build/libs/iceberg-spark-runtime-3.1_2.12-676d024.dirty.jar
       -rw-rw-r-- 1 dhuo dhuo 26992919 Dec 21 00:05 ./spark/v3.2/spark-runtime/build/libs/iceberg-spark-runtime-3.2_2.12-676d024.dirty.jar
       -rw-rw-r-- 1 dhuo dhuo 27032773 Dec 21 00:05 ./spark/v3.3/spark-runtime/build/libs/iceberg-spark-runtime-3.3_2.12-676d024.dirty.jar
       [dhuo@SDP_DevVM-dhuo ~/Snowflake-Labs/iceberg]$ jar tf ./spark/v3.1/spark-runtime/build/libs/iceberg-spark-runtime-3.1_2.12-676d024.dirty.jar > sparkclasses-3.1.after.txt
       [dhuo@SDP_DevVM-dhuo ~/Snowflake-Labs/iceberg]$ jar tf ./spark/v3.2/spark-runtime/build/libs/iceberg-spark-runtime-3.2_2.12-676d024.dirty.jar > sparkclasses-3.2.after.txt
       [dhuo@SDP_DevVM-dhuo ~/Snowflake-Labs/iceberg]$ jar tf ./spark/v3.3/spark-runtime/build/libs/iceberg-spark-runtime-3.3_2.12-676d024.dirty.jar > sparkclasses-3.3.after.txt
       [dhuo@SDP_DevVM-dhuo ~/Snowflake-Labs/iceberg]$ diff sparkclasses-3.1.before.txt sparkclasses-3.1.after.txt
       421a422,437
       > org/apache/iceberg/snowflake/
       > org/apache/iceberg/snowflake/JdbcSnowflakeClient$1.class
       > org/apache/iceberg/snowflake/JdbcSnowflakeClient$QueryHarness.class
       > org/apache/iceberg/snowflake/JdbcSnowflakeClient$ResultSetParser.class
       > org/apache/iceberg/snowflake/JdbcSnowflakeClient.class
       > org/apache/iceberg/snowflake/NamespaceHelpers$1.class
       > org/apache/iceberg/snowflake/NamespaceHelpers.class
       > org/apache/iceberg/snowflake/SnowflakeCatalog$1.class
       > org/apache/iceberg/snowflake/SnowflakeCatalog$FileIOFactory.class
       > org/apache/iceberg/snowflake/SnowflakeCatalog.class
       > org/apache/iceberg/snowflake/SnowflakeClient.class
       > org/apache/iceberg/snowflake/SnowflakeIdentifier$1.class
       > org/apache/iceberg/snowflake/SnowflakeIdentifier$Type.class
       > org/apache/iceberg/snowflake/SnowflakeIdentifier.class
       > org/apache/iceberg/snowflake/SnowflakeTableMetadata.class
       > org/apache/iceberg/snowflake/SnowflakeTableOperations.class
       [dhuo@SDP_DevVM-dhuo ~/Snowflake-Labs/iceberg]$ diff sparkclasses-3.2.before.txt sparkclasses-3.2.after.txt
       486a487,502
       > org/apache/iceberg/snowflake/
       > org/apache/iceberg/snowflake/JdbcSnowflakeClient$1.class
       > org/apache/iceberg/snowflake/JdbcSnowflakeClient$QueryHarness.class
       > org/apache/iceberg/snowflake/JdbcSnowflakeClient$ResultSetParser.class
       > org/apache/iceberg/snowflake/JdbcSnowflakeClient.class
       > org/apache/iceberg/snowflake/NamespaceHelpers$1.class
       > org/apache/iceberg/snowflake/NamespaceHelpers.class
       > org/apache/iceberg/snowflake/SnowflakeCatalog$1.class
       > org/apache/iceberg/snowflake/SnowflakeCatalog$FileIOFactory.class
       > org/apache/iceberg/snowflake/SnowflakeCatalog.class
       > org/apache/iceberg/snowflake/SnowflakeClient.class
       > org/apache/iceberg/snowflake/SnowflakeIdentifier$1.class
       > org/apache/iceberg/snowflake/SnowflakeIdentifier$Type.class
       > org/apache/iceberg/snowflake/SnowflakeIdentifier.class
       > org/apache/iceberg/snowflake/SnowflakeTableMetadata.class
       > org/apache/iceberg/snowflake/SnowflakeTableOperations.class
       [dhuo@SDP_DevVM-dhuo ~/Snowflake-Labs/iceberg]$ diff sparkclasses-3.3.before.txt sparkclasses-3.3.after.txt
       498a499,514
       > org/apache/iceberg/snowflake/
       > org/apache/iceberg/snowflake/JdbcSnowflakeClient$1.class
       > org/apache/iceberg/snowflake/JdbcSnowflakeClient$QueryHarness.class
       > org/apache/iceberg/snowflake/JdbcSnowflakeClient$ResultSetParser.class
       > org/apache/iceberg/snowflake/JdbcSnowflakeClient.class
       > org/apache/iceberg/snowflake/NamespaceHelpers$1.class
       > org/apache/iceberg/snowflake/NamespaceHelpers.class
       > org/apache/iceberg/snowflake/SnowflakeCatalog$1.class
       > org/apache/iceberg/snowflake/SnowflakeCatalog$FileIOFactory.class
       > org/apache/iceberg/snowflake/SnowflakeCatalog.class
       > org/apache/iceberg/snowflake/SnowflakeClient.class
       > org/apache/iceberg/snowflake/SnowflakeIdentifier$1.class
       > org/apache/iceberg/snowflake/SnowflakeIdentifier$Type.class
       > org/apache/iceberg/snowflake/SnowflakeIdentifier.class
       > org/apache/iceberg/snowflake/SnowflakeTableMetadata.class
       > org/apache/iceberg/snowflake/SnowflakeTableOperations.class
   



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051660823


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.hadoop.HadoopFileIO";
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    LOG.debug("listTables with namespace: {}", namespace);
+    SnowflakeIdentifier scope = NamespaceHelpers.getSnowflakeIdentifierForNamespace(namespace);
+    Preconditions.checkArgument(
+        scope.getType() == SnowflakeIdentifier.Type.ROOT
+            || scope.getType() == SnowflakeIdentifier.Type.DATABASE
+            || scope.getType() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at ROOT, DATABASE, or SCHEMA level; got %s from namespace %s",
+        scope,
+        namespace);
+
+    List<SnowflakeIdentifier> sfTables = snowflakeClient.listIcebergTables(scope);

Review Comment:
   I believe this behavior deviates from other implementations.  This appears to be listing everything under the current level rather than listing the current level.  I believe list tables should only return results for the snowflake schema (not database or root).



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051662857


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.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.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+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 {

Review Comment:
   I don't think this needs to be public



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051663154


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/entities/SnowflakeIdentifier.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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;

Review Comment:
   You might want to drop the `entities` package as it doesn't provide a lot of logical separation and results in having to expose some of these internals as `public`, which they really don't need to be.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1052715267


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.hadoop.HadoopFileIO";
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    LOG.debug("listTables with namespace: {}", namespace);
+    SnowflakeIdentifier scope = NamespaceHelpers.getSnowflakeIdentifierForNamespace(namespace);
+    Preconditions.checkArgument(
+        scope.getType() == SnowflakeIdentifier.Type.ROOT
+            || scope.getType() == SnowflakeIdentifier.Type.DATABASE
+            || scope.getType() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at ROOT, DATABASE, or SCHEMA level; got %s from namespace %s",
+        scope,
+        namespace);
+
+    List<SnowflakeIdentifier> sfTables = snowflakeClient.listIcebergTables(scope);
+
+    return sfTables.stream()
+        .map(
+            table ->
+                TableIdentifier.of(
+                    table.getDatabaseName(), table.getSchemaName(), table.getTableName()))
+        .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) {
+    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);
+
+    String fileIOImpl = DEFAULT_FILE_IO_IMPL;
+    if (properties.containsKey(CatalogProperties.FILE_IO_IMPL)) {
+      fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL);
+    }
+
+    initialize(
+        name,
+        new JdbcSnowflakeClient(connectionPool),
+        CatalogUtil.loadFileIO(fileIOImpl, properties, conf),
+        properties);
+  }
+
+  /**
+   * Initialize using caller-supplied SnowflakeClient and FileIO.
+   *
+   * @param name The name of the catalog, defaults to "snowflake_catalog"
+   * @param snowflakeClient The client encapsulating network communication with Snowflake
+   * @param fileIO The {@link FileIO} to use for table operations
+   * @param properties The catalog options to use and propagate to dependencies
+   */
+  @SuppressWarnings("checkstyle:HiddenField")
+  public void initialize(

Review Comment:
   Changed to package-private



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1052725312


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.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.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+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 {

Review Comment:
   Changed to package-private



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1052729488


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/entities/SnowflakeIdentifier.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class SnowflakeIdentifier {
+  public enum Type {
+    ROOT,
+    DATABASE,
+    SCHEMA,
+    TABLE
+  }
+
+  private String databaseName;
+  private String schemaName;
+  private String tableName;
+
+  protected SnowflakeIdentifier(String databaseName, String schemaName, String tableName) {
+    this.databaseName = databaseName;
+    this.schemaName = schemaName;
+    this.tableName = tableName;
+  }
+
+  public static SnowflakeIdentifier ofRoot() {
+    return new SnowflakeIdentifier(null, null, null);
+  }
+
+  public static SnowflakeIdentifier ofDatabase(String databaseName) {
+    Preconditions.checkArgument(null != databaseName, "databaseName must be non-null");
+    return new SnowflakeIdentifier(databaseName, null, null);
+  }
+
+  public static SnowflakeIdentifier ofSchema(String databaseName, String schemaName) {
+    Preconditions.checkArgument(null != databaseName, "databaseName must be non-null");
+    Preconditions.checkArgument(null != schemaName, "schemaName must be non-null");
+    return new SnowflakeIdentifier(databaseName, schemaName, null);
+  }
+
+  public static SnowflakeIdentifier ofTable(
+      String databaseName, String schemaName, String tableName) {
+    Preconditions.checkArgument(null != databaseName, "databaseName must be non-null");
+    Preconditions.checkArgument(null != schemaName, "schemaName must be non-null");
+    Preconditions.checkArgument(null != tableName, "tableName must be non-null");
+    return new SnowflakeIdentifier(databaseName, schemaName, tableName);
+  }
+
+  /**
+   * If type is TABLE, expect non-null databaseName, schemaName, and tableName. If type is SCHEMA,
+   * expect non-null databaseName and schemaName. If type is DATABASE, expect non-null databaseName.
+   * If type is ROOT, expect all of databaseName, schemaName, and tableName to be null.
+   */
+  public Type getType() {

Review Comment:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1052732312


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/NamespaceHelpers.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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 org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class NamespaceHelpers {
+  private static final int MAX_NAMESPACE_DEPTH = 2;
+  private static final int NAMESPACE_ROOT_LEVEL = 0;
+  private static final int NAMESPACE_DB_LEVEL = 1;
+  private static final int NAMESPACE_SCHEMA_LEVEL = 2;
+
+  private static final Logger LOG = LoggerFactory.getLogger(NamespaceHelpers.class);
+
+  /**
+   * Converts a Namespace into a SnowflakeIdentifier representing ROOT, a DATABASE, or a SCHEMA.
+   *
+   * @throws IllegalArgumentException if the namespace is not a supported depth.
+   */
+  public static SnowflakeIdentifier getSnowflakeIdentifierForNamespace(Namespace namespace) {

Review Comment:
   Done. Simplified to `toSnowflakeIdentifier` here and below.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051680606


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/entities/SnowflakeTableMetadata.java:
##########
@@ -0,0 +1,158 @@
+/*
+ * 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 com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.commons.dbutils.ResultSetHandler;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+public class SnowflakeTableMetadata {
+  public static final Pattern SNOWFLAKE_AZURE_PATTERN =
+      Pattern.compile("azure://([^/]+)/([^/]+)/(.*)");
+
+  private String snowflakeMetadataLocation;
+  private String icebergMetadataLocation;
+  private String status;
+
+  // Note: Since not all sources will necessarily come from a raw JSON representation, this raw
+  // JSON should only be considered a convenient debugging field. Equality of two
+  // SnowflakeTableMetadata instances should not depend on equality of this field.
+  private String rawJsonVal;
+
+  public SnowflakeTableMetadata(
+      String snowflakeMetadataLocation,
+      String icebergMetadataLocation,
+      String status,
+      String rawJsonVal) {
+    this.snowflakeMetadataLocation = snowflakeMetadataLocation;
+    this.icebergMetadataLocation = icebergMetadataLocation;
+    this.status = status;
+    this.rawJsonVal = rawJsonVal;
+  }
+
+  /** Storage location of table metadata in Snowflake's path syntax. */
+  public String getSnowflakeMetadataLocation() {
+    return snowflakeMetadataLocation;
+  }
+
+  /** Storage location of table metadata in Iceberg's path syntax. */
+  public String getIcebergMetadataLocation() {
+    return icebergMetadataLocation;
+  }
+
+  public String getStatus() {
+    return status;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    } else if (!(o instanceof SnowflakeTableMetadata)) {
+      return false;
+    }
+
+    // Only consider parsed fields, not the raw JSON that may or may not be the original source of
+    // this instance.
+    SnowflakeTableMetadata that = (SnowflakeTableMetadata) o;
+    return Objects.equal(this.snowflakeMetadataLocation, that.snowflakeMetadataLocation)
+        && Objects.equal(this.icebergMetadataLocation, that.icebergMetadataLocation)
+        && Objects.equal(this.status, that.status);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(snowflakeMetadataLocation, icebergMetadataLocation, status);
+  }
+
+  @Override
+  public String toString() {
+    return String.format(
+        "snowflakeMetadataLocation: '%s', icebergMetadataLocation: '%s', status: '%s",
+        snowflakeMetadataLocation, icebergMetadataLocation, status);
+  }
+
+  /**
+   * Translates from Snowflake's path syntax to Iceberg's path syntax for paths matching known
+   * non-compatible Snowflake paths. Throws IllegalArgumentException if the prefix of the
+   * snowflakeLocation is a known non-compatible path syntax but fails to match the expected path
+   * components for a successful translation.
+   */
+  public static String getIcebergLocationFromSnowflakeLocation(String snowflakeLocation) {
+    if (snowflakeLocation.startsWith("azure://")) {
+      // Convert from expected path of the form:
+      // azure://account.blob.core.windows.net/container/volumepath
+      // to:
+      // wasbs://container@account.blob.core.windows.net/volumepath
+      Matcher matcher = SNOWFLAKE_AZURE_PATTERN.matcher(snowflakeLocation);
+      Preconditions.checkArgument(
+          matcher.matches(),
+          "Location '%s' failed to match pattern '%s'",
+          snowflakeLocation,
+          SNOWFLAKE_AZURE_PATTERN);
+      return String.format(
+          "wasbs://%s@%s/%s", matcher.group(2), matcher.group(1), matcher.group(3));
+    } else if (snowflakeLocation.startsWith("gcs://")) {
+      // Convert from expected path of the form:
+      // gcs://bucket/path
+      // to:
+      // gs://bucket/path
+      return "gs" + snowflakeLocation.substring(3);
+    }
+    return snowflakeLocation;

Review Comment:
   Style: there should be an empty line between control flow blocks and the following statement.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051678623


##########
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:
   Yeah, this is the correct behavior. It's a little weird, but nice to avoid Hadoop requirements.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051682372


##########
versions.props:
##########
@@ -28,6 +28,8 @@ org.scala-lang.modules:scala-collection-compat_2.12 = 2.6.0
 org.scala-lang.modules:scala-collection-compat_2.13 = 2.6.0
 com.emc.ecs:object-client-bundle = 3.3.2
 org.immutables:value = 2.9.2
+commons-dbutils:commons-dbutils = 1.7

Review Comment:
   Iceberg avoids runtime dependencies on Apache Commons. What is this used for? Can we remove it easily?



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1052896404


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeClient.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.snowflake;
+
+import java.io.Closeable;
+import java.util.List;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+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 {
+  /**
+   * Lists all Snowflake schemas within a given scope. Returned SnowflakeIdentifiers must have
+   * getType() == SnowflakeIdentifier.Type.SCHEMA.
+   *
+   * @param scope The scope in which to list, which may be ROOT or a single DATABASE.
+   */
+  List<SnowflakeIdentifier> listSchemas(SnowflakeIdentifier scope);
+
+  /**
+   * Lists all Snowflake Iceberg tables within a given scope. Returned SnowflakeIdentifiers must
+   * have getType() == SnowflakeIdentifier.Type.TABLE.
+   *
+   * @param scope The scope in which to list, which may be ROOT, a DATABASE, or a SCHEMA.
+   */
+  List<SnowflakeIdentifier> listIcebergTables(SnowflakeIdentifier scope);
+
+  /**
+   * Returns Snowflake-level metadata containing locations to more detailed metadata.
+   *
+   * @param tableIdentifier The fully-qualified identifier that must be of type
+   *     SnowflakeIdentifier.Type.TABLE.
+   */
+  SnowflakeTableMetadata getTableMetadata(SnowflakeIdentifier tableIdentifier);

Review Comment:
   Changed to `loadTableMetadata`



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#issuecomment-1360990536

   Thanks @rdblue @danielcweeks and @nastra for the continued reviews! Should be ready for re-review now; with the latest refactor I've also confirmed that no additional dependencies end up in the spark-runtime jars other than the classes directly implemented in this PR. Please let me know if I missed any comments.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1056616842


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java:
##########
@@ -0,0 +1,362 @@
+/*
+ * 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.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+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.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+/**
+ * This implementation of SnowflakeClient builds on top of Snowflake's JDBC driver to interact with
+ * Snowflake's Iceberg-aware resource model.
+ */
+class JdbcSnowflakeClient implements SnowflakeClient {
+  public static final String EXPECTED_JDBC_IMPL = "net.snowflake.client.jdbc.SnowflakeDriver";

Review Comment:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#issuecomment-1376346320

   @dennishuo  #6538 is merged, so you might want to rebase on top of that.  I'm good, but I it would be great to have @nastra sign off as well since there are couple comments open.


-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] nastra commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1065914096


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeTableOperations.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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 org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class SnowflakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeTableOperations.class);
+
+  private final FileIO fileIO;
+  private final TableIdentifier tableIdentifier;
+  private final SnowflakeIdentifier snowflakeIdentifierForTable;
+  private final String fullTableName;
+
+  private final SnowflakeClient snowflakeClient;
+
+  protected SnowflakeTableOperations(
+      SnowflakeClient snowflakeClient,
+      FileIO fileIO,
+      String catalogName,
+      TableIdentifier tableIdentifier) {
+    this.snowflakeClient = snowflakeClient;
+    this.fileIO = fileIO;
+    this.tableIdentifier = tableIdentifier;
+    this.snowflakeIdentifierForTable = NamespaceHelpers.toSnowflakeIdentifier(tableIdentifier);
+    this.fullTableName = String.format("%s.%s", catalogName, tableIdentifier);
+  }
+
+  @Override
+  public void doRefresh() {
+    LOG.debug("Getting metadata location for table {}", tableIdentifier);
+    String location = loadTableMetadataLocation();
+    Preconditions.checkState(
+        location != null && !location.isEmpty(),
+        "Got null or empty location %s for table %s",

Review Comment:
   nit: if location is really null/empty, then it's probably not worth including it in the message itself? `Invalid location for table %s: null or empty` might be slightly better here



##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeTableMetadata.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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 com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+class SnowflakeTableMetadata {
+  public static final Pattern SNOWFLAKE_AZURE_PATTERN =
+      Pattern.compile("azure://([^/]+)/([^/]+)/(.*)");
+
+  private final String snowflakeMetadataLocation;
+  private final String icebergMetadataLocation;
+  private final String status;
+
+  // Note: Since not all sources will necessarily come from a raw JSON representation, this raw
+  // JSON should only be considered a convenient debugging field. Equality of two
+  // SnowflakeTableMetadata instances should not depend on equality of this field.
+  private final String rawJsonVal;
+
+  SnowflakeTableMetadata(
+      String snowflakeMetadataLocation,
+      String icebergMetadataLocation,
+      String status,
+      String rawJsonVal) {
+    this.snowflakeMetadataLocation = snowflakeMetadataLocation;
+    this.icebergMetadataLocation = icebergMetadataLocation;
+    this.status = status;
+    this.rawJsonVal = rawJsonVal;
+  }
+
+  /** Storage location of table metadata in Snowflake's path syntax. */
+  public String snowflakeMetadataLocation() {
+    return snowflakeMetadataLocation;
+  }
+
+  /** Storage location of table metadata in Iceberg's path syntax. */
+  public String icebergMetadataLocation() {
+    return icebergMetadataLocation;
+  }
+
+  public String getStatus() {
+    return status;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    } else if (!(o instanceof SnowflakeTableMetadata)) {
+      return false;
+    }
+
+    // Only consider parsed fields, not the raw JSON that may or may not be the original source of
+    // this instance.
+    SnowflakeTableMetadata that = (SnowflakeTableMetadata) o;
+    return Objects.equal(this.snowflakeMetadataLocation, that.snowflakeMetadataLocation)
+        && Objects.equal(this.icebergMetadataLocation, that.icebergMetadataLocation)
+        && Objects.equal(this.status, that.status);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(snowflakeMetadataLocation, icebergMetadataLocation, status);
+  }
+
+  @Override
+  public String toString() {
+    return String.format(
+        "snowflakeMetadataLocation: '%s', icebergMetadataLocation: '%s', status: '%s', rawJsonVal: %s",

Review Comment:
   nit: curious whether we really want to have the raw json in the string representation as it might potentially be large? I'm ok leaving it here, but if if largeness might be a concern, then it could be worth having a separate `toDebugString()` method where `rawJsonVal` is included?



##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  private static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  private static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.io.ResolvingFileIO";
+
+  static class FileIOFactory {
+    public FileIO newFileIO(String impl, Map<String, String> properties, Object hadoopConf) {
+      return CatalogUtil.loadFileIO(impl, properties, hadoopConf);
+    }
+  }
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIOFactory fileIOFactory;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    SnowflakeIdentifier scope = NamespaceHelpers.toSnowflakeIdentifier(namespace);
+    Preconditions.checkArgument(
+        scope.type() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at SCHEMA level; got %s from namespace %s",
+        scope,
+        namespace);
+
+    List<SnowflakeIdentifier> sfTables = snowflakeClient.listIcebergTables(scope);
+
+    return sfTables.stream()
+        .map(NamespaceHelpers::toIcebergTableIdentifier)
+        .collect(Collectors.toList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support dropTable");
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support renameTable");
+  }
+
+  @Override
+  public void initialize(String name, Map<String, String> properties) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkArgument(null != 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 communication.
+      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);
+
+    initialize(name, new JdbcSnowflakeClient(connectionPool), new FileIOFactory(), properties);
+  }
+
+  /**
+   * Initialize using caller-supplied SnowflakeClient and FileIO.
+   *
+   * @param name The name of the catalog, defaults to "snowflake_catalog"
+   * @param snowflakeClient The client encapsulating network communication with Snowflake
+   * @param fileIOFactory The {@link FileIOFactory} to use to instantiate a new FileIO for each new
+   *     table operation
+   * @param properties The catalog options to use and propagate to dependencies
+   */
+  @SuppressWarnings("checkstyle:HiddenField")
+  void initialize(
+      String name,
+      SnowflakeClient snowflakeClient,
+      FileIOFactory fileIOFactory,
+      Map<String, String> properties) {
+    Preconditions.checkArgument(null != snowflakeClient, "snowflakeClient must be non-null");
+    Preconditions.checkArgument(null != fileIOFactory, "fileIOFactory must be non-null");
+    this.catalogName = name == null ? DEFAULT_CATALOG_NAME : name;
+    this.snowflakeClient = snowflakeClient;
+    this.fileIOFactory = fileIOFactory;
+    this.catalogProperties = properties;
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(snowflakeClient);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (null != closeableGroup) {
+      closeableGroup.close();
+    }
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support createNamespace");
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    SnowflakeIdentifier scope = NamespaceHelpers.toSnowflakeIdentifier(namespace);
+    List<SnowflakeIdentifier> results = null;
+    switch (scope.type()) {
+      case ROOT:
+        results = snowflakeClient.listDatabases();
+        break;
+      case DATABASE:
+        results = snowflakeClient.listSchemas(scope);
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format(
+                "listNamespaces must be at either ROOT or DATABASE level; got %s from namespace %s",
+                scope, namespace));
+    }
+
+    List<Namespace> namespaceList =

Review Comment:
   nit: seems like this variable is redundant and we can just return the result directly here



##########
snowflake/src/test/java/org/apache/iceberg/snowflake/NamespaceHelpersTest.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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 org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+public class NamespaceHelpersTest {
+  @Test
+  public void testToSnowflakeIdentifierRoot() {
+    Assertions.assertThat(NamespaceHelpers.toSnowflakeIdentifier(Namespace.empty()))
+        .isEqualTo(SnowflakeIdentifier.ofRoot());
+  }
+
+  @Test
+  public void testToSnowflakeIdentifierDatabase() {
+    Assertions.assertThat(NamespaceHelpers.toSnowflakeIdentifier(Namespace.of("DB1")))
+        .isEqualTo(SnowflakeIdentifier.ofDatabase("DB1"));
+  }
+
+  @Test
+  public void testToSnowflakeIdentifierSchema() {
+    Assertions.assertThat(NamespaceHelpers.toSnowflakeIdentifier(Namespace.of("DB1", "SCHEMA1")))
+        .isEqualTo(SnowflakeIdentifier.ofSchema("DB1", "SCHEMA1"));
+  }
+
+  @Test
+  public void testToSnowflakeIdentifierMaxNamespaceLevel() {
+    Assertions.assertThatExceptionOfType(IllegalArgumentException.class)
+        .isThrownBy(
+            () ->
+                NamespaceHelpers.toSnowflakeIdentifier(
+                    Namespace.of("DB1", "SCHEMA1", "THIRD_NS_LVL")))
+        .withMessageContaining("max namespace level");
+  }
+
+  @Test
+  public void testToSnowflakeIdentifierTable() {
+    Assertions.assertThat(
+            NamespaceHelpers.toSnowflakeIdentifier(TableIdentifier.of("DB1", "SCHEMA1", "TABLE1")))
+        .isEqualTo(SnowflakeIdentifier.ofTable("DB1", "SCHEMA1", "TABLE1"));
+  }
+
+  @Test
+  public void testToSnowflakeIdentifierTableBadNamespace() {
+    Assertions.assertThatExceptionOfType(IllegalArgumentException.class)
+        .isThrownBy(
+            () ->
+                NamespaceHelpers.toSnowflakeIdentifier(
+                    TableIdentifier.of(Namespace.of("DB1_WITHOUT_SCHEMA"), "TABLE1")))
+        .withMessageContaining("must be at the SCHEMA level");
+  }
+
+  @Test
+  public void testToIcebergNamespaceRoot() {
+    Assertions.assertThat(NamespaceHelpers.toIcebergNamespace(SnowflakeIdentifier.ofRoot()))

Review Comment:
   nit: I think it's ok to combine `toSnowflakeIdentifier` and `toIcebergNamespace` into a single test method rather than having two separate ones. That way you directly test the round-trip of each in a single method.
   
   The same applies for all the other test methods. So instead of having `testToSnowflakeIdentifierX` & `testToIcebergX` I would just have one named `testX` or something similar



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1056622204


##########
snowflake/src/test/java/org/apache/iceberg/snowflake/FakeSnowflakeClient.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.jdbc.UncheckedSQLException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class FakeSnowflakeClient implements SnowflakeClient {
+  // In-memory lookup by database/schema/tableName to table metadata.
+  private Map<String, Map<String, Map<String, SnowflakeTableMetadata>>> databases =

Review Comment:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1063854690


##########
core/src/test/java/org/apache/iceberg/io/InMemoryFileIO.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.io;
+
+import java.util.Map;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class InMemoryFileIO implements FileIO {
+
+  private Map<String, byte[]> inMemoryFiles = Maps.newHashMap();
+  private boolean closed = false;
+
+  public void addFile(String path, byte[] contents) {
+    Preconditions.checkState(!closed, "Cannot call addFile after calling close()");
+    inMemoryFiles.put(path, contents);
+  }
+
+  public boolean fileExists(String path) {
+    return inMemoryFiles.containsKey(path);
+  }
+
+  @Override
+  public InputFile newInputFile(String path) {
+    Preconditions.checkState(!closed, "Cannot call newInputFile after calling close()");
+    if (!inMemoryFiles.containsKey(path)) {
+      throw new NotFoundException("No in-memory file found for path: %s", path);
+    }
+    return new InMemoryInputFile(path, inMemoryFiles.get(path));
+  }
+
+  @Override
+  public OutputFile newOutputFile(String path) {
+    Preconditions.checkState(!closed, "Cannot call newInputFile after calling close()");

Review Comment:
   Fixed



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1063898457


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java:
##########
@@ -0,0 +1,386 @@
+/*
+ * 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.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+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.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+/**
+ * This implementation of SnowflakeClient builds on top of Snowflake's JDBC driver to interact with
+ * Snowflake's Iceberg-aware resource model.
+ */
+class JdbcSnowflakeClient implements SnowflakeClient {
+  static final String EXPECTED_JDBC_IMPL = "net.snowflake.client.jdbc.SnowflakeDriver";
+
+  @FunctionalInterface
+  interface ResultSetParser<T> {
+    T parse(ResultSet rs) throws SQLException;
+  }
+
+  /**
+   * This class wraps the basic boilerplate of setting up PreparedStatements and applying a
+   * ResultSetParser to translate a ResultSet into parsed objects. Allows easily injecting
+   * subclasses for debugging/testing purposes.
+   */
+  static class QueryHarness {
+    public <T> T query(Connection conn, String sql, ResultSetParser<T> parser, String... args)
+        throws SQLException {
+      try (PreparedStatement statement = conn.prepareStatement(sql)) {
+        if (args != null) {
+          for (int i = 0; i < args.length; ++i) {
+            statement.setString(i + 1, args[i]);
+          }
+        }
+
+        try (ResultSet rs = statement.executeQuery()) {
+          return parser.parse(rs);
+        }
+      }
+    }
+  }
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Database identifiers,
+   * containing "name" (representing databaseName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> DATABASE_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> databases = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("name");
+          databases.add(SnowflakeIdentifier.ofDatabase(databaseName));
+        }
+        return databases;
+      };
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Schema identifiers,
+   * containing "database_name" and "name" (representing schemaName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> SCHEMA_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> schemas = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("database_name");
+          String schemaName = rs.getString("name");
+          schemas.add(SnowflakeIdentifier.ofSchema(databaseName, schemaName));
+        }
+        return schemas;
+      };
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Table identifiers,
+   * containing "database_name", "schema_name", and "name" (representing tableName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> TABLE_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> tables = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("database_name");
+          String schemaName = rs.getString("schema_name");
+          String tableName = rs.getString("name");
+          tables.add(SnowflakeIdentifier.ofTable(databaseName, schemaName, tableName));
+        }
+        return tables;
+      };
+
+  /**
+   * Expects to handle ResultSets representing a single record holding Snowflake Iceberg metadata.
+   */
+  public static final ResultSetParser<SnowflakeTableMetadata> TABLE_METADATA_RESULT_SET_HANDLER =
+      rs -> {
+        if (!rs.next()) {
+          return null;
+        }
+
+        String rawJsonVal = rs.getString("METADATA");
+        return SnowflakeTableMetadata.parseJson(rawJsonVal);
+      };
+
+  private final JdbcClientPool connectionPool;
+  private QueryHarness queryHarness;
+
+  JdbcSnowflakeClient(JdbcClientPool conn) {
+    Preconditions.checkArgument(null != conn, "JdbcClientPool must be non-null");
+    connectionPool = conn;
+    queryHarness = new QueryHarness();
+  }
+
+  @VisibleForTesting
+  void setQueryHarness(QueryHarness queryHarness) {
+    this.queryHarness = queryHarness;
+  }
+
+  /**
+   * For rare cases where PreparedStatements aren't supported for user-supplied identifiers intended
+   * for use in special LIKE clauses, we can sanitize by "broadening" the identifier with
+   * single-character wildcards and manually post-filter client-side.
+   *
+   * <p>Note: This sanitization approach intentionally "broadens" the scope of matching results;
+   * callers must be able to handle this method returning an all-wildcard expression; i.e. the
+   * caller must treat the usage of the LIKE clause as only an optional optimization, and should
+   * post-filter for correctness as if the LIKE clause wasn't present in the query at all.
+   */
+  @VisibleForTesting
+  String sanitizeIdentifierWithWildcardForLikeClause(String identifier) {
+    // Restrict identifiers to the "Unquoted object identifiers" synax documented at
+    // https://docs.snowflake.com/en/sql-reference/identifiers-syntax.html
+    //
+    // Use a strict allowlist of characters, replace everything *not* matching the character set
+    // with "_", which is used as a single-character wildcard in Snowflake.
+    String sanitized = identifier.replaceAll("[^a-zA-Z0-9_$]", "_");
+    if (sanitized.startsWith("$")) {
+      sanitized = "_" + sanitized.substring(1);
+    }
+    return sanitized;
+  }
+
+  @Override
+  public boolean databaseExists(SnowflakeIdentifier database) {
+    Preconditions.checkArgument(
+        database.type() == SnowflakeIdentifier.Type.DATABASE,
+        "databaseExists requires a DATABASE identifier, got '%s'",
+        database);
+
+    // Due to current limitations in PreparedStatement parameters for the LIKE clause in
+    // SHOW DATABASES queries, we'll use a fairly limited allowlist for identifier characters,
+    // using wildcards for non-allowed characters, and post-filter for matching.
+    final String finalQuery =
+        String.format(
+            "SHOW DATABASES LIKE '%s' IN ACCOUNT",
+            sanitizeIdentifierWithWildcardForLikeClause(database.databaseName()));
+    List<SnowflakeIdentifier> databases;
+    try {
+      databases =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn, finalQuery, DATABASE_RESULT_SET_HANDLER, (String[]) null));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to check if database '%s' exists", database);
+    } catch (InterruptedException e) {
+      throw new UncheckedInterruptedException(
+          e, "Interrupted while checking if database '%s' exists", database);
+    }
+
+    // Filter to handle the edge case of '_' appearing as a wildcard that can't be remapped the way
+    // it can for predicates in SELECT statements.
+    databases.removeIf(db -> !database.databaseName().equalsIgnoreCase(db.databaseName()));
+    return !databases.isEmpty();
+  }
+
+  @Override
+  public boolean schemaExists(SnowflakeIdentifier schema) {
+    Preconditions.checkArgument(
+        schema.type() == SnowflakeIdentifier.Type.SCHEMA,
+        "schemaExists requires a SCHEMA identifier, got '%s'",
+        schema);
+
+    if (!databaseExists(SnowflakeIdentifier.ofDatabase(schema.databaseName()))) {
+      return false;
+    }
+
+    // Due to current limitations in PreparedStatement parameters for the LIKE clause in
+    // SHOW SCHEMAS queries, we'll use a fairly limited allowlist for identifier characters,
+    // using wildcards for non-allowed characters, and post-filter for matching.
+    final String finalQuery =
+        String.format(
+            "SHOW SCHEMAS LIKE '%s' IN DATABASE IDENTIFIER(?)",
+            sanitizeIdentifierWithWildcardForLikeClause(schema.schemaName()));
+    List<SnowflakeIdentifier> schemas;
+    try {
+      schemas =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn,
+                      finalQuery,
+                      SCHEMA_RESULT_SET_HANDLER,
+                      new String[] {schema.databaseName()}));

Review Comment:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1063835703


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeClient.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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;
+
+/**
+ * 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.
+ */
+interface SnowflakeClient extends Closeable {
+
+  /** Returns true if the database exists, false otherwise. */
+  boolean databaseExists(SnowflakeIdentifier database);
+
+  /** Returns true if the schema and its parent database exists, false otherwise. */
+  boolean schemaExists(SnowflakeIdentifier schema);
+
+  /** Lists all Snowflake databases within the currently configured account. */
+  List<SnowflakeIdentifier> listDatabases();
+
+  /**
+   * Lists all Snowflake schemas within a given scope. Returned SnowflakeIdentifiers must have
+   * type() == SnowflakeIdentifier.Type.SCHEMA.
+   *
+   * @param scope The scope in which to list, which may be ROOT or a single DATABASE.
+   */
+  List<SnowflakeIdentifier> listSchemas(SnowflakeIdentifier scope);
+
+  /**
+   * Lists all Snowflake Iceberg tables within a given scope. Returned SnowflakeIdentifiers must
+   * have type() == SnowflakeIdentifier.Type.TABLE.
+   *
+   * @param scope The scope in which to list, which may be ROOT, a DATABASE, or a SCHEMA.

Review Comment:
   I believe listing should only be allowed within the scope of a `SCHEMA` (not ROOT or DATABASE).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1063902735


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeTableMetadata.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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 com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+class SnowflakeTableMetadata {
+  public static final Pattern SNOWFLAKE_AZURE_PATTERN =
+      Pattern.compile("azure://([^/]+)/([^/]+)/(.*)");
+
+  private String snowflakeMetadataLocation;
+  private String icebergMetadataLocation;
+  private String status;

Review Comment:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1066454772


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  private static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  private static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.io.ResolvingFileIO";
+
+  static class FileIOFactory {
+    public FileIO newFileIO(String impl, Map<String, String> properties, Object hadoopConf) {
+      return CatalogUtil.loadFileIO(impl, properties, hadoopConf);
+    }
+  }
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIOFactory fileIOFactory;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    SnowflakeIdentifier scope = NamespaceHelpers.toSnowflakeIdentifier(namespace);
+    Preconditions.checkArgument(
+        scope.type() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at SCHEMA level; got %s from namespace %s",
+        scope,
+        namespace);
+
+    List<SnowflakeIdentifier> sfTables = snowflakeClient.listIcebergTables(scope);
+
+    return sfTables.stream()
+        .map(NamespaceHelpers::toIcebergTableIdentifier)
+        .collect(Collectors.toList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support dropTable");
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support renameTable");
+  }
+
+  @Override
+  public void initialize(String name, Map<String, String> properties) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkArgument(null != 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 communication.
+      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);
+
+    initialize(name, new JdbcSnowflakeClient(connectionPool), new FileIOFactory(), properties);
+  }
+
+  /**
+   * Initialize using caller-supplied SnowflakeClient and FileIO.
+   *
+   * @param name The name of the catalog, defaults to "snowflake_catalog"
+   * @param snowflakeClient The client encapsulating network communication with Snowflake
+   * @param fileIOFactory The {@link FileIOFactory} to use to instantiate a new FileIO for each new
+   *     table operation
+   * @param properties The catalog options to use and propagate to dependencies
+   */
+  @SuppressWarnings("checkstyle:HiddenField")
+  void initialize(
+      String name,
+      SnowflakeClient snowflakeClient,
+      FileIOFactory fileIOFactory,
+      Map<String, String> properties) {
+    Preconditions.checkArgument(null != snowflakeClient, "snowflakeClient must be non-null");
+    Preconditions.checkArgument(null != fileIOFactory, "fileIOFactory must be non-null");
+    this.catalogName = name == null ? DEFAULT_CATALOG_NAME : name;
+    this.snowflakeClient = snowflakeClient;
+    this.fileIOFactory = fileIOFactory;
+    this.catalogProperties = properties;
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(snowflakeClient);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (null != closeableGroup) {
+      closeableGroup.close();
+    }
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support createNamespace");
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    SnowflakeIdentifier scope = NamespaceHelpers.toSnowflakeIdentifier(namespace);
+    List<SnowflakeIdentifier> results = null;
+    switch (scope.type()) {
+      case ROOT:
+        results = snowflakeClient.listDatabases();
+        break;
+      case DATABASE:
+        results = snowflakeClient.listSchemas(scope);
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format(
+                "listNamespaces must be at either ROOT or DATABASE level; got %s from namespace %s",
+                scope, namespace));
+    }
+
+    List<Namespace> namespaceList =

Review Comment:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1063860728


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeIdentifier.java:
##########
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.snowflake;
+
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Since the SnowflakeCatalog supports exactly two levels of Iceberg Namespaces, corresponding
+ * directly to the "database" and "schema" portions of Snowflake's resource model, this class
+ * represents a pre-validated and structured representation of a fully-qualified Snowflake resource
+ * identifier. Snowflake-specific helper libraries should operate on this representation instead of
+ * directly operating on TableIdentifiers or Namespaces wherever possible to avoid duplication of
+ * parsing/validation logic for Iceberg TableIdentifier/Namespace levels.
+ */
+class SnowflakeIdentifier {
+  public enum Type {
+    ROOT,
+    DATABASE,
+    SCHEMA,
+    TABLE
+  }
+
+  private String databaseName;
+  private String schemaName;
+  private String tableName;
+  private Type type;

Review Comment:
   These can all be final



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1063815527


##########
core/src/test/java/org/apache/iceberg/io/InMemoryFileIO.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.io;
+
+import java.util.Map;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class InMemoryFileIO implements FileIO {
+
+  private Map<String, byte[]> inMemoryFiles = Maps.newHashMap();
+  private boolean closed = false;
+
+  public void addFile(String path, byte[] contents) {
+    Preconditions.checkState(!closed, "Cannot call addFile after calling close()");
+    inMemoryFiles.put(path, contents);
+  }
+
+  public boolean fileExists(String path) {
+    return inMemoryFiles.containsKey(path);
+  }
+
+  @Override
+  public InputFile newInputFile(String path) {
+    Preconditions.checkState(!closed, "Cannot call newInputFile after calling close()");
+    if (!inMemoryFiles.containsKey(path)) {
+      throw new NotFoundException("No in-memory file found for path: %s", path);
+    }
+    return new InMemoryInputFile(path, inMemoryFiles.get(path));
+  }
+
+  @Override
+  public OutputFile newOutputFile(String path) {
+    Preconditions.checkState(!closed, "Cannot call newInputFile after calling close()");

Review Comment:
   Message says `newInputFile` but should be `newOutputFile`



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1063842045


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,254 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  private static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  private static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.io.ResolvingFileIO";
+
+  static class FileIOFactory {
+    public FileIO newFileIO(String impl, Map<String, String> properties, Object hadoopConf) {
+      return CatalogUtil.loadFileIO(impl, properties, hadoopConf);
+    }
+  }
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIOFactory fileIOFactory;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    SnowflakeIdentifier scope = NamespaceHelpers.toSnowflakeIdentifier(namespace);
+    Preconditions.checkArgument(
+        scope.type() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at SCHEMA level; got %s from namespace %s",
+        scope,
+        namespace);
+
+    List<SnowflakeIdentifier> sfTables = snowflakeClient.listIcebergTables(scope);
+
+    return sfTables.stream()
+        .map(NamespaceHelpers::toIcebergTableIdentifier)
+        .collect(Collectors.toList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support dropTable");
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support renameTable");
+  }
+
+  @Override
+  public void initialize(String name, Map<String, String> properties) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkArgument(null != 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 communication.
+      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);
+
+    initialize(name, new JdbcSnowflakeClient(connectionPool), new FileIOFactory(), properties);
+  }
+
+  /**
+   * Initialize using caller-supplied SnowflakeClient and FileIO.
+   *
+   * @param name The name of the catalog, defaults to "snowflake_catalog"
+   * @param snowflakeClient The client encapsulating network communication with Snowflake
+   * @param fileIOFactory The {@link FileIOFactory} to use to instantiate a new FileIO for each new
+   *     table operation
+   * @param properties The catalog options to use and propagate to dependencies
+   */
+  @SuppressWarnings("checkstyle:HiddenField")
+  void initialize(
+      String name,
+      SnowflakeClient snowflakeClient,
+      FileIOFactory fileIOFactory,
+      Map<String, String> properties) {
+    Preconditions.checkArgument(null != snowflakeClient, "snowflakeClient must be non-null");
+    Preconditions.checkArgument(null != fileIOFactory, "fileIOFactory must be non-null");
+    this.catalogName = name == null ? DEFAULT_CATALOG_NAME : name;
+    this.snowflakeClient = snowflakeClient;
+    this.fileIOFactory = fileIOFactory;
+    this.catalogProperties = properties;
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(snowflakeClient);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (null != closeableGroup) {
+      closeableGroup.close();
+    }
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support createNamespace");
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    SnowflakeIdentifier scope = NamespaceHelpers.toSnowflakeIdentifier(namespace);
+    List<SnowflakeIdentifier> results = null;
+    switch (scope.type()) {
+      case ROOT:
+        results = snowflakeClient.listDatabases();
+        break;
+      case DATABASE:
+        results = snowflakeClient.listSchemas(scope);
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format(
+                "listNamespaces must be at either ROOT or DATABASE level; got %s from namespace %s",
+                scope, namespace));
+    }
+
+    List<Namespace> namespaceList =
+        results.stream().map(NamespaceHelpers::toIcebergNamespace).collect(Collectors.toList());
+    return namespaceList;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace)
+      throws NoSuchNamespaceException {
+    SnowflakeIdentifier id = NamespaceHelpers.toSnowflakeIdentifier(namespace);
+    boolean namespaceExists;
+    switch (id.type()) {
+      case DATABASE:
+        namespaceExists = snowflakeClient.databaseExists(id);
+        break;
+      case SCHEMA:
+        namespaceExists = snowflakeClient.schemaExists(id);
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format(
+                "loadNamespaceMetadat must be at either DATABASE or SCHEMA level; got %s from namespace %s",
+                id, namespace));
+    }
+    if (namespaceExists) {
+      return ImmutableMap.of();
+    } else {
+      throw new NoSuchNamespaceException(
+          "Namespace '%s' with snowflake identifier '%s' doesn't exist", namespace, id);
+    }
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support dropNamespace");
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support setProperties");
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support removeProperties");
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    String fileIOImpl = DEFAULT_FILE_IO_IMPL;
+    if (catalogProperties.containsKey(CatalogProperties.FILE_IO_IMPL)) {
+      fileIOImpl = catalogProperties.get(CatalogProperties.FILE_IO_IMPL);
+    }
+
+    // Initialize a fresh FileIO for each TableOperations created, because some FileIO
+    // implementations such as S3FileIO can become bound to a single S3 bucket. Additionally,
+    // FileIO implementations often support only a finite set of one or more URI schemes (i.e.
+    // S3FileIO only supports s3/s3a/s3n, and even ResolvingFileIO only supports the combination
+    // of schemes registered for S3FileIO and HadoopFileIO). Individual catalogs may need to
+    // support tables across different cloud/storage providers with disjoint FileIO implementations.
+    FileIO fileIO = fileIOFactory.newFileIO(fileIOImpl, catalogProperties, conf);
+    closeableGroup.addCloseable(fileIO);
+    return new SnowflakeTableOperations(snowflakeClient, fileIO, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier tableIdentifier) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support defaultWarehouseLocation");
+  }
+
+  @Override
+  public void setConf(Object conf) {
+    this.conf = conf;
+  }
+
+  public Object getConf() {

Review Comment:
   I don't believe this is necessary since we only dynamically set the configuraiton when dynamically loading a catalog implementation. 



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1063840398


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeClient.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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;
+
+/**
+ * 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.
+ */
+interface SnowflakeClient extends Closeable {
+
+  /** Returns true if the database exists, false otherwise. */
+  boolean databaseExists(SnowflakeIdentifier database);
+
+  /** Returns true if the schema and its parent database exists, false otherwise. */
+  boolean schemaExists(SnowflakeIdentifier schema);
+
+  /** Lists all Snowflake databases within the currently configured account. */
+  List<SnowflakeIdentifier> listDatabases();
+
+  /**
+   * Lists all Snowflake schemas within a given scope. Returned SnowflakeIdentifiers must have
+   * type() == SnowflakeIdentifier.Type.SCHEMA.
+   *
+   * @param scope The scope in which to list, which may be ROOT or a single DATABASE.

Review Comment:
   ok, I see that in the SnowflakeCatalog we're limiting this.  It's probably ok to leave this for the internal client.



##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeClient.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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;
+
+/**
+ * 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.
+ */
+interface SnowflakeClient extends Closeable {
+
+  /** Returns true if the database exists, false otherwise. */
+  boolean databaseExists(SnowflakeIdentifier database);
+
+  /** Returns true if the schema and its parent database exists, false otherwise. */
+  boolean schemaExists(SnowflakeIdentifier schema);
+
+  /** Lists all Snowflake databases within the currently configured account. */
+  List<SnowflakeIdentifier> listDatabases();
+
+  /**
+   * Lists all Snowflake schemas within a given scope. Returned SnowflakeIdentifiers must have
+   * type() == SnowflakeIdentifier.Type.SCHEMA.
+   *
+   * @param scope The scope in which to list, which may be ROOT or a single DATABASE.
+   */
+  List<SnowflakeIdentifier> listSchemas(SnowflakeIdentifier scope);
+
+  /**
+   * Lists all Snowflake Iceberg tables within a given scope. Returned SnowflakeIdentifiers must
+   * have type() == SnowflakeIdentifier.Type.TABLE.
+   *
+   * @param scope The scope in which to list, which may be ROOT, a DATABASE, or a SCHEMA.

Review Comment:
   ok, I see that in the SnowflakeCatalog we're limiting this.  It's probably ok to leave this for the internal client.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1063902516


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeIdentifier.java:
##########
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.snowflake;
+
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Since the SnowflakeCatalog supports exactly two levels of Iceberg Namespaces, corresponding
+ * directly to the "database" and "schema" portions of Snowflake's resource model, this class
+ * represents a pre-validated and structured representation of a fully-qualified Snowflake resource
+ * identifier. Snowflake-specific helper libraries should operate on this representation instead of
+ * directly operating on TableIdentifiers or Namespaces wherever possible to avoid duplication of
+ * parsing/validation logic for Iceberg TableIdentifier/Namespace levels.
+ */
+class SnowflakeIdentifier {
+  public enum Type {
+    ROOT,
+    DATABASE,
+    SCHEMA,
+    TABLE
+  }
+
+  private String databaseName;
+  private String schemaName;
+  private String tableName;
+  private Type type;

Review Comment:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1070166470


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  private static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  private static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.io.ResolvingFileIO";
+
+  static class FileIOFactory {
+    public FileIO newFileIO(String impl, Map<String, String> properties, Object hadoopConf) {
+      return CatalogUtil.loadFileIO(impl, properties, hadoopConf);

Review Comment:
   Added code comment to clarify



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051230626


##########
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:
   Done. Also added test cases to SnowflakeCatalogTest to ensure the close() propagates through to both `snowflakeClient` and `fileIO` and that we don't have a NullPointerException if close() is called before initialize creates the closeableGroup.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051657795


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.hadoop.HadoopFileIO";
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    LOG.debug("listTables with namespace: {}", namespace);
+    SnowflakeIdentifier scope = NamespaceHelpers.getSnowflakeIdentifierForNamespace(namespace);
+    Preconditions.checkArgument(
+        scope.getType() == SnowflakeIdentifier.Type.ROOT
+            || scope.getType() == SnowflakeIdentifier.Type.DATABASE
+            || scope.getType() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at ROOT, DATABASE, or SCHEMA level; got %s from namespace %s",
+        scope,
+        namespace);
+
+    List<SnowflakeIdentifier> sfTables = snowflakeClient.listIcebergTables(scope);
+
+    return sfTables.stream()
+        .map(
+            table ->
+                TableIdentifier.of(
+                    table.getDatabaseName(), table.getSchemaName(), table.getTableName()))
+        .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) {
+    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);
+
+    String fileIOImpl = DEFAULT_FILE_IO_IMPL;
+    if (properties.containsKey(CatalogProperties.FILE_IO_IMPL)) {
+      fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL);
+    }
+
+    initialize(
+        name,
+        new JdbcSnowflakeClient(connectionPool),
+        CatalogUtil.loadFileIO(fileIOImpl, properties, conf),
+        properties);
+  }
+
+  /**
+   * Initialize using caller-supplied SnowflakeClient and FileIO.
+   *
+   * @param name The name of the catalog, defaults to "snowflake_catalog"
+   * @param snowflakeClient The client encapsulating network communication with Snowflake
+   * @param fileIO The {@link FileIO} to use for table operations
+   * @param properties The catalog options to use and propagate to dependencies
+   */
+  @SuppressWarnings("checkstyle:HiddenField")
+  public void initialize(
+      String name, SnowflakeClient snowflakeClient, FileIO fileIO, Map<String, String> properties) {
+    Preconditions.checkArgument(null != snowflakeClient, "snowflakeClient must be non-null");
+    Preconditions.checkArgument(null != fileIO, "fileIO must be non-null");
+    this.catalogName = name == null ? DEFAULT_CATALOG_NAME : name;
+    this.snowflakeClient = snowflakeClient;
+    this.fileIO = fileIO;
+    this.catalogProperties = properties;
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(snowflakeClient);
+    closeableGroup.addCloseable(fileIO);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (null != closeableGroup) {
+      closeableGroup.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);
+    SnowflakeIdentifier scope = NamespaceHelpers.getSnowflakeIdentifierForNamespace(namespace);
+    Preconditions.checkArgument(
+        scope.getType() == SnowflakeIdentifier.Type.ROOT
+            || scope.getType() == SnowflakeIdentifier.Type.DATABASE,
+        "listNamespaces must be at either ROOT or DATABASE level; got %s from namespace %s",
+        scope,
+        namespace);
+    List<SnowflakeIdentifier> sfSchemas = snowflakeClient.listSchemas(scope);
+
+    List<Namespace> namespaceList =
+        sfSchemas.stream()
+            .map(
+                schema -> {
+                  Preconditions.checkState(
+                      schema.getType() == SnowflakeIdentifier.Type.SCHEMA,
+                      "Got identifier of type %s from listSchemas for %s",
+                      schema.getType(),
+                      namespace);
+                  return Namespace.of(schema.getDatabaseName(), schema.getSchemaName());
+                })
+            .collect(Collectors.toList());
+    return namespaceList;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace)
+      throws NoSuchNamespaceException {
+    LOG.debug("loadNamespaceMetadata with namespace: {}", namespace);
+    return ImmutableMap.of();
+  }
+
+  @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) {
+    return new SnowflakeTableOperations(

Review Comment:
   Each table should be loaded with its own FileIO instance



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051691247


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.hadoop.HadoopFileIO";
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    LOG.debug("listTables with namespace: {}", namespace);
+    SnowflakeIdentifier scope = NamespaceHelpers.getSnowflakeIdentifierForNamespace(namespace);
+    Preconditions.checkArgument(
+        scope.getType() == SnowflakeIdentifier.Type.ROOT
+            || scope.getType() == SnowflakeIdentifier.Type.DATABASE
+            || scope.getType() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at ROOT, DATABASE, or SCHEMA level; got %s from namespace %s",
+        scope,
+        namespace);
+
+    List<SnowflakeIdentifier> sfTables = snowflakeClient.listIcebergTables(scope);
+
+    return sfTables.stream()
+        .map(
+            table ->
+                TableIdentifier.of(
+                    table.getDatabaseName(), table.getSchemaName(), table.getTableName()))
+        .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) {
+    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);
+
+    String fileIOImpl = DEFAULT_FILE_IO_IMPL;
+    if (properties.containsKey(CatalogProperties.FILE_IO_IMPL)) {
+      fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL);
+    }
+
+    initialize(
+        name,
+        new JdbcSnowflakeClient(connectionPool),
+        CatalogUtil.loadFileIO(fileIOImpl, properties, conf),
+        properties);
+  }
+
+  /**
+   * Initialize using caller-supplied SnowflakeClient and FileIO.
+   *
+   * @param name The name of the catalog, defaults to "snowflake_catalog"
+   * @param snowflakeClient The client encapsulating network communication with Snowflake
+   * @param fileIO The {@link FileIO} to use for table operations
+   * @param properties The catalog options to use and propagate to dependencies
+   */
+  @SuppressWarnings("checkstyle:HiddenField")
+  public void initialize(
+      String name, SnowflakeClient snowflakeClient, FileIO fileIO, Map<String, String> properties) {
+    Preconditions.checkArgument(null != snowflakeClient, "snowflakeClient must be non-null");
+    Preconditions.checkArgument(null != fileIO, "fileIO must be non-null");
+    this.catalogName = name == null ? DEFAULT_CATALOG_NAME : name;
+    this.snowflakeClient = snowflakeClient;
+    this.fileIO = fileIO;
+    this.catalogProperties = properties;
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(snowflakeClient);
+    closeableGroup.addCloseable(fileIO);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (null != closeableGroup) {
+      closeableGroup.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);
+    SnowflakeIdentifier scope = NamespaceHelpers.getSnowflakeIdentifierForNamespace(namespace);
+    Preconditions.checkArgument(
+        scope.getType() == SnowflakeIdentifier.Type.ROOT
+            || scope.getType() == SnowflakeIdentifier.Type.DATABASE,
+        "listNamespaces must be at either ROOT or DATABASE level; got %s from namespace %s",
+        scope,
+        namespace);
+    List<SnowflakeIdentifier> sfSchemas = snowflakeClient.listSchemas(scope);
+
+    List<Namespace> namespaceList =
+        sfSchemas.stream()
+            .map(
+                schema -> {
+                  Preconditions.checkState(
+                      schema.getType() == SnowflakeIdentifier.Type.SCHEMA,
+                      "Got identifier of type %s from listSchemas for %s",
+                      schema.getType(),
+                      namespace);
+                  return Namespace.of(schema.getDatabaseName(), schema.getSchemaName());
+                })
+            .collect(Collectors.toList());
+    return namespaceList;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace)
+      throws NoSuchNamespaceException {
+    LOG.debug("loadNamespaceMetadata with namespace: {}", namespace);
+    return ImmutableMap.of();
+  }
+
+  @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) {
+    return new SnowflakeTableOperations(

Review Comment:
   Reusing the FileIO will cause problems in certain cases though, so I feel like we shouldn't introduce those here even if there are other catalogs that currently use that approach.  For example, if you want to use S3FileIO (or any other native FileIO), then you'll be limited to a single bucket.  Once the FileIO is initialized, it will only be able to load the table metadata if it was the same as what the catalog was initialized with.  
   
   In Snowflake you can register multiple buckets and arbitrarily point tables to different locations, so we don't want to limit what can be addressed by reusing the same FileIO.  HadoopFileIO is relying on Hadoop's implementation of abstracting the access across buckets to enable this to work, but we shouldn't rely on Hadoop's behavior here.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] nastra commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1049312756


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

Review Comment:
   just curious, would it be possible to extend the existing `CatalogTests` class as it already defines/tests a lot of common behavior



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051262362


##########
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:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051274163


##########
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:
   As @manisin mentioned, the Namespace validation in Catalog is mostly to try to isolate Iceberg-specific business logic from the mechanics of contacting Snowflake in an underlying client, hopefully to keep the client impl(s) as thin as possible.
   
   Agree that it's still a little bit tangled right now though - when we considered how to refactor the client interface cleanly it felt like too much boilerplate to remove touching the "Namespace" class in the JdbcSnowflakeClient entirely.
   
   But thinking about it more along with your comments about `entities/` and `SnowflakeResources`, maybe it's worth squishing `SnowflakeSchema` and `SnowflakeTable` into a single `SnowflakeIdentifier` entity that can represent all of: root namespace, a fully-qualified database, a fully-qualified schema, or a fully-qualified table.
   
   Essentially, it's indeed a bit of boilerplate that is technically the same as an Iceberg TableIdentifier, but just with constraints on Namespace having 0, 1, or 2 elements, and that those namespace elements correspond to first-class concepts of "databaseName" and "schemaName".
   
   The goal of having the SnowflakeIdentifier would be to isolate all the handling of Namespace level numbers into a single place. I'll try to refactor.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051272184


##########
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:
   Done, moved checks into SnowflakeTableOperations class instead and added unittest.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051274939


##########
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:
   Yeah, as @manisin mentions we'll ideally maintain an abstraction layer to keep the underlying client impl(s) clean from Namespace-level-processing stuff, but agree the current entity layout seems a bit verbose. I'll see if we can squish into a SnowflakeIdentifier instead.



##########
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:
   Done



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1052912874


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.hadoop.HadoopFileIO";
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    LOG.debug("listTables with namespace: {}", namespace);
+    SnowflakeIdentifier scope = NamespaceHelpers.getSnowflakeIdentifierForNamespace(namespace);
+    Preconditions.checkArgument(
+        scope.getType() == SnowflakeIdentifier.Type.ROOT
+            || scope.getType() == SnowflakeIdentifier.Type.DATABASE
+            || scope.getType() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at ROOT, DATABASE, or SCHEMA level; got %s from namespace %s",
+        scope,
+        namespace);
+
+    List<SnowflakeIdentifier> sfTables = snowflakeClient.listIcebergTables(scope);
+
+    return sfTables.stream()
+        .map(
+            table ->
+                TableIdentifier.of(

Review Comment:
   Moved to `NamespaceHelpers` for consistency with the inverse operations; planning to keep `SnowflakeIdentifier` itself free of Iceberg-specific dependencies for now so that the entire lower-level `SnowflakeClient/JdbcSnowflakeClient` stack could theoretically sit outside of Iceberg dependencies.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] nastra commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1051938403


##########
snowflake/src/test/java/org/apache/iceberg/snowflake/SnowflakeCatalogTest.java:
##########
@@ -0,0 +1,277 @@
+/*
+ * 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.IOException;
+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.Maps;
+import org.apache.iceberg.snowflake.entities.SnowflakeTableMetadata;
+import org.apache.iceberg.types.Types;
+import org.assertj.core.api.Assertions;
+import org.junit.Before;
+import org.junit.Test;
+
+public class SnowflakeCatalogTest {
+
+  static final String TEST_CATALOG_NAME = "slushLog";
+  private SnowflakeCatalog catalog;
+  private FakeSnowflakeClient fakeClient;
+  private InMemoryFileIO fakeFileIO;
+  private Map<String, String> properties;
+
+  @Before
+  public void before() {
+    catalog = new SnowflakeCatalog();
+
+    fakeClient = new FakeSnowflakeClient();
+    fakeClient.addTable(
+        "DB_1",
+        "SCHEMA_1",
+        "TAB_1",
+        SnowflakeTableMetadata.parseJson(
+            "{\"metadataLocation\":\"s3://tab1/metadata/v3.metadata.json\",\"status\":\"success\"}"));
+    fakeClient.addTable(
+        "DB_1",
+        "SCHEMA_1",
+        "TAB_2",
+        SnowflakeTableMetadata.parseJson(
+            "{\"metadataLocation\":\"s3://tab2/metadata/v1.metadata.json\",\"status\":\"success\"}"));
+    fakeClient.addTable(
+        "DB_2",
+        "SCHEMA_2",
+        "TAB_3",
+        SnowflakeTableMetadata.parseJson(
+            "{\"metadataLocation\":\"azure://myaccount.blob.core.windows.net/mycontainer/tab3/metadata/v334.metadata.json\",\"status\":\"success\"}"));
+    fakeClient.addTable(
+        "DB_2",
+        "SCHEMA_2",
+        "TAB_4",
+        SnowflakeTableMetadata.parseJson(
+            "{\"metadataLocation\":\"azure://myaccount.blob.core.windows.net/mycontainer/tab4/metadata/v323.metadata.json\",\"status\":\"success\"}"));
+    fakeClient.addTable(
+        "DB_3",
+        "SCHEMA_3",
+        "TAB_5",
+        SnowflakeTableMetadata.parseJson(
+            "{\"metadataLocation\":\"gcs://tab5/metadata/v793.metadata.json\",\"status\":\"success\"}"));
+    fakeClient.addTable(
+        "DB_3",
+        "SCHEMA_4",
+        "TAB_6",
+        SnowflakeTableMetadata.parseJson(
+            "{\"metadataLocation\":\"gcs://tab6/metadata/v123.metadata.json\",\"status\":\"success\"}"));
+
+    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://mycontainer@myaccount.blob.core.windows.net/tab3/metadata/v334.metadata.json",
+        TableMetadataParser.toJson(
+                TableMetadata.newTableMetadata(
+                    schema,
+                    partitionSpec,
+                    "wasbs://mycontainer@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());
+
+    properties = Maps.newHashMap();
+    catalog.initialize(TEST_CATALOG_NAME, fakeClient, fakeFileIO, properties);
+  }
+
+  @Test
+  public void testInitializeNullClient() {
+    Assertions.assertThatExceptionOfType(IllegalArgumentException.class)
+        .isThrownBy(() -> catalog.initialize(TEST_CATALOG_NAME, null, fakeFileIO, properties))
+        .withMessageContaining("snowflakeClient must be non-null");
+  }
+
+  @Test
+  public void testInitializeNullFileIO() {
+    Assertions.assertThatExceptionOfType(IllegalArgumentException.class)
+        .isThrownBy(() -> catalog.initialize(TEST_CATALOG_NAME, fakeClient, null, properties))
+        .withMessageContaining("fileIO must be non-null");
+  }
+
+  @Test
+  public void testListNamespace() {
+    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"));
+  }
+
+  @Test
+  public void testListNamespaceWithinDB() {
+    String dbName = "DB_1";
+    Assertions.assertThat(catalog.listNamespaces(Namespace.of(dbName)))
+        .containsExactly(Namespace.of(dbName, "SCHEMA_1"));
+  }
+
+  @Test
+  public void testListNamespaceWithinNonExistentDB() {
+    // Existence check for nonexistent parent namespaces is optional in the SupportsNamespaces
+    // interface.
+    String dbName = "NONEXISTENT_DB";
+    Assertions.assertThatExceptionOfType(RuntimeException.class)
+        .isThrownBy(() -> catalog.listNamespaces(Namespace.of(dbName)))
+        .withMessageContaining("does not exist")
+        .withMessageContaining(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";
+    Assertions.assertThatExceptionOfType(IllegalArgumentException.class)
+        .isThrownBy(() -> catalog.listNamespaces(Namespace.of(dbName, schemaName)))
+        .withMessageContaining("level")
+        .withMessageContaining("DB_3.SCHEMA_4");
+  }
+
+  @Test
+  public void testListTables() {
+    Assertions.assertThat(catalog.listTables(Namespace.empty()))
+        .containsExactly(
+            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"));
+  }
+
+  @Test
+  public void testListTablesWithinDB() {
+    String dbName = "DB_1";
+    Assertions.assertThat(catalog.listTables(Namespace.of(dbName)))
+        .containsExactly(
+            TableIdentifier.of("DB_1", "SCHEMA_1", "TAB_1"),
+            TableIdentifier.of("DB_1", "SCHEMA_1", "TAB_2"));
+  }
+
+  @Test
+  public void testListTablesWithinNonexistentDB() {
+    String dbName = "NONEXISTENT_DB";
+    Assertions.assertThatExceptionOfType(RuntimeException.class)
+        .isThrownBy(() -> catalog.listTables(Namespace.of(dbName)))
+        .withMessageContaining("does not exist")
+        .withMessageContaining(dbName);
+  }
+
+  @Test
+  public void testListTablesWithinSchema() {
+    String dbName = "DB_2";
+    String schemaName = "SCHEMA_2";
+    Assertions.assertThat(catalog.listTables(Namespace.of(dbName, schemaName)))
+        .containsExactly(
+            TableIdentifier.of("DB_2", "SCHEMA_2", "TAB_3"),
+            TableIdentifier.of("DB_2", "SCHEMA_2", "TAB_4"));
+  }
+
+  @Test
+  public void testListTablesWithinNonexistentSchema() {
+    String dbName = "DB_2";
+    String schemaName = "NONEXISTENT_SCHEMA";
+    Assertions.assertThatExceptionOfType(RuntimeException.class)
+        .isThrownBy(() -> catalog.listTables(Namespace.of(dbName, schemaName)))
+        .withMessageContaining("does not exist")
+        .withMessageContaining("DB_2.NONEXISTENT_SCHEMA");
+  }
+
+  @Test
+  public void testLoadS3Table() {
+    Table table = catalog.loadTable(TableIdentifier.of(Namespace.of("DB_1", "SCHEMA_1"), "TAB_1"));
+    Assertions.assertThat(table.location()).isEqualTo("s3://tab1/");
+  }
+
+  @Test
+  public void testLoadAzureTable() {
+    Table table = catalog.loadTable(TableIdentifier.of(Namespace.of("DB_2", "SCHEMA_2"), "TAB_3"));
+    Assertions.assertThat(table.location())
+        .isEqualTo("wasbs://mycontainer@myaccount.blob.core.windows.net/tab1/");
+  }
+
+  @Test
+  public void testLoadGcsTable() {
+    Table table = catalog.loadTable(TableIdentifier.of(Namespace.of("DB_3", "SCHEMA_3"), "TAB_5"));
+    Assertions.assertThat(table.location()).isEqualTo("gs://tab5/");
+  }
+
+  @Test
+  public void testLoadTableWithMalformedTableIdentifier() {
+    Assertions.assertThatExceptionOfType(IllegalArgumentException.class)
+        .isThrownBy(
+            () ->
+                catalog.loadTable(
+                    TableIdentifier.of(Namespace.of("DB_1", "SCHEMA_1", "BAD_NS_LEVEL"), "TAB_1")))
+        .withMessageContaining("level")
+        .withMessageContaining("DB_1.SCHEMA_1.BAD_NS_LEVEL");
+    Assertions.assertThatExceptionOfType(IllegalArgumentException.class)
+        .isThrownBy(
+            () -> catalog.loadTable(TableIdentifier.of(Namespace.of("DB_WITHOUT_SCHEMA"), "TAB_1")))
+        .withMessageContaining("level")
+        .withMessageContaining("DB_WITHOUT_SCHEMA.TAB_1");
+  }
+
+  @Test
+  public void testCloseBeforeInitialize() throws IOException {
+    catalog = new SnowflakeCatalog();
+    catalog.close();

Review Comment:
   what does this test exactly?



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1054082154


##########
versions.props:
##########
@@ -28,6 +28,8 @@ org.scala-lang.modules:scala-collection-compat_2.12 = 2.6.0
 org.scala-lang.modules:scala-collection-compat_2.13 = 2.6.0
 com.emc.ecs:object-client-bundle = 3.3.2
 org.immutables:value = 2.9.2
+commons-dbutils:commons-dbutils = 1.7

Review Comment:
   Just QueryRunner with its statement-configuration/parameter-handling functionality along with the ResultSetHandler interface. Turned out to be surprisingly easy to replace the whole dbutils dependency with analogous but significantly simplified interfaces inline in JdbcSnowflakeClient.java, especially since we're only using String parameters in PreparedStatements for now and we probably don't need to worry about some of the more advanced dbutils configuration of things like maxFieldSize, fetchSize, etc. for now.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1052715720


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeClient.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.snowflake;
+
+import java.io.Closeable;
+import java.util.List;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+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 {

Review Comment:
   Changed to package-private



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1052736631


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/entities/SnowflakeIdentifier.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class SnowflakeIdentifier {
+  public enum Type {
+    ROOT,
+    DATABASE,
+    SCHEMA,
+    TABLE
+  }
+
+  private String databaseName;
+  private String schemaName;
+  private String tableName;
+
+  protected SnowflakeIdentifier(String databaseName, String schemaName, String tableName) {
+    this.databaseName = databaseName;
+    this.schemaName = schemaName;
+    this.tableName = tableName;
+  }
+
+  public static SnowflakeIdentifier ofRoot() {
+    return new SnowflakeIdentifier(null, null, null);
+  }
+
+  public static SnowflakeIdentifier ofDatabase(String databaseName) {
+    Preconditions.checkArgument(null != databaseName, "databaseName must be non-null");
+    return new SnowflakeIdentifier(databaseName, null, null);
+  }
+
+  public static SnowflakeIdentifier ofSchema(String databaseName, String schemaName) {
+    Preconditions.checkArgument(null != databaseName, "databaseName must be non-null");
+    Preconditions.checkArgument(null != schemaName, "schemaName must be non-null");
+    return new SnowflakeIdentifier(databaseName, schemaName, null);
+  }
+
+  public static SnowflakeIdentifier ofTable(
+      String databaseName, String schemaName, String tableName) {
+    Preconditions.checkArgument(null != databaseName, "databaseName must be non-null");
+    Preconditions.checkArgument(null != schemaName, "schemaName must be non-null");
+    Preconditions.checkArgument(null != tableName, "tableName must be non-null");
+    return new SnowflakeIdentifier(databaseName, schemaName, tableName);
+  }
+
+  /**
+   * If type is TABLE, expect non-null databaseName, schemaName, and tableName. If type is SCHEMA,
+   * expect non-null databaseName and schemaName. If type is DATABASE, expect non-null databaseName.
+   * If type is ROOT, expect all of databaseName, schemaName, and tableName to be null.
+   */
+  public Type getType() {
+    if (null != tableName) {
+      return Type.TABLE;
+    } else if (null != schemaName) {
+      return Type.SCHEMA;
+    } else if (null != databaseName) {
+      return Type.DATABASE;
+    } else {
+      return Type.ROOT;
+    }
+  }
+
+  public String getTableName() {
+    return tableName;
+  }
+
+  public String getDatabaseName() {
+    return databaseName;
+  }
+
+  public String getSchemaName() {
+    return schemaName;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    } else if (!(o instanceof SnowflakeIdentifier)) {
+      return false;
+    }
+
+    SnowflakeIdentifier that = (SnowflakeIdentifier) o;
+    return Objects.equal(this.databaseName, that.databaseName)
+        && Objects.equal(this.schemaName, that.schemaName)
+        && Objects.equal(this.tableName, that.tableName);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(databaseName, schemaName, tableName);
+  }
+
+  /** Returns this identifier as a String suitable for use in a Snowflake IDENTIFIER param. */
+  public String toIdentifierString() {
+    switch (getType()) {
+      case TABLE:
+        return String.format("%s.%s.%s", databaseName, schemaName, tableName);
+      case SCHEMA:
+        return String.format("%s.%s", databaseName, schemaName);
+      case DATABASE:
+        return databaseName;
+      default:
+        return "";
+    }
+  }
+
+  @Override
+  public String toString() {
+    return String.format("%s: '%s'", getType(), toIdentifierString());
+  }
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Schema identifiers,
+   * containing "database_name" and "name" (representing schemaName).
+   */
+  public static ResultSetHandler<List<SnowflakeIdentifier>> createSchemaHandler() {
+    return rs -> {

Review Comment:
   Done, turned into public static finals defined at top of class instead of method calls.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1052897044


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.snowflake.entities.SnowflakeIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.hadoop.HadoopFileIO";
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIO fileIO;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    LOG.debug("listTables with namespace: {}", namespace);
+    SnowflakeIdentifier scope = NamespaceHelpers.getSnowflakeIdentifierForNamespace(namespace);
+    Preconditions.checkArgument(
+        scope.getType() == SnowflakeIdentifier.Type.ROOT
+            || scope.getType() == SnowflakeIdentifier.Type.DATABASE
+            || scope.getType() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at ROOT, DATABASE, or SCHEMA level; got %s from namespace %s",
+        scope,
+        namespace);
+
+    List<SnowflakeIdentifier> sfTables = snowflakeClient.listIcebergTables(scope);
+
+    return sfTables.stream()
+        .map(
+            table ->
+                TableIdentifier.of(
+                    table.getDatabaseName(), table.getSchemaName(), table.getTableName()))
+        .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) {
+    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

Review Comment:
   Fixed



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] nastra commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1056076021


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.io.ResolvingFileIO";

Review Comment:
   looks like `DEFAULT_CATALOG_NAME` & `DEFAULT_FILE_IO_IMPL` are only used inside `SnowflakeCatalog`, thus both could be made private



##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeTableOperations.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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 org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class SnowflakeTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeTableOperations.class);
+
+  private final FileIO fileIO;
+  private final TableIdentifier tableIdentifier;
+  private final SnowflakeIdentifier snowflakeIdentifierForTable;
+  private final String fullTableName;
+
+  private final SnowflakeClient snowflakeClient;
+
+  protected SnowflakeTableOperations(
+      SnowflakeClient snowflakeClient,
+      FileIO fileIO,
+      String catalogName,
+      TableIdentifier tableIdentifier) {
+    this.snowflakeClient = snowflakeClient;
+    this.fileIO = fileIO;
+    this.tableIdentifier = tableIdentifier;
+    this.snowflakeIdentifierForTable = NamespaceHelpers.toSnowflakeIdentifier(tableIdentifier);
+    this.fullTableName = String.format("%s.%s", catalogName, tableIdentifier.toString());

Review Comment:
   ```suggestion
       this.fullTableName = String.format("%s.%s", catalogName, tableIdentifier);
   ```



##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.io.ResolvingFileIO";
+
+  static class FileIOFactory {
+    public FileIO newFileIO(String impl, Map<String, String> properties, Object hadoopConf) {
+      return CatalogUtil.loadFileIO(impl, properties, hadoopConf);
+    }
+  }
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIOFactory fileIOFactory;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    SnowflakeIdentifier scope = NamespaceHelpers.toSnowflakeIdentifier(namespace);
+    Preconditions.checkArgument(
+        scope.type() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at SCHEMA level; got %s from namespace %s",
+        scope,
+        namespace);
+
+    List<SnowflakeIdentifier> sfTables = snowflakeClient.listIcebergTables(scope);
+
+    return sfTables.stream()
+        .map(NamespaceHelpers::toIcebergTableIdentifier)
+        .collect(Collectors.toList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support dropTable");
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support renameTable");
+  }
+
+  @Override
+  public void initialize(String name, Map<String, String> properties) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");

Review Comment:
   here we do a `Preconditions.checkNotNull(uri, ..)` while in the other `initialize()` method we use `Preconditions.checkArgument(null != ...)`. Would be great to align this to `Preconditions.checkArgument(..)`



##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.io.ResolvingFileIO";
+
+  static class FileIOFactory {
+    public FileIO newFileIO(String impl, Map<String, String> properties, Object hadoopConf) {
+      return CatalogUtil.loadFileIO(impl, properties, hadoopConf);
+    }
+  }
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIOFactory fileIOFactory;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    SnowflakeIdentifier scope = NamespaceHelpers.toSnowflakeIdentifier(namespace);
+    Preconditions.checkArgument(
+        scope.type() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at SCHEMA level; got %s from namespace %s",
+        scope,
+        namespace);
+
+    List<SnowflakeIdentifier> sfTables = snowflakeClient.listIcebergTables(scope);
+
+    return sfTables.stream()
+        .map(NamespaceHelpers::toIcebergTableIdentifier)
+        .collect(Collectors.toList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support dropTable");
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support renameTable");
+  }
+
+  @Override
+  public void initialize(String name, Map<String, String> properties) {
+    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 communication.
+      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);
+
+    initialize(name, new JdbcSnowflakeClient(connectionPool), new FileIOFactory(), properties);
+  }
+
+  /**
+   * Initialize using caller-supplied SnowflakeClient and FileIO.
+   *
+   * @param name The name of the catalog, defaults to "snowflake_catalog"
+   * @param snowflakeClient The client encapsulating network communication with Snowflake
+   * @param fileIOFactory The {@link FileIOFactory} to use to instantiate a new FileIO for each new
+   *     table operation
+   * @param properties The catalog options to use and propagate to dependencies
+   */
+  @SuppressWarnings("checkstyle:HiddenField")
+  void initialize(
+      String name,
+      SnowflakeClient snowflakeClient,
+      FileIOFactory fileIOFactory,
+      Map<String, String> properties) {
+    Preconditions.checkArgument(null != snowflakeClient, "snowflakeClient must be non-null");
+    Preconditions.checkArgument(null != fileIOFactory, "fileIOFactory must be non-null");
+    this.catalogName = name == null ? DEFAULT_CATALOG_NAME : name;
+    this.snowflakeClient = snowflakeClient;
+    this.fileIOFactory = fileIOFactory;
+    this.catalogProperties = properties;
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(snowflakeClient);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (null != closeableGroup) {
+      closeableGroup.close();
+    }
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support createNamespace");
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    SnowflakeIdentifier scope = NamespaceHelpers.toSnowflakeIdentifier(namespace);
+    List<SnowflakeIdentifier> results = null;
+    switch (scope.type()) {
+      case ROOT:
+        results = snowflakeClient.listDatabases();
+        break;
+      case DATABASE:
+        results = snowflakeClient.listSchemas(scope);
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format(
+                "listNamespaces must be at either ROOT or DATABASE level; got %s from namespace %s",
+                scope, namespace));
+    }
+
+    List<Namespace> namespaceList =
+        results.stream()
+            .map(
+                result -> {
+                  Preconditions.checkState(
+                      result.type() == SnowflakeIdentifier.Type.SCHEMA
+                          || result.type() == SnowflakeIdentifier.Type.DATABASE,
+                      "Got identifier of type %s from listNamespaces for %s",
+                      result.type(),
+                      namespace);
+                  return NamespaceHelpers.toIcebergNamespace(result);
+                })
+            .collect(Collectors.toList());
+    return namespaceList;
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace)
+      throws NoSuchNamespaceException {
+    SnowflakeIdentifier id = NamespaceHelpers.toSnowflakeIdentifier(namespace);
+    boolean namespaceExists;
+    switch (id.type()) {
+      case DATABASE:
+        namespaceExists = snowflakeClient.databaseExists(id);
+        break;
+      case SCHEMA:
+        namespaceExists = snowflakeClient.schemaExists(id);
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format(
+                "loadNamespaceMetadat must be at either DATABASE or SCHEMA level; got %s from namespace %s",
+                id, namespace));
+    }
+    if (namespaceExists) {
+      return ImmutableMap.of();
+    } else {
+      throw new NoSuchNamespaceException(
+          "Namespace '%s' with snowflake identifier '%s' doesn't exist", namespace, id);
+    }
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support dropNamespace");
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support setProperties");
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support removeProperties");
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    String fileIOImpl = DEFAULT_FILE_IO_IMPL;
+    if (catalogProperties.containsKey(CatalogProperties.FILE_IO_IMPL)) {
+      fileIOImpl = catalogProperties.get(CatalogProperties.FILE_IO_IMPL);
+    }
+    FileIO fileIO = fileIOFactory.newFileIO(fileIOImpl, catalogProperties, conf);

Review Comment:
   could you elaborate please why we're initializing FileIO here? Not that it's a bad thing to do, but I'd just like to understand the context (maybe a comment would help here as well)



##########
snowflake/src/main/java/org/apache/iceberg/snowflake/NamespaceHelpers.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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 org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class NamespaceHelpers {
+  private static final int MAX_NAMESPACE_DEPTH = 2;
+  private static final int NAMESPACE_ROOT_LEVEL = 0;
+  private static final int NAMESPACE_DB_LEVEL = 1;
+  private static final int NAMESPACE_SCHEMA_LEVEL = 2;
+
+  private static final Logger LOG = LoggerFactory.getLogger(NamespaceHelpers.class);
+
+  private NamespaceHelpers() {}
+
+  /**
+   * Converts a Namespace into a SnowflakeIdentifier representing ROOT, a DATABASE, or a SCHEMA.
+   *
+   * @throws IllegalArgumentException if the namespace is not a supported depth.
+   */
+  public static SnowflakeIdentifier toSnowflakeIdentifier(Namespace namespace) {

Review Comment:
   should there be a `TestNamespaceHelpers` class that makes sure the functionality here is tested?



##########
snowflake/src/main/java/org/apache/iceberg/snowflake/NamespaceHelpers.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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 org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class NamespaceHelpers {
+  private static final int MAX_NAMESPACE_DEPTH = 2;
+  private static final int NAMESPACE_ROOT_LEVEL = 0;
+  private static final int NAMESPACE_DB_LEVEL = 1;
+  private static final int NAMESPACE_SCHEMA_LEVEL = 2;
+
+  private static final Logger LOG = LoggerFactory.getLogger(NamespaceHelpers.class);
+
+  private NamespaceHelpers() {}
+
+  /**
+   * Converts a Namespace into a SnowflakeIdentifier representing ROOT, a DATABASE, or a SCHEMA.
+   *
+   * @throws IllegalArgumentException if the namespace is not a supported depth.
+   */
+  public static SnowflakeIdentifier toSnowflakeIdentifier(Namespace namespace) {
+    SnowflakeIdentifier identifier = null;
+    switch (namespace.length()) {
+      case NAMESPACE_ROOT_LEVEL:
+        identifier = SnowflakeIdentifier.ofRoot();
+        break;
+      case NAMESPACE_DB_LEVEL:
+        identifier = SnowflakeIdentifier.ofDatabase(namespace.level(NAMESPACE_DB_LEVEL - 1));
+        break;
+      case NAMESPACE_SCHEMA_LEVEL:
+        identifier =
+            SnowflakeIdentifier.ofSchema(
+                namespace.level(NAMESPACE_DB_LEVEL - 1),
+                namespace.level(NAMESPACE_SCHEMA_LEVEL - 1));
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format(
+                "Snowflake max namespace level is %d, got namespace '%s'",
+                MAX_NAMESPACE_DEPTH, namespace));
+    }
+    LOG.debug("toSnowflakeIdentifier({}) -> {}", namespace, identifier);
+    return identifier;
+  }
+
+  /**
+   * Converts a TableIdentifier into a SnowflakeIdentifier of type TABLE; the identifier must have
+   * exactly the right namespace depth to represent a fully-qualified Snowflake table identifier.
+   */
+  public static SnowflakeIdentifier toSnowflakeIdentifier(TableIdentifier identifier) {
+    SnowflakeIdentifier namespaceScope = toSnowflakeIdentifier(identifier.namespace());
+    Preconditions.checkArgument(
+        namespaceScope.type() == SnowflakeIdentifier.Type.SCHEMA,
+        "Namespace portion of '%s' must be at the SCHEMA level, got namespaceScope '%s'",
+        identifier,
+        namespaceScope);
+    SnowflakeIdentifier ret =
+        SnowflakeIdentifier.ofTable(
+            namespaceScope.databaseName(), namespaceScope.schemaName(), identifier.name());
+    LOG.debug("toSnowflakeIdentifier({}) -> {}", identifier, ret);
+    return ret;
+  }
+
+  /**
+   * Converts a SnowflakeIdentifier of type ROOT, DATABASE, or SCHEMA into an equivalent Iceberg
+   * Namespace; throws IllegalArgumentException if not an appropriate type.
+   */
+  public static Namespace toIcebergNamespace(SnowflakeIdentifier identifier) {
+    Namespace namespace = null;
+    switch (identifier.type()) {
+      case ROOT:
+        namespace = Namespace.of();

Review Comment:
   nit: `Namespace.empty()`



##########
snowflake/src/test/java/org/apache/iceberg/snowflake/SnowflakeCatalogTest.java:
##########
@@ -0,0 +1,310 @@
+/*
+ * 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.IOException;
+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.io.FileIO;
+import org.apache.iceberg.io.InMemoryFileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Types;
+import org.assertj.core.api.Assertions;
+import org.junit.Before;
+import org.junit.Test;
+
+public class SnowflakeCatalogTest {
+
+  static final String TEST_CATALOG_NAME = "slushLog";

Review Comment:
   nit: can probably be `private`



##########
snowflake/src/test/java/org/apache/iceberg/snowflake/FakeSnowflakeClient.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.jdbc.UncheckedSQLException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class FakeSnowflakeClient implements SnowflakeClient {

Review Comment:
   this can probably be a follow up, but would it make sense to eventually run the snowflake client against a real instance? I'm thinking whether it would be possible to use TestContainers with a Snowflake Docker image to do a real integration test for this? I think this should be doable and I could help with that.



##########
snowflake/src/test/java/org/apache/iceberg/snowflake/FakeSnowflakeClient.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.jdbc.UncheckedSQLException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class FakeSnowflakeClient implements SnowflakeClient {
+  // In-memory lookup by database/schema/tableName to table metadata.
+  private Map<String, Map<String, Map<String, SnowflakeTableMetadata>>> databases =

Review Comment:
   nit: can be final



##########
snowflake/src/test/java/org/apache/iceberg/snowflake/FakeSnowflakeClient.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.jdbc.UncheckedSQLException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class FakeSnowflakeClient implements SnowflakeClient {
+  // In-memory lookup by database/schema/tableName to table metadata.
+  private Map<String, Map<String, Map<String, SnowflakeTableMetadata>>> databases =
+      Maps.newTreeMap();
+  private boolean closed = false;
+
+  public FakeSnowflakeClient() {}
+
+  /**
+   * Also adds parent database/schema if they don't already exist. If the tableName already exists
+   * under the given database/schema, the value is replaced with the provided metadata.
+   */
+  public void addTable(

Review Comment:
   it seems that it would probably make sense to use `SnowflakeIdentifier` as a parameter rather than `database` / `schema` / `tableName`



##########
snowflake/src/test/java/org/apache/iceberg/snowflake/JdbcSnowflakeClientTest.java:
##########
@@ -0,0 +1,620 @@
+/*
+ * 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 static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+import org.apache.iceberg.ClientPool;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.jdbc.UncheckedInterruptedException;
+import org.apache.iceberg.jdbc.UncheckedSQLException;
+import org.assertj.core.api.Assertions;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.ArgumentMatchers;
+import org.mockito.Mock;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.junit.MockitoJUnitRunner;
+import org.mockito.stubbing.Answer;
+
+@RunWith(MockitoJUnitRunner.class)
+public class JdbcSnowflakeClientTest {
+  @Mock private Connection mockConnection;
+  @Mock private JdbcClientPool mockClientPool;
+  @Mock private JdbcSnowflakeClient.QueryHarness mockQueryHarness;
+  @Mock private ResultSet mockResultSet;
+
+  private JdbcSnowflakeClient snowflakeClient;
+
+  @Before
+  public void before() throws SQLException, InterruptedException {
+    snowflakeClient = new JdbcSnowflakeClient(mockClientPool);
+    snowflakeClient.setQueryHarness(mockQueryHarness);
+
+    doAnswer(
+            new Answer() {
+              @Override
+              public Object answer(InvocationOnMock invocation) throws Throwable {
+                return ((ClientPool.Action) invocation.getArguments()[0]).run(mockConnection);
+              }
+            })
+        .when(mockClientPool)
+        .run(any(ClientPool.Action.class));
+    doAnswer(
+            new Answer() {
+              @Override
+              public Object answer(InvocationOnMock invocation) throws Throwable {
+                return ((JdbcSnowflakeClient.ResultSetParser) invocation.getArguments()[2])
+                    .parse(mockResultSet);
+              }
+            })
+        .when(mockQueryHarness)
+        .query(
+            any(Connection.class),
+            any(String.class),
+            any(JdbcSnowflakeClient.ResultSetParser.class),
+            ArgumentMatchers.<String>any());
+  }
+
+  @Test
+  public void testNullClientPoolInConstructor() {
+    Assertions.assertThatExceptionOfType(IllegalArgumentException.class)
+        .isThrownBy(() -> new JdbcSnowflakeClient(null))
+        .withMessageContaining("JdbcClientPool must be non-null");
+  }
+
+  @Test
+  public void testDatabaseExists() throws SQLException {
+    when(mockResultSet.next()).thenReturn(true).thenReturn(false);
+    when(mockResultSet.getString("name")).thenReturn("DB_1");
+
+    Assertions.assertThat(snowflakeClient.databaseExists(SnowflakeIdentifier.ofDatabase("DB_1")))
+        .isTrue();
+
+    verify(mockQueryHarness)
+        .query(
+            eq(mockConnection),
+            eq("SHOW DATABASES LIKE 'DB_1' IN ACCOUNT"),
+            any(JdbcSnowflakeClient.ResultSetParser.class),
+            eq((String[]) null));

Review Comment:
   nit: unnecessary cast



##########
snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java:
##########
@@ -0,0 +1,362 @@
+/*
+ * 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.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+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.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+/**
+ * This implementation of SnowflakeClient builds on top of Snowflake's JDBC driver to interact with
+ * Snowflake's Iceberg-aware resource model.
+ */
+class JdbcSnowflakeClient implements SnowflakeClient {
+  public static final String EXPECTED_JDBC_IMPL = "net.snowflake.client.jdbc.SnowflakeDriver";

Review Comment:
   nit: can probably be package-private



##########
snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java:
##########
@@ -0,0 +1,362 @@
+/*
+ * 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.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+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.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+/**
+ * This implementation of SnowflakeClient builds on top of Snowflake's JDBC driver to interact with
+ * Snowflake's Iceberg-aware resource model.
+ */
+class JdbcSnowflakeClient implements SnowflakeClient {
+  public static final String EXPECTED_JDBC_IMPL = "net.snowflake.client.jdbc.SnowflakeDriver";
+
+  @FunctionalInterface
+  interface ResultSetParser<T> {
+    T parse(ResultSet rs) throws SQLException;
+  }
+
+  /**
+   * This class wraps the basic boilerplate of setting up PreparedStatements and applying a
+   * ResultSetParser to translate a ResultSet into parsed objects. Allows easily injecting
+   * subclasses for debugging/testing purposes.
+   */
+  static class QueryHarness {
+    public <T> T query(Connection conn, String sql, ResultSetParser<T> parser, String... args)
+        throws SQLException {
+      try (PreparedStatement statement = conn.prepareStatement(sql)) {
+        if (args != null) {
+          for (int i = 0; i < args.length; ++i) {
+            statement.setString(i + 1, args[i]);
+          }
+        }
+
+        try (ResultSet rs = statement.executeQuery()) {
+          return parser.parse(rs);
+        }
+      }
+    }
+  }
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Database identifiers,
+   * containing "name" (representing databaseName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> DATABASE_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> databases = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("name");
+          databases.add(SnowflakeIdentifier.ofDatabase(databaseName));
+        }
+        return databases;
+      };
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Schema identifiers,
+   * containing "database_name" and "name" (representing schemaName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> SCHEMA_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> schemas = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("database_name");
+          String schemaName = rs.getString("name");
+          schemas.add(SnowflakeIdentifier.ofSchema(databaseName, schemaName));
+        }
+        return schemas;
+      };
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Table identifiers,
+   * containing "database_name", "schema_name", and "name" (representing tableName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> TABLE_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> tables = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("database_name");
+          String schemaName = rs.getString("schema_name");
+          String tableName = rs.getString("name");
+          tables.add(SnowflakeIdentifier.ofTable(databaseName, schemaName, tableName));
+        }
+        return tables;
+      };
+
+  /**
+   * Expects to handle ResultSets representing a single record holding Snowflake Iceberg metadata.
+   */
+  public static final ResultSetParser<SnowflakeTableMetadata> TABLE_METADATA_RESULT_SET_HANDLER =
+      rs -> {
+        if (!rs.next()) {
+          return null;
+        }
+
+        String rawJsonVal = rs.getString("METADATA");
+        return SnowflakeTableMetadata.parseJson(rawJsonVal);
+      };
+
+  private final JdbcClientPool connectionPool;
+  private QueryHarness queryHarness;
+
+  JdbcSnowflakeClient(JdbcClientPool conn) {
+    Preconditions.checkArgument(null != conn, "JdbcClientPool must be non-null");
+    connectionPool = conn;
+    queryHarness = new QueryHarness();
+  }
+
+  @VisibleForTesting
+  void setQueryHarness(QueryHarness queryHarness) {
+    this.queryHarness = queryHarness;
+  }
+
+  /**
+   * For rare cases where PreparedStatements aren't supported for user-supplied identifiers intended
+   * for use in special LIKE clauses, we can sanitize by "broadening" the identifier with
+   * single-character wildcards and manually post-filter client-side.
+   *
+   * <p>Note: This sanitization approach intentionally "broadens" the scope of matching results;
+   * callers must be able to handle this method returning an all-wildcard expression; i.e. the
+   * caller must treat the usage of the LIKE clause as only an optional optimization, and should
+   * post-filter for correctness as if the LIKE clause wasn't present in the query at all.
+   */
+  @VisibleForTesting
+  String sanitizeIdentifierWithWildcardForLikeClause(String identifier) {
+    // Restrict identifiers to the "Unquoted object identifiers" synax documented at
+    // https://docs.snowflake.com/en/sql-reference/identifiers-syntax.html
+    //
+    // Use a strict allowlist of characters, replace everything *not* matching the character set
+    // with "_", which is used as a single-character wildcard in Snowflake.
+    String sanitized = identifier.replaceAll("[^a-zA-Z0-9_$]", "_");
+    if (sanitized.startsWith("$")) {
+      sanitized = "_" + sanitized.substring(1);
+    }
+    return sanitized;
+  }
+
+  @Override
+  public boolean databaseExists(SnowflakeIdentifier database) {
+    Preconditions.checkArgument(
+        database.type() == SnowflakeIdentifier.Type.DATABASE,
+        "databaseExists requires a DATABASE identifier, got '%s'",
+        database);
+
+    // Due to current limitations in PreparedStatement parameters for the LIKE clause in
+    // SHOW DATABASES queries, we'll use a fairly limited allowlist for identifier characters,
+    // using wildcards for non-allowed characters, and post-filter for matching.
+    final String finalQuery =
+        String.format(
+            "SHOW DATABASES LIKE '%s' IN ACCOUNT",
+            sanitizeIdentifierWithWildcardForLikeClause(database.databaseName()));
+    List<SnowflakeIdentifier> databases;
+    try {
+      databases =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn, finalQuery, DATABASE_RESULT_SET_HANDLER, (String[]) null));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to check if database exists");

Review Comment:
   should this maybe mention the `database` name being checked? Same for the exception messages in the other methods in this class



##########
core/src/test/java/org/apache/iceberg/io/InMemoryFileIO.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.io;
+
+import java.util.Map;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class InMemoryFileIO implements FileIO {

Review Comment:
   I would probably consider introducing this as a separate PR. That way it can be reviewed independently from the Snowflake Catalog PR



##########
snowflake/src/main/java/org/apache/iceberg/snowflake/NamespaceHelpers.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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 org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class NamespaceHelpers {
+  private static final int MAX_NAMESPACE_DEPTH = 2;
+  private static final int NAMESPACE_ROOT_LEVEL = 0;
+  private static final int NAMESPACE_DB_LEVEL = 1;
+  private static final int NAMESPACE_SCHEMA_LEVEL = 2;
+
+  private static final Logger LOG = LoggerFactory.getLogger(NamespaceHelpers.class);
+
+  private NamespaceHelpers() {}
+
+  /**
+   * Converts a Namespace into a SnowflakeIdentifier representing ROOT, a DATABASE, or a SCHEMA.
+   *
+   * @throws IllegalArgumentException if the namespace is not a supported depth.
+   */
+  public static SnowflakeIdentifier toSnowflakeIdentifier(Namespace namespace) {
+    SnowflakeIdentifier identifier = null;
+    switch (namespace.length()) {
+      case NAMESPACE_ROOT_LEVEL:
+        identifier = SnowflakeIdentifier.ofRoot();
+        break;
+      case NAMESPACE_DB_LEVEL:
+        identifier = SnowflakeIdentifier.ofDatabase(namespace.level(NAMESPACE_DB_LEVEL - 1));
+        break;
+      case NAMESPACE_SCHEMA_LEVEL:
+        identifier =
+            SnowflakeIdentifier.ofSchema(
+                namespace.level(NAMESPACE_DB_LEVEL - 1),
+                namespace.level(NAMESPACE_SCHEMA_LEVEL - 1));
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format(
+                "Snowflake max namespace level is %d, got namespace '%s'",
+                MAX_NAMESPACE_DEPTH, namespace));
+    }
+    LOG.debug("toSnowflakeIdentifier({}) -> {}", namespace, identifier);
+    return identifier;
+  }
+
+  /**
+   * Converts a TableIdentifier into a SnowflakeIdentifier of type TABLE; the identifier must have
+   * exactly the right namespace depth to represent a fully-qualified Snowflake table identifier.
+   */
+  public static SnowflakeIdentifier toSnowflakeIdentifier(TableIdentifier identifier) {
+    SnowflakeIdentifier namespaceScope = toSnowflakeIdentifier(identifier.namespace());
+    Preconditions.checkArgument(
+        namespaceScope.type() == SnowflakeIdentifier.Type.SCHEMA,
+        "Namespace portion of '%s' must be at the SCHEMA level, got namespaceScope '%s'",
+        identifier,
+        namespaceScope);
+    SnowflakeIdentifier ret =
+        SnowflakeIdentifier.ofTable(
+            namespaceScope.databaseName(), namespaceScope.schemaName(), identifier.name());
+    LOG.debug("toSnowflakeIdentifier({}) -> {}", identifier, ret);
+    return ret;
+  }
+
+  /**
+   * Converts a SnowflakeIdentifier of type ROOT, DATABASE, or SCHEMA into an equivalent Iceberg
+   * Namespace; throws IllegalArgumentException if not an appropriate type.
+   */
+  public static Namespace toIcebergNamespace(SnowflakeIdentifier identifier) {
+    Namespace namespace = null;
+    switch (identifier.type()) {
+      case ROOT:
+        namespace = Namespace.of();
+        break;
+      case DATABASE:
+        namespace = Namespace.of(identifier.databaseName());
+        break;
+      case SCHEMA:
+        namespace = Namespace.of(identifier.databaseName(), identifier.schemaName());
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format("Cannot convert identifier '%s' to Namespace", identifier));
+    }
+    LOG.debug("toIcebergNamespace({}) -> {}", identifier, namespace);

Review Comment:
   nit: I feel like this is just adding unnecessary overhead to the logic of the method. For example, rather than storing the result in a `namespace` variable, it would be simpler to just do a `return Namespace.of(..)` in each case.
   
   This applies to all methods in this class, so I'm not sure if we're gaining anything from the debug logging here? 



##########
snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.io.IOException;
+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.CloseableGroup;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SnowflakeCatalog extends BaseMetastoreCatalog
+    implements Closeable, SupportsNamespaces, Configurable<Object> {
+  public static final String DEFAULT_CATALOG_NAME = "snowflake_catalog";
+  public static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.io.ResolvingFileIO";
+
+  static class FileIOFactory {
+    public FileIO newFileIO(String impl, Map<String, String> properties, Object hadoopConf) {
+      return CatalogUtil.loadFileIO(impl, properties, hadoopConf);
+    }
+  }
+
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeCatalog.class);
+
+  private CloseableGroup closeableGroup;
+  private Object conf;
+  private String catalogName;
+  private Map<String, String> catalogProperties;
+  private FileIOFactory fileIOFactory;
+  private SnowflakeClient snowflakeClient;
+
+  public SnowflakeCatalog() {}
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    SnowflakeIdentifier scope = NamespaceHelpers.toSnowflakeIdentifier(namespace);
+    Preconditions.checkArgument(
+        scope.type() == SnowflakeIdentifier.Type.SCHEMA,
+        "listTables must be at SCHEMA level; got %s from namespace %s",
+        scope,
+        namespace);
+
+    List<SnowflakeIdentifier> sfTables = snowflakeClient.listIcebergTables(scope);
+
+    return sfTables.stream()
+        .map(NamespaceHelpers::toIcebergTableIdentifier)
+        .collect(Collectors.toList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support dropTable");
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support renameTable");
+  }
+
+  @Override
+  public void initialize(String name, Map<String, String> properties) {
+    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 communication.
+      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);
+
+    initialize(name, new JdbcSnowflakeClient(connectionPool), new FileIOFactory(), properties);
+  }
+
+  /**
+   * Initialize using caller-supplied SnowflakeClient and FileIO.
+   *
+   * @param name The name of the catalog, defaults to "snowflake_catalog"
+   * @param snowflakeClient The client encapsulating network communication with Snowflake
+   * @param fileIOFactory The {@link FileIOFactory} to use to instantiate a new FileIO for each new
+   *     table operation
+   * @param properties The catalog options to use and propagate to dependencies
+   */
+  @SuppressWarnings("checkstyle:HiddenField")
+  void initialize(
+      String name,
+      SnowflakeClient snowflakeClient,
+      FileIOFactory fileIOFactory,
+      Map<String, String> properties) {
+    Preconditions.checkArgument(null != snowflakeClient, "snowflakeClient must be non-null");
+    Preconditions.checkArgument(null != fileIOFactory, "fileIOFactory must be non-null");
+    this.catalogName = name == null ? DEFAULT_CATALOG_NAME : name;
+    this.snowflakeClient = snowflakeClient;
+    this.fileIOFactory = fileIOFactory;
+    this.catalogProperties = properties;
+    this.closeableGroup = new CloseableGroup();
+    closeableGroup.addCloseable(snowflakeClient);
+    closeableGroup.setSuppressCloseFailure(true);
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (null != closeableGroup) {
+      closeableGroup.close();
+    }
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    throw new UnsupportedOperationException(
+        "SnowflakeCatalog does not currently support createNamespace");
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) {
+    SnowflakeIdentifier scope = NamespaceHelpers.toSnowflakeIdentifier(namespace);
+    List<SnowflakeIdentifier> results = null;
+    switch (scope.type()) {
+      case ROOT:
+        results = snowflakeClient.listDatabases();
+        break;
+      case DATABASE:
+        results = snowflakeClient.listSchemas(scope);
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format(
+                "listNamespaces must be at either ROOT or DATABASE level; got %s from namespace %s",
+                scope, namespace));
+    }
+
+    List<Namespace> namespaceList =
+        results.stream()
+            .map(
+                result -> {
+                  Preconditions.checkState(

Review Comment:
   to me it feels like this check should be somewhere else - maybe in the `snowflakeClient.listDatabases()` /  `snowflakeClient.listSchemas(scope)` that then guarantess that the correct type comes back?



##########
snowflake/src/test/java/org/apache/iceberg/snowflake/JdbcSnowflakeClientTest.java:
##########
@@ -0,0 +1,620 @@
+/*
+ * 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 static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+import org.apache.iceberg.ClientPool;
+import org.apache.iceberg.jdbc.JdbcClientPool;
+import org.apache.iceberg.jdbc.UncheckedInterruptedException;
+import org.apache.iceberg.jdbc.UncheckedSQLException;
+import org.assertj.core.api.Assertions;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.ArgumentMatchers;
+import org.mockito.Mock;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.junit.MockitoJUnitRunner;
+import org.mockito.stubbing.Answer;
+
+@RunWith(MockitoJUnitRunner.class)
+public class JdbcSnowflakeClientTest {
+  @Mock private Connection mockConnection;
+  @Mock private JdbcClientPool mockClientPool;
+  @Mock private JdbcSnowflakeClient.QueryHarness mockQueryHarness;
+  @Mock private ResultSet mockResultSet;
+
+  private JdbcSnowflakeClient snowflakeClient;
+
+  @Before
+  public void before() throws SQLException, InterruptedException {
+    snowflakeClient = new JdbcSnowflakeClient(mockClientPool);
+    snowflakeClient.setQueryHarness(mockQueryHarness);
+
+    doAnswer(
+            new Answer() {

Review Comment:
   nit: this and other places can be replaced with a Lambda



##########
snowflake/src/main/java/org/apache/iceberg/snowflake/NamespaceHelpers.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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 org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class NamespaceHelpers {
+  private static final int MAX_NAMESPACE_DEPTH = 2;
+  private static final int NAMESPACE_ROOT_LEVEL = 0;
+  private static final int NAMESPACE_DB_LEVEL = 1;
+  private static final int NAMESPACE_SCHEMA_LEVEL = 2;
+
+  private static final Logger LOG = LoggerFactory.getLogger(NamespaceHelpers.class);
+
+  private NamespaceHelpers() {}
+
+  /**
+   * Converts a Namespace into a SnowflakeIdentifier representing ROOT, a DATABASE, or a SCHEMA.
+   *
+   * @throws IllegalArgumentException if the namespace is not a supported depth.
+   */
+  public static SnowflakeIdentifier toSnowflakeIdentifier(Namespace namespace) {
+    SnowflakeIdentifier identifier = null;
+    switch (namespace.length()) {
+      case NAMESPACE_ROOT_LEVEL:
+        identifier = SnowflakeIdentifier.ofRoot();
+        break;
+      case NAMESPACE_DB_LEVEL:
+        identifier = SnowflakeIdentifier.ofDatabase(namespace.level(NAMESPACE_DB_LEVEL - 1));
+        break;
+      case NAMESPACE_SCHEMA_LEVEL:
+        identifier =
+            SnowflakeIdentifier.ofSchema(
+                namespace.level(NAMESPACE_DB_LEVEL - 1),
+                namespace.level(NAMESPACE_SCHEMA_LEVEL - 1));
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format(
+                "Snowflake max namespace level is %d, got namespace '%s'",
+                MAX_NAMESPACE_DEPTH, namespace));
+    }
+    LOG.debug("toSnowflakeIdentifier({}) -> {}", namespace, identifier);
+    return identifier;
+  }
+
+  /**
+   * Converts a TableIdentifier into a SnowflakeIdentifier of type TABLE; the identifier must have
+   * exactly the right namespace depth to represent a fully-qualified Snowflake table identifier.
+   */
+  public static SnowflakeIdentifier toSnowflakeIdentifier(TableIdentifier identifier) {
+    SnowflakeIdentifier namespaceScope = toSnowflakeIdentifier(identifier.namespace());
+    Preconditions.checkArgument(
+        namespaceScope.type() == SnowflakeIdentifier.Type.SCHEMA,
+        "Namespace portion of '%s' must be at the SCHEMA level, got namespaceScope '%s'",
+        identifier,
+        namespaceScope);
+    SnowflakeIdentifier ret =
+        SnowflakeIdentifier.ofTable(
+            namespaceScope.databaseName(), namespaceScope.schemaName(), identifier.name());
+    LOG.debug("toSnowflakeIdentifier({}) -> {}", identifier, ret);
+    return ret;
+  }
+
+  /**
+   * Converts a SnowflakeIdentifier of type ROOT, DATABASE, or SCHEMA into an equivalent Iceberg
+   * Namespace; throws IllegalArgumentException if not an appropriate type.
+   */
+  public static Namespace toIcebergNamespace(SnowflakeIdentifier identifier) {
+    Namespace namespace = null;
+    switch (identifier.type()) {
+      case ROOT:
+        namespace = Namespace.of();
+        break;
+      case DATABASE:
+        namespace = Namespace.of(identifier.databaseName());
+        break;
+      case SCHEMA:
+        namespace = Namespace.of(identifier.databaseName(), identifier.schemaName());
+        break;
+      default:
+        throw new IllegalArgumentException(
+            String.format("Cannot convert identifier '%s' to Namespace", identifier));
+    }
+    LOG.debug("toIcebergNamespace({}) -> {}", identifier, namespace);
+    return namespace;
+  }
+
+  /**
+   * Converts a SnowflakeIdentifier to an equivalent Iceberg TableIdentifier; the identifier must be
+   * of type TABLE.
+   */
+  public static TableIdentifier toIcebergTableIdentifier(SnowflakeIdentifier identifier) {
+    Preconditions.checkArgument(
+        identifier.type() == SnowflakeIdentifier.Type.TABLE,
+        "SnowflakeIdentifier must be type TABLE, get '%s'",

Review Comment:
   nit: get -> got



##########
snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java:
##########
@@ -0,0 +1,362 @@
+/*
+ * 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.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+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.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+/**
+ * This implementation of SnowflakeClient builds on top of Snowflake's JDBC driver to interact with
+ * Snowflake's Iceberg-aware resource model.
+ */
+class JdbcSnowflakeClient implements SnowflakeClient {
+  public static final String EXPECTED_JDBC_IMPL = "net.snowflake.client.jdbc.SnowflakeDriver";
+
+  @FunctionalInterface
+  interface ResultSetParser<T> {
+    T parse(ResultSet rs) throws SQLException;
+  }
+
+  /**
+   * This class wraps the basic boilerplate of setting up PreparedStatements and applying a
+   * ResultSetParser to translate a ResultSet into parsed objects. Allows easily injecting
+   * subclasses for debugging/testing purposes.
+   */
+  static class QueryHarness {
+    public <T> T query(Connection conn, String sql, ResultSetParser<T> parser, String... args)
+        throws SQLException {
+      try (PreparedStatement statement = conn.prepareStatement(sql)) {
+        if (args != null) {
+          for (int i = 0; i < args.length; ++i) {
+            statement.setString(i + 1, args[i]);
+          }
+        }
+
+        try (ResultSet rs = statement.executeQuery()) {
+          return parser.parse(rs);
+        }
+      }
+    }
+  }
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Database identifiers,
+   * containing "name" (representing databaseName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> DATABASE_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> databases = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("name");
+          databases.add(SnowflakeIdentifier.ofDatabase(databaseName));
+        }
+        return databases;
+      };
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Schema identifiers,
+   * containing "database_name" and "name" (representing schemaName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> SCHEMA_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> schemas = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("database_name");
+          String schemaName = rs.getString("name");
+          schemas.add(SnowflakeIdentifier.ofSchema(databaseName, schemaName));
+        }
+        return schemas;
+      };
+
+  /**
+   * Expects to handle ResultSets representing fully-qualified Snowflake Table identifiers,
+   * containing "database_name", "schema_name", and "name" (representing tableName).
+   */
+  public static final ResultSetParser<List<SnowflakeIdentifier>> TABLE_RESULT_SET_HANDLER =
+      rs -> {
+        List<SnowflakeIdentifier> tables = Lists.newArrayList();
+        while (rs.next()) {
+          String databaseName = rs.getString("database_name");
+          String schemaName = rs.getString("schema_name");
+          String tableName = rs.getString("name");
+          tables.add(SnowflakeIdentifier.ofTable(databaseName, schemaName, tableName));
+        }
+        return tables;
+      };
+
+  /**
+   * Expects to handle ResultSets representing a single record holding Snowflake Iceberg metadata.
+   */
+  public static final ResultSetParser<SnowflakeTableMetadata> TABLE_METADATA_RESULT_SET_HANDLER =
+      rs -> {
+        if (!rs.next()) {
+          return null;
+        }
+
+        String rawJsonVal = rs.getString("METADATA");
+        return SnowflakeTableMetadata.parseJson(rawJsonVal);
+      };
+
+  private final JdbcClientPool connectionPool;
+  private QueryHarness queryHarness;
+
+  JdbcSnowflakeClient(JdbcClientPool conn) {
+    Preconditions.checkArgument(null != conn, "JdbcClientPool must be non-null");
+    connectionPool = conn;
+    queryHarness = new QueryHarness();
+  }
+
+  @VisibleForTesting
+  void setQueryHarness(QueryHarness queryHarness) {
+    this.queryHarness = queryHarness;
+  }
+
+  /**
+   * For rare cases where PreparedStatements aren't supported for user-supplied identifiers intended
+   * for use in special LIKE clauses, we can sanitize by "broadening" the identifier with
+   * single-character wildcards and manually post-filter client-side.
+   *
+   * <p>Note: This sanitization approach intentionally "broadens" the scope of matching results;
+   * callers must be able to handle this method returning an all-wildcard expression; i.e. the
+   * caller must treat the usage of the LIKE clause as only an optional optimization, and should
+   * post-filter for correctness as if the LIKE clause wasn't present in the query at all.
+   */
+  @VisibleForTesting
+  String sanitizeIdentifierWithWildcardForLikeClause(String identifier) {
+    // Restrict identifiers to the "Unquoted object identifiers" synax documented at
+    // https://docs.snowflake.com/en/sql-reference/identifiers-syntax.html
+    //
+    // Use a strict allowlist of characters, replace everything *not* matching the character set
+    // with "_", which is used as a single-character wildcard in Snowflake.
+    String sanitized = identifier.replaceAll("[^a-zA-Z0-9_$]", "_");
+    if (sanitized.startsWith("$")) {
+      sanitized = "_" + sanitized.substring(1);
+    }
+    return sanitized;
+  }
+
+  @Override
+  public boolean databaseExists(SnowflakeIdentifier database) {
+    Preconditions.checkArgument(
+        database.type() == SnowflakeIdentifier.Type.DATABASE,
+        "databaseExists requires a DATABASE identifier, got '%s'",
+        database);
+
+    // Due to current limitations in PreparedStatement parameters for the LIKE clause in
+    // SHOW DATABASES queries, we'll use a fairly limited allowlist for identifier characters,
+    // using wildcards for non-allowed characters, and post-filter for matching.
+    final String finalQuery =
+        String.format(
+            "SHOW DATABASES LIKE '%s' IN ACCOUNT",
+            sanitizeIdentifierWithWildcardForLikeClause(database.databaseName()));
+    List<SnowflakeIdentifier> databases;
+    try {
+      databases =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn, finalQuery, DATABASE_RESULT_SET_HANDLER, (String[]) null));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to check if database exists");
+    } catch (InterruptedException e) {
+      throw new UncheckedInterruptedException(e, "Interrupted while checking if database exists");
+    }
+
+    // Filter to handle the edge case of '_' appearing as a wildcard that can't be remapped the way
+    // it can for predicates in SELECT statements.
+    databases.removeIf(db -> !database.databaseName().equalsIgnoreCase(db.databaseName()));
+    return !databases.isEmpty();
+  }
+
+  @Override
+  public boolean schemaExists(SnowflakeIdentifier schema) {
+    Preconditions.checkArgument(
+        schema.type() == SnowflakeIdentifier.Type.SCHEMA,
+        "schemaExists requires a SCHEMA identifier, got '%s'",
+        schema);
+
+    if (!databaseExists(SnowflakeIdentifier.ofDatabase(schema.databaseName()))) {
+      return false;
+    }
+
+    // Due to current limitations in PreparedStatement parameters for the LIKE clause in
+    // SHOW SCHEMAS queries, we'll use a fairly limited allowlist for identifier characters,
+    // using wildcards for non-allowed characters, and post-filter for matching.
+    final String finalQuery =
+        String.format(
+            "SHOW SCHEMAS LIKE '%s' IN DATABASE IDENTIFIER(?)",
+            sanitizeIdentifierWithWildcardForLikeClause(schema.schemaName()));
+    List<SnowflakeIdentifier> schemas;
+    try {
+      schemas =
+          connectionPool.run(
+              conn ->
+                  queryHarness.query(
+                      conn,
+                      finalQuery,
+                      SCHEMA_RESULT_SET_HANDLER,
+                      new String[] {schema.databaseName()}));
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to check if schema exists");

Review Comment:
   should this mention the `schema` name being checked?



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [iceberg] dennishuo commented on a diff in pull request #6428: Add new SnowflakeCatalog implementation to enable directly using Snowflake-managed Iceberg tables

Posted by GitBox <gi...@apache.org>.
dennishuo commented on code in PR #6428:
URL: https://github.com/apache/iceberg/pull/6428#discussion_r1056620988


##########
core/src/test/java/org/apache/iceberg/io/InMemoryFileIO.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.io;
+
+import java.util.Map;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class InMemoryFileIO implements FileIO {

Review Comment:
   Yeah, started out with this just being in the snowflake package; I expanded this and moved it here due to https://github.com/apache/iceberg/pull/6428#discussion_r1051680729
   
   I could move the expanded functionality back into snowflake package and then send a future followup just moving the file back into the shared package.
   
   @rdblue what do you think?



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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