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

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


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

Reply via email to